From 10ee4e64e5b226d414389a221369f61a8c090b5d Mon Sep 17 00:00:00 2001 From: t-bast Date: Wed, 27 Nov 2024 15:33:01 +0100 Subject: [PATCH 1/3] Add support for official trampoline payments We add support for the official version of trampoline payments, as specified in https://github.com/lightning/bolts/pull/836. We keep supporting trampoline payments that use the legacy protocol to allow a smooth transition. We hardcode the legacy feature bit 149 in a few places to make this work, which is a bit hacky but simple and should be removed 6 months after releasing the official version. We also keep supporting payments from trampoline wallets to nodes that don't support trampoline: this is bad from a privacy standpoint, but will be fixed when recipients start supporting Bolt 12. --- docs/release-notes/eclair-vnext.md | 14 ++ eclair-core/src/main/resources/reference.conf | 3 +- .../main/scala/fr/acinq/eclair/Features.scala | 21 +-- .../scala/fr/acinq/eclair/NodeParams.scala | 2 - .../acinq/eclair/payment/PaymentPacket.scala | 12 +- .../payment/receive/MultiPartHandler.scala | 7 +- .../eclair/payment/relay/NodeRelay.scala | 6 +- .../acinq/eclair/payment/relay/Relayer.scala | 4 +- .../acinq/eclair/payment/send/Recipient.scala | 6 +- .../send/TrampolinePaymentLifecycle.scala | 2 +- .../eclair/wire/protocol/PaymentOnion.scala | 53 ++++-- .../scala/fr/acinq/eclair/TestConstants.scala | 4 +- .../integration/PaymentIntegrationSpec.scala | 22 +-- .../eclair/payment/Bolt11InvoiceSpec.scala | 26 ++- .../eclair/payment/MultiPartHandlerSpec.scala | 19 +- .../eclair/payment/PaymentInitiatorSpec.scala | 2 +- .../eclair/payment/PaymentPacketSpec.scala | 170 +++++++++++++++++- .../eclair/payment/relay/RelayerSpec.scala | 7 +- .../wire/protocol/PaymentOnionSpec.scala | 94 +++++----- 19 files changed, 332 insertions(+), 142 deletions(-) diff --git a/docs/release-notes/eclair-vnext.md b/docs/release-notes/eclair-vnext.md index c590cf5a69..fb47926880 100644 --- a/docs/release-notes/eclair-vnext.md +++ b/docs/release-notes/eclair-vnext.md @@ -4,6 +4,20 @@ ## Major changes +### Trampoline payments + +Trampoline payments allow nodes running on constrained devices to sync only a small portion of the network and leverage trampoline nodes to calculate the missing parts of the payment route, while providing the same privacy as fully source-routed payments. + +Eclair started supporting [trampoline payments](https://github.com/lightning/bolts/pull/829) in v0.3.3. +The specification has evolved since then and has recently been added to the [BOLTs](https://github.com/lightning/bolts/pull/836). + +With this release, eclair nodes are able to relay and receive trampoline payments (activated by default). +This feature can be disabled if you don't want to relay or receive trampoline payments: + +```conf +eclair.features.trampoline_routing = disabled +``` + ### New MPP splitting strategy Eclair can send large payments using multiple low-capacity routes by sending as much as it can through each route (if `randomize-route-selection = false`) or some random fraction (if `randomize-route-selection = true`). diff --git a/eclair-core/src/main/resources/reference.conf b/eclair-core/src/main/resources/reference.conf index a6c7b837bd..fafd9d7c3a 100644 --- a/eclair-core/src/main/resources/reference.conf +++ b/eclair-core/src/main/resources/reference.conf @@ -49,7 +49,6 @@ eclair { node-alias = "eclair" node-color = "49daaa" - trampoline-payments-enable = false // TODO: @t-bast: once spec-ed this should use a global feature flag // see https://github.com/lightningnetwork/lightning-rfc/blob/master/09-features.md features { // option_upfront_shutdown_script is not activated by default. @@ -88,7 +87,7 @@ eclair { option_zeroconf = disabled keysend = disabled option_simple_close=optional - trampoline_payment_prototype = disabled + trampoline_routing = optional async_payment_prototype = disabled on_the_fly_funding = disabled } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Features.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Features.scala index 12e5127208..1db3d00edb 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Features.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Features.scala @@ -307,6 +307,11 @@ object Features { val mandatory = 54 } + case object TrampolinePayment extends Feature with InitFeature with NodeFeature with Bolt11Feature with Bolt12Feature { + val rfcName = "trampoline_routing" + val mandatory = 56 + } + case object SimpleClose extends Feature with InitFeature with NodeFeature { val rfcName = "option_simple_close" val mandatory = 60 @@ -323,17 +328,6 @@ object Features { val mandatory = 132 } - // TODO: @t-bast: update feature bits once spec-ed (currently reserved here: https://github.com/lightningnetwork/lightning-rfc/issues/605) - // We're not advertising these bits yet in our announcements, clients have to assume support. - // This is why we haven't added them yet to `areSupported`. - // The version of trampoline enabled by this feature bit does not match the latest spec PR: once the spec is accepted, - // we will introduce a new version of trampoline that will work in parallel to this legacy one, until we can safely - // deprecate it. - case object TrampolinePaymentPrototype extends Feature with InitFeature with NodeFeature with Bolt11Feature { - val rfcName = "trampoline_payment_prototype" - val mandatory = 148 - } - // TODO: @remyers update feature bits once spec-ed (currently reserved here: https://github.com/lightning/bolts/pull/989) case object AsyncPaymentPrototype extends Feature with InitFeature with Bolt11Feature { val rfcName = "async_payment_prototype" @@ -402,7 +396,7 @@ object Features { SimpleTaprootChannelsPhoenix, SimpleTaprootChannelsStaging, WakeUpNotificationClient, - TrampolinePaymentPrototype, + TrampolinePayment, AsyncPaymentPrototype, SplicePrototype, OnTheFlyFunding, @@ -418,11 +412,10 @@ object Features { AnchorOutputs -> (StaticRemoteKey :: Nil), AnchorOutputsZeroFeeHtlcTx -> (StaticRemoteKey :: Nil), RouteBlinding -> (VariableLengthOnion :: Nil), - TrampolinePaymentPrototype -> (PaymentSecret :: Nil), KeySend -> (VariableLengthOnion :: Nil), SimpleClose -> (ShutdownAnySegwit :: Nil), SimpleTaprootChannelsPhoenix -> (ChannelType :: SimpleClose :: Nil), - AsyncPaymentPrototype -> (TrampolinePaymentPrototype :: Nil), + AsyncPaymentPrototype -> (TrampolinePayment :: Nil), OnTheFlyFunding -> (SplicePrototype :: Nil), FundingFeeCredit -> (OnTheFlyFunding :: Nil) ) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala b/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala index e824e684dd..303464e9c9 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala @@ -84,7 +84,6 @@ case class NodeParams(nodeKeyManager: NodeKeyManager, socksProxy_opt: Option[Socks5ProxyParams], maxPaymentAttempts: Int, paymentFinalExpiry: PaymentFinalExpiryConf, - enableTrampolinePayment: Boolean, balanceCheckInterval: FiniteDuration, blockchainWatchdogThreshold: Int, blockchainWatchdogSources: Seq[String], @@ -683,7 +682,6 @@ object NodeParams extends Logging { socksProxy_opt = socksProxy_opt, maxPaymentAttempts = config.getInt("max-payment-attempts"), paymentFinalExpiry = PaymentFinalExpiryConf(CltvExpiryDelta(config.getInt("send.recipient-final-expiry.min-delta")), CltvExpiryDelta(config.getInt("send.recipient-final-expiry.max-delta"))), - enableTrampolinePayment = config.getBoolean("trampoline-payments-enable"), balanceCheckInterval = FiniteDuration(config.getDuration("balance-check-interval").getSeconds, TimeUnit.SECONDS), blockchainWatchdogThreshold = config.getInt("blockchain-watchdog.missing-blocks-threshold"), blockchainWatchdogSources = config.getStringList("blockchain-watchdog.sources").asScala.toSeq, diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala index 9a77355dcb..8636fd53e2 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala @@ -162,8 +162,9 @@ object IncomingPaymentPacket { case None if add.pathKey_opt.isDefined => Left(InvalidOnionBlinding(Sphinx.hash(add.onionRoutingPacket))) case None => // We check if the payment is using trampoline: if it is, we may not be the final recipient. - payload.get[OnionPaymentPayloadTlv.TrampolineOnion] match { - case Some(OnionPaymentPayloadTlv.TrampolineOnion(trampolinePacket)) => + val trampolinePacket_opt = payload.get[OnionPaymentPayloadTlv.TrampolineOnion].map(_.packet).orElse(payload.get[OnionPaymentPayloadTlv.LegacyTrampolineOnion].map(_.packet)) + trampolinePacket_opt match { + case Some(trampolinePacket) => val outerPayload = payload.get[OnionPaymentPayloadTlv.PaymentData] match { case Some(_) => payload // The spec allows omitting the payment_secret field when not using MPP to reach the trampoline node. @@ -252,7 +253,7 @@ object IncomingPaymentPacket { case innerPayload => // We merge contents from the outer and inner payloads. // We must use the inner payload's total amount and payment secret because the payment may be split between multiple trampoline payments (#reckless). - val trampolinePacket = outerPayload.records.get[OnionPaymentPayloadTlv.TrampolineOnion].map(_.packet) + val trampolinePacket = outerPayload.records.get[OnionPaymentPayloadTlv.TrampolineOnion].map(_.packet).orElse(outerPayload.records.get[OnionPaymentPayloadTlv.LegacyTrampolineOnion].map(_.packet)) Right(FinalPacket(add, FinalPayload.Standard.createPayload(outerPayload.amount, innerPayload.totalAmount, innerPayload.expiry, innerPayload.paymentSecret, innerPayload.paymentMetadata, trampolinePacket), TimestampMilli.now())) } } @@ -323,7 +324,10 @@ object OutgoingPaymentPacket { * In that case, packetPayloadLength_opt must be greater than the actual onion's content. */ def buildOnion(payloads: Seq[NodePayload], associatedData: ByteVector32, packetPayloadLength_opt: Option[Int]): Either[OutgoingPaymentError, Sphinx.PacketAndSecrets] = { - val sessionKey = randomKey() + buildOnion(randomKey(), payloads, associatedData, packetPayloadLength_opt) + } + + def buildOnion(sessionKey: PrivateKey, payloads: Seq[NodePayload], associatedData: ByteVector32, packetPayloadLength_opt: Option[Int]): Either[OutgoingPaymentError, Sphinx.PacketAndSecrets] = { val nodeIds = payloads.map(_.nodeId) val payloadsBin = payloads .map(p => PaymentOnionCodecs.perHopPayloadCodec.encode(p.payload.records)) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala index 6822aa721a..9070661dc1 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala @@ -322,11 +322,6 @@ object MultiPartHandler { val paymentHash = Crypto.sha256(paymentPreimage) val expirySeconds = r.expirySeconds_opt.getOrElse(nodeParams.invoiceExpiry.toSeconds) val paymentMetadata = hex"2a" - val featuresTrampolineOpt = if (nodeParams.enableTrampolinePayment) { - nodeParams.features.bolt11Features().add(Features.TrampolinePaymentPrototype, FeatureSupport.Optional) - } else { - nodeParams.features.bolt11Features() - } val invoice = Bolt11Invoice( nodeParams.chainHash, r.amount_opt, @@ -338,7 +333,7 @@ object MultiPartHandler { expirySeconds = Some(expirySeconds), extraHops = r.extraHops, paymentMetadata = Some(paymentMetadata), - features = featuresTrampolineOpt + features = nodeParams.features.bolt11Features() ) context.log.debug("generated invoice={} from amount={}", invoice.toString, r.amount_opt) nodeParams.db.payments.addIncomingPayment(invoice, paymentPreimage, r.paymentType) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala index 07fb03c030..2815fdd6f2 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala @@ -43,7 +43,7 @@ import fr.acinq.eclair.router.Router.{ChannelHop, HopRelayParams, Route, RoutePa import fr.acinq.eclair.router.{BalanceTooLow, RouteNotFound} import fr.acinq.eclair.wire.protocol.PaymentOnion.IntermediatePayload import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{Alias, CltvExpiry, CltvExpiryDelta, EncodedNodeId, Features, InitFeature, Logs, MilliSatoshi, MilliSatoshiLong, NodeParams, TimestampMilli, UInt64, nodeFee, randomBytes32} +import fr.acinq.eclair.{Alias, CltvExpiry, CltvExpiryDelta, EncodedNodeId, FeatureSupport, Features, InitFeature, InvoiceFeature, Logs, MilliSatoshi, MilliSatoshiLong, NodeParams, TimestampMilli, UInt64, UnknownFeature, nodeFee, randomBytes32} import scodec.bits.ByteVector import java.util.UUID @@ -251,7 +251,9 @@ class NodeRelay private(nodeParams: NodeParams, nextPayload match { case payloadOut: IntermediatePayload.NodeRelay.Standard => val paymentSecret = randomBytes32() // we generate a new secret to protect against probing attacks - val recipient = ClearRecipient(payloadOut.outgoingNodeId, Features.empty, payloadOut.amountToForward, payloadOut.outgoingCltv, paymentSecret, nextTrampolineOnion_opt = nextPacket_opt) + // If the recipient is using the legacy trampoline feature, we will use the legacy onion format. + val features = if (payloadOut.isLegacy) Features(Map.empty[InvoiceFeature, FeatureSupport], Set(UnknownFeature(149))) else Features.empty[InvoiceFeature] + val recipient = ClearRecipient(payloadOut.outgoingNodeId, features, payloadOut.amountToForward, payloadOut.outgoingCltv, paymentSecret, nextTrampolineOnion_opt = nextPacket_opt) context.log.debug("forwarding payment to the next trampoline node {}", recipient.nodeId) attemptWakeUpIfRecipientIsWallet(upstream, recipient, nextPayload, nextPacket_opt) case payloadOut: IntermediatePayload.NodeRelay.ToNonTrampoline => diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/Relayer.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/Relayer.scala index 920274be82..26d73a32ce 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/Relayer.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/Relayer.scala @@ -30,7 +30,7 @@ import fr.acinq.eclair.db.PendingCommandsDb import fr.acinq.eclair.payment._ import fr.acinq.eclair.reputation.{Reputation, ReputationRecorder} import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{CltvExpiryDelta, Logs, MilliSatoshi, NodeParams, RealShortChannelId, TimestampMilli} +import fr.acinq.eclair.{CltvExpiryDelta, Features, Logs, MilliSatoshi, NodeParams, RealShortChannelId, TimestampMilli} import grizzled.slf4j.Logging import scala.concurrent.Promise @@ -72,7 +72,7 @@ class Relayer(nodeParams: NodeParams, router: ActorRef, register: ActorRef, paym case Right(r: IncomingPaymentPacket.ChannelRelayPacket) => channelRelayer ! ChannelRelayer.Relay(r, originNode, incomingChannelOccupancy) case Right(r: IncomingPaymentPacket.NodeRelayPacket) => - if (!nodeParams.enableTrampolinePayment) { + if (!nodeParams.features.hasFeature(Features.TrampolinePayment)) { log.warning(s"rejecting htlc #${add.id} from channelId=${add.channelId} reason=trampoline disabled") val attribution = FailureAttributionData(htlcReceivedAt = r.receivedAt, trampolineReceivedAt_opt = None) PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, add.channelId, CMD_FAIL_HTLC(add.id, FailureReason.LocalFailure(RequiredNodeFeatureMissing()), Some(attribution), commit = true)) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Recipient.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Recipient.scala index 09b46a48e2..0ea1392ac8 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Recipient.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Recipient.scala @@ -25,7 +25,7 @@ import fr.acinq.eclair.payment.{Bolt11Invoice, Bolt12Invoice} import fr.acinq.eclair.router.Router._ import fr.acinq.eclair.wire.protocol.PaymentOnion.{FinalPayload, IntermediatePayload, OutgoingBlindedPerHopPayload} import fr.acinq.eclair.wire.protocol.{GenericTlv, OnionRoutingPacket} -import fr.acinq.eclair.{CltvExpiry, Features, InvoiceFeature, MilliSatoshi, ShortChannelId} +import fr.acinq.eclair.{CltvExpiry, Features, InvoiceFeature, MilliSatoshi, ShortChannelId, UnknownFeature} import scodec.bits.ByteVector /** @@ -74,9 +74,13 @@ case class ClearRecipient(nodeId: PublicKey, paymentMetadata_opt: Option[ByteVector] = None, nextTrampolineOnion_opt: Option[OnionRoutingPacket] = None, customTlvs: Set[GenericTlv] = Set.empty) extends Recipient { + // Feature bit used by the legacy trampoline feature. + private val isLegacyTrampoline = features.unknown.contains(UnknownFeature(149)) + override def buildPayloads(paymentHash: ByteVector32, route: Route): Either[OutgoingPaymentError, PaymentPayloads] = { ClearRecipient.validateRoute(nodeId, route).map(_ => { val finalPayload = nextTrampolineOnion_opt match { + case Some(trampolinePacket) if isLegacyTrampoline => NodePayload(nodeId, FinalPayload.Standard.createLegacyTrampolinePayload(route.amount, totalAmount, expiry, paymentSecret, trampolinePacket)) case Some(trampolinePacket) => NodePayload(nodeId, FinalPayload.Standard.createTrampolinePayload(route.amount, totalAmount, expiry, paymentSecret, trampolinePacket)) case None => NodePayload(nodeId, FinalPayload.Standard.createPayload(route.amount, totalAmount, expiry, paymentSecret, paymentMetadata_opt, trampolineOnion_opt = None, customTlvs = customTlvs)) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala index bf99fb55b3..d91c7d8d12 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala @@ -279,7 +279,7 @@ object TrampolinePayment { def buildOutgoingPayment(trampolineNodeId: PublicKey, invoice: Invoice, amount: MilliSatoshi, expiry: CltvExpiry, trampolinePaymentSecret_opt: Option[ByteVector32], attemptNumber: Int): OutgoingPayment = { val totalAmount = invoice.amount_opt.get val trampolineOnion = invoice match { - case invoice: Bolt11Invoice if invoice.features.hasFeature(Features.TrampolinePaymentPrototype) => + case invoice: Bolt11Invoice if invoice.features.hasFeature(Features.TrampolinePayment) => val finalPayload = PaymentOnion.FinalPayload.Standard.createPayload(amount, totalAmount, expiry, invoice.paymentSecret, invoice.paymentMetadata) val trampolinePayload = PaymentOnion.IntermediatePayload.NodeRelay.Standard(totalAmount, expiry, invoice.nodeId) buildOnion(NodePayload(trampolineNodeId, trampolinePayload) :: NodePayload(invoice.nodeId, finalPayload) :: Nil, invoice.paymentHash, None).toOption.get diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/PaymentOnion.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/PaymentOnion.scala index f32933e263..d322470d0a 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/PaymentOnion.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/PaymentOnion.scala @@ -143,9 +143,6 @@ object OnionPaymentPayloadTlv { */ case class PaymentData(secret: ByteVector32, totalAmount: MilliSatoshi) extends OnionPaymentPayloadTlv - /** Id of the next node. */ - case class OutgoingNodeId(nodeId: PublicKey) extends OnionPaymentPayloadTlv - /** * Route blinding lets the recipient provide some encrypted data for each intermediate node in the blinded part of the * route. This data cannot be decrypted or modified by the sender and usually contains information to locate the next @@ -156,8 +153,11 @@ object OnionPaymentPayloadTlv { /** Path key for the introduction node of a blinded route. */ case class PathKey(publicKey: PublicKey) extends OnionPaymentPayloadTlv - /** Total amount in blinded multi-part payments. */ - case class TotalAmount(totalAmount: MilliSatoshi) extends OnionPaymentPayloadTlv + /** Id of the next node (when using legacy trampoline payments). */ + case class LegacyOutgoingNodeId(nodeId: PublicKey) extends OnionPaymentPayloadTlv + + /** Id of the next node. */ + case class OutgoingNodeId(nodeId: PublicKey) extends OnionPaymentPayloadTlv /** * When payment metadata is included in a Bolt 11 invoice, we should send it as-is to the recipient. @@ -165,6 +165,12 @@ object OnionPaymentPayloadTlv { */ case class PaymentMetadata(data: ByteVector) extends OnionPaymentPayloadTlv + /** Total amount in blinded multi-part payments. */ + case class TotalAmount(totalAmount: MilliSatoshi) extends OnionPaymentPayloadTlv + + /** An encrypted trampoline onion packet. */ + case class TrampolineOnion(packet: OnionRoutingPacket) extends OnionPaymentPayloadTlv + /** * Invoice feature bits. Only included for intermediate trampoline nodes when they should convert to a legacy payment * because the final recipient doesn't support trampoline. @@ -177,8 +183,8 @@ object OnionPaymentPayloadTlv { */ case class InvoiceRoutingInfo(extraHops: List[List[Bolt11Invoice.ExtraHop]]) extends OnionPaymentPayloadTlv - /** An encrypted trampoline onion packet. */ - case class TrampolineOnion(packet: OnionRoutingPacket) extends OnionPaymentPayloadTlv + /** An encrypted trampoline onion packet (when using legacy trampoline payments). */ + case class LegacyTrampolineOnion(packet: OnionRoutingPacket) extends OnionPaymentPayloadTlv /** Pre-image included by the sender of a payment in case of a donation */ case class KeySend(paymentPreimage: ByteVector32) extends OnionPaymentPayloadTlv @@ -290,6 +296,7 @@ object PaymentOnion { sealed trait NodeRelay extends IntermediatePayload { // @formatter:off + def isLegacy: Boolean def outgoingAmount(incomingAmount: MilliSatoshi): MilliSatoshi def outgoingExpiry(incomingCltv: CltvExpiry): CltvExpiry // @formatter:on @@ -299,7 +306,8 @@ object PaymentOnion { case class Standard(records: TlvStream[OnionPaymentPayloadTlv]) extends NodeRelay { val amountToForward = records.get[AmountToForward].get.amount val outgoingCltv = records.get[OutgoingCltv].get.cltv - val outgoingNodeId = records.get[OutgoingNodeId].get.nodeId + val outgoingNodeId = records.get[OutgoingNodeId].map(_.nodeId).orElse(records.get[LegacyOutgoingNodeId].map(_.nodeId)).get + val isLegacy: Boolean = records.get[LegacyOutgoingNodeId].nonEmpty val isAsyncPayment: Boolean = records.get[AsyncPayment].isDefined // @formatter:off @@ -316,7 +324,7 @@ object PaymentOnion { def validate(records: TlvStream[OnionPaymentPayloadTlv]): Either[InvalidTlvPayload, Standard] = { if (records.get[AmountToForward].isEmpty) return Left(MissingRequiredTlv(UInt64(2))) if (records.get[OutgoingCltv].isEmpty) return Left(MissingRequiredTlv(UInt64(4))) - if (records.get[OutgoingNodeId].isEmpty) return Left(MissingRequiredTlv(UInt64(66098))) + if (records.get[OutgoingNodeId].isEmpty && records.get[LegacyOutgoingNodeId].isEmpty) return Left(MissingRequiredTlv(UInt64(14))) if (records.get[EncryptedRecipientData].nonEmpty) return Left(ForbiddenTlv(UInt64(10))) if (records.get[PathKey].nonEmpty) return Left(ForbiddenTlv(UInt64(12))) Right(Standard(records)) @@ -332,7 +340,7 @@ object PaymentOnion { case class ToNonTrampoline(records: TlvStream[OnionPaymentPayloadTlv]) extends NodeRelay { val amountToForward = records.get[AmountToForward].get.amount val outgoingCltv = records.get[OutgoingCltv].get.cltv - val outgoingNodeId = records.get[OutgoingNodeId].get.nodeId + val outgoingNodeId = records.get[OutgoingNodeId].map(_.nodeId).orElse(records.get[LegacyOutgoingNodeId].map(_.nodeId)).get val totalAmount = records.get[PaymentData].map(_.totalAmount match { case MilliSatoshi(0) => amountToForward case totalAmount => totalAmount @@ -341,6 +349,7 @@ object PaymentOnion { val paymentMetadata = records.get[PaymentMetadata].map(_.data) val invoiceFeatures = records.get[InvoiceFeatures].map(_.features).getOrElse(ByteVector.empty) val invoiceRoutingInfo = records.get[InvoiceRoutingInfo].map(_.extraHops).get + val isLegacy: Boolean = records.get[LegacyOutgoingNodeId].nonEmpty // @formatter:off override def outgoingAmount(incomingAmount: MilliSatoshi): MilliSatoshi = amountToForward @@ -366,7 +375,7 @@ object PaymentOnion { if (records.get[AmountToForward].isEmpty) return Left(MissingRequiredTlv(UInt64(2))) if (records.get[OutgoingCltv].isEmpty) return Left(MissingRequiredTlv(UInt64(4))) if (records.get[PaymentData].isEmpty) return Left(MissingRequiredTlv(UInt64(8))) - if (records.get[OutgoingNodeId].isEmpty) return Left(MissingRequiredTlv(UInt64(66098))) + if (records.get[OutgoingNodeId].isEmpty && records.get[LegacyOutgoingNodeId].isEmpty) return Left(MissingRequiredTlv(UInt64(14))) if (records.get[InvoiceRoutingInfo].isEmpty) return Left(MissingRequiredTlv(UInt64(66099))) if (records.get[EncryptedRecipientData].nonEmpty) return Left(ForbiddenTlv(UInt64(10))) if (records.get[PathKey].nonEmpty) return Left(ForbiddenTlv(UInt64(12))) @@ -380,6 +389,7 @@ object PaymentOnion { val outgoingCltv = records.get[OutgoingCltv].get.cltv val outgoingBlindedPaths = records.get[OutgoingBlindedPaths].get.paths val invoiceFeatures = records.get[InvoiceFeatures].get.features + val isLegacy: Boolean = records.get[LegacyTrampolineOnion].nonEmpty // @formatter:off override def outgoingAmount(incomingAmount: MilliSatoshi): MilliSatoshi = amountToForward @@ -467,6 +477,11 @@ object PaymentOnion { def createTrampolinePayload(amount: MilliSatoshi, totalAmount: MilliSatoshi, expiry: CltvExpiry, paymentSecret: ByteVector32, trampolinePacket: OnionRoutingPacket): Standard = { Standard(TlvStream(AmountToForward(amount), OutgoingCltv(expiry), PaymentData(paymentSecret, totalAmount), TrampolineOnion(trampolinePacket))) } + + /** Create a trampoline outer payload for legacy trampoline payments. */ + def createLegacyTrampolinePayload(amount: MilliSatoshi, totalAmount: MilliSatoshi, expiry: CltvExpiry, paymentSecret: ByteVector32, trampolinePacket: OnionRoutingPacket): Standard = { + Standard(TlvStream(AmountToForward(amount), OutgoingCltv(expiry), PaymentData(paymentSecret, totalAmount), LegacyTrampolineOnion(trampolinePacket))) + } } /** @@ -581,6 +596,7 @@ object PaymentOnionCodecs { private val pathKey: Codec[PathKey] = fixedLengthTlvField(33, publicKey) private val outgoingNodeId: Codec[OutgoingNodeId] = fixedLengthTlvField(33, publicKey) + private val legacyOutgoingNodeId: Codec[LegacyOutgoingNodeId] = fixedLengthTlvField(33, publicKey) private val paymentMetadata: Codec[PaymentMetadata] = tlvField(bytes) @@ -591,6 +607,7 @@ object PaymentOnionCodecs { private val invoiceRoutingInfo: Codec[InvoiceRoutingInfo] = tlvField(list(listOfN(uint8, Bolt11Invoice.Codecs.extraHopCodec))) private val trampolineOnion: Codec[TrampolineOnion] = tlvField(OnionRoutingCodecs.variableSizeOnionRoutingPacketCodec) + private val legacyTrampolineOnion: Codec[LegacyTrampolineOnion] = tlvField(OnionRoutingCodecs.variableSizeOnionRoutingPacketCodec) private val paymentBlindedRoute: Codec[PaymentBlindedRoute] = (("route" | OfferCodecs.blindedRouteCodec) :: @@ -610,14 +627,18 @@ object PaymentOnionCodecs { .typecase(UInt64(8), paymentData) .typecase(UInt64(10), encryptedRecipientData) .typecase(UInt64(12), pathKey) + .typecase(UInt64(14), outgoingNodeId) .typecase(UInt64(16), paymentMetadata) .typecase(UInt64(18), totalAmount) + .typecase(UInt64(20), trampolineOnion) + .typecase(UInt64(21), invoiceFeatures) + .typecase(UInt64(22), outgoingBlindedPaths) // Types below aren't specified - use cautiously when deploying (be careful with backwards-compatibility). - .typecase(UInt64(66097), invoiceFeatures) - .typecase(UInt64(66098), outgoingNodeId) - .typecase(UInt64(66099), invoiceRoutingInfo) - .typecase(UInt64(66100), trampolineOnion) - .typecase(UInt64(66102), outgoingBlindedPaths) + .typecase(UInt64(66097), invoiceFeatures) // should be removed when removing support for legacy trampoline payments + .typecase(UInt64(66098), legacyOutgoingNodeId) // should be removed when removing support for legacy trampoline payments + .typecase(UInt64(66099), invoiceRoutingInfo) // trampoline payments to non-trampoline Bolt 11 recipients: should not be removed + .typecase(UInt64(66100), legacyTrampolineOnion) // should be removed when removing support for legacy trampoline payments + .typecase(UInt64(66102), outgoingBlindedPaths) // should be removed when removing support for legacy trampoline payments .typecase(UInt64(181324718L), asyncPayment) .typecase(UInt64(5482373484L), keySend) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala index 433d97a3ff..046cf3a120 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala @@ -107,6 +107,7 @@ object TestConstants { Features.VariableLengthOnion -> FeatureSupport.Mandatory, Features.PaymentSecret -> FeatureSupport.Mandatory, Features.BasicMultiPartPayment -> FeatureSupport.Optional, + Features.TrampolinePayment -> FeatureSupport.Optional, Features.Wumbo -> FeatureSupport.Optional, Features.PaymentMetadata -> FeatureSupport.Optional, Features.RouteBlinding -> FeatureSupport.Optional, @@ -241,7 +242,6 @@ object TestConstants { socksProxy_opt = None, maxPaymentAttempts = 5, paymentFinalExpiry = PaymentFinalExpiryConf(CltvExpiryDelta(1), CltvExpiryDelta(1)), - enableTrampolinePayment = true, instanceId = UUID.fromString("aaaaaaaa-aaaa-aaaa-aaaa-aaaaaaaaaaaa"), balanceCheckInterval = 1 hour, blockchainWatchdogThreshold = 6, @@ -301,6 +301,7 @@ object TestConstants { Features.VariableLengthOnion -> FeatureSupport.Mandatory, Features.PaymentSecret -> FeatureSupport.Mandatory, Features.BasicMultiPartPayment -> FeatureSupport.Optional, + Features.TrampolinePayment -> FeatureSupport.Optional, Features.Wumbo -> FeatureSupport.Optional, Features.PaymentMetadata -> FeatureSupport.Optional, Features.RouteBlinding -> FeatureSupport.Optional, @@ -433,7 +434,6 @@ object TestConstants { socksProxy_opt = None, maxPaymentAttempts = 5, paymentFinalExpiry = PaymentFinalExpiryConf(CltvExpiryDelta(1), CltvExpiryDelta(1)), - enableTrampolinePayment = true, instanceId = UUID.fromString("bbbbbbbb-bbbb-bbbb-bbbb-bbbbbbbbbbbb"), balanceCheckInterval = 1 hour, blockchainWatchdogThreshold = 6, diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala index 6ff332ba06..cf20a9429a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala @@ -64,13 +64,13 @@ import scala.jdk.CollectionConverters._ class PaymentIntegrationSpec extends IntegrationSpec { test("start eclair nodes") { - instantiateEclairNode("A", ConfigFactory.parseMap(Map("eclair.node-alias" -> "A", "eclair.channel.expiry-delta-blocks" -> 130, "eclair.server.port" -> 29730, "eclair.api.port" -> 28080, "eclair.channel.channel-flags.announce-channel" -> false).asJava).withFallback(withStaticRemoteKey).withFallback(commonConfig)) // A's channels are private - instantiateEclairNode("B", ConfigFactory.parseMap(Map("eclair.node-alias" -> "B", "eclair.channel.expiry-delta-blocks" -> 131, "eclair.server.port" -> 29731, "eclair.api.port" -> 28081, "eclair.trampoline-payments-enable" -> true, "eclair.onion-messages.relay-policy" -> "relay-all").asJava).withFallback(withStaticRemoteKey).withFallback(commonConfig)) - instantiateEclairNode("C", ConfigFactory.parseMap(Map("eclair.node-alias" -> "C", "eclair.channel.expiry-delta-blocks" -> 132, "eclair.server.port" -> 29732, "eclair.api.port" -> 28082, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(withDualFunding).withFallback(commonConfig)) - instantiateEclairNode("D", ConfigFactory.parseMap(Map("eclair.node-alias" -> "D", "eclair.channel.expiry-delta-blocks" -> 133, "eclair.server.port" -> 29733, "eclair.api.port" -> 28083, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(withStaticRemoteKey).withFallback(commonConfig)) + instantiateEclairNode("A", ConfigFactory.parseMap(Map("eclair.node-alias" -> "A", "eclair.channel.expiry-delta-blocks" -> 130, "eclair.server.port" -> 29730, "eclair.api.port" -> 28080, "eclair.channel.channel-flags.announce-channel" -> false, "eclair.features.trampoline_routing" -> "disabled").asJava).withFallback(withStaticRemoteKey).withFallback(commonConfig)) // A's channels are private + instantiateEclairNode("B", ConfigFactory.parseMap(Map("eclair.node-alias" -> "B", "eclair.channel.expiry-delta-blocks" -> 131, "eclair.server.port" -> 29731, "eclair.api.port" -> 28081, "eclair.onion-messages.relay-policy" -> "relay-all").asJava).withFallback(withStaticRemoteKey).withFallback(commonConfig)) + instantiateEclairNode("C", ConfigFactory.parseMap(Map("eclair.node-alias" -> "C", "eclair.channel.expiry-delta-blocks" -> 132, "eclair.server.port" -> 29732, "eclair.api.port" -> 28082).asJava).withFallback(withDualFunding).withFallback(commonConfig)) + instantiateEclairNode("D", ConfigFactory.parseMap(Map("eclair.node-alias" -> "D", "eclair.channel.expiry-delta-blocks" -> 133, "eclair.server.port" -> 29733, "eclair.api.port" -> 28083).asJava).withFallback(withStaticRemoteKey).withFallback(commonConfig)) instantiateEclairNode("E", ConfigFactory.parseMap(Map("eclair.node-alias" -> "E", "eclair.channel.expiry-delta-blocks" -> 134, "eclair.server.port" -> 29734, "eclair.api.port" -> 28084).asJava).withFallback(withDualFunding).withFallback(commonConfig)) - instantiateEclairNode("F", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F", "eclair.channel.expiry-delta-blocks" -> 135, "eclair.server.port" -> 29735, "eclair.api.port" -> 28085, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(commonConfig)) - instantiateEclairNode("G", ConfigFactory.parseMap(Map("eclair.node-alias" -> "G", "eclair.channel.expiry-delta-blocks" -> 136, "eclair.server.port" -> 29736, "eclair.api.port" -> 28086, "eclair.relay.fees.public-channels.fee-base-msat" -> 1010, "eclair.relay.fees.public-channels.fee-proportional-millionths" -> 102, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(commonConfig)) + instantiateEclairNode("F", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F", "eclair.channel.expiry-delta-blocks" -> 135, "eclair.server.port" -> 29735, "eclair.api.port" -> 28085).asJava).withFallback(commonConfig)) + instantiateEclairNode("G", ConfigFactory.parseMap(Map("eclair.node-alias" -> "G", "eclair.channel.expiry-delta-blocks" -> 136, "eclair.server.port" -> 29736, "eclair.api.port" -> 28086, "eclair.relay.fees.public-channels.fee-base-msat" -> 1010, "eclair.relay.fees.public-channels.fee-proportional-millionths" -> 102).asJava).withFallback(commonConfig)) } test("connect nodes") { @@ -480,7 +480,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { sender.send(nodes("F").paymentHandler, ReceiveStandardPayment(sender.ref, Some(amount), Left("like trampoline much?"))) val invoice = sender.expectMsgType[Bolt11Invoice] assert(invoice.features.hasFeature(Features.BasicMultiPartPayment)) - assert(invoice.features.hasFeature(Features.TrampolinePaymentPrototype)) + assert(invoice.features.hasFeature(Features.TrampolinePayment)) // The best route from G is G -> C -> F. val payment = SendTrampolinePayment(sender.ref, invoice, nodes("G").nodeParams.nodeId, routeParams = integrationTestRouteParams) @@ -517,7 +517,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { sender.send(nodes("B").paymentHandler, ReceiveStandardPayment(sender.ref, Some(amount), Left("trampoline-MPP is so #reckless"))) val invoice = sender.expectMsgType[Bolt11Invoice] assert(invoice.features.hasFeature(Features.BasicMultiPartPayment)) - assert(invoice.features.hasFeature(Features.TrampolinePaymentPrototype)) + assert(invoice.features.hasFeature(Features.TrampolinePayment)) assert(invoice.paymentMetadata.nonEmpty) // The direct route C -> B does not have enough capacity, the payment will be split between C -> B and C -> G -> B @@ -556,7 +556,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { sender.send(nodes("A").paymentHandler, ReceiveStandardPayment(sender.ref, Some(amount), Left("trampoline to non-trampoline is so #vintage"), extraHops = routingHints)) val invoice = sender.expectMsgType[Bolt11Invoice] assert(invoice.features.hasFeature(Features.BasicMultiPartPayment)) - assert(!invoice.features.hasFeature(Features.TrampolinePaymentPrototype)) + assert(!invoice.features.hasFeature(Features.TrampolinePayment)) assert(invoice.paymentMetadata.nonEmpty) val payment = SendTrampolinePayment(sender.ref, invoice, nodes("C").nodeParams.nodeId, routeParams = integrationTestRouteParams) @@ -596,7 +596,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { sender.send(nodes("D").paymentHandler, ReceiveStandardPayment(sender.ref, Some(amount), Left("I iz Satoshi"))) val invoice = sender.expectMsgType[Bolt11Invoice] assert(invoice.features.hasFeature(Features.BasicMultiPartPayment)) - assert(invoice.features.hasFeature(Features.TrampolinePaymentPrototype)) + assert(invoice.features.hasFeature(Features.TrampolinePayment)) nodes("B").system.eventStream.subscribe(holdTimesRecorder.ref, classOf[Router.ReportedHoldTimes]) val payment = SendTrampolinePayment(sender.ref, invoice, nodes("C").nodeParams.nodeId, routeParams = integrationTestRouteParams) @@ -618,7 +618,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { sender.send(nodes("D").paymentHandler, ReceiveStandardPayment(sender.ref, Some(amount), Left("I iz not Satoshi"))) val invoice = sender.expectMsgType[Bolt11Invoice] assert(invoice.features.hasFeature(Features.BasicMultiPartPayment)) - assert(invoice.features.hasFeature(Features.TrampolinePaymentPrototype)) + assert(invoice.features.hasFeature(Features.TrampolinePayment)) val payment = SendTrampolinePayment(sender.ref, invoice, nodes("B").nodeParams.nodeId, routeParams = integrationTestRouteParams) sender.send(nodes("A").paymentInitiator, payment) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt11InvoiceSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt11InvoiceSpec.scala index dc185b4dca..6d40292a2d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt11InvoiceSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt11InvoiceSpec.scala @@ -327,7 +327,7 @@ class Bolt11InvoiceSpec extends AnyFunSuite { assert(features2bits(invoice.features) == bin"1000000000000000000000000000000000000000000000000000000000000000000000000000000000000100000100000000") assert(!invoice.features.hasFeature(BasicMultiPartPayment)) assert(invoice.features.hasFeature(PaymentSecret, Some(Mandatory))) - assert(!invoice.features.hasFeature(TrampolinePaymentPrototype)) + assert(!invoice.features.hasFeature(TrampolinePayment)) assert(TestConstants.Alice.nodeParams.features.invoiceFeatures().areSupported(invoice.features)) assert(invoice.sign(priv).toString == ref.toLowerCase) } @@ -347,7 +347,7 @@ class Bolt11InvoiceSpec extends AnyFunSuite { assert(features2bits(invoice.features) == bin"000011000000000000000000000000000000000000000000000000000000000000000000000000000000000000100000100000000") assert(!invoice.features.hasFeature(BasicMultiPartPayment)) assert(invoice.features.hasFeature(PaymentSecret, Some(Mandatory))) - assert(!invoice.features.hasFeature(TrampolinePaymentPrototype)) + assert(!invoice.features.hasFeature(TrampolinePayment)) assert(!TestConstants.Alice.nodeParams.features.invoiceFeatures().areSupported(invoice.features)) assert(invoice.sign(priv).toString == ref) } @@ -603,19 +603,15 @@ class Bolt11InvoiceSpec extends AnyFunSuite { } test("trampoline") { - val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(123 msat), ByteVector32.One, priv, Left("Some invoice"), CltvExpiryDelta(18)) - assert(!invoice.features.hasFeature(TrampolinePaymentPrototype)) - - val pr1 = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(123 msat), ByteVector32.One, priv, Left("Some invoice"), CltvExpiryDelta(18), features = Features(VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, TrampolinePaymentPrototype -> Optional)) - assert(!pr1.features.hasFeature(BasicMultiPartPayment)) - assert(pr1.features.hasFeature(TrampolinePaymentPrototype)) - - val pr2 = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(123 msat), ByteVector32.One, priv, Left("Some invoice"), CltvExpiryDelta(18), features = Features(VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, TrampolinePaymentPrototype -> Optional)) - assert(pr2.features.hasFeature(BasicMultiPartPayment)) - assert(pr2.features.hasFeature(TrampolinePaymentPrototype)) + val invoice1 = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(123 msat), ByteVector32.One, priv, Left("Some invoice"), CltvExpiryDelta(18)) + assert(!invoice1.features.hasFeature(TrampolinePayment)) + + val invoice2 = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(123 msat), ByteVector32.One, priv, Left("Some invoice"), CltvExpiryDelta(18), features = Features(VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, TrampolinePayment -> Optional)) + assert(invoice2.features.hasFeature(BasicMultiPartPayment)) + assert(invoice2.features.hasFeature(TrampolinePayment)) - val Success(pr3) = Bolt11Invoice.fromString("lnbc40n1pw9qjvwsp5q56txjvpzwz5lkd4atjpc894l8ppc0eeylfnelaytcgfaxjy76tspp5qq3w2ln6krepcslqszkrsfzwy49y0407hvks30ec6pu9s07jur3sdpstfshq5n9v9jzucm0d5s8vmm5v5s8qmmnwssyj3p6yqenwdencqzysxqrrss7ju0s4dwx6w8a95a9p2xc5vudl09gjl0w2n02sjrvffde632nxwh2l4w35nqepj4j5njhh4z65wyfc724yj6dn9wajvajfn5j7em6wsqakczwg") - assert(!pr3.features.hasFeature(TrampolinePaymentPrototype)) + val Success(invoice3) = Bolt11Invoice.fromString("lnbc40n1pw9qjvwsp5q56txjvpzwz5lkd4atjpc894l8ppc0eeylfnelaytcgfaxjy76tspp5qq3w2ln6krepcslqszkrsfzwy49y0407hvks30ec6pu9s07jur3sdpstfshq5n9v9jzucm0d5s8vmm5v5s8qmmnwssyj3p6yqenwdencqzysxqrrss7ju0s4dwx6w8a95a9p2xc5vudl09gjl0w2n02sjrvffde632nxwh2l4w35nqepj4j5njhh4z65wyfc724yj6dn9wajvajfn5j7em6wsqakczwg") + assert(!invoice3.features.hasFeature(TrampolinePayment)) } test("nonreg") { @@ -696,7 +692,7 @@ class Bolt11InvoiceSpec extends AnyFunSuite { val invoiceFeatures = TestConstants.Alice.nodeParams.features.bolt11Features() assert(invoiceFeatures.unknown.nonEmpty) val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(123 msat), ByteVector32.One, priv, Left("Some invoice"), CltvExpiryDelta(18), features = invoiceFeatures) - assert(invoice.features == Features(PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, PaymentMetadata -> Optional, VariableLengthOnion -> Mandatory)) + assert(invoice.features == Features(PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, PaymentMetadata -> Optional, VariableLengthOnion -> Mandatory, TrampolinePayment -> Optional)) assert(Bolt11Invoice.fromString(invoice.toString).get == invoice) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala index 14fe0755d8..d48b64e644 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala @@ -235,32 +235,25 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike val sender = TestProbe() { - val handler = TestActorRef[PaymentHandler](PaymentHandler.props(Alice.nodeParams.copy(enableTrampolinePayment = false, features = featuresWithoutMpp), TestProbe().ref, TestProbe().ref)) + val handler = TestActorRef[PaymentHandler](PaymentHandler.props(Alice.nodeParams.copy(features = featuresWithoutMpp), TestProbe().ref, TestProbe().ref)) sender.send(handler, ReceiveStandardPayment(sender.ref, Some(42 msat), Left("1 coffee"))) val invoice = sender.expectMsgType[Bolt11Invoice] assert(!invoice.features.hasFeature(BasicMultiPartPayment)) - assert(!invoice.features.hasFeature(TrampolinePaymentPrototype)) + assert(!invoice.features.hasFeature(TrampolinePayment)) } { - val handler = TestActorRef[PaymentHandler](PaymentHandler.props(Alice.nodeParams.copy(enableTrampolinePayment = false, features = featuresWithMpp), TestProbe().ref, TestProbe().ref)) + val handler = TestActorRef[PaymentHandler](PaymentHandler.props(Alice.nodeParams.copy(features = featuresWithMpp), TestProbe().ref, TestProbe().ref)) sender.send(handler, ReceiveStandardPayment(sender.ref, Some(42 msat), Left("1 coffee"))) val invoice = sender.expectMsgType[Bolt11Invoice] assert(invoice.features.hasFeature(BasicMultiPartPayment)) - assert(!invoice.features.hasFeature(TrampolinePaymentPrototype)) + assert(!invoice.features.hasFeature(TrampolinePayment)) } { - val handler = TestActorRef[PaymentHandler](PaymentHandler.props(Alice.nodeParams.copy(enableTrampolinePayment = true, features = featuresWithoutMpp), TestProbe().ref, TestProbe().ref)) - sender.send(handler, ReceiveStandardPayment(sender.ref, Some(42 msat), Left("1 coffee"))) - val invoice = sender.expectMsgType[Bolt11Invoice] - assert(!invoice.features.hasFeature(BasicMultiPartPayment)) - assert(invoice.features.hasFeature(TrampolinePaymentPrototype)) - } - { - val handler = TestActorRef[PaymentHandler](PaymentHandler.props(Alice.nodeParams.copy(enableTrampolinePayment = true, features = featuresWithMpp), TestProbe().ref, TestProbe().ref)) + val handler = TestActorRef[PaymentHandler](PaymentHandler.props(Alice.nodeParams.copy(features = featuresWithMpp.add(TrampolinePayment, Optional)), TestProbe().ref, TestProbe().ref)) sender.send(handler, ReceiveStandardPayment(sender.ref, Some(42 msat), Left("1 coffee"))) val invoice = sender.expectMsgType[Bolt11Invoice] assert(invoice.features.hasFeature(BasicMultiPartPayment)) - assert(invoice.features.hasFeature(TrampolinePaymentPrototype)) + assert(invoice.features.hasFeature(TrampolinePayment)) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala index d81b19a2da..a5d6f38b0a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentInitiatorSpec.scala @@ -74,7 +74,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, - TrampolinePaymentPrototype -> Optional, + Features.TrampolinePayment -> Optional, ) case class FakePaymentFactory(payFsm: TestProbe, trampolinePayFsm: TestProbe, multiPartPayFsm: TestProbe) extends PaymentInitiator.MultiPartPaymentFactory { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala index f216d20b59..f9da074e56 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala @@ -17,6 +17,7 @@ package fr.acinq.eclair.payment import akka.actor.ActorRef +import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.scalacompat.DeterministicWallet.ExtendedPrivateKey import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, ByteVector64, Crypto, DeterministicWallet, OutPoint, Satoshi, SatoshiLong, Transaction, TxOut} import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional} @@ -39,7 +40,7 @@ import fr.acinq.eclair.transactions.Transactions.DefaultCommitmentFormat import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, Offer, PaymentInfo} import fr.acinq.eclair.wire.protocol.PaymentOnion.{FinalPayload, IntermediatePayload, OutgoingBlindedPerHopPayload} import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{BlockHeight, Bolt11Feature, Bolt12Feature, CltvExpiry, CltvExpiryDelta, EncodedNodeId, Features, MilliSatoshi, MilliSatoshiLong, ShortChannelId, TestConstants, TimestampMilli, TimestampSecondLong, UInt64, nodeFee, randomBytes32, randomKey} +import fr.acinq.eclair.{BlockHeight, Bolt11Feature, Bolt12Feature, CltvExpiry, CltvExpiryDelta, EncodedNodeId, FeatureSupport, Features, InvoiceFeature, MilliSatoshi, MilliSatoshiLong, ShortChannelId, TestConstants, TimestampMilli, TimestampSecondLong, UInt64, UnknownFeature, nodeFee, randomBytes32, randomKey} import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite import scodec.bits.{ByteVector, HexStringSyntax} @@ -317,7 +318,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { // .----. // / \ // b -> c e - val invoiceFeatures = Features[Bolt11Feature](VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, PaymentMetadata -> Optional, TrampolinePaymentPrototype -> Optional) + val invoiceFeatures = Features[Bolt11Feature](VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, PaymentMetadata -> Optional, Features.TrampolinePayment -> Optional) val invoice = Bolt11Invoice(Block.RegtestGenesisBlock.hash, Some(finalAmount), paymentHash, priv_e.privateKey, Left("invoice"), CltvExpiryDelta(6), paymentSecret = paymentSecret, features = invoiceFeatures, paymentMetadata = Some(hex"010203")) val payment = TrampolinePayment.buildOutgoingPayment(c, invoice, finalExpiry) testRelayTrampolinePayment(invoice, payment) @@ -328,13 +329,168 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { // .----. // / \ // b -> c e - val invoiceFeatures = Features[Bolt11Feature](VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, PaymentMetadata -> Optional, TrampolinePaymentPrototype -> Optional) + val invoiceFeatures = Features[Bolt11Feature](VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, PaymentMetadata -> Optional, Features.TrampolinePayment -> Optional) val invoice = Bolt11Invoice(Block.RegtestGenesisBlock.hash, Some(finalAmount), paymentHash, priv_e.privateKey, Left("invoice"), CltvExpiryDelta(6), paymentSecret = paymentSecret, features = invoiceFeatures, paymentMetadata = Some(hex"010203")) // Note that we don't include a payment_secret for the trampoline node, since the spec allows omitting it when not using MPP to reach the trampoline node. val payment = TrampolinePayment.buildOutgoingPayment(c, invoice, finalAmount, finalExpiry, trampolinePaymentSecret_opt = None, attemptNumber = 0) testRelayTrampolinePayment(invoice, payment) } + test("build legacy outgoing trampoline payment") { + // simple trampoline route to e between legacy wallets: + // .----. + // / \ + // b -> c e + val invoiceFeatures = Features[Bolt11Feature](VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, PaymentMetadata -> Optional, Features.TrampolinePayment -> Optional) + val invoice = Bolt11Invoice(Block.RegtestGenesisBlock.hash, Some(finalAmount), paymentHash, priv_e.privateKey, Left("invoice"), CltvExpiryDelta(6), paymentSecret = paymentSecret, features = invoiceFeatures, paymentMetadata = Some(hex"010203")) + val trampolineOnion = { + val finalPayload = PaymentOnion.FinalPayload.Standard.createPayload(finalAmount, finalAmount, finalExpiry, invoice.paymentSecret, invoice.paymentMetadata) + val trampolinePayload = PaymentOnion.IntermediatePayload.NodeRelay.Standard(finalAmount, finalExpiry, invoice.nodeId) + buildOnion(NodePayload(c, trampolinePayload) :: NodePayload(invoice.nodeId, finalPayload) :: Nil, invoice.paymentHash, None).toOption.get + } + val payment = { + val trampolineAmount = finalAmount * 1.005 // 0.5% fees + val trampolineExpiry = finalExpiry + CltvExpiryDelta(72) + val payload = PaymentOnion.FinalPayload.Standard.createLegacyTrampolinePayload(trampolineAmount, trampolineAmount, trampolineExpiry, randomBytes32(), trampolineOnion.packet) + val paymentOnion = buildOnion(NodePayload(c, payload) :: Nil, invoice.paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get + TrampolinePayment.OutgoingPayment(trampolineAmount, trampolineExpiry, paymentOnion, trampolineOnion) + } + + val add_c = UpdateAddHtlc(randomBytes32(), 2, payment.trampolineAmount, paymentHash, payment.trampolineExpiry, payment.onion.packet, None, Reputation.maxEndorsement, None) + val Right(RelayToTrampolinePacket(add_c2, outer_c, inner_c, trampolinePacket_e, _)) = decrypt(add_c, priv_c.privateKey, Features.empty) + assert(add_c2 == add_c) + assert(outer_c.amount == payment.trampolineAmount) + assert(outer_c.totalAmount == payment.trampolineAmount) + assert(outer_c.expiry == payment.trampolineExpiry) + assert(outer_c.records.get[OnionPaymentPayloadTlv.LegacyTrampolineOnion].nonEmpty) + assert(inner_c.amountToForward == finalAmount) + assert(inner_c.outgoingCltv == finalExpiry) + assert(inner_c.outgoingNodeId == e) + + // c forwards the trampoline payment to e through d using a legacy trampoline onion. + val recipient_e = ClearRecipient(e, Features(Map.empty[InvoiceFeature, FeatureSupport], Set(UnknownFeature(149))), inner_c.amountToForward, inner_c.outgoingCltv, randomBytes32(), nextTrampolineOnion_opt = Some(trampolinePacket_e)) + val Right(payment_e) = buildOutgoingPayment(Origin.Hot(ActorRef.noSender, Upstream.Hot.Trampoline(List(Upstream.Hot.Channel(add_c, TimestampMilli(1687345927000L), b, 0.1)))), paymentHash, Route(inner_c.amountToForward, afterTrampolineChannelHops, None), recipient_e, Reputation.Score.max) + assert(payment_e.outgoingChannel == channelUpdate_cd.shortChannelId) + assert(payment_e.cmd.amount == amount_cd) + assert(payment_e.cmd.cltvExpiry == expiry_cd) + val add_d = UpdateAddHtlc(randomBytes32(), 3, payment_e.cmd.amount, paymentHash, payment_e.cmd.cltvExpiry, payment_e.cmd.onion, None, Reputation.maxEndorsement, None) + val Right(ChannelRelayPacket(add_d2, payload_d, packet_e, _)) = decrypt(add_d, priv_d.privateKey, Features.empty) + assert(add_d2 == add_d) + assert(payload_d == IntermediatePayload.ChannelRelay.Standard(channelUpdate_de.shortChannelId, amount_de, expiry_de)) + + val add_e = UpdateAddHtlc(randomBytes32(), 4, amount_de, paymentHash, expiry_de, packet_e, None, Reputation.maxEndorsement, None) + val Right(FinalPacket(add_e2, payload_e, _)) = decrypt(add_e, priv_e.privateKey, Features.empty) + assert(add_e2 == add_e) + assert(payload_e.isInstanceOf[FinalPayload.Standard]) + assert(payload_e.amount == finalAmount) + assert(payload_e.expiry == finalExpiry) + assert(payload_e.asInstanceOf[FinalPayload.Standard].paymentSecret == paymentSecret) + assert(payload_e.asInstanceOf[FinalPayload.Standard].paymentMetadata.contains(hex"010203")) + assert(payload_e.asInstanceOf[FinalPayload.Standard].isTrampoline) + } + + // See bolt04/trampoline-payment-onion-test.json + test("build outgoing trampoline payment (reference test vector)") { + // .-> Dave -. + // / \ + // Alice -> Bob -> Carol Eve + val paymentHash = ByteVector32.fromValidHex("4242424242424242424242424242424242424242424242424242424242424242") + + // Alice creates a trampoline onion Carol -> Eve. + val trampolineOnionForCarol = { + val paymentSecret = ByteVector32.fromValidHex("2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a") + val carol = PublicKey(hex"027f31ebc5462c1fdce1b737ecff52d37d75dea43ce11c74d25aa297165faa2007") + val eve = PublicKey(hex"02edabbd16b41c8371b92ef2f04c1185b4f03b6dcd52ba9b78d9d7c89c8f221145") + val payloads = Seq( + NodePayload(carol, PaymentOnion.IntermediatePayload.NodeRelay.Standard(100_000_000 msat, CltvExpiry(800_000), eve)), + NodePayload(eve, PaymentOnion.FinalPayload.Standard.createPayload(100_000_000 msat, 100_000_000 msat, CltvExpiry(800_000), paymentSecret)), + ) + val sessionKey = PrivateKey(hex"0303030303030303030303030303030303030303030303030303030303030303") + val trampolineOnionForCarol = OutgoingPaymentPacket.buildOnion(sessionKey, payloads, paymentHash, packetPayloadLength_opt = None).toOption.get.packet + val encoded = OnionRoutingCodecs.onionRoutingPacketCodec(trampolineOnionForCarol.payload.length.toInt).encode(trampolineOnionForCarol).require.bytes + assert(encoded == hex"0002531fe6068134503d2723133227c867ac8fa6c83c537e9a44c3c5bdbdcb1fe3371860c0749bfd613056cfc5718beecc25a2f255fc7abbea3cd75ff820e9d30807d19b30f33626452fa54bb2d822e918558ed3e6714deb3f9a2a10895e7553c6f088c9a852043530dbc9abcc486030894364b205f5de60171b451ff462664ebce23b672579bf2a444ebfe0a81875c26d2fa16d426795b9b02ccbc4bdf909c583f0c2ebe9136510645917153ecb05181ca0c1b207824578ee841804a148f4c3df7306dcea52d94222907c9187bc31c0880fc084f0d88716e195c0abe7672d15217623") + trampolineOnionForCarol + } + + // Alice wraps it into a payment onion Alice -> Bob -> Carol. + val onionForBob = { + val paymentSecret = ByteVector32.fromValidHex("2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b") + val bob = PublicKey(hex"0324653eac434488002cc06bbfb7f10fe18991e35f9fe4302dbea6d2353dc0ab1c") + val carol = PublicKey(hex"027f31ebc5462c1fdce1b737ecff52d37d75dea43ce11c74d25aa297165faa2007") + val payloads = Seq( + NodePayload(bob, PaymentOnion.IntermediatePayload.ChannelRelay.Standard(ShortChannelId.fromCoordinates("572330x7x1105").get, 100_005_000 msat, CltvExpiry(800_250))), + NodePayload(carol, PaymentOnion.FinalPayload.Standard.createTrampolinePayload(100_005_000 msat, 100_005_000 msat, CltvExpiry(800_250), paymentSecret, trampolineOnionForCarol)), + ) + val sessionKey = PrivateKey(hex"0404040404040404040404040404040404040404040404040404040404040404") + val onionForBob = OutgoingPaymentPacket.buildOnion(sessionKey, payloads, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get.packet + val encoded = PaymentOnionCodecs.paymentOnionPacketCodec.encode(onionForBob).require.bytes + assert(encoded == hex"0003462779ad4aad39514614751a71085f2f10e1c7a593e4e030efb5b8721ce55b0b9149ce01cce1709194109ab594037113e897ab6120025c770527dd8537997e2528082b984fe078a5667978a573abeaf7977d9b8b6ee4f124d3352f7eea52cc66c0e76b8f6d7a25d4501a04ae190b17baff8e6378b36f165815f714559dfef275278eba897f5f229be70fc8a1980cf859d1c25fe90c77f006419770e19d29ba80be8f613d039dd05600734e0d1e218af441fe30877e717a26b7b37c2c071d62bf6d61dd17f7abfb81546d2c722c9a6dc581aa97fb6f3b513e5fbaf0d669fbf0714b2b016a0a8e356d55f267fa144f7501792f2a59269c5a22e555a914e2eb71eba5af519564f246cf58983ea3fa2674e3ab7d9969d8dffbb2bda2b2752657417937d46601eb8ebf1837221d4bdf55a4d6a97ecffde5a09bd409717fa19e440e55d775890aed89f72e65af515757e94a9b501e6bad048af55e1583adb2960a84f60fb5efd0352e77a34045fc6b221498f62810bd8294d995d9f513696f8633f29faaa9668d0c6fa0d0dd7fa13e2c185572485762bd2810dc12187f521fbefa9c320762ac1e107f7988d81c6ee201ab68a95d45d578027e271b6526154317877037dca17134ccd955a22a8481b8e1996d896fc4bf006154ed18ef279d4f255e3f1233d037aea2560011069a0ae56d6bfdd8327054ded12d85d120b8982bff970986db333baae7c95f85677726a8f74cc8bd1e5aca3d433c113048305ecce8e35caf0485a53df00284b52b42291a9ffe686b96442135b3107d8856bc652d674ee9a148e79b02c9972d3ca8c2b02609f3b358c4a67c540ba6769c4d83169bceda640b1d18b74d12b6df605b417dacf6f82d79d43bb40920898f818dc8344c036ae9c8bbf9ef52ea1ccf225c8825a4d8503df190b999e15a4be34c9d7bbf60d3b93bb7d6559f4a5916f5e40c3defeeca9337ccd1280e46d6727c5c91c2d898b685543d4ca7cfee23981323c43260b6387e7febb0fffb200a8c974ef36b3253d0fb9fe0c1c6017f2dbbdc169f3f061d9781521e8118164aeec31c3e59c199016f1025c295d8f7bdeb627d357105a2708c4c3a856b9e83ff37ed69f59f2d2e464ed1db5882925ebe2493a7ddb707e1a308fa445172a24b3ea60732f75f5c69b41fc11467ee93f37c9a6f7285ba42f716e2a0e30909056ea3e4f7985d14ca9ab280cc184ce98e2a0722d0447aa1a2eedc5e53ddfa53731df7eced406b10627b0bebd768a30bde0d470c0f1d10adc070f8d3029cacceec74e4833f4dc8c52c3f41733f5f896fceb425d0737e717a63bfb033df46286d99594dd01e2bd0a942ab792874177b32842f4833bc0340ddb74852e9cd6f29f1d997a4a4bf05dd5d12011f95e6ce18928e3a9b83b24d15f989bdf43370bcc657c3ac6601eaf5e951efdbd7ee69b1623dc5039b2dfc640692378ef032f17bc36cc00293ad90b7e18f5feb8f287a7061ed9713929aed9b14b8d566199fc7822b1c38daa16b6d83077b10af0e2b6e531ccc34ea248ea593128c9ff17defcee6618c29cd2d93cfed99b90319104b1fdcfea91e98b41d792782840fb7b25280d8565b0bcd874e79b1b323139e7fc88eb5f80f690ce30fcd81111076adb31de6aeced879b538c0b5f2b74c027cc582a540133952cb021424510312f13e15d403f700f3e15b41d677c5a1e7c4e692c5880cb4522c48e993381996a29615d2956781509cd74aec6a3c73b8536d1817e473dad4cbb1787e046606b692a44e5d21ef6b5219658b002f674367e90a2b610924e9ac543362257d4567728f2e61f61231cb5d7816e100bb6f6bd9a42329b728b18d7a696711650c16fd476e2f471f38af0f6b00d45c6e1fa492cc7962814953ab6ad1ce3d3f3dc950e64d18a8fdce6aabc14321576f06") + onionForBob + } + + // Bob decrypts the payment onion and forwards it to Carol. + val onionForCarol = { + val priv = PrivateKey(hex"4242424242424242424242424242424242424242424242424242424242424242") + val add = UpdateAddHtlc(randomBytes32(), 1, 100_006_000 msat, paymentHash, CltvExpiry(800_300), onionForBob, None, Reputation.maxEndorsement, None) + val Right(ChannelRelayPacket(_, _, onionForCarol, _)) = decrypt(add, priv, Features.empty) + val encoded = PaymentOnionCodecs.paymentOnionPacketCodec.encode(onionForCarol).require.bytes + assert(encoded == hex"00036d7fa1507cac3a1fda3308b465b71d817a2ee8dfde598c6bdb1dec73c7acf0165136fdd0fd9f3f7eac074f42b015825614214ac3b7ec95234538c9cfd04fc1a5128fa47c8d56e21e51bb843da8252c0abafb72395cf6ca8a186bd1de72341cb0f988e79988c39e4d444a4495120ccf3577576177a45c2a0fdc88776291d3af9e62d700c06206c769260859715ba5e1e7c0dc5f97dbf80decb564c885d0d6f0e10bddb225ee3d82a1e02b6a3735ea81ab91dada382a5752a940814e38c709e62d3427d69bfd09a19955c507aea300bf10578e3bda3d632a5de159f3fc0ff9311b2fc5d4a6c03582c4cd85c92d29bc285971f1019cb468942a7d3706e096f6ab105e7d8d525586a4f7987135af70d166317dc2b5b6c58345c54e87615d277e7ade5f0b9f8baed5f16e1b340492c4fa6b443f94544a4f083e4dfb778badf1084c0c39e998cd67ff5f1a6526fb163cfd48e04ff34d928a91f061781463b9f668a0d084e6c5bb80413968ee3185abd545b38f63f496d9fa16e67d84c08414df8c03d0efb1925edcdd14a4134424f65372166be4a8e66906a428eb726ae43ea6cf81256f082382e18b765e78cd21819045b5bcc5f4464b812215f8838acf73c5a4748a09ee10b6bcec9c201dc38ef009b23b9072d653c81316a59b36533732f4c4eeb29863bcf420155aa90378a111f0393599fb9dd42f69808c3552654b7352a6a1e2a71db0a0214c8d9021ef52d667da4d351a9a44a0cdbff34894d1994e7cced665061b6979f9e508d98ac9b2193f01694597e8189122daf0bd3c82743f5994678b4efb309028be23987bc18720388bc78be39b02276a0f3577390e36a5cd0dbab97b08a5c7f45a5a952681a2669e653004977b2a1e098a5bfee2ee927c2f51fc9dc66af120b5a40b01738c5db1e091f7141096e3c4d5905a695f02c852fd40412c7288c15befb522eec41232899863c17f66cbfefb3597c346fc7483a03d0f3f2dcaa6ae56d508d0df9298d80b2bcfcb91b30b298ca2415a3cbc8284bb2f4a5cfc244efe2d78a446d36d350bebd7ff30d70a2015679f1a3a63d841e2333fa30ebabf9d84576616f3c93a78a42948d991e1c628c3dbb3ad856fe97f9a3ce3d2b7e8e3ff2e1c3b2eb494dd9c947878120a8912afda70ca7d7829b9011f13c848d10e69274f4bd918c4c5531c8382e5f1c0b72ecabbd34d14190cde1a3247e4473c8016a122077f4a9cddf21c11680c2c25c342dacc7676304dd2466b47a172641e33de3cf9c2f476f57e0a90cdb7f8398dc012fd65df9a685a73b8f6f02a2ba3045e0cb308a72645370c827ac43da67f614e2d68b7811805b8144e6f21e94b679003486aa79bad22db09735d72e5a32c5831c3e44c9100322ae70c74df52ba98653624361b62d9500b704450e6e23d3373aae9697fed5e6133d1d1677608be513344590fd72569c6e19c070d303e8aa6f7196e7ac0f4039912cf1e9f050c6927340f9a96de229adbe7906072bc87c2214dc476d8dc7d81f2cb56d5a7407fe9fb378703f04fe19f1bb4b8f938c84072a4ac0b18de581b4b8b5971ce411cc82a0484764a6df49f8ffc3c858a299b4ffc9f96b933bd12f4fc876b6ce34f7c022ded91d51a03c5f14c29a9f7b28e45395782d74a3d795ac596b44ba36f805d62e3ba7976f10904784af7f5994cc57817979a0adf87e3b3e32047f0a4d68c2609c9405612b264094c49dd27836f4bdab4d68256b2b4d8e10411ff166065265fdd0c04c6c3ad989530f258b9549128765f0cc6af5e50cf15d3fd856e91580bf66a7ebce267726aee798b580df6deaee59fa90c5a35e06f36d4960c326d0418adcbe3ff4248bf04dc24a3758de2c58f97fd9e4333beae43428d184e3872ad52d2b4dd4d770da0dca339bf70a6b22dd05cf8547ec0a7a8d49543") + onionForCarol + } + + // Carol decrypts the payment onion and the inner trampoline onion. + val trampolineOnionForEve = { + val priv = PrivateKey(hex"4343434343434343434343434343434343434343434343434343434343434343") + val eve = PublicKey(hex"02edabbd16b41c8371b92ef2f04c1185b4f03b6dcd52ba9b78d9d7c89c8f221145") + val add = UpdateAddHtlc(randomBytes32(), 1, 100_005_000 msat, paymentHash, CltvExpiry(800_250), onionForCarol, None, Reputation.maxEndorsement, None) + val Right(RelayToTrampolinePacket(_, outer_c, inner_c, trampolineOnionForEve, _)) = decrypt(add, priv, Features.empty) + assert(outer_c.amount == 100_005_000.msat) + assert(outer_c.expiry == CltvExpiry(800_250)) + assert(inner_c.amountToForward == 100_000_000.msat) + assert(inner_c.outgoingCltv == CltvExpiry(800_000)) + assert(inner_c.outgoingNodeId == eve) + val encoded = OnionRoutingCodecs.onionRoutingPacketCodec(trampolineOnionForEve.payload.length.toInt).encode(trampolineOnionForEve).require.bytes + assert(encoded == hex"00035e5c85814fdb522b4efeef99b44fe8a5d3d3412057bc213b98d6f605edb022c2ae4a9141f6ac403790afeed975061f024e2723d485f9cb35a3eaf881732f468dc19009bf195b561590798fb895b7b7065b5537018dec330e509e8618700c9c6e1df5d15b900ac3c34104b6abb1099fd2eca3b640d7d5fda9370e20c09035168fc64d954baa80361b965314c400da2d7a64d0536bf9e494aebb80aec358327a4a1a667fcff1daf241c99dd8c4fa907de5b931fb9daed083c157f5ea1dd960d142952f8ebe4e1ccaee4d565a093e2b91f94b04a884ce2e8c60aced3565e8d2d10de5") + trampolineOnionForEve + } + + // Carol wraps the trampoline onion for Eve into a payment Carol -> Dave -> Eve. + val onionForDave = { + val paymentSecret = ByteVector32.fromValidHex("2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c") + val dave = PublicKey(hex"032c0b7cf95324a07d05398b240174dc0c2be444d96b159aa6c7f7b1e668680991") + val eve = PublicKey(hex"02edabbd16b41c8371b92ef2f04c1185b4f03b6dcd52ba9b78d9d7c89c8f221145") + val payloads = Seq( + NodePayload(dave, PaymentOnion.IntermediatePayload.ChannelRelay.Standard(ShortChannelId.fromCoordinates("572330x42x1729").get, 100_000_000 msat, CltvExpiry(800_000))), + NodePayload(eve, PaymentOnion.FinalPayload.Standard.createTrampolinePayload(100_000_000 msat, 100_000_000 msat, CltvExpiry(800_000), paymentSecret, trampolineOnionForEve)), + ) + val sessionKey = PrivateKey(hex"0505050505050505050505050505050505050505050505050505050505050505") + val onionForDave = OutgoingPaymentPacket.buildOnion(sessionKey, payloads, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get.packet + val encoded = PaymentOnionCodecs.paymentOnionPacketCodec.encode(onionForDave).require.bytes + assert(encoded == hex"000362c0a046dacce86ddd0343c6d3c7c79c2208ba0d9c9cf24a6d046d21d21f90f74125c590a7877d17303ddcdce79b0b33e006eaf4eff557631c70c7ab9a61105ffd738239016e84c2639ff5246f3d2167ea7ea7932138435a9cc427f7f9d838daa2b7f4c3bfb8c44e8e48d2fd744c1c5a7626d188b5690d36900eb0a498cd0b4139424bc1b65d74409a72fca8e36f239f4c80644963e80391ca1c707f727e3dc9656de66bfdf77823b0b5746c55c31978faffd65937b2c526478e4f30d08cc371fb9d045f65316af2d416c9a82ac412db84e4386901877670c8a2fcdd1b2f3276c5384f2feb23d4c62788cce78edc1194bf4fbd2af5670d2917cc940c41897fea944ebf908a1a90a1bd208b42209ccf2d480d2590bfce320ce185f12e77703f906e98b1a9ff701490b792a60faba11d75d691c2cecf867bb63062ec8c3bd1c2665dbd380e59cdffbfd028e5c86a1371fd3d5141e50986247a9f21143df0d1099e6df7e2044f1f4a87bc759cb7c2354616e39ace2d06165a580206ae9c5bc5005a6654215e7ab1bb619eb2df5bc11e3a8cfbc0a9c7e515c0f6d9d02512ef856d4782e54192ea63a173b4fcf02a11e85d2da6de47a6f8dd9bbfb30dccecd5e2195d5c9b0bf0bfc8b571b4962deacba4669afa017294b45e2668ad87168b9589f00f56275022f049f0cdece8c9e1f0f35035aa1af4a70103a7e8aa2b7a6579accf554c6a4f305981f5732036894765e086c167f5f342f313e4617da53b79303c72e0a6f03c3f592cb9c035c509c02dc09e5ea20b158a3f47b1722db86d354f7dfccbdaf6be21c7f473e143b459b2b06a21984f29ba80dfcd52696c76fb2a11f66383e33d88226f451317125fcfa02671015c359db52ee1462b1b820588d5c874765de4e7cc83b84dde8630b2a21325116cf53fd1eb369bed1330dfcbe0633698c518a376312624d78011922621e32e9b316a9329c3d1f967069d35844e60caf53e7a2bbbe695808de2e91dc16a9dd408ab2a8c363f2a5c34124f9c79010db4706e1315e1ff230741a9ab7e069318db587004bd0ccb71aad37c616b276bc0fe883865ba730b4d86ce2ae710185747d0860e00bb37b97fe71d79492a2e9a3bc07e5693f92de886fab3802ac62a8a4adcbf041eec05152cd28fd77154799e99674c8ea571519186ad9eb84a26edcef86473621e05515f3278810f931c662d037d9da73612c2f9d7d64e573595c402e9166299cbe356119ca38a3c6da77d6f864d61062b4300e388b631f60c25cb364b76561b4064c13e9e25d1ecb491472047157ea04fbbf6ccfe36cb2c030250b0335ae00255cf3670a61a5f207d72fccaac0b36a74d041f62341bc3759cd17d6e1c81aafcbbdc0f29906e54bc66dc1217031f881c9782eabe09de6835cdf4426113fb28e3bc0a73b007521c9a5abdc4a602c3c3358f0d3d81c8d84da5cc8acf1d15c9dd038ca64229097c666099a701b47bcf3a35e2541d4554a7bc1e3d4693b031c35f33b063d339558911870dd8bc3a52895612bee20ea8a7b0110da64362a357a4f9dbd7ff01155278c1173c57dd3d1b0947e58b571673544dbeff1c19cdb0ab9901671b3d043c4173fbcdf8e9cb03585bb9987414080046b6f283fc7c3aa245152941138636cd1b201e59080b8a7257bc2d7046c18d738c64804b088ac0983fbaeb92624f3ddc175afa3afc85cc8d83815bea41a195e883a4044a6406dbcb67682fc0522d2c920bc1d2372e95ea31408fcbe53e91c787e6da85255c40d0c9dbb0d4a5ded5886c90664bec4396f94782851fcd8565562a9df646025ad224078add8a05b8614ad0ce33141213a4650590ebaef22ef10b9cca5c4025eeaf58796baf052824d239586d7c706431fa1240f36a4f882d36ca608ece021b803386356f13a22bf3f42ef39d") + onionForDave + } + + // Dave decrypts the payment onion and forwards it to Eve. + val onionForEve = { + val priv = PrivateKey(hex"4444444444444444444444444444444444444444444444444444444444444444") + val add = UpdateAddHtlc(randomBytes32(), 1, 100_001_000 msat, paymentHash, CltvExpiry(800_100), onionForDave, None, Reputation.maxEndorsement, None) + val Right(ChannelRelayPacket(_, _, onionForEve, _)) = decrypt(add, priv, Features.empty) + val encoded = PaymentOnionCodecs.paymentOnionPacketCodec.encode(onionForEve).require.bytes + assert(encoded == hex"00037d517980f2321ce95c8ecea4aebceb2f62ebbbac598973439c79e9f66d28ce5c728d3226c796b85df07009baec8b4e46d73bf6bbf6f8dfa8bcf610bda5de6ebaf395b5a8572e30e91e402688834a13db55d04c28dc1bfdcc07c602532330ee6ce1bd6acce875c81fd53b8f7f4243ed940bb5e4897252763968a00c2b59d6cbfdf73bd86e4b9135a63dcf99612da557962da6b525c68b3159e4f56cd49cf654ca6240ec5a0c2365731266eb4263e16c90aed2fbd662de9aa22ce7bf8af18687d99550e48477c6c46f8a84957d24ac323381e69b57342d82e06082c645fcd96eb77ed4f563f04e7e7913e4bac16a78e56d223baead194b4cd80c97fa7d892d0288780ac90f1020e0cb43e267721bbbdd6fb759da9df2744882f4259a4e5bab60aaca2847311122c1d60a483c978d7b3042ae189892f85e1e7e3ad89d48769404b5dea1ddf1794b3c6b002286995e976b1de9c2457895a00952a06983986a619863e4c60f17e40a210e89273fa7f55ebd83887d451b5e658b9092e81540de49a4e4a05a757aa103ca5dc63194094869f067d5cc36e2d59de9d038f3b5a4b6fa4fd5b276db7b77182ddc96eaf53bfbfa2f988b785643047a5639965dde3baafeac2db4efbdf04da3520766c012c988d64c9e3aa2f723baa4926e413b18b93bdeec4e0761ef55bedea1de8751b49cb8a67a15ddeae511e06f03d36c2158aba897997c53a2f12e2db98214b093c0ee3c5fe6763dc7a3418db28a571a88da50a851eac78f78c29c489d6a09751976f4e456ffa23b71b3894e9263476d490e842de6a41fd085bf218691b1de3b4cf077f560fc86dd7f8d24c06912e5b9d53fc7b36d3f5bcde6cb9f22d5db09c0ec4e870466d0549f5fcd0e6849aa925f3f238b1a613c022ea22dc308899330113b60576b7fc8904233a77cf24ad2f9482cdc1265f6e74353d92d4fbff4a0a42dfebb92ac71c7fc2c79ccd1b187bd4542ed2d1808735179bebaba664f49a75d2823f7e7041e1cc0f717899b7eb2c2b9550be185f1a0b2245a48fdc205c5339742ad14e370193158997f4d4edff05297a4668705f667b2a858a0b8af56aa4b93fb41b30e16a50a75fdc0ce33dd94da254d8b1e55c40aa49444aacf4796a6979f0feca13924ff3a886d3e859e51d2d585ee919abcc82c396da1df7a4e97f415a01c25c1a5ae2fe65f4cc385e16d91e54836e12e7588d89ee41dfef277b97eb7d6c6ebfc6ed4f89b13d776904fad6e405123bca86068dc558dbbc284c65947f295b8828e7e35e80fd0981ba46229d47e646afa73f55070ae4a202b8c46719e6449632a4377eedbe83a69d1f422e73bc159172e631165dc5fe63e09dcace38218de2598204127255535d6c2197003383195af636cfd8615bd8a5db96057fe4ee67156685351d90c3db5bf61d3e573877572f58f982d9fbb35bd678143ccc1f2cccf1fd34c20e0a59b4c837540fac3964068eec3ffb8981a2ab774c542f74168ccd7fa9b08141cd0bda0d99ecee10a3857818370456c3c00d3f7b514f30ff6c31f11147851c8438411de3fc71719fbf79df3cab963231732c95850d59df90144161c2ef84a8b1c76a9494b8dd7234782bc61a6fc23222599a14163f78e117c99f33b3d2a4b11339903b41e7cfc253f1319d4c3ab1dc3d31a503c0bf9c233cb9216201d71abf915b8e50c0612b1fdba8ea8f248767256597151ba2f58dd67d470f8cfdfdc0bffceba618587f652a2155c58717a85e1eff38149b521f99449b35ed2a5ecb474fe60257d261017386ae08ea61cf907ebb7d2d5b1a55e50088449563d1d788d8b4f18ee57e24c6cab40dcd569495c6ea13fa1ca68dbeb6fed7462444ca94b6561471b4e1a75945d7327e5e56348bbd5cae106bf74976cc9288394a731b3555401e59c2718001171b6d6") + onionForEve + } + + // Eve decrypts the payment onion and the inner trampoline onion. + val priv = PrivateKey(hex"4545454545454545454545454545454545454545454545454545454545454545") + val add = UpdateAddHtlc(randomBytes32(), 1, 100_000_000.msat, paymentHash, CltvExpiry(800_000), onionForEve, None, Reputation.maxEndorsement, None) + val Right(FinalPacket(_, payloadForEve, _)) = decrypt(add, priv, Features.empty) + assert(payloadForEve.isInstanceOf[FinalPayload.Standard]) + assert(payloadForEve.amount == 100_000_000.msat) + assert(payloadForEve.expiry == CltvExpiry(800_000)) + assert(payloadForEve.asInstanceOf[FinalPayload.Standard].paymentSecret == ByteVector32.fromValidHex("2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a")) + assert(payloadForEve.asInstanceOf[FinalPayload.Standard].isTrampoline) + } + test("build outgoing trampoline payment with non-trampoline recipient") { // simple trampoline route to e where e doesn't support trampoline: // .----. @@ -402,7 +558,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { val payment = { val trampolineAmount = finalAmount * 1.001 // 0.1% fees val trampolineExpiry = finalExpiry + CltvExpiryDelta(144) - val payload = PaymentOnion.FinalPayload.Standard.createTrampolinePayload(trampolineAmount, trampolineAmount, trampolineExpiry, randomBytes32(), trampolineOnion.packet) + val payload = PaymentOnion.FinalPayload.Standard.createLegacyTrampolinePayload(trampolineAmount, trampolineAmount, trampolineExpiry, randomBytes32(), trampolineOnion.packet) val paymentOnion = buildOnion(NodePayload(c, payload) :: Nil, invoice.paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get TrampolinePayment.OutgoingPayment(trampolineAmount, trampolineExpiry, paymentOnion, trampolineOnion) } @@ -413,7 +569,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { assert(outer_c.totalAmount == payment.trampolineAmount) assert(outer_c.expiry == payment.trampolineExpiry) assert(outer_c.paymentSecret != invoice.paymentSecret) - assert(outer_c.records.get[OnionPaymentPayloadTlv.TrampolineOnion].get.packet.payload.size < 400) + assert(outer_c.records.get[OnionPaymentPayloadTlv.LegacyTrampolineOnion].get.packet.payload.size < 400) assert(inner_c.amountToForward == finalAmount) assert(inner_c.totalAmount == finalAmount) assert(inner_c.outgoingCltv == finalExpiry) @@ -469,7 +625,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { } test("fail to decrypt when the trampoline onion is invalid") { - val invoiceFeatures = Features[Bolt11Feature](VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, PaymentMetadata -> Optional, TrampolinePaymentPrototype -> Optional) + val invoiceFeatures = Features[Bolt11Feature](VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, PaymentMetadata -> Optional, Features.TrampolinePayment -> Optional) val invoice = Bolt11Invoice(Block.RegtestGenesisBlock.hash, Some(finalAmount), paymentHash, priv_e.privateKey, Left("invoice"), CltvExpiryDelta(6), paymentSecret = paymentSecret, features = invoiceFeatures, paymentMetadata = Some(hex"010203")) val payment = TrampolinePayment.buildOutgoingPayment(c, invoice, finalExpiry) @@ -590,7 +746,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { // // and return the HTLC sent by b to c. def createIntermediateTrampolinePayment(): UpdateAddHtlc = { - val invoiceFeatures = Features[Bolt11Feature](VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, TrampolinePaymentPrototype -> Optional) + val invoiceFeatures = Features[Bolt11Feature](VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, Features.TrampolinePayment -> Optional) val invoice = Bolt11Invoice(Block.RegtestGenesisBlock.hash, Some(finalAmount), paymentHash, priv_e.privateKey, Left("invoice"), CltvExpiryDelta(6), paymentSecret = paymentSecret, features = invoiceFeatures) val payment = TrampolinePayment.buildOutgoingPayment(c, invoice, finalExpiry) UpdateAddHtlc(randomBytes32(), 2, payment.trampolineAmount, paymentHash, payment.trampolineExpiry, payment.onion.packet, None, Reputation.maxEndorsement, None) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala index af8c75a240..ae02934542 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala @@ -21,6 +21,7 @@ import akka.actor.typed.eventstream.EventStream import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.adapter.{TypedActorContextOps, TypedActorRefOps} import akka.testkit.TestKit.awaitCond +import com.softwaremill.quicklens.ModifyPimp import com.typesafe.config.ConfigFactory import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, Crypto, TxId} import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional} @@ -58,8 +59,8 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat override def withFixture(test: OneArgTest): Outcome = { // we are node B in the route A -> B -> C -> .... - val disableTrampoline = test.tags.contains("trampoline-disabled") - val nodeParams = TestConstants.Bob.nodeParams.copy(enableTrampolinePayment = !disableTrampoline) + val nodeParams = TestConstants.Bob.nodeParams + .modify(_.features).usingIf(test.tags.contains("trampoline-disabled"))(_.remove(Features.TrampolinePayment)) val router = TestProbe[Any]("router") val register = TestProbe[Any]("register") val paymentHandler = TestProbe[Any]("payment-handler") @@ -201,7 +202,7 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat import f._ // we use this to build a valid trampoline onion inside a normal onion - val invoiceFeatures = Features[Bolt11Feature](VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, PaymentMetadata -> Optional, TrampolinePaymentPrototype -> Optional) + val invoiceFeatures = Features[Bolt11Feature](VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, PaymentMetadata -> Optional, Features.TrampolinePayment -> Optional) val invoice = Bolt11Invoice(Block.RegtestGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("invoice"), CltvExpiryDelta(6), paymentSecret = paymentSecret, features = invoiceFeatures) val payment = TrampolinePayment.buildOutgoingPayment(b, invoice, finalExpiry) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/PaymentOnionSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/PaymentOnionSpec.scala index d933fb2175..37d3a50c76 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/PaymentOnionSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/PaymentOnionSpec.scala @@ -138,18 +138,21 @@ class PaymentOnionSpec extends AnyFunSuite { test("encode/decode node relay per-hop payload") { val nodeId = PublicKey(hex"02eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619") - val expected = TlvStream[OnionPaymentPayloadTlv](AmountToForward(561 msat), OutgoingCltv(CltvExpiry(42)), OutgoingNodeId(nodeId)) - val bin = hex"2e 02020231 04012a fe000102322102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619" - - val decoded = perHopPayloadCodec.decode(bin.bits).require.value - assert(decoded == expected) - val Right(payload) = IntermediatePayload.NodeRelay.Standard.validate(decoded) - assert(payload.amountToForward == 561.msat) - assert(payload.outgoingCltv == CltvExpiry(42)) - assert(payload.outgoingNodeId == nodeId) + val testCases = Seq( + TlvStream[OnionPaymentPayloadTlv](AmountToForward(561 msat), OutgoingCltv(CltvExpiry(42)), LegacyOutgoingNodeId(nodeId)) -> hex"2e 02020231 04012a fe000102322102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619", + TlvStream[OnionPaymentPayloadTlv](AmountToForward(561 msat), OutgoingCltv(CltvExpiry(42)), OutgoingNodeId(nodeId)) -> hex"2a 02020231 04012a 0e2102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619", + ) + testCases.foreach { case (expected, bin) => + val decoded = perHopPayloadCodec.decode(bin.bits).require.value + assert(decoded == expected) + val Right(payload) = IntermediatePayload.NodeRelay.Standard.validate(decoded) + assert(payload.amountToForward == 561.msat) + assert(payload.outgoingCltv == CltvExpiry(42)) + assert(payload.outgoingNodeId == nodeId) - val encoded = perHopPayloadCodec.encode(expected).require.bytes - assert(encoded == bin) + val encoded = perHopPayloadCodec.encode(expected).require.bytes + assert(encoded == bin) + } } test("encode/decode node relay to legacy per-hop payload") { @@ -160,22 +163,27 @@ class PaymentOnionSpec extends AnyFunSuite { List(ExtraHop(node1, ShortChannelId(1), 10 msat, 100, CltvExpiryDelta(144))), List(ExtraHop(node2, ShortChannelId(2), 20 msat, 150, CltvExpiryDelta(12)), ExtraHop(node3, ShortChannelId(3), 30 msat, 200, CltvExpiryDelta(24))) ) - val expected = TlvStream[OnionPaymentPayloadTlv](AmountToForward(561 msat), OutgoingCltv(CltvExpiry(42)), PaymentData(ByteVector32(hex"eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), 1105 msat), InvoiceFeatures(features), OutgoingNodeId(nodeId), InvoiceRoutingInfo(routingHints)) - val bin = hex"fa 02020231 04012a 0822eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f2836866190451 fe00010231010a fe000102322102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619 fe000102339b01036d6caac248af96f6afa7f904f550253a0f3ef3f5aa2fe6838a95b216691468e200000000000000010000000a00000064009002025f7117a78150fe2ef97db7cfc83bd57b2e2c0d0dd25eaf467a4a1c2a45ce148600000000000000020000001400000096000c02a051267759c3a149e3e72372f4e0c4054ba597ebfd0eda78a2273023667205ee00000000000000030000001e000000c80018" - - val decoded = perHopPayloadCodec.decode(bin.bits).require.value - assert(decoded == expected) - val Right(payload) = IntermediatePayload.NodeRelay.ToNonTrampoline.validate(decoded) - assert(payload.amountToForward == 561.msat) - assert(payload.totalAmount == 1105.msat) - assert(payload.paymentSecret == ByteVector32(hex"eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619")) - assert(payload.outgoingCltv == CltvExpiry(42)) - assert(payload.outgoingNodeId == nodeId) - assert(payload.invoiceFeatures == features) - assert(payload.invoiceRoutingInfo == routingHints) - - val encoded = perHopPayloadCodec.encode(expected).require.bytes - assert(encoded == bin) + val expected = TlvStream[OnionPaymentPayloadTlv](AmountToForward(561 msat), OutgoingCltv(CltvExpiry(42)), PaymentData(ByteVector32(hex"eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), 1105 msat), InvoiceFeatures(features), LegacyOutgoingNodeId(nodeId), InvoiceRoutingInfo(routingHints)) + // This is the encoding used by the experimental trampoline prototype. + val legacyBin = hex"fa 02020231 04012a 0822eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f2836866190451 fe00010231010a fe000102322102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619 fe000102339b01036d6caac248af96f6afa7f904f550253a0f3ef3f5aa2fe6838a95b216691468e200000000000000010000000a00000064009002025f7117a78150fe2ef97db7cfc83bd57b2e2c0d0dd25eaf467a4a1c2a45ce148600000000000000020000001400000096000c02a051267759c3a149e3e72372f4e0c4054ba597ebfd0eda78a2273023667205ee00000000000000030000001e000000c80018" + // This is the encoding used by the official trampoline feature. + val bin = hex"f6 02020231 04012a 0822eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f2836866190451 15010a fe000102322102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619 fe000102339b01036d6caac248af96f6afa7f904f550253a0f3ef3f5aa2fe6838a95b216691468e200000000000000010000000a00000064009002025f7117a78150fe2ef97db7cfc83bd57b2e2c0d0dd25eaf467a4a1c2a45ce148600000000000000020000001400000096000c02a051267759c3a149e3e72372f4e0c4054ba597ebfd0eda78a2273023667205ee00000000000000030000001e000000c80018" + + Seq(legacyBin, bin).foreach(encoded => { + val decoded = perHopPayloadCodec.decode(encoded.bits).require.value + assert(decoded == expected) + val Right(payload) = IntermediatePayload.NodeRelay.ToNonTrampoline.validate(decoded) + assert(payload.amountToForward == 561.msat) + assert(payload.totalAmount == 1105.msat) + assert(payload.paymentSecret == ByteVector32(hex"eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619")) + assert(payload.outgoingCltv == CltvExpiry(42)) + assert(payload.outgoingNodeId == nodeId) + assert(payload.invoiceFeatures == features) + assert(payload.invoiceRoutingInfo == routingHints) + + val reEncoded = perHopPayloadCodec.encode(expected).require.bytes + assert(reEncoded == bin) + }) } test("encode/decode node relay to blinded paths per-hop payload") { @@ -187,18 +195,23 @@ class PaymentOnionSpec extends AnyFunSuite { ) val path = PaymentBlindedRoute(blindedRoute, OfferTypes.PaymentInfo(1000 msat, 678, CltvExpiryDelta(82), 300 msat, 4000000 msat, ByteVector.empty)) val expected = TlvStream[OnionPaymentPayloadTlv](AmountToForward(341 msat), OutgoingCltv(CltvExpiry(826483)), OutgoingBlindedPaths(Seq(path)), InvoiceFeatures(features)) - val bin = hex"82 02020155 04030c9c73 fe0001023103020000 fe000102366a0100000000000001d40232882c4982576e00f0d6bd4998f5b3e92d47ecc8fbad5b6a5e7521819d891d9e0103823aa560d631e9d7b686be4a9227e577009afb5173023b458a6a6aff056ac9800000000003e8000002a60052000000000000012c00000000003d09000000" + // This is the encoding used by the experimental trampoline prototype. + val legacyBin = hex"82 02020155 04030c9c73 fe0001023103020000 fe000102366a0100000000000001d40232882c4982576e00f0d6bd4998f5b3e92d47ecc8fbad5b6a5e7521819d891d9e0103823aa560d631e9d7b686be4a9227e577009afb5173023b458a6a6aff056ac9800000000003e8000002a60052000000000000012c00000000003d09000000" + // This is the encoding used by the official trampoline feature. + val bin = hex"7a 02020155 04030c9c73 1503020000 166a0100000000000001d40232882c4982576e00f0d6bd4998f5b3e92d47ecc8fbad5b6a5e7521819d891d9e0103823aa560d631e9d7b686be4a9227e577009afb5173023b458a6a6aff056ac9800000000003e8000002a60052000000000000012c00000000003d09000000" - val decoded = perHopPayloadCodec.decode(bin.bits).require.value - assert(decoded == expected) - val Right(payload) = IntermediatePayload.NodeRelay.ToBlindedPaths.validate(decoded) - assert(payload.amountToForward == 341.msat) - assert(payload.outgoingCltv == CltvExpiry(826483)) - assert(payload.outgoingBlindedPaths == Seq(path)) - assert(payload.invoiceFeatures == features) - - val encoded = perHopPayloadCodec.encode(expected).require.bytes - assert(encoded == bin) + Seq(legacyBin, bin).foreach(encoded => { + val decoded = perHopPayloadCodec.decode(encoded.bits).require.value + assert(decoded == expected) + val Right(payload) = IntermediatePayload.NodeRelay.ToBlindedPaths.validate(decoded) + assert(payload.amountToForward == 341.msat) + assert(payload.outgoingCltv == CltvExpiry(826483)) + assert(payload.outgoingBlindedPaths == Seq(path)) + assert(payload.invoiceFeatures == features) + + val reEncoded = perHopPayloadCodec.encode(expected).require.bytes + assert(reEncoded == bin) + }) } test("encode/decode final per-hop payload") { @@ -210,7 +223,8 @@ class PaymentOnionSpec extends AnyFunSuite { TlvStream[OnionPaymentPayloadTlv](AmountToForward(561 msat), OutgoingCltv(CltvExpiry(42)), PaymentData(ByteVector32(hex"eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), 1099511627775L msat)) -> hex"2e 02020231 04012a 0825eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619ffffffffff", TlvStream[OnionPaymentPayloadTlv](AmountToForward(561 msat), OutgoingCltv(CltvExpiry(42)), OutgoingChannelId(ShortChannelId(1105)), PaymentData(ByteVector32(hex"eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), 0 msat)) -> hex"33 02020231 04012a 06080000000000000451 0820eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619", TlvStream[OnionPaymentPayloadTlv](Set[OnionPaymentPayloadTlv](AmountToForward(561 msat), OutgoingCltv(CltvExpiry(42)), PaymentData(ByteVector32(hex"eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), 0 msat)), Set(GenericTlv(65535, hex"06c1"))) -> hex"2f 02020231 04012a 0820eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619 fdffff0206c1", - TlvStream[OnionPaymentPayloadTlv](AmountToForward(561 msat), OutgoingCltv(CltvExpiry(42)), PaymentData(ByteVector32(hex"eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), 0 msat), TrampolineOnion(OnionRoutingPacket(0, hex"02eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619", hex"cff34152f3a36e52ca94e74927203a560392b9cc7ce3c45809c6be52166c24a595716880f95f178bf5b30ca63141f74db6e92795c6130877cfdac3d4bd3087ee73c65d627ddd709112a848cc99e303f3706509aa43ba7c8a88cba175fccf9a8f5016ef06d3b935dbb15196d7ce16dc1a7157845566901d7b2197e52cab4ce487014b14816e5805f9fcacb4f8f88b8ff176f1b94f6ce6b00bc43221130c17d20ef629db7c5f7eafaa166578c720619561dd14b3277db557ec7dcdb793771aef0f2f667cfdbeae3ac8d331c5994779dffb31e5fc0dbdedc0c592ca6d21c18e47fe3528d6975c19517d7e2ea8c5391cf17d0fe30c80913ed887234ccb48808f7ef9425bcd815c3b586210979e3bb286ef2851bf9ce04e28c40a203df98fd648d2f1936fd2f1def0e77eecb277229b4b682322371c0a1dbfcd723a991993df8cc1f2696b84b055b40a1792a29f710295a18fbd351b0f3ff34cd13941131b8278ba79303c89117120eea691738a9954908195143b039dbeed98f26a92585f3d15cf742c953799d3272e0545e9b744be9d3b4c", ByteVector32(hex"bb079bfc4b35190eee9f59a1d7b41ba2f773179f322dafb4b1af900c289ebd6c")))) -> hex"fd0203 02020231 04012a 0820eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619 fe00010234fd01d20002eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619cff34152f3a36e52ca94e74927203a560392b9cc7ce3c45809c6be52166c24a595716880f95f178bf5b30ca63141f74db6e92795c6130877cfdac3d4bd3087ee73c65d627ddd709112a848cc99e303f3706509aa43ba7c8a88cba175fccf9a8f5016ef06d3b935dbb15196d7ce16dc1a7157845566901d7b2197e52cab4ce487014b14816e5805f9fcacb4f8f88b8ff176f1b94f6ce6b00bc43221130c17d20ef629db7c5f7eafaa166578c720619561dd14b3277db557ec7dcdb793771aef0f2f667cfdbeae3ac8d331c5994779dffb31e5fc0dbdedc0c592ca6d21c18e47fe3528d6975c19517d7e2ea8c5391cf17d0fe30c80913ed887234ccb48808f7ef9425bcd815c3b586210979e3bb286ef2851bf9ce04e28c40a203df98fd648d2f1936fd2f1def0e77eecb277229b4b682322371c0a1dbfcd723a991993df8cc1f2696b84b055b40a1792a29f710295a18fbd351b0f3ff34cd13941131b8278ba79303c89117120eea691738a9954908195143b039dbeed98f26a92585f3d15cf742c953799d3272e0545e9b744be9d3b4cbb079bfc4b35190eee9f59a1d7b41ba2f773179f322dafb4b1af900c289ebd6c" + TlvStream[OnionPaymentPayloadTlv](AmountToForward(561 msat), OutgoingCltv(CltvExpiry(42)), PaymentData(ByteVector32(hex"eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), 0 msat), TrampolineOnion(OnionRoutingPacket(0, hex"02eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619", hex"cff34152f3a36e52ca94e74927203a560392b9cc7ce3c45809c6be52166c24a595716880f95f178bf5b30ca63141f74db6e92795c6130877cfdac3d4bd3087ee73c65d627ddd709112a848cc99e303f3706509aa43ba7c8a88cba175fccf9a8f5016ef06d3b935dbb15196d7ce16dc1a7157845566901d7b2197e52cab4ce487014b14816e5805f9fcacb4f8f88b8ff176f1b94f6ce6b00bc43221130c17d20ef629db7c5f7eafaa166578c720619561dd14b3277db557ec7dcdb793771aef0f2f667cfdbeae3ac8d331c5994779dffb31e5fc0dbdedc0c592ca6d21c18e47fe3528d6975c19517d7e2ea8c5391cf17d0fe30c80913ed887234ccb48808f7ef9425bcd815c3b586210979e3bb286ef2851bf9ce04e28c40a203df98fd648d2f1936fd2f1def0e77eecb277229b4b682322371c0a1dbfcd723a991993df8cc1f2696b84b055b40a1792a29f710295a18fbd351b0f3ff34cd13941131b8278ba79303c89117120eea691738a9954908195143b039dbeed98f26a92585f3d15cf742c953799d3272e0545e9b744be9d3b4c", ByteVector32(hex"bb079bfc4b35190eee9f59a1d7b41ba2f773179f322dafb4b1af900c289ebd6c")))) -> hex"fd01ff 02020231 04012a 0820eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619 14fd01d20002eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619cff34152f3a36e52ca94e74927203a560392b9cc7ce3c45809c6be52166c24a595716880f95f178bf5b30ca63141f74db6e92795c6130877cfdac3d4bd3087ee73c65d627ddd709112a848cc99e303f3706509aa43ba7c8a88cba175fccf9a8f5016ef06d3b935dbb15196d7ce16dc1a7157845566901d7b2197e52cab4ce487014b14816e5805f9fcacb4f8f88b8ff176f1b94f6ce6b00bc43221130c17d20ef629db7c5f7eafaa166578c720619561dd14b3277db557ec7dcdb793771aef0f2f667cfdbeae3ac8d331c5994779dffb31e5fc0dbdedc0c592ca6d21c18e47fe3528d6975c19517d7e2ea8c5391cf17d0fe30c80913ed887234ccb48808f7ef9425bcd815c3b586210979e3bb286ef2851bf9ce04e28c40a203df98fd648d2f1936fd2f1def0e77eecb277229b4b682322371c0a1dbfcd723a991993df8cc1f2696b84b055b40a1792a29f710295a18fbd351b0f3ff34cd13941131b8278ba79303c89117120eea691738a9954908195143b039dbeed98f26a92585f3d15cf742c953799d3272e0545e9b744be9d3b4cbb079bfc4b35190eee9f59a1d7b41ba2f773179f322dafb4b1af900c289ebd6c", + TlvStream[OnionPaymentPayloadTlv](AmountToForward(561 msat), OutgoingCltv(CltvExpiry(42)), PaymentData(ByteVector32(hex"eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), 0 msat), LegacyTrampolineOnion(OnionRoutingPacket(0, hex"02eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619", hex"cff34152f3a36e52ca94e74927203a560392b9cc7ce3c45809c6be52166c24a595716880f95f178bf5b30ca63141f74db6e92795c6130877cfdac3d4bd3087ee73c65d627ddd709112a848cc99e303f3706509aa43ba7c8a88cba175fccf9a8f5016ef06d3b935dbb15196d7ce16dc1a7157845566901d7b2197e52cab4ce487014b14816e5805f9fcacb4f8f88b8ff176f1b94f6ce6b00bc43221130c17d20ef629db7c5f7eafaa166578c720619561dd14b3277db557ec7dcdb793771aef0f2f667cfdbeae3ac8d331c5994779dffb31e5fc0dbdedc0c592ca6d21c18e47fe3528d6975c19517d7e2ea8c5391cf17d0fe30c80913ed887234ccb48808f7ef9425bcd815c3b586210979e3bb286ef2851bf9ce04e28c40a203df98fd648d2f1936fd2f1def0e77eecb277229b4b682322371c0a1dbfcd723a991993df8cc1f2696b84b055b40a1792a29f710295a18fbd351b0f3ff34cd13941131b8278ba79303c89117120eea691738a9954908195143b039dbeed98f26a92585f3d15cf742c953799d3272e0545e9b744be9d3b4c", ByteVector32(hex"bb079bfc4b35190eee9f59a1d7b41ba2f773179f322dafb4b1af900c289ebd6c")))) -> hex"fd0203 02020231 04012a 0820eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619 fe00010234fd01d20002eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619cff34152f3a36e52ca94e74927203a560392b9cc7ce3c45809c6be52166c24a595716880f95f178bf5b30ca63141f74db6e92795c6130877cfdac3d4bd3087ee73c65d627ddd709112a848cc99e303f3706509aa43ba7c8a88cba175fccf9a8f5016ef06d3b935dbb15196d7ce16dc1a7157845566901d7b2197e52cab4ce487014b14816e5805f9fcacb4f8f88b8ff176f1b94f6ce6b00bc43221130c17d20ef629db7c5f7eafaa166578c720619561dd14b3277db557ec7dcdb793771aef0f2f667cfdbeae3ac8d331c5994779dffb31e5fc0dbdedc0c592ca6d21c18e47fe3528d6975c19517d7e2ea8c5391cf17d0fe30c80913ed887234ccb48808f7ef9425bcd815c3b586210979e3bb286ef2851bf9ce04e28c40a203df98fd648d2f1936fd2f1def0e77eecb277229b4b682322371c0a1dbfcd723a991993df8cc1f2696b84b055b40a1792a29f710295a18fbd351b0f3ff34cd13941131b8278ba79303c89117120eea691738a9954908195143b039dbeed98f26a92585f3d15cf742c953799d3272e0545e9b744be9d3b4cbb079bfc4b35190eee9f59a1d7b41ba2f773179f322dafb4b1af900c289ebd6c", ) for ((expected, bin) <- testCases) { @@ -323,7 +337,7 @@ class PaymentOnionSpec extends AnyFunSuite { val testCases = Seq( (MissingRequiredTlv(UInt64(2)), hex"2a 04012a fe000102322102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), // missing amount (MissingRequiredTlv(UInt64(4)), hex"2b 02020231 fe000102322102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), // missing cltv - (MissingRequiredTlv(UInt64(66098)), hex"07 02020231 04012a"), // missing node id + (MissingRequiredTlv(UInt64(14)), hex"07 02020231 04012a"), // missing node id (ForbiddenTlv(UInt64(10)), hex"34 02020231 04012a 0a04ffffffff fe000102322102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), // forbidden encrypted data (ForbiddenTlv(UInt64(12)), hex"51 02020231 04012a 0c21036d6caac248af96f6afa7f904f550253a0f3ef3f5aa2fe6838a95b216691468e2 fe000102322102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), // forbidden path key ) From 7a901c6d65114620b2e7937a73764b77ed9cc5cc Mon Sep 17 00:00:00 2001 From: t-bast Date: Fri, 29 Nov 2024 11:52:26 +0100 Subject: [PATCH 2/3] Add support for blinded trampoline payments We add support for trampoline payments to blinded recipients, where each node of the blinded path is used as trampoline node. This is particularly useful to include custom TLVs from the payer to the recipient. --- .../acinq/eclair/payment/PaymentPacket.scala | 82 +++-- .../eclair/payment/relay/NodeRelay.scala | 25 ++ .../acinq/eclair/payment/send/Recipient.scala | 8 +- .../send/TrampolinePaymentLifecycle.scala | 75 ++++- .../eclair/wire/protocol/PaymentOnion.scala | 51 ++- .../integration/PaymentIntegrationSpec.scala | 38 ++- .../eclair/payment/PaymentPacketSpec.scala | 305 +++++++++++++++++- .../wire/protocol/PaymentOnionSpec.scala | 57 ++++ 8 files changed, 589 insertions(+), 52 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala index 8636fd53e2..92f4254aef 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala @@ -63,6 +63,7 @@ object IncomingPaymentPacket { def innerPayload: IntermediatePayload.NodeRelay } case class RelayToTrampolinePacket(add: UpdateAddHtlc, outerPayload: FinalPayload.Standard, innerPayload: IntermediatePayload.NodeRelay.Standard, nextPacket: OnionRoutingPacket, receivedAt: TimestampMilli) extends NodeRelayPacket + case class RelayToBlindedTrampolinePacket(add: UpdateAddHtlc, outerPayload: FinalPayload.Standard, innerPayload: IntermediatePayload.NodeRelay.Blinded, nextPacket: OnionRoutingPacket, receivedAt: TimestampMilli) extends NodeRelayPacket case class RelayToNonTrampolinePacket(add: UpdateAddHtlc, outerPayload: FinalPayload.Standard, innerPayload: IntermediatePayload.NodeRelay.ToNonTrampoline, receivedAt: TimestampMilli) extends NodeRelayPacket case class RelayToBlindedPathsPacket(add: UpdateAddHtlc, outerPayload: FinalPayload.Standard, innerPayload: IntermediatePayload.NodeRelay.ToBlindedPaths, receivedAt: TimestampMilli) extends NodeRelayPacket // @formatter:on @@ -176,9 +177,14 @@ object IncomingPaymentPacket { val totalAmount = payload.get[OnionPaymentPayloadTlv.AmountToForward].map(_.amount).getOrElse(add.amountMsat) payload.copy(records = payload.records + OnionPaymentPayloadTlv.PaymentData(dummyPaymentSecret, totalAmount)) } - // NB: when we enable blinded trampoline routes, we will need to check if the outer onion contains a - // path key and use it to derive the decryption key for the blinded trampoline onion. - decryptOnion(add.paymentHash, privateKey, trampolinePacket).flatMap { + // If we are an intermediate trampoline node inside a blinded path, the payer doesn't know our node_id + // and has encrypted the trampoline onion to our blinded node_id: in that case, the previous trampoline + // node will provide the path key in the outer onion. + val trampolineOnionDecryptionKey = payload.get[OnionPaymentPayloadTlv.PathKey].map(_.publicKey) match { + case Some(pathKey) => Sphinx.RouteBlinding.derivePrivateKey(privateKey, pathKey) + case None => privateKey + } + decryptOnion(add.paymentHash, trampolineOnionDecryptionKey, trampolinePacket).flatMap { case DecodedOnionPacket(innerPayload, Some(next)) => // We are an intermediate trampoline node. if (innerPayload.get[InvoiceRoutingInfo].isDefined) { @@ -187,7 +193,8 @@ object IncomingPaymentPacket { // The payer is a wallet using the legacy trampoline feature. validateTrampolineToNonTrampoline(add, outerPayload, innerPayload) } else { - validateNodeRelay(add, outerPayload, innerPayload, next) + // The recipient supports trampoline (and may support blinded payments). + validateNodeRelay(add, privateKey, outerPayload, innerPayload, next) } case DecodedOnionPacket(innerPayload, None) => if (innerPayload.get[OutgoingBlindedPaths].isDefined) { @@ -199,8 +206,8 @@ object IncomingPaymentPacket { // They can be reached with the invoice data provided. validateTrampolineToNonTrampoline(add, outerPayload, innerPayload) } else { - // We're the final recipient of this trampoline payment. - validateTrampolineFinalPayload(add, outerPayload, innerPayload) + // We're the final recipient of this trampoline payment (which may be blinded). + validateTrampolineFinalPayload(add, privateKey, outerPayload, innerPayload) } } case None => @@ -242,31 +249,50 @@ object IncomingPaymentPacket { } } - private def validateTrampolineFinalPayload(add: UpdateAddHtlc, outerPayload: TlvStream[OnionPaymentPayloadTlv], innerPayload: TlvStream[OnionPaymentPayloadTlv]): Either[FailureMessage, FinalPacket] = { - // The outer payload cannot use route blinding, but the inner payload may (but it's not supported yet). - FinalPayload.Standard.validate(outerPayload).left.map(_.failureMessage).flatMap { outerPayload => - FinalPayload.Standard.validate(innerPayload).left.map(_.failureMessage).flatMap { - case _ if add.amountMsat < outerPayload.amount => Left(FinalIncorrectHtlcAmount(add.amountMsat)) - case _ if add.cltvExpiry < outerPayload.expiry => Left(FinalIncorrectCltvExpiry(add.cltvExpiry)) - case innerPayload if outerPayload.expiry < innerPayload.expiry => Left(FinalIncorrectCltvExpiry(add.cltvExpiry)) // previous trampoline didn't forward the right expiry - case innerPayload if outerPayload.totalAmount < innerPayload.amount => Left(FinalIncorrectHtlcAmount(outerPayload.totalAmount)) // previous trampoline didn't forward the right amount - case innerPayload => - // We merge contents from the outer and inner payloads. - // We must use the inner payload's total amount and payment secret because the payment may be split between multiple trampoline payments (#reckless). - val trampolinePacket = outerPayload.records.get[OnionPaymentPayloadTlv.TrampolineOnion].map(_.packet).orElse(outerPayload.records.get[OnionPaymentPayloadTlv.LegacyTrampolineOnion].map(_.packet)) - Right(FinalPacket(add, FinalPayload.Standard.createPayload(outerPayload.amount, innerPayload.totalAmount, innerPayload.expiry, innerPayload.paymentSecret, innerPayload.paymentMetadata, trampolinePacket), TimestampMilli.now())) - } + private def validateTrampolineFinalPayload(add: UpdateAddHtlc, privateKey: PrivateKey, outerPayload: TlvStream[OnionPaymentPayloadTlv], innerPayload: TlvStream[OnionPaymentPayloadTlv]): Either[FailureMessage, FinalPacket] = { + // The outer payload cannot use route blinding, but the inner payload may. + FinalPayload.Standard.validate(outerPayload).left.map(_.failureMessage).flatMap { + case outerPayload if add.amountMsat < outerPayload.amount => Left(FinalIncorrectHtlcAmount(add.amountMsat)) + case outerPayload if add.cltvExpiry < outerPayload.expiry => Left(FinalIncorrectCltvExpiry(add.cltvExpiry)) + case outerPayload => + innerPayload.get[OnionPaymentPayloadTlv.EncryptedRecipientData] match { + case Some(encrypted) => + decryptEncryptedRecipientData(add, privateKey, outerPayload.records, encrypted.data).flatMap { + case DecodedEncryptedRecipientData(blindedPayload, _) => validateBlindedFinalPayload(add, innerPayload, blindedPayload) + } + case None => + FinalPayload.Standard.validate(innerPayload).left.map(_.failureMessage).flatMap { + case innerPayload if outerPayload.expiry < innerPayload.expiry => Left(FinalIncorrectCltvExpiry(add.cltvExpiry)) // previous trampoline didn't forward the right expiry + case innerPayload if outerPayload.totalAmount < innerPayload.amount => Left(FinalIncorrectHtlcAmount(outerPayload.totalAmount)) // previous trampoline didn't forward the right amount + case innerPayload => + // We merge contents from the outer and inner payloads. + // We must use the inner payload's total amount and payment secret because the payment may be split between multiple trampoline payments (#reckless). + val trampolinePacket = outerPayload.records.get[OnionPaymentPayloadTlv.TrampolineOnion].map(_.packet).orElse(outerPayload.records.get[OnionPaymentPayloadTlv.LegacyTrampolineOnion].map(_.packet)) + Right(FinalPacket(add, FinalPayload.Standard.createPayload(outerPayload.amount, innerPayload.totalAmount, innerPayload.expiry, innerPayload.paymentSecret, innerPayload.paymentMetadata, trampolinePacket), TimestampMilli.now())) + } + } } } - private def validateNodeRelay(add: UpdateAddHtlc, outerPayload: TlvStream[OnionPaymentPayloadTlv], innerPayload: TlvStream[OnionPaymentPayloadTlv], next: OnionRoutingPacket): Either[FailureMessage, RelayToTrampolinePacket] = { - // The outer payload cannot use route blinding, but the inner payload may (but it's not supported yet). - FinalPayload.Standard.validate(outerPayload).left.map(_.failureMessage).flatMap { outerPayload => - IntermediatePayload.NodeRelay.Standard.validate(innerPayload).left.map(_.failureMessage).flatMap { - case _ if add.amountMsat < outerPayload.amount => Left(FinalIncorrectHtlcAmount(add.amountMsat)) - case _ if add.cltvExpiry != outerPayload.expiry => Left(FinalIncorrectCltvExpiry(add.cltvExpiry)) - case innerPayload => Right(RelayToTrampolinePacket(add, outerPayload, innerPayload, next, TimestampMilli.now())) - } + private def validateNodeRelay(add: UpdateAddHtlc, privateKey: PrivateKey, outerPayload: TlvStream[OnionPaymentPayloadTlv], innerPayload: TlvStream[OnionPaymentPayloadTlv], next: OnionRoutingPacket): Either[FailureMessage, IncomingPaymentPacket] = { + // The outer payload cannot use route blinding, but the inner payload may. + FinalPayload.Standard.validate(outerPayload).left.map(_.failureMessage).flatMap { + case outerPayload if add.amountMsat < outerPayload.amount => Left(FinalIncorrectHtlcAmount(add.amountMsat)) + case outerPayload if add.cltvExpiry != outerPayload.expiry => Left(FinalIncorrectCltvExpiry(add.cltvExpiry)) + case outerPayload => + innerPayload.get[OnionPaymentPayloadTlv.EncryptedRecipientData] match { + case Some(encrypted) => + // The path key can be found: + // - in the inner payload if we are the introduction node of the blinded path (provided by the payer). + // - in the outer payload if we are an intermediate node in the blinded path (provided by the previous trampoline node). + val pathKey_opt = innerPayload.get[OnionPaymentPayloadTlv.PathKey].orElse(outerPayload.records.get[OnionPaymentPayloadTlv.PathKey]).map(_.publicKey) + decryptEncryptedRecipientData(add, privateKey, pathKey_opt, encrypted.data).flatMap { + case DecodedEncryptedRecipientData(blindedPayload, nextPathKey) => + IntermediatePayload.NodeRelay.Blinded.validate(innerPayload, blindedPayload, nextPathKey).left.map(_.failureMessage).map(innerPayload => RelayToBlindedTrampolinePacket(add, outerPayload, innerPayload, next, TimestampMilli.now())) + } + case None => + IntermediatePayload.NodeRelay.Standard.validate(innerPayload).left.map(_.failureMessage).map(innerPayload => RelayToTrampolinePacket(add, outerPayload, innerPayload, next, TimestampMilli.now())) + } } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala index 2815fdd6f2..69848628e8 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala @@ -67,6 +67,7 @@ object NodeRelay { private case class WrappedPaymentSent(paymentSent: PaymentSent) extends Command private case class WrappedPaymentFailed(paymentFailed: PaymentFailed) extends Command private case class WrappedPeerReadyResult(result: PeerReadyNotifier.Result) extends Command + private case class WrappedOutgoingNodeId(nodeId_opt: Option[PublicKey]) extends Command private case class WrappedResolvedPaths(resolved: Seq[ResolvedPath]) extends Command private case class WrappedPeerInfo(remoteFeatures_opt: Option[Features[InitFeature]]) extends Command private case class WrappedOnTheFlyFundingResponse(result: Peer.ProposeOnTheFlyFundingResponse) extends Command @@ -111,6 +112,7 @@ object NodeRelay { val incomingPaymentHandler = context.actorOf(MultiPartPaymentFSM.props(nodeParams, paymentHash, totalAmountIn, mppFsmAdapters)) val nextPacket_opt = nodeRelayPacket match { case IncomingPaymentPacket.RelayToTrampolinePacket(_, _, _, nextPacket, _) => Some(nextPacket) + case IncomingPaymentPacket.RelayToBlindedTrampolinePacket(_, _, _, nextPacket, _) => Some(nextPacket) case _: IncomingPaymentPacket.RelayToNonTrampolinePacket => None case _: IncomingPaymentPacket.RelayToBlindedPathsPacket => None } @@ -187,6 +189,7 @@ object NodeRelay { case nextPayload: IntermediatePayload.NodeRelay.Standard => failures.collectFirst { case RemoteFailure(_, _, e) if e.originNode == nextPayload.outgoingNodeId => e.failureMessage } case nextPayload: IntermediatePayload.NodeRelay.ToNonTrampoline => failures.collectFirst { case RemoteFailure(_, _, e) if e.originNode == nextPayload.outgoingNodeId => e.failureMessage } // When using blinded paths, we will never get a failure from the final node (for privacy reasons). + case _: IntermediatePayload.NodeRelay.Blinded => None case _: IntermediatePayload.NodeRelay.ToBlindedPaths => None } val otherNodeFailure = failures.collectFirst { case RemoteFailure(_, _, e) => e.failureMessage } @@ -256,6 +259,28 @@ class NodeRelay private(nodeParams: NodeParams, val recipient = ClearRecipient(payloadOut.outgoingNodeId, features, payloadOut.amountToForward, payloadOut.outgoingCltv, paymentSecret, nextTrampolineOnion_opt = nextPacket_opt) context.log.debug("forwarding payment to the next trampoline node {}", recipient.nodeId) attemptWakeUpIfRecipientIsWallet(upstream, recipient, nextPayload, nextPacket_opt) + case payloadOut: IntermediatePayload.NodeRelay.Blinded => + // Blinded paths in Bolt 12 invoices may use an scid to reference the next node, if it is one of our peers. + // We need to resolve that to a nodeId in order to create a payment onion. + payloadOut.outgoing match { + case Left(outgoingNodeId) => context.self ! WrappedOutgoingNodeId(Some(outgoingNodeId.publicKey)) + case Right(outgoingChannelId) => register ! Register.GetNextNodeId(context.messageAdapter[Option[PublicKey]](WrappedOutgoingNodeId), outgoingChannelId) + } + Behaviors.receiveMessagePartial { + rejectExtraHtlcPartialFunction orElse { + case WrappedOutgoingNodeId(Some(outgoingNodeId)) => + val outgoingAmount = nextPayload.outgoingAmount(upstream.amountIn) + val outgoingExpiry = nextPayload.outgoingExpiry(upstream.expiryIn) + val paymentSecret = randomBytes32() // we generate a new secret to protect against probing attacks + val recipient = ClearRecipient(outgoingNodeId, Features.empty, outgoingAmount, outgoingExpiry, paymentSecret, nextTrampolineOnion_opt = nextPacket_opt, trampolinePathKey_opt = Some(payloadOut.nextPathKey)) + context.log.debug("forwarding payment to the next blinded trampoline node {}", recipient.nodeId) + attemptWakeUpIfRecipientIsWallet(upstream, recipient, nextPayload, nextPacket_opt) + case WrappedOutgoingNodeId(None) => + context.log.warn("rejecting trampoline payment to blinded trampoline: cannot identify next node for scid={}", payloadOut.outgoing) + rejectPayment(upstream, Some(UnknownNextPeer())) + stopping() + } + } case payloadOut: IntermediatePayload.NodeRelay.ToNonTrampoline => val paymentSecret = payloadOut.paymentSecret val features = Features(payloadOut.invoiceFeatures).invoiceFeatures() diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Recipient.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Recipient.scala index 0ea1392ac8..e3f03b08e2 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Recipient.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Recipient.scala @@ -73,6 +73,8 @@ case class ClearRecipient(nodeId: PublicKey, extraEdges: Seq[ExtraEdge] = Nil, paymentMetadata_opt: Option[ByteVector] = None, nextTrampolineOnion_opt: Option[OnionRoutingPacket] = None, + // Must be provided if the payer is using a blinded trampoline path. + trampolinePathKey_opt: Option[PublicKey] = None, customTlvs: Set[GenericTlv] = Set.empty) extends Recipient { // Feature bit used by the legacy trampoline feature. private val isLegacyTrampoline = features.unknown.contains(UnknownFeature(149)) @@ -81,8 +83,8 @@ case class ClearRecipient(nodeId: PublicKey, ClearRecipient.validateRoute(nodeId, route).map(_ => { val finalPayload = nextTrampolineOnion_opt match { case Some(trampolinePacket) if isLegacyTrampoline => NodePayload(nodeId, FinalPayload.Standard.createLegacyTrampolinePayload(route.amount, totalAmount, expiry, paymentSecret, trampolinePacket)) - case Some(trampolinePacket) => NodePayload(nodeId, FinalPayload.Standard.createTrampolinePayload(route.amount, totalAmount, expiry, paymentSecret, trampolinePacket)) - case None => NodePayload(nodeId, FinalPayload.Standard.createPayload(route.amount, totalAmount, expiry, paymentSecret, paymentMetadata_opt, trampolineOnion_opt = None, customTlvs = customTlvs)) + case Some(trampolinePacket) => NodePayload(nodeId, FinalPayload.Standard.createTrampolinePayload(route.amount, totalAmount, expiry, paymentSecret, trampolinePacket, trampolinePathKey_opt)) + case None => NodePayload(nodeId, FinalPayload.Standard.createPayload(route.amount, totalAmount, expiry, paymentSecret, paymentMetadata_opt, trampolineOnion_opt = None, customTlvs)) } Recipient.buildPayloads(PaymentPayloads(route.amount, expiry, Seq(finalPayload), None), route.hops) }) @@ -91,7 +93,7 @@ case class ClearRecipient(nodeId: PublicKey, object ClearRecipient { def apply(invoice: Bolt11Invoice, totalAmount: MilliSatoshi, expiry: CltvExpiry, customTlvs: Set[GenericTlv]): ClearRecipient = { - ClearRecipient(invoice.nodeId, invoice.features, totalAmount, expiry, invoice.paymentSecret, invoice.extraEdges, invoice.paymentMetadata, None, customTlvs) + ClearRecipient(invoice.nodeId, invoice.features, totalAmount, expiry, invoice.paymentSecret, invoice.extraEdges, invoice.paymentMetadata, None, None, customTlvs) } def validateRoute(nodeId: PublicKey, route: Route): Either[OutgoingPaymentError, Route] = { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala index d91c7d8d12..e5cde7e99c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala @@ -32,8 +32,9 @@ import fr.acinq.eclair.payment.send.TrampolinePayment.{buildOutgoingPayment, com import fr.acinq.eclair.reputation.Reputation import fr.acinq.eclair.router.Router import fr.acinq.eclair.router.Router.RouteParams -import fr.acinq.eclair.wire.protocol.{PaymentOnion, PaymentOnionCodecs} -import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, Features, Logs, MilliSatoshi, NodeParams, randomBytes32} +import fr.acinq.eclair.wire.protocol._ +import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, EncodedNodeId, Features, Logs, MilliSatoshi, MilliSatoshiLong, NodeParams, ShortChannelId, randomBytes32, randomKey} +import scodec.bits.ByteVector import java.util.UUID @@ -113,7 +114,7 @@ object TrampolinePaymentLifecycle { val add = CMD_ADD_HTLC(addHtlcAdapter.toClassic, outgoing.trampolineAmount, paymentHash, outgoing.trampolineExpiry, outgoing.onion.packet, None, Reputation.Score.max, None, origin, commit = true) channelInfo.channel ! add val channelId = channelInfo.data.asInstanceOf[DATA_NORMAL].channelId - val part = PartialPayment(cmd.paymentId, amount, computeFees(amount, attemptNumber), channelId, None) + val part = PartialPayment(cmd.paymentId, amount, computeFees(amount, attemptNumber, cmd.invoice), channelId, None) waitForSettlement(part, outgoing.onion.sharedSecrets, outgoing.trampolineOnion.sharedSecrets) } @@ -182,7 +183,7 @@ class TrampolinePaymentLifecycle private(nodeParams: NodeParams, } private def sendPayment(channels: Seq[Peer.ChannelInfo], attemptNumber: Int): Behavior[Command] = { - val trampolineAmount = computeTrampolineAmount(totalAmount, attemptNumber) + val trampolineAmount = computeTrampolineAmount(totalAmount, attemptNumber, cmd.invoice) // We always use MPP to verify that the trampoline node is able to handle it. // This is a very naive way of doing MPP that simply splits the payment in two HTLCs. val filtered = channels.flatMap(c => { @@ -251,7 +252,7 @@ class TrampolinePaymentLifecycle private(nodeParams: NodeParams, } private def retryOrStop(attemptNumber: Int): Behavior[Command] = { - val nextFees = computeFees(totalAmount, attemptNumber) + val nextFees = computeFees(totalAmount, attemptNumber, cmd.invoice) if (attemptNumber > 3) { context.log.warn("cannot retry trampoline payment: retries exceeded") cmd.replyTo ! PaymentFailed(cmd.paymentId, paymentHash, LocalFailure(totalAmount, Nil, new RuntimeException("maximum trampoline retries exceeded")) :: Nil) @@ -286,27 +287,75 @@ object TrampolinePayment { case invoice: Bolt11Invoice => val trampolinePayload = PaymentOnion.IntermediatePayload.NodeRelay.ToNonTrampoline(totalAmount, totalAmount, expiry, invoice.nodeId, invoice) buildOnion(NodePayload(trampolineNodeId, trampolinePayload) :: Nil, invoice.paymentHash, None).toOption.get + case invoice: Bolt12Invoice if invoice.features.hasFeature(Features.TrampolinePayment) => + val path = invoice.blindedPaths.head + require(path.route.firstNodeId.isInstanceOf[EncodedNodeId.WithPublicKey], "blinded path must provide the introduction node_id") + val introductionNodeId = path.route.firstNodeId.asInstanceOf[EncodedNodeId.WithPublicKey].publicKey + // We're creating blinded payloads for an outgoing payment: we don't have access to the decrypted data, so we use dummy data. + val dummyPathId = RouteBlindingEncryptedDataTlv.PathId(ByteVector.empty) + val dummyPaymentRelayData = BlindedRouteData.PaymentRelayData(TlvStream( + RouteBlindingEncryptedDataTlv.OutgoingChannelId(ShortChannelId(0)), + RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(0), 0, 0 msat), + RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(0), 0 msat) + )) + val finalPayload = NodePayload(path.route.blindedHops.last.blindedPublicKey, PaymentOnion.FinalPayload.Blinded( + records = TlvStream( + OnionPaymentPayloadTlv.AmountToForward(totalAmount), + OnionPaymentPayloadTlv.OutgoingCltv(expiry), + OnionPaymentPayloadTlv.TotalAmount(totalAmount), + OnionPaymentPayloadTlv.EncryptedRecipientData(path.route.blindedHops.last.encryptedPayload), + ), + blindedRecords = TlvStream(dummyPathId), + )) + val intermediatePayloads = path.route.blindedHops.drop(1).dropRight(1).map { b => + NodePayload(b.blindedPublicKey, PaymentOnion.IntermediatePayload.ChannelRelay.Blinded( + records = TlvStream(OnionPaymentPayloadTlv.EncryptedRecipientData(b.encryptedPayload)), + paymentRelayData = dummyPaymentRelayData, + nextPathKey = randomKey().publicKey, + )) + } + val introductionPayload = NodePayload(introductionNodeId, PaymentOnion.IntermediatePayload.ChannelRelay.Blinded( + records = TlvStream(OnionPaymentPayloadTlv.EncryptedRecipientData(path.route.encryptedPayloads.head), OnionPaymentPayloadTlv.PathKey(path.route.firstPathKey)), + paymentRelayData = dummyPaymentRelayData, + nextPathKey = randomKey().publicKey, + )) + // We use our trampoline node to reach the introduction node of the blinded path. + val blindedAmount = totalAmount + path.paymentInfo.fee(totalAmount) + val blindedExpiry = expiry + path.paymentInfo.cltvExpiryDelta + val trampolinePayload = NodePayload(trampolineNodeId, PaymentOnion.IntermediatePayload.NodeRelay.Standard(blindedAmount, blindedExpiry, introductionNodeId)) + buildOnion(trampolinePayload +: introductionPayload +: intermediatePayloads :+ finalPayload, invoice.paymentHash, None).toOption.get case invoice: Bolt12Invoice => val trampolinePayload = PaymentOnion.IntermediatePayload.NodeRelay.ToBlindedPaths(totalAmount, expiry, invoice) buildOnion(NodePayload(trampolineNodeId, trampolinePayload) :: Nil, invoice.paymentHash, None).toOption.get } - val trampolineAmount = computeTrampolineAmount(amount, attemptNumber) - val trampolineTotalAmount = computeTrampolineAmount(totalAmount, attemptNumber) - val trampolineExpiry = computeTrampolineExpiry(expiry, attemptNumber) + val trampolineAmount = computeTrampolineAmount(amount, attemptNumber, invoice) + val trampolineTotalAmount = computeTrampolineAmount(totalAmount, attemptNumber, invoice) + val trampolineExpiry = computeTrampolineExpiry(expiry, attemptNumber, invoice) val payload = trampolinePaymentSecret_opt match { - case Some(trampolinePaymentSecret) => PaymentOnion.FinalPayload.Standard.createTrampolinePayload(trampolineAmount, trampolineTotalAmount, trampolineExpiry, trampolinePaymentSecret, trampolineOnion.packet) + case Some(trampolinePaymentSecret) => PaymentOnion.FinalPayload.Standard.createTrampolinePayload(trampolineAmount, trampolineTotalAmount, trampolineExpiry, trampolinePaymentSecret, trampolineOnion.packet, trampolinePathKey_opt = None) case None => PaymentOnion.TrampolineWithoutMppPayload.create(trampolineAmount, trampolineExpiry, trampolineOnion.packet) } val paymentOnion = buildOnion(NodePayload(trampolineNodeId, payload) :: Nil, invoice.paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get OutgoingPayment(trampolineAmount, trampolineExpiry, paymentOnion, trampolineOnion) } - // We increase the fees paid by 0.2% of the amount sent at each attempt. - def computeFees(amount: MilliSatoshi, attemptNumber: Int): MilliSatoshi = amount * (attemptNumber + 1) * 0.002 + def computeFees(amount: MilliSatoshi, attemptNumber: Int, invoice: Invoice): MilliSatoshi = invoice match { + case _: Bolt11Invoice => + // We increase the fees paid by 0.2% of the amount sent at each attempt. + amount * (attemptNumber + 1) * 0.002 + case _ => + // We increase the fees paid by 1% of the amount sent at each attempt. + // That's because integration tests are much easier to write if the first attempt uses large enough fees and + // expiry because we must inject a plugin actor to respond to HandleInvoiceRequest and HandlePayment. + amount * (attemptNumber + 1) * 0.01 + } - def computeTrampolineAmount(amount: MilliSatoshi, attemptNumber: Int): MilliSatoshi = amount + computeFees(amount, attemptNumber) + def computeTrampolineAmount(amount: MilliSatoshi, attemptNumber: Int, invoice: Invoice): MilliSatoshi = amount + computeFees(amount, attemptNumber, invoice) // We increase the trampoline expiry delta at each attempt. - private def computeTrampolineExpiry(expiry: CltvExpiry, attemptNumber: Int): CltvExpiry = expiry + CltvExpiryDelta(144) * (attemptNumber + 1) + private def computeTrampolineExpiry(expiry: CltvExpiry, attemptNumber: Int, invoice: Invoice): CltvExpiry = invoice match { + case _: Bolt11Invoice => expiry + CltvExpiryDelta(144) * (attemptNumber + 1) + case _ => expiry + CltvExpiryDelta(432) + CltvExpiryDelta(144) * (attemptNumber + 1) + } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/PaymentOnion.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/PaymentOnion.scala index d322470d0a..2c91dd41e1 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/PaymentOnion.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/PaymentOnion.scala @@ -336,6 +336,36 @@ object PaymentOnion { } } + /** + * @param paymentRelayData decrypted relaying data from the encrypted_recipient_data tlv. + * @param nextPathKey path key that must be forwarded to the next hop in the outer onion. + */ + case class Blinded(records: TlvStream[OnionPaymentPayloadTlv], paymentRelayData: PaymentRelayData, nextPathKey: PublicKey) extends NodeRelay { + // @formatter:off + val isLegacy: Boolean = false + val outgoing: Either[EncodedNodeId.WithPublicKey, ShortChannelId] = paymentRelayData.outgoing + override def outgoingAmount(incomingAmount: MilliSatoshi): MilliSatoshi = paymentRelayData.amountToForward(incomingAmount) + override def outgoingExpiry(incomingCltv: CltvExpiry): CltvExpiry = paymentRelayData.outgoingCltv(incomingCltv) + // @formatter:on + } + + object Blinded { + def validate(records: TlvStream[OnionPaymentPayloadTlv], blindedRecords: TlvStream[RouteBlindingEncryptedDataTlv], nextPathKey: PublicKey): Either[InvalidTlvPayload, Blinded] = { + if (records.get[EncryptedRecipientData].isEmpty) return Left(MissingRequiredTlv(UInt64(10))) + // Bolt 4: MUST return an error if the payload contains other tlv fields than `encrypted_recipient_data` and `current_path_key`. + if (records.unknown.nonEmpty) return Left(ForbiddenTlv(records.unknown.head.tag)) + records.records.find { + case _: EncryptedRecipientData => false + case _: PathKey => false + case _ => true + } match { + case Some(_) => return Left(ForbiddenTlv(UInt64(0))) + case None => // no forbidden tlv found + } + BlindedRouteData.validatePaymentRelayData(blindedRecords).map(paymentRelayData => Blinded(records, paymentRelayData, nextPathKey)) + } + } + /** We relay to a payment recipient that doesn't support trampoline, which exposes its identity. */ case class ToNonTrampoline(records: TlvStream[OnionPaymentPayloadTlv]) extends NodeRelay { val amountToForward = records.get[AmountToForward].get.amount @@ -474,8 +504,15 @@ object PaymentOnion { } /** Create a trampoline outer payload. */ - def createTrampolinePayload(amount: MilliSatoshi, totalAmount: MilliSatoshi, expiry: CltvExpiry, paymentSecret: ByteVector32, trampolinePacket: OnionRoutingPacket): Standard = { - Standard(TlvStream(AmountToForward(amount), OutgoingCltv(expiry), PaymentData(paymentSecret, totalAmount), TrampolineOnion(trampolinePacket))) + def createTrampolinePayload(amount: MilliSatoshi, totalAmount: MilliSatoshi, expiry: CltvExpiry, paymentSecret: ByteVector32, trampolinePacket: OnionRoutingPacket, trampolinePathKey_opt: Option[PublicKey]): Standard = { + val tlvs: Set[OnionPaymentPayloadTlv] = Set( + Some(AmountToForward(amount)), + Some(OutgoingCltv(expiry)), + Some(PaymentData(paymentSecret, totalAmount)), + trampolinePathKey_opt.map(k => PathKey(k)), + Some(TrampolineOnion(trampolinePacket)) + ).flatten + Standard(TlvStream(tlvs)) } /** Create a trampoline outer payload for legacy trampoline payments. */ @@ -558,8 +595,14 @@ object PaymentOnion { } object TrampolineWithoutMppPayload { - def create(amount: MilliSatoshi, expiry: CltvExpiry, trampolinePacket: OnionRoutingPacket): TrampolineWithoutMppPayload = { - TrampolineWithoutMppPayload(TlvStream(AmountToForward(amount), OutgoingCltv(expiry), TrampolineOnion(trampolinePacket))) + def create(amount: MilliSatoshi, expiry: CltvExpiry, trampolinePacket: OnionRoutingPacket, trampolinePathKey_opt: Option[PublicKey] = None): TrampolineWithoutMppPayload = { + val tlvs: Set[OnionPaymentPayloadTlv] = Set( + Some(AmountToForward(amount)), + Some(OutgoingCltv(expiry)), + trampolinePathKey_opt.map(k => PathKey(k)), + Some(TrampolineOnion(trampolinePacket)) + ).flatten + TrampolineWithoutMppPayload(TlvStream(tlvs)) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala index cf20a9429a..9161a2b882 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala @@ -32,9 +32,10 @@ import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel.{BroadcastChannelUpdate, PeriodicRefresh} import fr.acinq.eclair.crypto.Sphinx.DecryptedFailurePacket import fr.acinq.eclair.crypto.Sphinx.RouteBlinding.BlindedRoute -import fr.acinq.eclair.crypto.TransportHandler +import fr.acinq.eclair.crypto.{Sphinx, TransportHandler} import fr.acinq.eclair.db._ import fr.acinq.eclair.io.Peer.PeerRoutingMessage +import fr.acinq.eclair.message.OnionMessages import fr.acinq.eclair.message.OnionMessages.{IntermediateNode, Recipient, buildRoute} import fr.acinq.eclair.payment._ import fr.acinq.eclair.payment.offer.OfferManager @@ -819,6 +820,41 @@ class PaymentIntegrationSpec extends IntegrationSpec { assert(holdTimesRecorderA.expectMsgType[Router.ReportedHoldTimes].holdTimes.map(_.remoteNodeId) == Seq("B").map(nodes(_).nodeParams.nodeId)) } + test("send a blinded payment D->A with trampoline (non-trampoline recipient)") { + val amount = 10_000_000 msat + val chain = nodes("A").nodeParams.chainHash + val pathId = randomBytes32() + val offerPath = OnionMessages.buildRoute(randomKey(), Seq(IntermediateNode(nodes("B").nodeParams.nodeId)), Recipient(nodes("A").nodeParams.nodeId, Some(pathId))) + val offerKey = Sphinx.RouteBlinding.derivePrivateKey(nodes("A").nodeParams.privateKey, offerPath.lastPathKey) + val offer = Offer.withPaths(Some(amount), Some("test offer"), Seq(offerPath.route), nodes("A").nodeParams.features.bolt12Features(), chain) + val offerHandler = TypedProbe[HandlerCommand]()(nodes("A").system.toTyped) + nodes("A").offerManager ! RegisterOffer(offer, Some(offerKey), Some(pathId), offerHandler.ref) + + val sender = TestProbe() + val dave = new EclairImpl(nodes("D")) + dave.payOfferTrampoline(offer, amount, 1, nodes("C").nodeParams.nodeId, maxAttempts_opt = Some(1))(30 seconds).pipeTo(sender.ref) + + nodes("A").router ! Router.FinalizeRoute(sender.ref, Router.PredefinedNodeRoute(amount, Seq(nodes("B").nodeParams.nodeId, nodes("A").nodeParams.nodeId))) + val route = sender.expectMsgType[Router.RouteResponse].routes.head + + val handleInvoiceRequest = offerHandler.expectMessageType[HandleInvoiceRequest] + val receivingRoutes = Seq(OfferManager.InvoiceRequestActor.Route(route.hops, CltvExpiryDelta(500))) + handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, receivingRoutes, pluginData_opt = Some(hex"0123")) + + val handlePayment = offerHandler.expectMessageType[HandlePayment] + assert(handlePayment.offer == offer) + assert(handlePayment.invoiceData.pluginData_opt.contains(hex"0123")) + handlePayment.replyTo ! PaymentActor.AcceptPayment() + + val paymentSent = sender.expectMsgType[PaymentSent] + assert(paymentSent.recipientAmount == amount, paymentSent) + assert(paymentSent.feesPaid >= 0.msat, paymentSent) + + awaitCond(nodes("A").nodeParams.db.payments.getIncomingPayment(paymentSent.paymentHash).exists(_.status.isInstanceOf[IncomingPaymentStatus.Received])) + val Some(IncomingBlindedPayment(_, _, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("A").nodeParams.db.payments.getIncomingPayment(paymentSent.paymentHash) + assert(receivedAmount >= amount) + } + test("send to compact route") { val probe = TestProbe() val recipientKey = randomKey() diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala index f9da074e56..008d6ced38 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala @@ -419,7 +419,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { val carol = PublicKey(hex"027f31ebc5462c1fdce1b737ecff52d37d75dea43ce11c74d25aa297165faa2007") val payloads = Seq( NodePayload(bob, PaymentOnion.IntermediatePayload.ChannelRelay.Standard(ShortChannelId.fromCoordinates("572330x7x1105").get, 100_005_000 msat, CltvExpiry(800_250))), - NodePayload(carol, PaymentOnion.FinalPayload.Standard.createTrampolinePayload(100_005_000 msat, 100_005_000 msat, CltvExpiry(800_250), paymentSecret, trampolineOnionForCarol)), + NodePayload(carol, PaymentOnion.FinalPayload.Standard.createTrampolinePayload(100_005_000 msat, 100_005_000 msat, CltvExpiry(800_250), paymentSecret, trampolineOnionForCarol, None)), ) val sessionKey = PrivateKey(hex"0404040404040404040404040404040404040404040404040404040404040404") val onionForBob = OutgoingPaymentPacket.buildOnion(sessionKey, payloads, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get.packet @@ -461,7 +461,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { val eve = PublicKey(hex"02edabbd16b41c8371b92ef2f04c1185b4f03b6dcd52ba9b78d9d7c89c8f221145") val payloads = Seq( NodePayload(dave, PaymentOnion.IntermediatePayload.ChannelRelay.Standard(ShortChannelId.fromCoordinates("572330x42x1729").get, 100_000_000 msat, CltvExpiry(800_000))), - NodePayload(eve, PaymentOnion.FinalPayload.Standard.createTrampolinePayload(100_000_000 msat, 100_000_000 msat, CltvExpiry(800_000), paymentSecret, trampolineOnionForEve)), + NodePayload(eve, PaymentOnion.FinalPayload.Standard.createTrampolinePayload(100_000_000 msat, 100_000_000 msat, CltvExpiry(800_000), paymentSecret, trampolineOnionForEve, None)), ) val sessionKey = PrivateKey(hex"0505050505050505050505050505050505050505050505050505050505050505") val onionForDave = OutgoingPaymentPacket.buildOnion(sessionKey, payloads, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get.packet @@ -482,7 +482,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { // Eve decrypts the payment onion and the inner trampoline onion. val priv = PrivateKey(hex"4545454545454545454545454545454545454545454545454545454545454545") - val add = UpdateAddHtlc(randomBytes32(), 1, 100_000_000.msat, paymentHash, CltvExpiry(800_000), onionForEve, None, Reputation.maxEndorsement, None) + val add = UpdateAddHtlc(randomBytes32(), 1, 100_000_000 msat, paymentHash, CltvExpiry(800_000), onionForEve, None, Reputation.maxEndorsement, None) val Right(FinalPacket(_, payloadForEve, _)) = decrypt(add, priv, Features.empty) assert(payloadForEve.isInstanceOf[FinalPayload.Standard]) assert(payloadForEve.amount == 100_000_000.msat) @@ -491,6 +491,305 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { assert(payloadForEve.asInstanceOf[FinalPayload.Standard].isTrampoline) } + // See bolt04/trampoline-to-blinded-path-payment-onion-test.json + test("build outgoing trampoline payment to blinded recipient (reference test vector)") { + val preimage = ByteVector32.fromValidHex("8bb624f63457695115152f4bf9950bbd14972a5f49d882cb1a68aa064742c057") + val paymentHash = Crypto.sha256(preimage) + val evePriv = PrivateKey(hex"4545454545454545454545454545454545454545454545454545454545454545") + val eve = evePriv.publicKey + + // Eve creates a blinded path to herself going through Dave. + val pathId = hex"01caa1dcc994683479a217fb32ac30e9c2b6f7960121ca169ae732477e10349b560000000008f0d1808bb624f63457695115152f4bf9950bbd14972a5f49d882cb1a68aa064742c05702414343fd4a723942a86d5f60d2cfecb6c5e8a65595c9995332ec2dba8fe004a20000000000000001000000000000000068656c6c6f188d0b54e9a7df32f0e00c62ace869bac50e0e9c77bcf4c9850bae33d5e48ada65267a3d56111e3c3d84366f3cd53d998256929b0440355dff20455396fe6aac" + val path = { + val dave = PublicKey(hex"032c0b7cf95324a07d05398b240174dc0c2be444d96b159aa6c7f7b1e668680991") + val blindedPayloadEve = TlvStream[RouteBlindingEncryptedDataTlv](RouteBlindingEncryptedDataTlv.PathId(pathId)) + val blindedPayloadDave = TlvStream[RouteBlindingEncryptedDataTlv]( + RouteBlindingEncryptedDataTlv.OutgoingNodeId(EncodedNodeId(eve)), + RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(36), 1000, 500 msat), + RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(850_000), 1 msat), + ) + assert(RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec.encode(blindedPayloadDave).require.bytes == hex"042102edabbd16b41c8371b92ef2f04c1185b4f03b6dcd52ba9b78d9d7c89c8f221145 0a080024000003e801f4 0c05000cf85001") + assert(RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec.encode(blindedPayloadEve).require.bytes == hex"06 bf 01caa1dcc994683479a217fb32ac30e9c2b6f7960121ca169ae732477e10349b560000000008f0d1808bb624f63457695115152f4bf9950bbd14972a5f49d882cb1a68aa064742c05702414343fd4a723942a86d5f60d2cfecb6c5e8a65595c9995332ec2dba8fe004a20000000000000001000000000000000068656c6c6f188d0b54e9a7df32f0e00c62ace869bac50e0e9c77bcf4c9850bae33d5e48ada65267a3d56111e3c3d84366f3cd53d998256929b0440355dff20455396fe6aac") + val sessionKey = PrivateKey(hex"090a684b173ac8da6716859095a779208943cf88680c38c249d3e8831e2caf7e") + val blindedRouteDetails = Sphinx.RouteBlinding.create(sessionKey, Seq(dave, eve), Seq(blindedPayloadDave, blindedPayloadEve).map { p => + RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec.encode(p).require.bytes + }) + assert(EncodedNodeId(dave) == blindedRouteDetails.route.firstNodeId) + assert(PublicKey(hex"02c952268f1501cf108839f4f5d0fbb41a97de778a6ead8caf161c569bd4df1ad7") == blindedRouteDetails.lastPathKey) + assert(PublicKey(hex"02988face71e92c345a068f740191fd8e53be14f0bb957ef730d3c5f76087b960e") == blindedRouteDetails.route.firstPathKey) + val blindedNodes = Seq( + PublicKey(hex"0295d40514096a8be54859e7dfe947b376eaafea8afe5cb4eb2c13ff857ed0b4be"), + PublicKey(hex"020e2dbadcc2005e859819ddebbe88a834ae8a6d2b049233c07335f15cd1dc5f22"), + ) + assert(blindedNodes == blindedRouteDetails.route.blindedNodeIds) + val encryptedPayloads = Seq( + hex"0ccf3c8a58deaa603f657ee2a5ed9d604eb5c8ca1e5f801989afa8f3ea6d789bbdde2c7e7a1ef9ca8c38d2c54760febad8446d3f273ddb537569ef56613846ccd3aba78a", + hex"bcd747394fbd4d99588da075a623316e15a576df5bc785cccc7cd6ec7b398acce6faf520175f9ec920f2ef261cdb83dc28cc3a0eeb970107b3306489bf771ef5b1213bca811d345285405861d08a655b6c237fa247a8b4491beee20c878a60e9816492026d8feb9dafa84585b253978db6a0aa2945df5ef445c61e801fb82f43d5f00716baf9fc9b3de50bc22950a36bda8fc27bfb1242e5860c7e687438d4133e058770361a19b6c271a2a07788d34dccc27e39b9829b061a4d960eac4a2c2b0f4de506c24f9af3868c0aff6dda27281c", + ) + assert(encryptedPayloads == blindedRouteDetails.route.encryptedPayloads) + val paymentInfo = OfferTypes.PaymentInfo(500 msat, 1000, CltvExpiryDelta(36), 1 msat, 500_000_000 msat, ByteVector.empty) + PaymentBlindedRoute(blindedRouteDetails.route, paymentInfo) + } + + // Alice creates a trampoline onion using Eve's blinded path and starting at Carol (Carol -> Dave -> Eve). + val trampolineOnion = { + val carol = PublicKey(hex"027f31ebc5462c1fdce1b737ecff52d37d75dea43ce11c74d25aa297165faa2007") + val dave = PublicKey(hex"032c0b7cf95324a07d05398b240174dc0c2be444d96b159aa6c7f7b1e668680991") + assert(EncodedNodeId(dave) == path.route.firstNodeId) + val payloadEve = PaymentOnion.FinalPayload.Blinded( + records = TlvStream( + OnionPaymentPayloadTlv.AmountToForward(150_000_000 msat), + OnionPaymentPayloadTlv.OutgoingCltv(CltvExpiry(800_000)), + OnionPaymentPayloadTlv.TotalAmount(150_000_000 msat), + OnionPaymentPayloadTlv.EncryptedRecipientData(path.route.encryptedPayloads.last), + ), + blindedRecords = TlvStream(RouteBlindingEncryptedDataTlv.PathId(pathId)), + ) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadEve.records).require.bytes == hex"e4 020408f0d180 04030c3500 0ad1bcd747394fbd4d99588da075a623316e15a576df5bc785cccc7cd6ec7b398acce6faf520175f9ec920f2ef261cdb83dc28cc3a0eeb970107b3306489bf771ef5b1213bca811d345285405861d08a655b6c237fa247a8b4491beee20c878a60e9816492026d8feb9dafa84585b253978db6a0aa2945df5ef445c61e801fb82f43d5f00716baf9fc9b3de50bc22950a36bda8fc27bfb1242e5860c7e687438d4133e058770361a19b6c271a2a07788d34dccc27e39b9829b061a4d960eac4a2c2b0f4de506c24f9af3868c0aff6dda27281c 120408f0d180") + val payloadDave = PaymentOnion.IntermediatePayload.NodeRelay.Blinded( + records = TlvStream( + OnionPaymentPayloadTlv.EncryptedRecipientData(path.route.encryptedPayloads.head), + OnionPaymentPayloadTlv.PathKey(path.route.firstPathKey), + ), + paymentRelayData = BlindedRouteData.PaymentRelayData(TlvStream( + RouteBlindingEncryptedDataTlv.OutgoingNodeId(EncodedNodeId(eve)), + RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(36), 1000, 500 msat), + RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(850_000), 1 msat), + )), + nextPathKey = randomKey().publicKey + ) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadDave.records).require.bytes == hex"69 0a440ccf3c8a58deaa603f657ee2a5ed9d604eb5c8ca1e5f801989afa8f3ea6d789bbdde2c7e7a1ef9ca8c38d2c54760febad8446d3f273ddb537569ef56613846ccd3aba78a 0c2102988face71e92c345a068f740191fd8e53be14f0bb957ef730d3c5f76087b960e") + val payloadCarol = PaymentOnion.IntermediatePayload.NodeRelay.Standard(150_150_500 msat, CltvExpiry(800_036), dave) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadCarol.records).require.bytes == hex"2e 020408f31d64 04030c3524 0e21032c0b7cf95324a07d05398b240174dc0c2be444d96b159aa6c7f7b1e668680991") + val sessionKey = PrivateKey(hex"a64feb81abd58e473df290e9e1c07dc3e56114495cadf33191f44ba5448ebe99") + val trampolineOnion = OutgoingPaymentPacket.buildOnion(sessionKey, NodePayload(carol, payloadCarol) :: NodePayload(dave, payloadDave) :: NodePayload(path.route.blindedNodeIds.last, payloadEve) :: Nil, paymentHash, None).toOption.get.packet + val encoded = OnionRoutingCodecs.onionRoutingPacketCodec(trampolineOnion.payload.length.toInt).encode(trampolineOnion).require.bytes + assert(encoded == hex"0002bc59a9abc893d75a8d4f56a6572f9a3507323a8de22abe0496ea8d37da166a8b4bba0e560f1a9deb602bfd98fe9167141d0b61d669df90c0149096d505b85d3d02806e6c12caeb308b878b6bc7f1b15839c038a6443cd3bec3a94c2293165375555f6d7720862b525930f41fddcc02260d197abd93fb58e60835fd97d9dc14e7979c12f59df08517b02e3e4d50e1817de4271df66d522c4e9675df71c635c4176a8381bc22b342ff4e9031cede87f74cc039fca74aa0a3786bc1db2e158a9a520ecb99667ef9a6bbfaf5f0e06f81c27ca48134ba2103229145937c5dc7b8ecc5201d6aeb592e78faa3c05d3a035df77628f0be9b1af3ef7d386dd5cc87b20778f47ebd40dbfcf12b9071c5d7112ab84c3e0c5c14867e684d09a18bc93ac47d73b7343e3403ef6e3b70366835988920e7d772c3719d3596e53c29c4017cb6938421a557ce81b4bb26701c25bf622d4c69f1359dc85857a375c5c74987a4d3152f66987001c68a50c4bf9e0b1dab4ad1a64b0535319bbf6c4fbe4f9c50cb65f5ef887bfb91b0a57c0f86ba3d91cbeea1607fb0c12c6c75d03bbb0d3a3019c40597027f5eebca23083e50ec79d41b1152131853525bf3fc13fb0be62c2e3ce733f59671eee5c4064863fb92ae74be9ca68b9c716f9519fd268478ee27d91d466b0de51404de3226b74217d28250ead9d2c95411e0230570f547d4cc7c1d589791623131aa73965dccc5aa17ec12b442215ce5d346df664d799190df5dd04a13") + trampolineOnion + } + + // Alice creates a payment onion for Carol (Alice -> Bob -> Carol). + val onionForBob = { + val sessionKey = PrivateKey(hex"4f777e8dac16e6dfe333066d9efb014f7a51d11762ff76eca4d3a95ada99ba3e") + val bob = PublicKey(hex"0324653eac434488002cc06bbfb7f10fe18991e35f9fe4302dbea6d2353dc0ab1c") + val carol = PublicKey(hex"027f31ebc5462c1fdce1b737ecff52d37d75dea43ce11c74d25aa297165faa2007") + val paymentSecret = ByteVector32.fromValidHex("7494b65bc092b48a75465e43e29be807eb2cc535ce8aaba31012b8ff1ceac5da") + val payloadBob = PaymentOnion.IntermediatePayload.ChannelRelay.Standard(ShortChannelId.fromCoordinates("572330x42x2821").get, 150_153_000 msat, CltvExpiry(800_060)) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadBob.records).require.bytes == hex"15 020408f32728 04030c353c 060808bbaa00002a0b05") + val payloadCarol = PaymentOnion.FinalPayload.Standard.createTrampolinePayload(150_153_000 msat, 150_153_000 msat, CltvExpiry(800_060), paymentSecret, trampolineOnion, None) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadCarol.records).require.bytes == hex"fd0255 020408f32728 04030c353c 08247494b65bc092b48a75465e43e29be807eb2cc535ce8aaba31012b8ff1ceac5da08f32728 14fd02200002bc59a9abc893d75a8d4f56a6572f9a3507323a8de22abe0496ea8d37da166a8b4bba0e560f1a9deb602bfd98fe9167141d0b61d669df90c0149096d505b85d3d02806e6c12caeb308b878b6bc7f1b15839c038a6443cd3bec3a94c2293165375555f6d7720862b525930f41fddcc02260d197abd93fb58e60835fd97d9dc14e7979c12f59df08517b02e3e4d50e1817de4271df66d522c4e9675df71c635c4176a8381bc22b342ff4e9031cede87f74cc039fca74aa0a3786bc1db2e158a9a520ecb99667ef9a6bbfaf5f0e06f81c27ca48134ba2103229145937c5dc7b8ecc5201d6aeb592e78faa3c05d3a035df77628f0be9b1af3ef7d386dd5cc87b20778f47ebd40dbfcf12b9071c5d7112ab84c3e0c5c14867e684d09a18bc93ac47d73b7343e3403ef6e3b70366835988920e7d772c3719d3596e53c29c4017cb6938421a557ce81b4bb26701c25bf622d4c69f1359dc85857a375c5c74987a4d3152f66987001c68a50c4bf9e0b1dab4ad1a64b0535319bbf6c4fbe4f9c50cb65f5ef887bfb91b0a57c0f86ba3d91cbeea1607fb0c12c6c75d03bbb0d3a3019c40597027f5eebca23083e50ec79d41b1152131853525bf3fc13fb0be62c2e3ce733f59671eee5c4064863fb92ae74be9ca68b9c716f9519fd268478ee27d91d466b0de51404de3226b74217d28250ead9d2c95411e0230570f547d4cc7c1d589791623131aa73965dccc5aa17ec12b442215ce5d346df664d799190df5dd04a13") + val onionForBob = OutgoingPaymentPacket.buildOnion(sessionKey, NodePayload(bob, payloadBob) :: NodePayload(carol, payloadCarol) :: Nil, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get.packet + val encoded = PaymentOnionCodecs.paymentOnionPacketCodec.encode(onionForBob).require.bytes + assert(encoded == hex"00025fd60556c134ae97e4baedba220a644037754ee67c54fd05e93bf40c17cbb73362fb9dee96001ff229945595b6edb59437a6bc143406d3f90f749892a84d8d430c6890437d26d5bfc599d565316ef51347521075bbab87c59c57bcf20af7e63d7192b46cf171e4f73cb11f9f603915389105d91ad630224bea95d735e3988add1e24b5bf28f1d7128db64284d90a839ba340d088c74b1fb1bd21136b1809428ec5399c8649e9bdf92d2dcfc694deae5046fa5b2bdf646847aaad73f5e95275763091c90e71031cae1f9a770fdea559642c9c02f424a2a28163dd0957e3874bd28a97bec67d18c0321b0e68bc804aa8345b17cb626e2348ca06c8312a167c989521056b0f25c55559d446507d6c491d50605cb79fa87929ce64b0a9860926eeaec2c431d926a1cadb9a1186e4061cb01671a122fc1f57602cbef06d6c194ec4b715c2e3dd4120baca3172cd81900b49fef857fb6d6afd24c983b608108b0a5ac0c1c6c52011f23b8778059ffadd1bb7cd06e2525417365f485a7fd1d4a9ba3818ede7cdc9e71afee8532252d08e2531ca52538655b7e8d912f7ec6d37bbcce8d7ec690709dbf9321e92c565b78e7fe2c22edf23e0902153d1ca15a112ad32fb19695ec65ce11ddf670da7915f05ad4b86c154fb908cb567315d1124f303f75fa075ebde8ef7bb12e27737ad9e4924439097338ea6d7a6fc3721b88c9b830a34e8d55f4c582b74a3895cc848fe57f4fe29f115dabeb6b3175be15d94408ed6771109cfaf57067ae658201082eae7605d26b1449af4425ae8e8f58cdda5c6265f1fd7a386fc6cea3074e4f25b909b96175883676f7610a00fdf34df9eb6c7b9a4ae89b839c69fd1f285e38cdceb634d782cc6d81179759bc9fd47d7fd060470d0b048287764c6837963274e708314f017ac7dc26d0554d59bfcfd3136225798f65f0b0fea337c6b256ebbb63a90b994c0ab93fd8b1d6bd4c74aebe535d6110014cd3d525394027dfe8faa98b4e9b2bee7949eb1961f1b026791092f84deea63afab66603dbe9b6365a102a1fef2f6b9744bc1bb091a8da9130d34d4d39f25dbad191649cfb67e10246364b7ce0c6ec072f9690cabb459d9fda0c849e17535de4357e9907270c75953fca3c845bb613926ecf73205219c7057a4b6bb244c184362bb4e2f24279dc4e60b94a5b1ec11c34081a628428ba5646c995b9558821053ba9c84a05afbf00dabd60223723096516d2f5668f3ec7e11612b01eb7a3a0506189a2272b88e89807943adb34291a17f6cb5516ffd6f945a1c42a524b21f096d66f350b1dad4db455741ae3d0e023309fbda5ef55fb0dc74f3297041448b2be76c525141963934c6afc53d263fb7836626df502d7c2ee9e79cbbd87afd84bbb8dfbf45248af3cd61ad5fac827e7683ca4f91dfad507a8eb9c17b2c9ac5ec051fe645a4a6cb37136f6f19b611e0ea8da7960af2d779507e55f57305bc74b7568928c5dd5132990fe54c22117df91c257d8c7b61935a018a28c1c3b17bab8e4294fa699161ec21123c9fc4e71079df31f300c2822e1246561e04765d3aab333eafd026c7431ac7616debb0e022746f4538e1c6348b600c988eeb2d051fc60c468dca260a84c79ab3ab8342dc345a764672848ea234e17332bc124799daf7c5fcb2e2358514a7461357e1c19c802c5ee32deccf1776885dd825bedd5f781d459984370a6b7ae885d4483a76ddb19b30f47ed47cd56aa5a079a89793dbcad461c59f2e002067ac98dd5a534e525c9c46c2af730741bf1f8629357ec0bfc0bc9ecb31af96777e507648ff4260dc3673716e098d9111dfd245f1d7c55a6de340deb8bd7a053e5d62d760f184dc70ca8fa255b9023b9b9aedfb6e419a5b5951ba0f83b603793830ee68d442d7b88ee1bbf6bbd1bcd6f68cc1af") + onionForBob + } + + // Bob decrypts the onion and relays to Carol. + val onionForCarol = { + val bobPriv = PrivateKey(hex"4242424242424242424242424242424242424242424242424242424242424242") + val add = UpdateAddHtlc(randomBytes32(), 1, 150_155_000 msat, paymentHash, CltvExpiry(800_100), onionForBob, None, Reputation.maxEndorsement, None) + val Right(packetForBob: ChannelRelayPacket) = decrypt(add, bobPriv, Features.empty) + assert(packetForBob.payload.outgoing.contains(ShortChannelId.fromCoordinates("572330x42x2821").get)) + assert(packetForBob.amountToForward == 150_153_000.msat) + assert(packetForBob.outgoingCltv == CltvExpiry(800_060)) + val encoded = PaymentOnionCodecs.paymentOnionPacketCodec.encode(packetForBob.nextPacket).require.bytes + assert(encoded == hex"0003dc6a4c9b34bdcd2191fc4dacfc1aeb20f71991acbd17847b9ab17d69579c1614da1276d18820e55534d7352839caa436aa79a6d5be26c6ecbd1c79f74442b1b7bf8ed8b739b736b9248769c2c422eebc85fb0d580e9561b1cda1be3fd4cfa6ed0d839a2feb878acf686b112febaae9c1494a2ad20d97b2d2f7e59d6d296a104ba1b29e5d06d7a7d0279e627c51d6eed9c6a56bdbde81b22dc92e07e151546fc9568d4b27d0a1e217c3caf42e8f6e629829ee76e2a077fd1eb38dcce22217458c529cc7a9df4adc507ead08b267a722cb6b06cfa3d2a35b6387f878fd7b18e6debe7c2ff03603687adfc654606756b2c609a891956a9f4c2918d625632833ca371fe605da31a10044393c240bf4db8a1e413da7f8a6ea9dc80b0031e0dc43c1f5922ca5003e87f405ef73fc492bc813962192e3c4c4801c0f03baab2e0aba3a6cd101f8d09de15c027bff835beacc5ba09420323f4d5f75e6818939fbe02cb2aaa3e6651d512eee37ded2f27406974a3fedf77c8364beb2ce60d869b0de5ce33c466406b45e5791e189f0795c623d786d794c3d9b927b9fb7fde99df2b4359da128496ceb1a336049f05f06937c45d0fd90f10f7654ea4bb5734d5a9a3e0b2ace8ac771494a6e442fedd2314772e761704ad16f8aa9a16832b30987535c43963e880acda194119407de24fcf23558596a2848d6c98430f504e9281127b2bf649a25c6c1d35783d509f17fec8c2c0ee4004778d66822db24f01bc3361454efb60cde453c6cfe33f2f0177a3b0e326a077a0d8c433f8a21613fb62ceb2f314aa69f81b7756e76ac9f6c6dcfdace9a320e1afc738b223d192fcbc0adf5cd84f4cc161abe0455c26cdaa1270b2823fedbe7ee982ca5af8c8b8bec0ba90c3bb65a8e1ee89cc6c44114d658fb89985c7ca8e8eeb32bcf3fedae62330eff3da9654fda0a58281480d4be76c916889b2db9210e3a66c9ccaa3f06232150d5d96cbf6c18916d603e1495ab6f17baacb5005ed5ec17864c2da1bb400e6e68975cc84325e18215a6052313c3c75e25163648c840506916c2d845760063d6a4385df4f54bd0ec5da029b837202e45e399d1ee794160f49cd6d0149457d3efc2537e2ec36ce6a02727a5104d6f37fd612fec4e96f169f4f7d66706ea7d9ba344bef8e2e57664bde30f26249664bdc3eeb1dfa88a9f33e6d790581b67d57c30255d43624751f269da3f98df459a6bafe6e37d62be589eb938d8d223c7e80038a8dae2313126822fe16771a6ab6598d15bc350ddd76180e0963b5765834365254c611e2de46bde204a0377e13dab44afdacf5d77465dc035ca1ff70603f5887a023cca650ab9e3b4244f3a1870bb07b2556e3bcd47fafa3adec659aee17a881310c208b2d696dc14fbdd2209d89e7e61bbee19263ea98eb994eef0dc97e2ae0e56a6fc9592f9e27de5a22d749c9dbed19f2b2b8602ff890e82fedbdb41e019fbaf74256d6bcaf31538fff1329ed30b4b7bd991e9f1b5c6ef4f119387fcb7875f4f21f2d39b0ad01500201da644158e1260a58b4bcd1712ca3bc6e093951424452197b4fb3ac2aaf16e70fcedddd9fa96fcf46c2d60cb40a64c807fffb2448672c5bb2afce2205fa1d356d4cdb907a25b82c27e4fb735375c1f532fbbdd43c415a27e603cf15ff7f00f1ac96f346c2dcc00ffeb682db175b912cb5356f7528a834fe84df2df7453e34dd01008a087e799c18656eefe3038a03c71803bbc0990cd50f4a413329e6f779107d57158e78886728a9fa039c385abc92e230179051a02727402e7a613364b48cd93b6a26cb6888d5c4cc1d3c6a39cb442c2de12bb1ad396205c9a10023d7edc951fc60a28813867f635c3d74d4206f398849e94750b98ba43c5faca8502bf46929e3c0debfac32fc4e4a09c2436a0590cd53c") + packetForBob.nextPacket + } + + // Carol decrypts the onion and relays to Dave, without knowing that a blinded path starts at Dave. + val onionForDave = { + val carolPriv = PrivateKey(hex"4343434343434343434343434343434343434343434343434343434343434343") + val dave = PublicKey(hex"032c0b7cf95324a07d05398b240174dc0c2be444d96b159aa6c7f7b1e668680991") + val add = UpdateAddHtlc(randomBytes32(), 1, 150_153_000 msat, paymentHash, CltvExpiry(800_060), onionForCarol, None, Reputation.maxEndorsement, None) + val Right(RelayToTrampolinePacket(_, payload, trampolinePayload, trampolineOnionForDave, _)) = decrypt(add, carolPriv, Features.empty) + assert(payload.amount == 150_153_000.msat) + assert(payload.expiry == CltvExpiry(800_060)) + assert(payload.paymentSecret == ByteVector32.fromValidHex("7494b65bc092b48a75465e43e29be807eb2cc535ce8aaba31012b8ff1ceac5da")) + assert(trampolinePayload.amountToForward == 150_150_500.msat) + assert(trampolinePayload.outgoingCltv == CltvExpiry(800_036)) + assert(trampolinePayload.outgoingNodeId == dave) + val sessionKey = PrivateKey(hex"e4acea94d5ddce1a557229bc39f8953ec1398171f9c2c6bb97d20152933be4c4") + val paymentSecret = ByteVector32.fromValidHex("d1818c04937ace92037fc46e57bafbfc7ec09521bf49ff1564b10832633c2a93") + val payloadDave = PaymentOnion.FinalPayload.Standard.createTrampolinePayload(150_150_500 msat, 150_150_500 msat, CltvExpiry(800_036), paymentSecret, trampolineOnionForDave, None) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadDave.records).require.bytes == hex"fd0255 020408f31d64 04030c3524 0824d1818c04937ace92037fc46e57bafbfc7ec09521bf49ff1564b10832633c2a9308f31d64 14fd02200003868a4ec9883bc8332dcd63bafc80086fe046bf2795c070445381f98f5b4678efc8c102fee084102c1ffb91cae87bbbdce3ef59e68af26deae97af39879713b71df2c31e56acbddf7cc8f85214162785839e981a3abb51749d7cab6e09956a7e384fa658323e144293c7328f6f9097705b05eed2f11107cafdf4f6f6de7a53512e192276386c83f91809462f8f2737b8729d35ce145999770edae36808757db3aa3e77dbd8dc517fb0437e2660b16ef728fbcadf7d7f3cb4395924d1bb50a14ce8ba68635e73a7fa3d55f2a9fa796635a8a1dc6c1a3b72c491d4b1fd5fe642e6decb93d28223e79e4a69ffe71bc6e595b949e4071a2ffa65bd9099d6af7bf7f26065f032969ce33b78195cc741e2c97f801311368aee7e75159de00f6dc2b0b2b2e77c583ce8fe4ae61b774491dfefacc2aa3dfb99d6d00689a344def2086405caa2e2dc2126dc7b47750f3393f492c8b5c96bcd609e1c56a2d713ec9f6c0618a33ddfb20f2f3cbe355424292de47b6374bc012390a433e02f31cfa8a9817bf6a5597ac42b063e1cf3aaf6666b5d420600c8fc8ce689678bd802ac3815f9aaf6a48d0d3a7f940f621bd74d3e738b40c4c67f5b54b258e57d15584cf84ee2ad61c8a1fabb0e035fdb67f92f54f14797fd20bddee25d3a1ea9982757778c311f77dba90013d37780535acc4ef2281ebabf1736cb188fe7f08dc861d61a4135f295d85eb02e3a8f0015c6bcd206c7b5162f0696c1d69a06e42918dbe8fd9affb") + val onionForDave = OutgoingPaymentPacket.buildOnion(sessionKey, NodePayload(dave, payloadDave) :: Nil, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get.packet + val encoded = PaymentOnionCodecs.paymentOnionPacketCodec.encode(onionForDave).require.bytes + assert(encoded == hex"0003626d5fa7ed0b8706f975ca58ae2c645889514153568aaab7835325ecd3739a57cd1f65ed63fff058e29e81f660c134ea43937b9d93fb0caf3846b176734c292b7c7fd7a576391e63c0fe0023bdeabcc9971f4a44db079bf8203b0a52ec47c86653207ea6596c8d029793acd2327ff788dcf8a9ee424c2aa7798c95ce53fae6c9be85a0d276ff20c814c26f6e4f19a2152ffbd931a9d054976cafe556b00365ef54edc9a3f8021c73f17884ef209be13e7710df3b93efa8edd2b4eda306766f6dabb517ed720076cc3e7395373dc81e018f347109bcbc0265f0d5098489f83212a2e5c1583c8411dea509a17735713e8cc66d4a89cd56d0a35e0d3f59553b6b017eb5c9b6dcd724ca79c0d4aa5706a34503895655d2fc186ec772a17ba46e8961c37c61fe173f448d83efd1c6f78ce5f64fdfa352c8ab1c7d3aeddeeacc409a92fe87f86a07cfec33c5a7a2f3834d8dfba1291bed88271cb13c210fbfb1ced93440bee260f6335044a95d7aaa14bed06e1d3b9ec7c82db822543ffa4eff34e88a715ff8f8a23017982c6ae882ee18e0bdabba6f7d0e4285d034a2fd2a903b9ccfce534f9fba5acdc91f91d723ae359b9c2823b7e24dd2ee8d7ca3f6384976f5d319172785c948964426332c811682a643ce8c9575431e84c9af7fcf0dfdcdc67bc3e0b2415719e5a0da868dafef91be595c280bdb67e1d586c184c71617bddddfc653f4a1083658a96fe86cfa0eb93f9182fddccbb64f9e9e2b8f0c4b6edeb2aa81f3e16a24ffce789d183e0fa3689e437a1180b44dac0a5bb3bacd1ab0dab53c55194419c0e194f31ca683cf7c9b3ef304ff19dd490a6ad80233c1855b680f34e0eb2d252689ad5cd1750a793529a83b194e7d410f6cf027ae78f94b5d0405feeef397e272050d27581bb996eab562cd71d6aec3e4a793b93c950dcbadeb1d8cef4e9b5a466a06f1051f0eab1d08896c3eeb20d55118dec43ac5dcb8e90e1e3bcb4a67c419c4d825111ff450a6241b31087ef70f2112da8940a834441f2e0f7883eed5ef2dd09d57c23eec75ccf443ac02197f2f6cbf8c47fd8753cca90e1375c2c04dc985500f3fe147f72121bc1020f430fb199161897d38765bf0480e8d0505cb09eb6cefc842eb93edbf7d7a99c1f9f7f09db3e6dd3e5116f29d1b1497f940fc341b6ed90f187c68a14b00a845303a248187f7699ab9ebb0e6c9355ade7772703faa4380cfd88c9600f2147747b402d3621d2410f3a6c60367b3bed950186f966db77c58d83bb29305748b8fbf6da1e8d3d1d7251ce0812170e999fa35128cada2ff53d7cb42f37ebc3e758c6f571eb3c8e94f3535d422ebd1b11788c9ac75292d15b759a612250c97efa01fa869187cf8cdadf95ebccafa18ca9b40076828d459d7a295528fd3c77c5f3978d7fa6244466a056ebdf59902907b13bd6255904aee68cbbb46a81ce1f5cd541a3935a2229d0501f1c272624157501841eaaf6703b6b40a12a010e68bb563e2794ed7d3976ff9b59bb1f6719b6d06a0cd3d561bdcd0b2761ccbec1c4decb2faf13065ba05f633d114cdf8d61fd33c3d6f149319adc4b0367df11b77c92c75a57de20bb8775d0582be511e6139b27bea77e60e7180ba292a94944f87a91a20cabc8346ccde196e74b5bfde5a613ea5536292971cd3737980efac61d8189926b14cc1f5ea553b0f41afc2c3f6bc7d19f078cff09b2181d1b9dd068ac8a8116dd96a418c1e24f7cf22c54f7dd7c40812b7e36805d7adccbf5c8a703e3891c05caf6b4434192940ea2f164d0ae84355d1a33859d45107abcd41598da0afa4fe5f8c5a3ecb9b4857ac0e736fc76b4f51325391530a645618656a3dae74cba34a34bb7a3c0e9523f6db6b31694e8ceb9c67ae658af7db5a4c8de9e8322c3172fed09f27aa4420ae9a0a") + onionForDave + } + + // Dave decrypts the onion and blinded path data and relays to Eve. + val onionForEve = { + val davePriv = PrivateKey(hex"4444444444444444444444444444444444444444444444444444444444444444") + val add = UpdateAddHtlc(randomBytes32(), 3, 150_150_500 msat, paymentHash, CltvExpiry(800_036), onionForDave, None, Reputation.maxEndorsement, None) + val Right(RelayToBlindedTrampolinePacket(_, payload, innerPayload, trampolineOnionForEve, _)) = decrypt(add, davePriv, Features(Features.RouteBlinding -> FeatureSupport.Optional)) + assert(payload.amount == 150_150_500.msat) + assert(payload.expiry == CltvExpiry(800_036)) + assert(payload.paymentSecret == ByteVector32.fromValidHex("d1818c04937ace92037fc46e57bafbfc7ec09521bf49ff1564b10832633c2a93")) + assert(innerPayload.outgoing == Left(EncodedNodeId.WithPublicKey.Plain(eve))) + assert(innerPayload.outgoingAmount(add.amountMsat) == 150_000_000.msat) + assert(innerPayload.outgoingExpiry(add.cltvExpiry) == CltvExpiry(800_000)) + val sessionKey = PrivateKey(hex"cfeb31c76b7b6905be8da966ce2d9a87e3abbb03d236d7346c2852862c87a4b8") + val paymentSecret = ByteVector32.fromValidHex("1221f15a9dece128347dac673d6171be13b3d92c9c77ff581506507045a1d2e8") + val payloadEve = PaymentOnion.FinalPayload.Standard.createTrampolinePayload(150_000_000 msat, 150_000_000 msat, CltvExpiry(800_000), paymentSecret, trampolineOnionForEve, Some(innerPayload.nextPathKey)) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadEve.records).require.bytes == hex"fd0278 020408f0d180 04030c3500 08241221f15a9dece128347dac673d6171be13b3d92c9c77ff581506507045a1d2e808f0d180 0c2102c952268f1501cf108839f4f5d0fbb41a97de778a6ead8caf161c569bd4df1ad7 14fd022000038da50a45c30a086668ad1c34a23c11ee94bf0b4e7b8b8b184b7914645aef9e1ecf8f95df787c87965210644a84d1da8baf1731a02d0a292ae9c6e685a36e0e1679a8e0c38c27de47014966aaacfea446571ddaf7afcff7c3517e7bf57f87388720a1f226cc9ba1f670396435163a6872d39d2460adafdefb355bc5a89d51e62d427aac45e40b18d2b34587ca19753a8a0a7d704e38c190034b0c5b253bd566e20845a22e81d2d6a74071dfdfefe6fceb555f3d52a7f7d6b99e8e74a6cf4893f7374b473e28e62c9d99fc386ed220dd0ecc50274883d9f6a63e4aabdc1d6604827367dd3b3ddf233c2a8a7d577bf75736ca77c5d7d43f85db51c7cc6e33513225428e525ac0c22f6ef6c509e4ebbe4074f1fb726a8fd1e8643893e9fa38ae1eb6fd761e7fb12db8d3f20b5b26483b3fb92e6eb9fabd647870ddc39d61de48bdc39ce26eedf2f4d8da60adc13876844ddda3cc902792a8bd113980011279cddc625b9bcda8b0cc91cacaa4061d565a0b6e5daecf21ef3ce1be4d195c28ddc7337754e1d58908c4d8ffb45d0fbe936b83beb9851b88e57026c80e3e6d7b5b984785b4dd67498f86a9afcfc0548837b87ce07ef524696b68dc5a42312588dd051ea608f46dec1613c558e11d64e32c5cfd6b0e1c93691c724b257033d93dc7fffebca7f494d2b6391492985eac16d6919dcf60f1ab49e6ae216c90776b48ace0404128313220af7b6e546d1b89ab356cab83059301ae2d3a0eff524a610649c8") + val onionForEve = OutgoingPaymentPacket.buildOnion(sessionKey, NodePayload(eve, payloadEve) :: Nil, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get.packet + val encoded = PaymentOnionCodecs.paymentOnionPacketCodec.encode(onionForEve).require.bytes + assert(encoded == hex"000318a5a814e6e22fa5e938e671528091a27ff4883b81353560bb24d65cf9c245e880d563cfd8717f5ffae349d5ebc982b674d752a867e515855af157e325294ce929e8a5418faa5c2c7913f9440d4aba5087aee5e1499f239b392fac71a935636b3bbf6146aaa995db6e090c3216191f5c0662796277854c618bfadac769cacf34c89e05435b0f739f505f8b1020055497bdfafd88cd35d35ad2e0657aadfa8cc47647ccadae9dde5ddedd4e2b3d9c90074406449bf51e22fcae388bab417573e3f104a80913305bc1f3078e2badb8cf519b77ebd5433d5d12ccb9ff420ae6451b1368fa648213fc831b456a700fcd88a387c5cdf1816b9a88dd696ee2bc9035cf19998740813b50ae131920a2879026fbdda8caea065950ee3288e875d9001b80c0f915d8ef3666473f8a1a2233ffb349a8b5d292cef9011deada08d839ca2a2b0f4910edb68d6ae4d3ee08ef97143aee5a6afcac6036d5be5bac5591ce46dd645438340839fff8e7e7ad6ec20bca4335ef8dab6e0e1ddf861352e33bb63c6d5a3531a68abd0b185d0ebe1545cb10b0ef4461068b3e8e95a549ac9756ce44653593bb198241e4169c3f419c3f0903d36220ad849323495606366e1adb3dc7db0ac6cbe7a43f2f954534d1eb4d8f617369e7ea4f281bcac1f8ff7e13625f6518d4e255f3695528afa24cc237ba69c4452b0645b95ff6465cab30ca6afd21f1957f331df501c997f2eb17e6e3c17619dc40671d29f68ab4844926c1d513c3a1209c8da2e9b4bac4c740ae3bc6953f1afa406c6b5999a9e2d72e41073171246fd66f52397c83cb9027e59f0d940b01596f5c631e43fdb5b320142c0e595383e39d0a333535b0c6ff9b9f8d3f892797c992e582a60f974788ffcaf252c691c550c56e9d43b6ca0278da91ef8b84eaf640d4551b7cc94c1b78968d3df85798fef2382353854ebba9c11da5d541535e2aca9d4adcc60fe9b978b9b096e35e3ffb7f740fc27721dab063db98100d581259f20a79fe66629fbe92c3576dfef5f8e23724bd68689553c88065ac19af4cabd19a6e7566278d212407884cd294947973acfb872788e6f7c2372410db55b1d6d7713c639957e2161dfd35c487ec706290e85a0bd0f12ac7f20c1cbfc12e1d13049d105d421a260b1b095c161df77f00eb74fc451c1bc3fee1ba6bb5ac0236ae249f11ee5f7b21e3e41c84fc423c9692f60a2cb4ad707757104b058033e57c3b23a6ef920a2a14efd1c39171c3085cb4f7f27fe93b6b8cd374d92b615406ca2b3dfa860dcebc22fb55a2c8a92e706d84798bf4268656c8ffd10436897b2f26ad4e7389915d7c10f82d20075a02eb24af6ef6d5e823837470c0c1ccaf6ae43d64a7ce1e0427b89e571ec38ad1d718065f6896656018f5ddb6189d8f60ad7f4e9218473cd203c8341259ee6a2ae1446057a493be99132521a3ee26944a67642ba6ee9071e9b15f6d644c93bb82bc4543105a36284c459e91ec5519afcea79f5b8b9485e1cabf3b551f959bd9664b3c301ee6fe2f562ce378cf570ddf3da5c35ec0273c9fe5f6b86c54298aa77c1bee5f20b77ee97d1928fb3684939768364b28313dafb7fad9fa690a882e52ddef1e6ae6730b55a1267ff7b05a92fa4ad77b60439b4a7b549a6f22130867da882c25ff512d5949702a72477c1b0c2b4d919eb92858eb7e67cfacb0ee368e278898d4dff3b489345a314502ad852a7037a208f143f240a3315a5d432c51ae4510e343df0d111d689963b624b4628e8e0a1604704f1778084e07807496d00d94d529284f55a81ee8de5077229501e7c02e80b7f82ce3c649246672c6cab48f0407e0e09772135524204bede73e3ab870edc6c8346f152ae6667fa381bad766a3312a17ab41cb059ad20be93f01d8fd59741e8871f9c6d0b1bf8dbfd042") + onionForEve + } + + // Eve receives the payment. + val add = UpdateAddHtlc(randomBytes32(), 1, 150_000_000 msat, paymentHash, CltvExpiry(800_000), onionForEve, None, Reputation.maxEndorsement, None) + val Right(FinalPacket(_, payload, _)) = decrypt(add, evePriv, Features(Features.RouteBlinding -> FeatureSupport.Optional)) + assert(payload.isInstanceOf[FinalPayload.Blinded]) + assert(payload.asInstanceOf[FinalPayload.Blinded].pathId == pathId) + assert(payload.asInstanceOf[FinalPayload.Blinded].amount == 150_000_000.msat) + assert(payload.asInstanceOf[FinalPayload.Blinded].totalAmount == 150_000_000.msat) + assert(payload.asInstanceOf[FinalPayload.Blinded].expiry == CltvExpiry(800_000)) + } + + // See bolt04/trampoline-to-blinded-path-payment-onion-test.json + test("build outgoing trampoline payment to blinded paths (reference test vector)") { + val preimage = ByteVector32.fromValidHex("8bb624f63457695115152f4bf9950bbd14972a5f49d882cb1a68aa064742c057") + val paymentHash = Crypto.sha256(preimage) + assert(paymentHash == ByteVector32.fromValidHex("e89bc505e84aaca09613833fc58c9069078fb43bfbea0488f34eec9db99b5f82")) + val evePriv = PrivateKey(hex"4545454545454545454545454545454545454545454545454545454545454545") + val eve = evePriv.publicKey + assert(eve == PublicKey(hex"02edabbd16b41c8371b92ef2f04c1185b4f03b6dcd52ba9b78d9d7c89c8f221145")) + + // Eve creates a blinded path to herself going through Dave. + val pathId = hex"0149792a42a127e421026a0c616e9490fb560d8fa5374a3d38d97aa618056a2ad70000000008f0d1808bb624f63457695115152f4bf9950bbd14972a5f49d882cb1a68aa064742c05702414343fd4a723942a86d5f60d2cfecb6c5e8a65595c9995332ec2dba8fe004a20000000000000001000000000000000068656c6c6f7bcdd1f21161675ee57f03e449abd395867d703a0fa3c1c92fe9111ad9da9fe216f8c170fc25726261af0195732366dad38384c0ab24060c7cd65c49d1de8411" + val (blindedPath, invoice) = { + val dave = PublicKey(hex"032c0b7cf95324a07d05398b240174dc0c2be444d96b159aa6c7f7b1e668680991") + val blindedPayloadEve = TlvStream[RouteBlindingEncryptedDataTlv](RouteBlindingEncryptedDataTlv.PathId(pathId)) + val blindedPayloadDave = TlvStream[RouteBlindingEncryptedDataTlv]( + RouteBlindingEncryptedDataTlv.OutgoingChannelId(ShortChannelId.fromCoordinates("572330x42x2465").get), + RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(36), 1000, 500 msat), + RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(850_000), 1 msat), + ) + assert(RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec.encode(blindedPayloadDave).require.bytes == hex"020808bbaa00002a09a1 0a080024000003e801f4 0c05000cf85001") + assert(RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec.encode(blindedPayloadEve).require.bytes == hex"06 bf 0149792a42a127e421026a0c616e9490fb560d8fa5374a3d38d97aa618056a2ad70000000008f0d1808bb624f63457695115152f4bf9950bbd14972a5f49d882cb1a68aa064742c05702414343fd4a723942a86d5f60d2cfecb6c5e8a65595c9995332ec2dba8fe004a20000000000000001000000000000000068656c6c6f7bcdd1f21161675ee57f03e449abd395867d703a0fa3c1c92fe9111ad9da9fe216f8c170fc25726261af0195732366dad38384c0ab24060c7cd65c49d1de8411") + val sessionKey = PrivateKey(hex"090a684b173ac8da6716859095a779208943cf88680c38c249d3e8831e2caf7e") + val blindedRouteDetails = Sphinx.RouteBlinding.create(sessionKey, Seq(dave, eve), Seq(blindedPayloadDave, blindedPayloadEve).map { p => + RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec.encode(p).require.bytes + }) + assert(blindedRouteDetails.route.firstNodeId == EncodedNodeId(dave)) + assert(blindedRouteDetails.lastPathKey == PublicKey(hex"02c952268f1501cf108839f4f5d0fbb41a97de778a6ead8caf161c569bd4df1ad7")) + assert(blindedRouteDetails.route.firstPathKey == PublicKey(hex"02988face71e92c345a068f740191fd8e53be14f0bb957ef730d3c5f76087b960e")) + val blindedNodes = Seq( + PublicKey(hex"0295d40514096a8be54859e7dfe947b376eaafea8afe5cb4eb2c13ff857ed0b4be"), + PublicKey(hex"020e2dbadcc2005e859819ddebbe88a834ae8a6d2b049233c07335f15cd1dc5f22"), + ) + assert(blindedNodes == blindedRouteDetails.route.blindedNodeIds) + val encryptedPayloads = Seq( + hex"0ae636dc5963bcfe2a4705538b3b6d2c5cd87dce29374d47cb64d16b3a0d95f21b1af81f31f61c01e81a86", + hex"bcd747ba974bc6ac175df8d5dbd462acb1dc4f3fa1de21da4c5774d233d8ecd9b84b7420175f9ec920f2ef261cdb83dc28cc3a0eeb970107b3306489bf771ef5b1213bca811d345285405861d08a655b6c237fa247a8b4491beee20c878a60e9816492026d8feb9dafa84585b253978db6a0aa2945df5ef445c61e801fb82f43d59347cc1c013a2351f094cdafb5e0d1f5ccb1055d6a5dd086a69cd75d34ea06067659cb7bb02dda9c2d89978dc725168f93ab2fe22dff354bce6017b60d0cc5b29b01540595e6d024f3812adda1960b4d", + ) + assert(encryptedPayloads == blindedRouteDetails.route.encryptedPayloads) + val paymentInfo = OfferTypes.PaymentInfo(500 msat, 1000, CltvExpiryDelta(36), 1 msat, 500_000_000 msat, ByteVector.empty) + val paymentRoute = PaymentBlindedRoute(blindedRouteDetails.route, paymentInfo) + val offerFeatures = Features(Features.BasicMultiPartPayment -> FeatureSupport.Optional).bolt12Features() + val offer = OfferTypes.Offer(None, Some("bolt12"), eve, offerFeatures, Block.RegtestGenesisBlock.hash) + val alicePayerKey = PrivateKey(hex"40086168e170767e1c2587d503fea0eaa66ef21069c5858ec6e532503d6a4bd6") + val invoiceRequest = OfferTypes.InvoiceRequest(offer, 150_000_000 msat, 1, offerFeatures, alicePayerKey, Block.RegtestGenesisBlock.hash) + val invoice = Bolt12Invoice(invoiceRequest, preimage, Sphinx.RouteBlinding.derivePrivateKey(evePriv, blindedRouteDetails.lastPathKey), 60 seconds, offerFeatures, Seq(paymentRoute)) + (paymentRoute, invoice) + } + + // Alice creates a trampoline onion for Carol that includes Eve's blinded path. + val trampolineOnion = { + val carol = PublicKey(hex"027f31ebc5462c1fdce1b737ecff52d37d75dea43ce11c74d25aa297165faa2007") + val trampolinePayload = PaymentOnion.IntermediatePayload.NodeRelay.ToBlindedPaths(150_000_000 msat, CltvExpiry(800_000), invoice) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(trampolinePayload.records).require.bytes == hex"fd01b5 020408f0d180 04030c3500 1503020000 16fd01a1032c0b7cf95324a07d05398b240174dc0c2be444d96b159aa6c7f7b1e66868099102988face71e92c345a068f740191fd8e53be14f0bb957ef730d3c5f76087b960e020295d40514096a8be54859e7dfe947b376eaafea8afe5cb4eb2c13ff857ed0b4be002b0ae636dc5963bcfe2a4705538b3b6d2c5cd87dce29374d47cb64d16b3a0d95f21b1af81f31f61c01e81a86020e2dbadcc2005e859819ddebbe88a834ae8a6d2b049233c07335f15cd1dc5f2200d1bcd747ba974bc6ac175df8d5dbd462acb1dc4f3fa1de21da4c5774d233d8ecd9b84b7420175f9ec920f2ef261cdb83dc28cc3a0eeb970107b3306489bf771ef5b1213bca811d345285405861d08a655b6c237fa247a8b4491beee20c878a60e9816492026d8feb9dafa84585b253978db6a0aa2945df5ef445c61e801fb82f43d59347cc1c013a2351f094cdafb5e0d1f5ccb1055d6a5dd086a69cd75d34ea06067659cb7bb02dda9c2d89978dc725168f93ab2fe22dff354bce6017b60d0cc5b29b01540595e6d024f3812adda1960b4d000001f4000003e800240000000000000001000000001dcd65000000") + val sessionKey = PrivateKey(hex"a64feb81abd58e473df290e9e1c07dc3e56114495cadf33191f44ba5448ebe99") + val trampolineOnion = OutgoingPaymentPacket.buildOnion(sessionKey, NodePayload(carol, trampolinePayload) :: Nil, paymentHash, None).toOption.get.packet + val encoded = OnionRoutingCodecs.onionRoutingPacketCodec(trampolineOnion.payload.length.toInt).encode(trampolineOnion).require.bytes + assert(encoded == hex"0002bc59a9abc893d75a8d4f56a6572f9a3507323a8de22abe0496ea8d37da166a8b98b9bf5cf80f093ee323cbb0c5b0713b14779893b07e4cc60110ce2d2240f16be3fd3c23062491fb57d229dac4edbad7a3b26242cffc2a2e9d5a0eae187390d4e096699d093f5ac82d86abdf0fdaae01bf16b80261e30f6ffda635ea7662dc0d124e1137367ab0178d6ed0de8e307a5c94a213b0b5705efcc94440308f477a185f5b41ab698e4c2dd7adea3aa47cccb5f47548c9ec2fee9573d32042eee6851a4f17406b6f6d13e2b794b0bd1676d0c3b33e4ee102823bb9e55f0ec29fc7f9df3332be5f9c68d4482ff60c0183c17742844baf01821cc1a2dbed1f764d124a5696f290db7f43608ddad007da504a56d0c714a0d34eeeed848d08c846609d29123df3f82484a7ae994c37487add9c878a737bb9d6e314139329b2eed131906a5717516f7790f0ec78f3e1a6c9b9c0680221dd290e3e219146039cb02f28eec46b88d5eceae7738182d9b1be14130636943dfa95aee4cf0f81bcdb04b8f92e3c9841f9928a7b39c3c8861dd4b73bf736b1e1b0d9a22c3bf3c12cdb1580c343a129b93cbda9e58675a52cde759040718c25504ea28df3b6da73e832b5bd7b51054a5663d407871c4a90e76824eca922ccde0bdd30e81f1ce9bed788416cc9660b016adccab6a45e0ac23d11030f7076b88184c247da4586d4fa3102e44f882ae88a46cf4a4dd874a9466c31eb94c834ac6c9cfb4bb9a6ef6a6a") + trampolineOnion + } + + // Alice creates a payment onion for Carol (Alice -> Bob -> Carol). + val onionForBob = { + val sessionKey = PrivateKey(hex"4f777e8dac16e6dfe333066d9efb014f7a51d11762ff76eca4d3a95ada99ba3e") + val bob = PublicKey(hex"0324653eac434488002cc06bbfb7f10fe18991e35f9fe4302dbea6d2353dc0ab1c") + val carol = PublicKey(hex"027f31ebc5462c1fdce1b737ecff52d37d75dea43ce11c74d25aa297165faa2007") + val paymentSecret = ByteVector32.fromValidHex("7494b65bc092b48a75465e43e29be807eb2cc535ce8aaba31012b8ff1ceac5da") + val payloadBob = PaymentOnion.IntermediatePayload.ChannelRelay.Standard(ShortChannelId.fromCoordinates("572330x42x2821").get, 150_153_000 msat, CltvExpiry(800_060)) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadBob.records).require.bytes == hex"15 020408f32728 04030c353c 060808bbaa00002a0b05") + val payloadCarol = PaymentOnion.FinalPayload.Standard.createTrampolinePayload(150_153_000 msat, 150_153_000 msat, CltvExpiry(800_060), paymentSecret, trampolineOnion, None) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadCarol.records).require.bytes == hex"fd024f 020408f32728 04030c353c 08247494b65bc092b48a75465e43e29be807eb2cc535ce8aaba31012b8ff1ceac5da08f32728 14fd021a0002bc59a9abc893d75a8d4f56a6572f9a3507323a8de22abe0496ea8d37da166a8b98b9bf5cf80f093ee323cbb0c5b0713b14779893b07e4cc60110ce2d2240f16be3fd3c23062491fb57d229dac4edbad7a3b26242cffc2a2e9d5a0eae187390d4e096699d093f5ac82d86abdf0fdaae01bf16b80261e30f6ffda635ea7662dc0d124e1137367ab0178d6ed0de8e307a5c94a213b0b5705efcc94440308f477a185f5b41ab698e4c2dd7adea3aa47cccb5f47548c9ec2fee9573d32042eee6851a4f17406b6f6d13e2b794b0bd1676d0c3b33e4ee102823bb9e55f0ec29fc7f9df3332be5f9c68d4482ff60c0183c17742844baf01821cc1a2dbed1f764d124a5696f290db7f43608ddad007da504a56d0c714a0d34eeeed848d08c846609d29123df3f82484a7ae994c37487add9c878a737bb9d6e314139329b2eed131906a5717516f7790f0ec78f3e1a6c9b9c0680221dd290e3e219146039cb02f28eec46b88d5eceae7738182d9b1be14130636943dfa95aee4cf0f81bcdb04b8f92e3c9841f9928a7b39c3c8861dd4b73bf736b1e1b0d9a22c3bf3c12cdb1580c343a129b93cbda9e58675a52cde759040718c25504ea28df3b6da73e832b5bd7b51054a5663d407871c4a90e76824eca922ccde0bdd30e81f1ce9bed788416cc9660b016adccab6a45e0ac23d11030f7076b88184c247da4586d4fa3102e44f882ae88a46cf4a4dd874a9466c31eb94c834ac6c9cfb4bb9a6ef6a6a") + val onionForBob = OutgoingPaymentPacket.buildOnion(sessionKey, NodePayload(bob, payloadBob) :: NodePayload(carol, payloadCarol) :: Nil, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get.packet + val encoded = PaymentOnionCodecs.paymentOnionPacketCodec.encode(onionForBob).require.bytes + assert(encoded == hex"00025fd60556c134ae97e4baedba220a644037754ee67c54fd05e93bf40c17cbb73362fb9dee96001ff229945595b6edb59437a6bc14340622675e61ad0fd4e5e9473ea41567f4f7b0938040e2076c378a98409260c7234f87c58657bcf20af7e63d7192b46cf171e4f73cb11f9f603915389105d91ad630224bea95d735e3988add1e24b5bf28f1d7128db64284d930839ba340d088c74b1fb1bd21136b1809428ec5399c8649e9bdf92d2dcfc694deae5095f9ea212871fc9229a545ddd273635939ed304ba8a2c0a80a1a2ff7f95df532cde150bb304cd84abf88abe6e09b405d10e5e422f6d839a245fd2a300b2f6b95eedecf88479a3950727e6eeac46a34b2930aa9b0d7dd02d021d59800c3f7d5bae58eb45d03f31cce59f04715d7f7158fb2413f9ffe83b869c52019a54f6e0e194479e2eb546a6efe27cdb69863b5ff4e218e57b3e7aff727296036ed6b6756b6b98b22607b699190ced7484df2fd487fd679bf7b327322afd8c9ed658564a2d715cd86e0d270f3fad64980ef2926b82c415cdc537ff5d037b0a2986a44857ce430dfabce135748b4bd4daf3afaac064b1571fbce1369b7d7166c2638d426b6a3a418e5f017699373f614815de8275c74cd57bcfb9f3c5a11183cbb8f488bb255f7a0c3299df1306fdeeca785d81a7bcba5036a4891cd20d1b16c5436c51992d4797e124df65f2d71479739923b46d3daa3a0ecc75404c0475e5cd01665bf375e3897b3a57d2fa89ce1fa7d667ecfe0c097cfb7d94634c5a2b7c6ad5a3de7f9980a0779b66dff957389bed1e19d4681299fbe762a6ca0f9fc0726c203dc2021e74375559453ba0d2c2825142ed007cefb1e1466bb99303dbf4ceaba5eb76d18204910df11e3e3747d6d147c599edfbaf898fbd09d867558dec280e8d1c44d6e440a3c8d3b1afcfe7f3b1731a42bee7b6f7227e755bcc936952b696740f387c0ab93fd8b1d6bd4c74aebe535d6110014cd3d525394027dfe8fad19477d69dc1671d1133f5d8d21b55ddc7f3c76dabf718ca6f02da0d6445e4326b781c6d9041e9e330e44950d10d5dbed7f98b708d1681b75f8fe56c99c7a424899c6a06f36e5b29f2c3db0050bebeffee8b729351518644f98246c1db892ff3305b7610cfb09d5465f5a94da4812d35275c42f4b3a9cbfe626cee01e1818cdbe71565104e112d1c2c74450488d06de19c831d3c3e5203e67229bd9619166aab97fb579623c418b23c05fabf39e21ba0d7baf4aa67034b1ba8c4bd4542825471c7dfb37c24cdfd2d8d06d0e7ddaca01f001449195cc04201a7ae2da86e74d515e2feef3707e29508768f18eb5741ef22dc07f078cf751da83ee2fe9927c760474cdce266fce9b66959d391d51b204fa50cd9a8ff7b45fdd043679a20afa0b440938a721fef14badb97b68ad5e5494dfb2aea8edc1cdb69eb6f13b75bbd496c8eb35a48f229a080ae6744dec87f58058296c2969f0916685ac57a0a44efe4691eb06236f414334f5747a11b100e1d6272ff6082510fa79c64bcfaa58e43525f9fbbea025aa741feb7b18925e2dbd0da2a73748a6c30fe625afb497189d7f188869602989a53892ad24624807e1581eeca2db2cef855aa65af66c4573f9c637699bcbe8ae5f6d9f0713ffe52d453faa39b44be3108e940b322db0d1dc008aff99d4909345ffcf996a382359e7e5b4592522d453fffa9744e1e32a21a237fff4c8c55c1f46fdc5b2e8de267419a3052b33c6065119f690e972ac9b19921bab489a572df128494a1158650665bc875bbc02de3cac75963cee5c10075768d921edacb382044c74848af73092641a57c2050ea0e68dbb6c6121b1bf012073c8812d68fac75a06a8a35bec984c71ff951eb3ef18e96e1158f653a837a9fec2df21cdd816d36bf998ee108b562a60a6") + onionForBob + } + + // Bob decrypts the onion and relays to Carol. + val onionForCarol = { + val bobPriv = PrivateKey(hex"4242424242424242424242424242424242424242424242424242424242424242") + val add = UpdateAddHtlc(randomBytes32(), 1, 150_155_000 msat, paymentHash, CltvExpiry(800_100), onionForBob, None, Reputation.maxEndorsement, None) + val Right(packetForBob: ChannelRelayPacket) = decrypt(add, bobPriv, Features.empty) + assert(packetForBob.payload.outgoing.contains(ShortChannelId.fromCoordinates("572330x42x2821").get)) + assert(packetForBob.amountToForward == 150_153_000.msat) + assert(packetForBob.outgoingCltv == CltvExpiry(800_060)) + val encoded = PaymentOnionCodecs.paymentOnionPacketCodec.encode(packetForBob.nextPacket).require.bytes + assert(encoded == hex"0003dc6a4c9b34bdcd2191fc4dacfc1aeb20f71991acbd17847b9ab17d69579c1614da126cd18820e55534d7352839caa436aa79a6d5be26c6ecbd1c79f74442b1b7bf8ed8b739b736b9248769c2c422eebc85fb0d580e95618bcda1be3fd4cfa6ed0d839a2feb878acf686b112febaae9c1494a2ad20d97b2d2f7e54e6e9860e75e35671d5530ff9cf131b16b00a89337781aa37e5b867995d56578e69c031b7b272c4697727210e10bc8456e5cd58ae958d07e2811e2fb767b702c792b26fd6c352306b31f808a0e46d28ababda518d0d33c8f3a301adef4dd4f12fd2f78da4d548b7c12b0d890b6ab24e724e569106ae47b1acea4f5de055ba6d910bbe824810a11349ce7ea557abf02c5104740b52c910cd0bfea5d8666a41448703c054ed0612775e8617eda8df2fdccebf65193301738ba4308b61f447016a0b801de0eff2a7db374e6ccdadc9efbcb2fe0fb56c34fbaffd1bf87e5bf46cca75b77cdb7161532402fbc9323af57304e0b6cfa5082af683ef82a2731e89734b9e377184c647486c63ab57e18d4f42e9ddd55189a064cfc3a2800b8abd291043aab068c8c8ce57f17ef169945bba4d434d67bc883b68ba2c2c92dad42c788a209b4d7a2e00b375b811766ff67fc630ae047b8d2781e00291f6d1e31495a797a7e4ed135585da237cdcd067d37641e49f562f22dd619240bb2411fab802f834d96aa6451fd4b3f585dbde15bc78e692f49a491dcd8e44a8ef035bcbb4863462d7cf6066e0df516dcb6209674abe54e7d2faca26d17019bc2b6ec59ec94b51fd62064e7ff2230e73c375fadf7f305c307870a1d3dcb4eecddce6eeca54bff76b945823364ca823f7f3dc273c5eaa6d7aa3b510bf3bd274c8bd73570d15bc1ff0ba90c3bb65a8e1ee89cc6c44114d658fb89985c7ca8e8eeb32bc8be1e3ab951ff1a720bcc0d4c298ae4c1d06c164615c8af5bbda93f431d5d2be8bc40320c9c3a002bd9f2e39828abb6e7bfde83421d7faed6b16f355b9bd86d018fb3ec0f98ffdcaae8d521bd5003e93382459fc7957e2590409e5c8a88d7c1488884da0e148b01ec99aafe96d418d7cd76d7437d3c1d9d79e79386e3286210fac073eac6cd90031ac1c5b70b494d60e74d243ee44bfb8d0fcc57d3f8683aaadc5a2d346fce681a8d4a4931e932a39e2ab443141eb5c29a475679c5ee4e8b94e9d5de731f03963acaddd7301be90c7ccdfab314f70e843037a98656c31b22c822312719434f7a503bac9f18eb2f0cbc2c2790e93fc1664b82726eb1265a4ffa1e8e72d2898df1d8db9da1586675d242ff565aa008a35aad1c65b50c07ae6c0452bcdaa2f5410600acb3326e335971eba42c1dbac36005b5299ab7b852812717048aa51f272e8ec21c11e22a25b48ef60ed98540d879f5ae6820ac94cfa29e5d0aa74d91ca30ee28e97cd94968b4f246b3f93f36ecbf1c84f12844867f0738c3c775981a827cb05ddc5bebd288b6312b0b3f7d46f6eb4ddaf91e7c6a3afdbc291ac5a151675f3c4ae23ab301a9c3f5e1ba62aef64dc50cd977a34ffe58a78feda76c27cc3d5a3a1e05303e9cdd72d60ed17cc90c88b015f3c4891651537b52d837ef0d5f9a90b01e05a9339a623034aea961f7bdc148f129f61f7e12d4ebd1ed37565935cdaec4ea6b7020e62d5db3bad4a3b1141ec3c78d679498bbb348091f56279a3c01662db7694ba54efd8d8f1271f4b06cba94804c3197f92ea97e93bdef8fcb348a405792855e84c1c9625153187495825c5a293e1efc7b672ddb609aa90caca1e7182ba301313a17364b48cd93b6a26cb6888d5c4cc1d3c6a39cb442c2de12bb1ad396205c9a10023d7edc951fc60a28813867f635c3d74d4206f398849e65eb5a8d8fdeb952ae813073c3b617ed68c7bf1a18a6b9f9e3af316029be4dd8") + packetForBob.nextPacket + } + + // Carol decrypts the onion and relays to the blinded path's introduction node Dave. + val onionForDave = { + val carolPriv = PrivateKey(hex"4343434343434343434343434343434343434343434343434343434343434343") + val dave = PublicKey(hex"032c0b7cf95324a07d05398b240174dc0c2be444d96b159aa6c7f7b1e668680991") + val add = UpdateAddHtlc(randomBytes32(), 1, 150_153_000 msat, paymentHash, CltvExpiry(800_060), onionForCarol, None, Reputation.maxEndorsement, None) + val Right(RelayToBlindedPathsPacket(_, payload, trampolinePayload, _)) = decrypt(add, carolPriv, Features(Features.RouteBlinding -> FeatureSupport.Optional)) + assert(payload.amount == 150_153_000.msat) + assert(payload.expiry == CltvExpiry(800_060)) + assert(payload.paymentSecret == ByteVector32.fromValidHex("7494b65bc092b48a75465e43e29be807eb2cc535ce8aaba31012b8ff1ceac5da")) + assert(trampolinePayload.invoiceFeatures == Features(Features.BasicMultiPartPayment -> FeatureSupport.Optional).toByteVector) + assert(trampolinePayload.outgoingBlindedPaths.length == 1) + assert(trampolinePayload.amountToForward == 150_000_000.msat) + assert(trampolinePayload.outgoingCltv == CltvExpiry(800_000)) + val outgoingPath = trampolinePayload.outgoingBlindedPaths.head + assert(outgoingPath.paymentInfo == blindedPath.paymentInfo) + assert(outgoingPath.route == blindedPath.route) + val sessionKey = PrivateKey(hex"e4acea94d5ddce1a557229bc39f8953ec1398171f9c2c6bb97d20152933be4c4") + val payloadDave = PaymentOnion.OutgoingBlindedPerHopPayload.createIntroductionPayload(outgoingPath.route.encryptedPayloads.head, outgoingPath.route.firstPathKey) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadDave.records).require.bytes == hex"50 0a2b0ae636dc5963bcfe2a4705538b3b6d2c5cd87dce29374d47cb64d16b3a0d95f21b1af81f31f61c01e81a86 0c2102988face71e92c345a068f740191fd8e53be14f0bb957ef730d3c5f76087b960e") + val payloadEve = PaymentOnion.OutgoingBlindedPerHopPayload.createFinalPayload(trampolinePayload.amountToForward, trampolinePayload.amountToForward, trampolinePayload.outgoingCltv, outgoingPath.route.encryptedPayloads.last) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadEve.records).require.bytes == hex"e4 020408f0d180 04030c3500 0ad1bcd747ba974bc6ac175df8d5dbd462acb1dc4f3fa1de21da4c5774d233d8ecd9b84b7420175f9ec920f2ef261cdb83dc28cc3a0eeb970107b3306489bf771ef5b1213bca811d345285405861d08a655b6c237fa247a8b4491beee20c878a60e9816492026d8feb9dafa84585b253978db6a0aa2945df5ef445c61e801fb82f43d59347cc1c013a2351f094cdafb5e0d1f5ccb1055d6a5dd086a69cd75d34ea06067659cb7bb02dda9c2d89978dc725168f93ab2fe22dff354bce6017b60d0cc5b29b01540595e6d024f3812adda1960b4d 120408f0d180") + val onionForDave = OutgoingPaymentPacket.buildOnion(sessionKey, NodePayload(dave, payloadDave) :: NodePayload(outgoingPath.route.blindedNodeIds.last, payloadEve) :: Nil, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get.packet + val encoded = PaymentOnionCodecs.paymentOnionPacketCodec.encode(onionForDave).require.bytes + assert(encoded == hex"0003626d5fa7ed0b8706f975ca58ae2c645889514153568aaab7835325ecd3739a5760171be581c1df1ce5267cd012e06f45a97fdbc5d8fc0c140c7432d34027b9ed0f2ab09fd388fe47525309f9c4509a7b9d74d88c28e40b1b6598e42fe2975f857e28b224316f266decf170cbfb5019dea2dab2ee7f1db089d44d1f974d6974bd06e515510cc1c178cd46c2442f07b9a083b4e4c7e9dd8d728508353959497fb8d25ebe8db83c60488566952fb1725267088ccb98acace7147d3846388464aa9512fe94f1962a54d8896d94105b185a41201e0dacf51f755da8e666a78261bc478ca77bd0ef5576bd7a4b24bb1fee9a97618bac0dcc4f1a34d64f5623446e2458089299be2f07592d69619bd6048c37a0460062b194f6f05da8f4ac1c5ff19681067398fdde459c60b4f448d5b3c1152988f6e29dc73b3a5407f1a502dcf2d656bcdba5f05eb4a7ecd3a1373c495dbc23109912aa456f0d9c1460f99f8151886ec8a69af2ac3ed76823ce372fb46a3c20ff114c04ee4a16ff673382b1abedcfc5e8d6f6e77c893dc346cf01f323bb043840546f9728b060514ddc4359c3ebc818abe56c8219260e26c833ff6faef7c02a3e669026dcc0a96ca4f0f8240185422355e0a5c9529bf65e7c52b384cfbe2eeb3ba32c118cfb6961068362b7bf41b2b1580cfc85757fd294840154cb8b13df456c6b86957a33391fd78e3aec6ff2fb9dabeefc63dc4faadfc12016d9d9501381c9751c581256cc24f8d8fecec2a1efaa9579935d5f5f7d14edb64ce97ab9deacb5c9d4c111325c70493ad7921966369315a1ec09c320c3cdc7a65ce52ff8ba5e9a71326e57f8b30766e6e5c77747b07e351c3e91efdf736a31410c9e278a683dfe1ca9c35d7868f11e1e429cb7655ff126438c83f69c3db2c5257e03f7d4f95e8d49400ee36e8d7f1629ac79f0b63430b115349df21e8d286a69b41d52e52e36553b16edf4c77acfb9d4596abd5054daf076b06abb3f84ecbea3e6d324965c7667ec7f83388ee05583f53f258291a806c025e300d63c81f5a411447b3ab3ee47b2dae485b8a87129224ad16fcc043a2d1b89e5c4f35f02675efa79730f5ee07d2de9d6ab503aa329f201ad0c9040d8c3437efde15c53b9212e93e0ece4a3ee7ae99a18b3fd75e8d1ee0ce9c73bfd5c2bbe30a91a3f92169a05887069dd31edf575265425d09998e2466bdf86919cdbfbbdb55c718b046197028b4370dd850833853b969a37e31f2cce96020a1fb22959b4529ae501f44d989b3f7473aaa787899ba200468b070079e2b9a3cd6b04b3caf2de5956aed477e4b3a9f0c93ac3f1042d16ee6a36744460e6d86144522215eeed052daebc7861d7189abe78edb67dd7ae47224b9bdb5907fdca6e6573dfe4bcf24ce1c6a4dfbae6991a8ac6976d9ec8a81f08dbdc34bf1cb18d93aa2e9d876335e0fce0d7a7b6c7080a70b1fc9bef912e4550931005210da7c46c76cf63fb02202df35d332e9ad779ef5ee086fd9fa993852be315691cf84c7e588ec61726b9fe5200ad30b2d43b1684f1dcd8df3f1598ae3841125eadfd534b074d560fd8e0eb9660c93a478ccfdd2308f587a45d5b933af280d39a77e19cd72c170931e4c8e44028bb6db77ec1e9b77af225e39db67bfb80afc6a0efe9864a80222fbfd6c4b6ad9afd43c76f2b9fc0cd0a4b07939147b005be7e6418295830a9bb114cc2c40bdb715077ef4219643455f2675ba00c0e6464f612c32cffa39f49d80ff91cf1363e109101e368114537fbd94428b6ed1934f6d8cd3b6cf2ec736ddcacf63007481fcd6dd9fca8ee39d9a4bbdd06349a5e86af75d8723eeeddc6f84575516997f7db931b91007bcfd21f1b5a8dc69ee846492493054b012e5a4ff3707d5aae44a4ca65210eb1c14c8d138441170f2e5e2920c1e4") + onionForDave + } + + // Dave decrypts the onion and relays to Eve. + val onionForEve = { + val davePriv = PrivateKey(hex"4444444444444444444444444444444444444444444444444444444444444444") + val add = UpdateAddHtlc(randomBytes32(), 1, 150_150_500 msat, paymentHash, CltvExpiry(800_036), onionForDave, None, Reputation.maxEndorsement, None) + val Right(packetForDave: ChannelRelayPacket) = decrypt(add, davePriv, Features(Features.RouteBlinding -> FeatureSupport.Optional)) + assert(packetForDave.payload.isInstanceOf[PaymentOnion.IntermediatePayload.ChannelRelay.Blinded]) + assert(packetForDave.amountToForward == 150_000_000.msat) + assert(packetForDave.outgoingCltv == CltvExpiry(800_000)) + val encoded = PaymentOnionCodecs.paymentOnionPacketCodec.encode(packetForDave.nextPacket).require.bytes + assert(encoded == hex"0002580900a2090fc8b09d77f87781aa5a5964372ee968bd8488da62e04e3f1d68bda5d48cb395a094d2d60f43d8af709c5b44bba1c51f3d590c462829104a18ec68d5c36989a3d6af086f2f61e791e619fd62bf6fccdcdb1dc01bb1798bff5550d385a3ea26ce909d6d218eb12cfa089d11d33a1cb1299510a4c5ac1f767ee18230960b2a37994dc05378ca9d6ce8c29c61dc543f11b676f1ffd3c0c0fe7d43168ecaa1760b115d397b4886c17daaf8dabaac2c5ce3e57f7b5441130e828c5368eb605c841045d84d137197512d9a6efa3bb8fb05a70af7b14f5d01518a61932717ebd04e5022e6925767f07f33b63894bbaff907967999001d6b4cb4b3bc42a9057b8b1d269172f638275688915ae9c07d276fa8aaf037a59069c3d2121a79e8eb869ade6b1dc5073922145d7e1246baa202544d5045fca6fb2974dabc145257d32f0ab5afdbb121b9d93dc1b3d345038714d70ed941be5dec56d4c5cb0582ebcb2d4a78356f75bf1696f82deecec4a97a23746b440082e07ee7d5ebac4c098a48fe0d64de53b303b960b52aabe8df029b9cb5b6079ecac2a2841dd662a2099e1c5995176b9dbc90054b789a07cfbd35e93c0da58eefa7150f7c793b37f4934e2541d6002be6953a0dfda018a881b4d7458d04d4ece3d6d570f1ac46e2eb7ad29652adae7f56ca804d88a1a92ecc17bad4ab7879e93aa56782c46f8b0fca6068a5c3593cddcd372db066bbd7ea615a0fc8b01b61849930959d3ec7951d619b93fc9feecd07c91ec6206a8a489023a55349c1d0b6c294104190090c2c82f1e00c1cf4c9349b09544157aedfed527fa5725408d38d8026916f6baac3218ab5469e157bc91475a5117947efab4af7a64373694cc62b08e0b8bfe1a35ba2f80fac95e043a17e850590bbfbb59d4c190a4fa1642d790e3403a34522f33de66e839c3e3706f9bea9d95efca2f9c7b012bfc39cb9f3e7dad4a1c7b52a8d02151a1b3524a64033d2868e9c450d496f66d71c414870c15911dee4365f1aab8a20b3968a67d04dd724955b0396a00dbdbaa0c0037a2bb8202061f6cc653a10e6ed8ce98a5b1315d5efa96603e989ce1cf315cf2e300f12696c96e45efd397776f8a781d12b8d4e3f265e49c8932cf54525af20977ab1c5b5e0a4f929074baf6b0d4fde175d02a78e0fccd4e814c0a2139475ea16517c33389a41160014f537c43c818f70ba1b9503987885f634f93b995c04f7302d1ff85add09232a2d2be27fbbf98d754c6a0e2c32f66b2a2cd6d5feef4ad10b62303ce05049e862e96987defc569cf6406585fedcc4bf4981ad67cb6af242e25f9bf701e5236deb61305bd0c20c2bfa0d17d6519979f3085427dcad1677959fa40565e16f2feee4b4974de401123f4b3e0f0e740305cdecc8f4b65d638cdd5b1af0013d5806c9d6661b96954463adee45cbacf33c16e836d8e544cab9eb47f9f661d415772a9dae0d4c3ffb44015bc6921e05e6bd8c5159893fd7e5291f6e40c84db19266a35c666afb1ec16d8c4bc507b887df09a2c71a599dbcdf75ced11eb8cd9c65f05a14a3a381971e615bdece5946affe0dbdbbb54a777e5d996e9cb9a5163bc503b88b15b31cd0fa3a8206701aa9f4068e6baac2b2f342e02f94ed22f43f285a6790ff1e216c917af77b5af726e403ce8615959b31e6d051c0a17f737ffef28264ec31c3f0f690f0f142c0b16c88507a44714516fdaee00b697288fdfea823a30bf11fa6cf3ae2215eb42b98aae1e80444c6f2688a5f8f80f1236fb3d12584f33bdfc33beb8c5b7bfdfeb94e25ed4c1fdf69f4a28f6cbb7fa0fb9424927e195908d0a8894555d02f285962a53a984fca3f6b3fb843e4d559e5294c2e01dd1dce5692664881c4dec168d52e42981c6d72f0a84caa78ebf409cb62584ec539f89147c1") + packetForDave.nextPacket + } + + // Eve receives the payment. + val pathKey = PublicKey(hex"02c952268f1501cf108839f4f5d0fbb41a97de778a6ead8caf161c569bd4df1ad7") + val add = UpdateAddHtlc(randomBytes32(), 1, 150_000_000 msat, paymentHash, CltvExpiry(800_000), onionForEve, Some(pathKey), Reputation.maxEndorsement, None) + val Right(FinalPacket(_, payload, _)) = decrypt(add, evePriv, Features(Features.RouteBlinding -> FeatureSupport.Optional)) + assert(payload.isInstanceOf[FinalPayload.Blinded]) + assert(payload.asInstanceOf[FinalPayload.Blinded].pathId == pathId) + assert(payload.asInstanceOf[FinalPayload.Blinded].amount == 150_000_000.msat) + assert(payload.asInstanceOf[FinalPayload.Blinded].totalAmount == 150_000_000.msat) + assert(payload.asInstanceOf[FinalPayload.Blinded].expiry == CltvExpiry(800_000)) + } + test("build outgoing trampoline payment with non-trampoline recipient") { // simple trampoline route to e where e doesn't support trampoline: // .----. diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/PaymentOnionSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/PaymentOnionSpec.scala index 37d3a50c76..1ad7b154f1 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/PaymentOnionSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/PaymentOnionSpec.scala @@ -155,6 +155,26 @@ class PaymentOnionSpec extends AnyFunSuite { } } + test("encode/decode node relay blinded per-hop payload") { + val nextNodeId = PublicKey(hex"0221cd519eba9c8b840a5e40b65dc2c040e159a766979723ed770efceb97260ec8") + val blindedTlvs = TlvStream[RouteBlindingEncryptedDataTlv]( + RouteBlindingEncryptedDataTlv.OutgoingNodeId(nextNodeId), + RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(144), 100, 10 msat), + RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(1500), 1 msat), + ) + val testCases = Seq( + TlvStream[OnionPaymentPayloadTlv](EncryptedRecipientData(hex"deadbeef")), + TlvStream[OnionPaymentPayloadTlv](EncryptedRecipientData(hex"deadbeef"), PathKey(randomKey().publicKey)), + ) + testCases.foreach(tlvs => { + val Right(payload) = IntermediatePayload.NodeRelay.Blinded.validate(tlvs, blindedTlvs, randomKey().publicKey) + assert(payload.outgoing == Left(EncodedNodeId.WithPublicKey.Plain(nextNodeId))) + assert(payload.outgoingAmount(10_000 msat) == 9990.msat) + assert(payload.outgoingExpiry(CltvExpiry(1000)) == CltvExpiry(856)) + assert(payload.paymentRelayData.allowedFeatures.isEmpty) + }) + } + test("encode/decode node relay to legacy per-hop payload") { val nodeId = PublicKey(hex"02eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619") val features = hex"0a" @@ -347,6 +367,43 @@ class PaymentOnionSpec extends AnyFunSuite { } } + test("decode invalid node relay blinded per-hop payload") { + val outgoingNodeId = PublicKey(hex"0221cd519eba9c8b840a5e40b65dc2c040e159a766979723ed770efceb97260ec8") + val validBlindedTlvs = TlvStream[RouteBlindingEncryptedDataTlv]( + RouteBlindingEncryptedDataTlv.OutgoingNodeId(outgoingNodeId), + RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(144), 100, 10 msat), + RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(1500), 1 msat), + ) + + case class TestCase(err: InvalidTlvPayload, bin: ByteVector, blindedTlvs: TlvStream[RouteBlindingEncryptedDataTlv]) + + val testCases = Seq( + // Forbidden non-encrypted amount. + TestCase(ForbiddenTlv(UInt64(0)), hex"0e 02020231 0a080123456789abcdef", validBlindedTlvs), + // Forbidden non-encrypted expiry. + TestCase(ForbiddenTlv(UInt64(0)), hex"0d 04012a 0a080123456789abcdef", validBlindedTlvs), + // Forbidden outgoing channel id. + TestCase(ForbiddenTlv(UInt64(0)), hex"14 06080000000000000451 0a080123456789abcdef", validBlindedTlvs), + // Forbidden unknown tlv. + TestCase(ForbiddenTlv(UInt64(51)), hex"0e 0a080123456789abcdef 33020102", validBlindedTlvs), + // Missing encrypted data. + TestCase(MissingRequiredTlv(UInt64(10)), hex"23 0c21036d6caac248af96f6afa7f904f550253a0f3ef3f5aa2fe6838a95b216691468e2", validBlindedTlvs), + // Missing encrypted outgoing node or outgoing channel. + TestCase(MissingRequiredTlv(UInt64(2)), hex"0a 0a080123456789abcdef", TlvStream(RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(144), 100, 10 msat), RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(1500), 1 msat))), + // Missing encrypted payment relay data. + TestCase(MissingRequiredTlv(UInt64(10)), hex"0a 0a080123456789abcdef", TlvStream(RouteBlindingEncryptedDataTlv.OutgoingNodeId(outgoingNodeId), RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(1500), 1 msat))), + // Missing encrypted payment constraint. + TestCase(MissingRequiredTlv(UInt64(12)), hex"0a 0a080123456789abcdef", TlvStream(RouteBlindingEncryptedDataTlv.OutgoingNodeId(outgoingNodeId), RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(144), 100, 10 msat))), + // Forbidden encrypted path id. + TestCase(ForbiddenTlv(UInt64(6)), hex"0a 0a080123456789abcdef", TlvStream(RouteBlindingEncryptedDataTlv.OutgoingNodeId(outgoingNodeId), RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(144), 100, 10 msat), RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(1500), 1 msat), RouteBlindingEncryptedDataTlv.PathId(hex"deadbeef"))), + ) + + for (testCase <- testCases) { + val decoded = perHopPayloadCodec.decode(testCase.bin.bits).require.value + assert(IntermediatePayload.NodeRelay.Blinded.validate(decoded, testCase.blindedTlvs, randomKey().publicKey) == Left(testCase.err)) + } + } + test("decode invalid final per-hop payload") { val testCases = Seq( (MissingRequiredTlv(UInt64(2)), hex"25 04012a 0820eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), // missing amount From d7739eb0ab8a908ed50aa7ac47a61980d82b973a Mon Sep 17 00:00:00 2001 From: t-bast Date: Thu, 5 Dec 2024 15:45:03 +0100 Subject: [PATCH 3/3] Add support for trampoline failure encryption When returning trampoline failures for the payer (the creator of the trampoline onion), they must be encrypted using the sphinx shared secret of the trampoline onion. When relaying a trampoline payment, we re-wrap the (peeled) trampoline onion inside a payment onion: if we receive a failure for the outgoing payment, it can be either coming from before the next trampoline node or after them. If it's coming from before, we can decrypt that error using the shared secrets we created for the payment onion: depending on the error, we can then return our own error to the payer. If it's coming from after the next trampoline onion, it will be encrypted for the payer, so we cannot decrypt it. We must peel the shared secrets of our payment onion, and then re-encrypted with the shared secret of the incoming trampoline onion. This way only the payer will be able to decrypt the failure, which is relayed back through each intermediate trampoline node. --- .../fr/acinq/eclair/payment/Monitoring.scala | 1 + .../acinq/eclair/payment/PaymentPacket.scala | 115 ++++- .../payment/receive/MultiPartHandler.scala | 7 +- .../eclair/payment/relay/NodeRelay.scala | 96 ++-- .../payment/relay/OnTheFlyFunding.scala | 14 +- .../relay/PostRestartHtlcCleaner.scala | 3 +- .../payment/send/PaymentLifecycle.scala | 4 +- .../send/TrampolinePaymentLifecycle.scala | 31 +- .../scala/fr/acinq/eclair/router/Router.scala | 7 +- .../eclair/wire/internal/CommandCodecs.scala | 2 + .../eclair/wire/protocol/FailureMessage.scala | 25 +- .../integration/PaymentIntegrationSpec.scala | 54 +- .../eclair/payment/MultiPartHandlerSpec.scala | 27 +- .../eclair/payment/PaymentPacketSpec.scala | 485 ++++++++++++++++++ .../payment/PostRestartHtlcCleanerSpec.scala | 4 +- .../payment/relay/NodeRelayerSpec.scala | 36 +- .../payment/relay/OnTheFlyFundingSpec.scala | 10 +- .../wire/internal/CommandCodecsSpec.scala | 1 + .../protocol/FailureMessageCodecsSpec.scala | 5 +- 19 files changed, 797 insertions(+), 130 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Monitoring.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Monitoring.scala index 6aa201aaa9..0ec9e8af67 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Monitoring.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Monitoring.scala @@ -130,6 +130,7 @@ object Monitoring { def apply(cmdFail: CMD_FAIL_HTLC): String = cmdFail.reason match { case _: FailureReason.EncryptedDownstreamFailure => Remote case FailureReason.LocalFailure(f) => f.getClass.getSimpleName + case FailureReason.LocalTrampolineFailure(f) => f.getClass.getSimpleName } def apply(pf: PaymentFailure): String = pf match { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala index 92f4254aef..f9b9d8a989 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala @@ -379,30 +379,81 @@ object OutgoingPaymentPacket { } } - private def buildHtlcFailure(nodeSecret: PrivateKey, useAttributableFailures: Boolean, reason: FailureReason, add: UpdateAddHtlc, holdTime: FiniteDuration): Either[CannotExtractSharedSecret, (ByteVector, TlvStream[UpdateFailHtlcTlv])] = { - extractSharedSecret(nodeSecret, add).map(sharedSecret => { - val (packet, attribution) = reason match { - case FailureReason.EncryptedDownstreamFailure(packet, attribution) => (packet, attribution) - case FailureReason.LocalFailure(failure) => (Sphinx.FailurePacket.create(sharedSecret, failure), None) + private def buildHtlcFailure(nodeSecret: PrivateKey, reason: FailureReason, add: UpdateAddHtlc, holdTime: FiniteDuration, trampolineHoldTime: FiniteDuration): Either[CannotExtractSharedSecret, (ByteVector, Option[ByteVector])] = { + extractSharedSecret(nodeSecret, add).map(ss => { + reason match { + case FailureReason.EncryptedDownstreamFailure(packet, previousAttribution_opt) => + ss.trampolineOnionSecret_opt match { + case Some(trampolineOnionSecret) if !ss.blinded => + // If we are unable to decrypt the downstream failure and the payment is using trampoline, the failure is + // intended for the payer. We encrypt it with the trampoline secret first and then the outer secret. + val trampolinePacket = Sphinx.FailurePacket.wrap(packet, trampolineOnionSecret) + val trampolineAttribution = Sphinx.Attribution.create(previousAttribution_opt, Some(packet), trampolineHoldTime, trampolineOnionSecret) + val outerAttribution = Sphinx.Attribution.create(Some(trampolineAttribution), Some(trampolinePacket), holdTime, ss.outerOnionSecret) + (Sphinx.FailurePacket.wrap(trampolinePacket, ss.outerOnionSecret), Some(outerAttribution)) + case Some(trampolineOnionSecret) => + // When we're inside a blinded path, we don't report our attribution data. + val trampolinePacket = Sphinx.FailurePacket.wrap(packet, trampolineOnionSecret) + (Sphinx.FailurePacket.wrap(trampolinePacket, ss.outerOnionSecret), None) + case None => + val attribution = Sphinx.Attribution.create(previousAttribution_opt, Some(packet), holdTime, ss.outerOnionSecret) + (Sphinx.FailurePacket.wrap(packet, ss.outerOnionSecret), Some(attribution)) + } + case FailureReason.LocalFailure(failure) => + // This isn't a trampoline failure, so we only encrypt it for the node who created the outer onion. + val packet = Sphinx.FailurePacket.create(ss.outerOnionSecret, failure) + val attribution = Sphinx.Attribution.create(previousAttribution_opt = None, Some(packet), holdTime, ss.outerOnionSecret) + (Sphinx.FailurePacket.wrap(packet, ss.outerOnionSecret), Some(attribution)) + case FailureReason.LocalTrampolineFailure(failure) => + // This is a trampoline failure: we try to encrypt it to the node who created the trampoline onion. + ss.trampolineOnionSecret_opt match { + case Some(trampolineOnionSecret) if !ss.blinded => + val packet = Sphinx.FailurePacket.create(trampolineOnionSecret, failure) + val trampolinePacket = Sphinx.FailurePacket.wrap(packet, trampolineOnionSecret) + val trampolineAttribution = Sphinx.Attribution.create(previousAttribution_opt = None, Some(packet), trampolineHoldTime, trampolineOnionSecret) + val outerAttribution = Sphinx.Attribution.create(Some(trampolineAttribution), Some(trampolinePacket), holdTime, ss.outerOnionSecret) + (Sphinx.FailurePacket.wrap(trampolinePacket, ss.outerOnionSecret), Some(outerAttribution)) + case Some(trampolineOnionSecret) => + val packet = Sphinx.FailurePacket.create(trampolineOnionSecret, failure) + val trampolinePacket = Sphinx.FailurePacket.wrap(packet, trampolineOnionSecret) + (Sphinx.FailurePacket.wrap(trampolinePacket, ss.outerOnionSecret), None) + case None => + // This shouldn't happen, we only generate trampoline failures when there was a trampoline onion. + val packet = Sphinx.FailurePacket.create(ss.outerOnionSecret, failure) + (Sphinx.FailurePacket.wrap(packet, ss.outerOnionSecret), None) + } } - val tlvs: TlvStream[UpdateFailHtlcTlv] = if (useAttributableFailures) { - TlvStream(UpdateFailHtlcTlv.AttributionData(Sphinx.Attribution.create(attribution, Some(packet), holdTime, sharedSecret))) - } else { - TlvStream.empty - } - (Sphinx.FailurePacket.wrap(packet, sharedSecret), tlvs) }) } + private case class HtlcSharedSecrets(outerOnionSecret: ByteVector32, trampolineOnionSecret_opt: Option[ByteVector32], blinded: Boolean) + /** * We decrypt the onion again to extract the shared secret used to encrypt onion failures. * We could avoid this by storing the shared secret after the initial onion decryption, but we would have to store it * in the database since we must be able to fail HTLCs after restarting our node. * It's simpler to extract it again from the encrypted onion. */ - private def extractSharedSecret(nodeSecret: PrivateKey, add: UpdateAddHtlc): Either[CannotExtractSharedSecret, ByteVector32] = { + private def extractSharedSecret(nodeSecret: PrivateKey, add: UpdateAddHtlc): Either[CannotExtractSharedSecret, HtlcSharedSecrets] = { Sphinx.peel(nodeSecret, Some(add.paymentHash), add.onionRoutingPacket) match { - case Right(Sphinx.DecryptedPacket(_, _, sharedSecret)) => Right(sharedSecret) + case Right(Sphinx.DecryptedPacket(payload, _, outerOnionSecret)) => + // Let's look at the onion payload to see if it contains a trampoline onion. + PaymentOnionCodecs.perHopPayloadCodec.decode(payload.bits) match { + case Attempt.Successful(DecodeResult(perHopPayload, _)) => + // We try to extract the trampoline shared secret, if we can find one. + val trampolineOnionSecret_opt = perHopPayload.get[OnionPaymentPayloadTlv.TrampolineOnion].map(_.packet).flatMap(trampolinePacket => { + val trampolinePathKey_opt = perHopPayload.get[OnionPaymentPayloadTlv.PathKey].map(_.publicKey) + val trampolineOnionDecryptionKey = trampolinePathKey_opt.map(pathKey => Sphinx.RouteBlinding.derivePrivateKey(nodeSecret, pathKey)).getOrElse(nodeSecret) + Sphinx.peel(trampolineOnionDecryptionKey, Some(add.paymentHash), trampolinePacket).toOption.map(_.sharedSecret) + }) + // We check if we are an intermediate node in a blinded (potentially trampoline) path. + val blinded = trampolineOnionSecret_opt match { + case Some(_) => perHopPayload.get[OnionPaymentPayloadTlv.PathKey].nonEmpty + case None => add.pathKey_opt.nonEmpty + } + Right(HtlcSharedSecrets(outerOnionSecret, trampolineOnionSecret_opt, blinded)) + case Attempt.Failure(_) => Right(HtlcSharedSecrets(outerOnionSecret, None, blinded = false)) + } case Left(_) => Left(CannotExtractSharedSecret(add.channelId, add)) } } @@ -414,26 +465,38 @@ object OutgoingPaymentPacket { val failure = InvalidOnionBlinding(Sphinx.hash(add.onionRoutingPacket)) Right(UpdateFailMalformedHtlc(add.channelId, add.id, failure.onionHash, failure.code)) case None => - // If the htlcReceivedAt was lost (because the node restarted), we use a hold time of 0 which should be ignored by the payer. + // If the attribution data was lost (because the node restarted), we use a hold time of 0 which should be ignored by the payer. + val trampolineHoldTime = cmd.attribution_opt.flatMap(_.trampolineReceivedAt_opt).map(now - _).getOrElse(0 millisecond) val holdTime = cmd.attribution_opt.map(now - _.htlcReceivedAt).getOrElse(0 millisecond) - buildHtlcFailure(nodeSecret, useAttributableFailures, cmd.reason, add, holdTime).map { - case (encryptedReason, tlvs) => UpdateFailHtlc(add.channelId, cmd.id, encryptedReason, tlvs) + buildHtlcFailure(nodeSecret, cmd.reason, add, holdTime, trampolineHoldTime).map { + case (encryptedReason, attributionData_opt) => + val tlvs: Set[UpdateFailHtlcTlv] = Set( + if (useAttributableFailures) attributionData_opt.map(UpdateFailHtlcTlv.AttributionData(_)) else None + ).flatten + UpdateFailHtlc(add.channelId, cmd.id, encryptedReason, TlvStream(tlvs)) } } } def buildHtlcFulfill(nodeSecret: PrivateKey, useAttributionData: Boolean, cmd: CMD_FULFILL_HTLC, add: UpdateAddHtlc, now: TimestampMilli = TimestampMilli.now()): UpdateFulfillHtlc = { // If we are part of a blinded route, we must not populate attribution data. - val tlvs: TlvStream[UpdateFulfillHtlcTlv] = if (useAttributionData && add.pathKey_opt.isEmpty) { - extractSharedSecret(nodeSecret, add) match { - case Left(_) => TlvStream.empty - case Right(sharedSecret) => - val holdTime = cmd.attribution_opt.map(now - _.htlcReceivedAt).getOrElse(0 millisecond) - TlvStream(UpdateFulfillHtlcTlv.AttributionData(Sphinx.Attribution.create(cmd.attribution_opt.flatMap(_.downstreamAttribution_opt), None, holdTime, sharedSecret))) - } - } else { - TlvStream.empty + val attributionData_opt = add.pathKey_opt match { + case None if useAttributionData => + val trampolineHoldTime = cmd.attribution_opt.flatMap(_.trampolineReceivedAt_opt).map(now - _).getOrElse(0 millisecond) + val holdTime = cmd.attribution_opt.map(now - _.htlcReceivedAt).getOrElse(0 millisecond) + extractSharedSecret(nodeSecret, add) match { + case Right(HtlcSharedSecrets(outerOnionSecret, None, _)) => + Some(Sphinx.Attribution.create(cmd.attribution_opt.flatMap(_.downstreamAttribution_opt), None, holdTime, outerOnionSecret)) + case Right(HtlcSharedSecrets(outerOnionSecret, Some(trampolineOnionSecret), blinded)) if !blinded => + val trampolineAttribution = Sphinx.Attribution.create(cmd.attribution_opt.flatMap(_.downstreamAttribution_opt), None, trampolineHoldTime, trampolineOnionSecret) + Some(Sphinx.Attribution.create(Some(trampolineAttribution), None, holdTime, outerOnionSecret)) + case _ => None + } + case _ => None } - UpdateFulfillHtlc(add.channelId, cmd.id, cmd.r, tlvs) + val tlvs: Set[UpdateFulfillHtlcTlv] = Set( + attributionData_opt.map(UpdateFulfillHtlcTlv.AttributionData(_)) + ).flatten + UpdateFulfillHtlc(add.channelId, cmd.id, cmd.r, TlvStream(tlvs)) } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala index 9070661dc1..5236182695 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala @@ -473,7 +473,12 @@ object MultiPartHandler { private def validateStandardPayment(nodeParams: NodeParams, add: UpdateAddHtlc, payload: FinalPayload.Standard, record: IncomingStandardPayment, receivedAt: TimestampMilli)(implicit log: LoggingAdapter): Option[CMD_FAIL_HTLC] = { // We send the same error regardless of the failure to avoid probing attacks. val attribution = FailureAttributionData(htlcReceivedAt = receivedAt, trampolineReceivedAt_opt = None) - val cmdFail = CMD_FAIL_HTLC(add.id, FailureReason.LocalFailure(IncorrectOrUnknownPaymentDetails(payload.totalAmount, nodeParams.currentBlockHeight)), Some(attribution), commit = true) + val failure = if (payload.isTrampoline) { + FailureReason.LocalTrampolineFailure(IncorrectOrUnknownPaymentDetails(payload.totalAmount, nodeParams.currentBlockHeight)) + } else { + FailureReason.LocalFailure(IncorrectOrUnknownPaymentDetails(payload.totalAmount, nodeParams.currentBlockHeight)) + } + val cmdFail = CMD_FAIL_HTLC(add.id, failure, Some(attribution), commit = true) val commonOk = validateCommon(nodeParams, add, payload, record) val secretOk = validatePaymentSecret(add, payload, record.invoice) if (commonOk && secretOk) None else Some(cmdFail) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala index 69848628e8..ff30c8abf6 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala @@ -129,12 +129,15 @@ object NodeRelay { val amountOut = outgoingAmount(upstream, payloadOut) val expiryOut = outgoingExpiry(upstream, payloadOut) val fee = nodeFee(nodeParams.relayParams.minTrampolineFees, amountOut) + // We don't know yet how costly it is to reach the next node: we use a rough first estimate of twice our trampoline + // fees. If we fail to find routes, we will return a different error with higher fees and expiry delta. + val failure = TrampolineFeeOrExpiryInsufficient(nodeParams.relayParams.minTrampolineFees.feeBase * 2, nodeParams.relayParams.minTrampolineFees.feeProportionalMillionths * 2, nodeParams.channelConf.expiryDelta * 2) if (upstream.amountIn - amountOut < fee) { - Some(TrampolineFeeInsufficient()) + Some(failure) } else if (upstream.expiryIn - expiryOut < nodeParams.channelConf.expiryDelta) { - Some(TrampolineExpiryTooSoon()) + Some(failure) } else if (expiryOut <= CltvExpiry(nodeParams.currentBlockHeight)) { - Some(TrampolineExpiryTooSoon()) + Some(failure) } else if (amountOut <= MilliSatoshi(0)) { Some(InvalidOnionPayload(UInt64(2), 0)) } else { @@ -170,31 +173,41 @@ object NodeRelay { * This helper method translates relaying errors (returned by the downstream nodes) to a BOLT 4 standard error that we * should return upstream. */ - private def translateError(nodeParams: NodeParams, failures: Seq[PaymentFailure], upstream: Upstream.Hot.Trampoline, nextPayload: IntermediatePayload.NodeRelay): Option[FailureMessage] = { + private def translateError(nodeParams: NodeParams, failures: Seq[PaymentFailure], upstream: Upstream.Hot.Trampoline, nextPayload: IntermediatePayload.NodeRelay): FailureReason = { val amountOut = outgoingAmount(upstream, nextPayload) val routeNotFound = failures.collectFirst { case f@LocalFailure(_, _, RouteNotFound) => f }.nonEmpty val routingFeeHigh = upstream.amountIn - amountOut >= nodeFee(nodeParams.relayParams.minTrampolineFees, amountOut) * 5 + val trampolineFeesFailure = TrampolineFeeOrExpiryInsufficient(nodeParams.relayParams.minTrampolineFees.feeBase * 5, nodeParams.relayParams.minTrampolineFees.feeProportionalMillionths * 5, nodeParams.channelConf.expiryDelta * 5) + // We select the best error we can from our downstream attempts. failures match { - case Nil => None + case Nil => FailureReason.LocalTrampolineFailure(TemporaryTrampolineFailure()) case LocalFailure(_, _, BalanceTooLow) :: Nil if routingFeeHigh => // We have direct channels to the target node, but not enough outgoing liquidity to use those channels. - // The routing fee proposed by the sender was high enough to find alternative, indirect routes, but didn't yield - // any result so we tell them that we don't have enough outgoing liquidity at the moment. - Some(TemporaryNodeFailure()) - case LocalFailure(_, _, BalanceTooLow) :: Nil => Some(TrampolineFeeInsufficient()) // a higher fee/cltv may find alternative, indirect routes - case _ if routeNotFound => Some(TrampolineFeeInsufficient()) // if we couldn't find routes, it's likely that the fee/cltv was insufficient + // The routing fee proposed by the sender was high enough to find alternative, indirect routes, but didn't + // yield any result so we tell them that we don't have enough outgoing liquidity at the moment. + FailureReason.LocalTrampolineFailure(TemporaryTrampolineFailure()) + case LocalFailure(_, _, BalanceTooLow) :: Nil => + // A higher fee/cltv may find alternative, indirect routes. + FailureReason.LocalTrampolineFailure(trampolineFeesFailure) + case _ if routeNotFound => + // If we couldn't find routes, it's likely that the fee/cltv was insufficient. + FailureReason.LocalTrampolineFailure(trampolineFeesFailure) case _ => - // Otherwise, we try to find a downstream error that we could decrypt. - val outgoingNodeFailure = nextPayload match { - case nextPayload: IntermediatePayload.NodeRelay.Standard => failures.collectFirst { case RemoteFailure(_, _, e) if e.originNode == nextPayload.outgoingNodeId => e.failureMessage } - case nextPayload: IntermediatePayload.NodeRelay.ToNonTrampoline => failures.collectFirst { case RemoteFailure(_, _, e) if e.originNode == nextPayload.outgoingNodeId => e.failureMessage } + nextPayload match { + case _: IntermediatePayload.NodeRelay.Standard => + // If we received a failure from the next trampoline node, we won't be able to decrypt it: we should encrypt + // it with our trampoline shared secret and relay it upstream, because only the sender can decrypt it. + // Note that we currently don't process the downstream attribution data, but we could! + failures.collectFirst { case UnreadableRemoteFailure(_, _, packet, _) => FailureReason.EncryptedDownstreamFailure(packet.unwrapped, packet.attribution_opt) } + .getOrElse(FailureReason.LocalTrampolineFailure(TemporaryTrampolineFailure())) + case nextPayload: IntermediatePayload.NodeRelay.ToNonTrampoline => + // The recipient doesn't support trampoline: if we received a failure from them, we forward it upstream. + failures.collectFirst { case RemoteFailure(_, _, e) if e.originNode == nextPayload.outgoingNodeId => FailureReason.LocalFailure(e.failureMessage) } + .getOrElse(FailureReason.LocalTrampolineFailure(TemporaryTrampolineFailure())) // When using blinded paths, we will never get a failure from the final node (for privacy reasons). - case _: IntermediatePayload.NodeRelay.Blinded => None - case _: IntermediatePayload.NodeRelay.ToBlindedPaths => None + case _: IntermediatePayload.NodeRelay.Blinded => FailureReason.LocalTrampolineFailure(TemporaryTrampolineFailure()) + case _: IntermediatePayload.NodeRelay.ToBlindedPaths => FailureReason.LocalTrampolineFailure(TemporaryTrampolineFailure()) } - val otherNodeFailure = failures.collectFirst { case RemoteFailure(_, _, e) => e.failureMessage } - val failure = outgoingNodeFailure.getOrElse(otherNodeFailure.getOrElse(TemporaryNodeFailure())) - Some(failure) } } @@ -234,7 +247,9 @@ class NodeRelay private(nodeParams: NodeParams, case WrappedMultiPartPaymentFailed(MultiPartPaymentFSM.MultiPartPaymentFailed(_, failure, parts)) => context.log.warn("could not complete incoming multi-part payment (parts={} paidAmount={} failure={})", parts.size, parts.map(_.amount).sum, failure) Metrics.recordPaymentRelayFailed(failure.getClass.getSimpleName, Tags.RelayType.Trampoline) - parts.collect { case p: MultiPartPaymentFSM.HtlcPart => rejectHtlc(p.htlc.id, p.htlc.channelId, p.amount, p.receivedAt, None, Some(failure)) } + // Note that we don't treat this as a trampoline failure, which would be encrypted for the payer. + // This is a failure of the previous trampoline node who didn't send a valid MPP payment. + parts.collect { case p: MultiPartPaymentFSM.HtlcPart => rejectHtlc(p.htlc.id, p.htlc.channelId, p.amount, p.receivedAt, None, Some(FailureReason.LocalFailure(failure))) } stopping() case WrappedMultiPartPaymentSucceeded(MultiPartPaymentFSM.MultiPartPaymentSucceeded(_, parts)) => context.log.info("completed incoming multi-part payment with parts={} paidAmount={}", parts.size, parts.map(_.amount).sum) @@ -242,7 +257,7 @@ class NodeRelay private(nodeParams: NodeParams, validateRelay(nodeParams, upstream, nextPayload) match { case Some(failure) => context.log.warn(s"rejecting trampoline payment reason=$failure") - rejectPayment(upstream, Some(failure)) + rejectPayment(upstream, FailureReason.LocalTrampolineFailure(failure), nextPayload.isLegacy) stopping() case None => resolveNextNode(upstream, nextPayload, nextPacket_opt) @@ -277,7 +292,7 @@ class NodeRelay private(nodeParams: NodeParams, attemptWakeUpIfRecipientIsWallet(upstream, recipient, nextPayload, nextPacket_opt) case WrappedOutgoingNodeId(None) => context.log.warn("rejecting trampoline payment to blinded trampoline: cannot identify next node for scid={}", payloadOut.outgoing) - rejectPayment(upstream, Some(UnknownNextPeer())) + rejectPayment(upstream, FailureReason.LocalTrampolineFailure(UnknownNextPeer()), nextPayload.isLegacy) stopping() } } @@ -297,7 +312,7 @@ class NodeRelay private(nodeParams: NodeParams, rejectExtraHtlcPartialFunction orElse { case WrappedResolvedPaths(resolved) if resolved.isEmpty => context.log.warn("rejecting trampoline payment to blinded paths: no usable blinded path") - rejectPayment(upstream, Some(UnknownNextPeer())) + rejectPayment(upstream, FailureReason.LocalTrampolineFailure(UnknownNextPeer()), nextPayload.isLegacy) stopping() case WrappedResolvedPaths(resolved) => // We don't have access to the invoice: we use the only node_id that somewhat makes sense for the recipient. @@ -352,7 +367,7 @@ class NodeRelay private(nodeParams: NodeParams, rejectExtraHtlcPartialFunction orElse { case WrappedPeerReadyResult(_: PeerReadyNotifier.PeerUnavailable) => context.log.warn("rejecting payment: failed to wake-up remote peer") - rejectPayment(upstream, Some(UnknownNextPeer())) + rejectPayment(upstream, FailureReason.LocalTrampolineFailure(UnknownNextPeer()), nextPayload.isLegacy) stopping() case WrappedPeerReadyResult(r: PeerReadyNotifier.PeerReady) => relay(upstream, recipient, Some(walletNodeId), Some(r.remoteFeatures), nextPayload, nextPacket_opt) @@ -428,7 +443,7 @@ class NodeRelay private(nodeParams: NodeParams, context.log.info("trampoline payment failed, attempting on-the-fly funding") attemptOnTheFlyFunding(upstream, walletNodeId, recipient, nextPayload, failures, startedAt) case _ => - rejectPayment(upstream, translateError(nodeParams, failures, upstream, nextPayload)) + rejectPayment(upstream, translateError(nodeParams, failures, upstream, nextPayload), nextPayload.isLegacy) recordRelayDuration(startedAt, isSuccess = false) stopping() } @@ -451,7 +466,7 @@ class NodeRelay private(nodeParams: NodeParams, OutgoingPaymentPacket.buildOutgoingPayment(Origin.Hot(ActorRef.noSender, upstream), paymentHash, dummyRoute, recipient, Reputation.Score.max) match { case Left(f) => context.log.warn("could not create payment onion for on-the-fly funding: {}", f.getMessage) - rejectPayment(upstream, translateError(nodeParams, failures, upstream, nextPayload)) + rejectPayment(upstream, translateError(nodeParams, failures, upstream, nextPayload), nextPayload.isLegacy) recordRelayDuration(startedAt, isSuccess = false) stopping() case Right(nextPacket) => @@ -470,7 +485,7 @@ class NodeRelay private(nodeParams: NodeParams, stopping() case ProposeOnTheFlyFundingResponse.NotAvailable(reason) => context.log.warn("could not propose on-the-fly funding: {}", reason) - rejectPayment(upstream, Some(UnknownNextPeer())) + rejectPayment(upstream, FailureReason.LocalTrampolineFailure(UnknownNextPeer()), nextPayload.isLegacy) recordRelayDuration(startedAt, isSuccess = false) stopping() } @@ -509,16 +524,31 @@ class NodeRelay private(nodeParams: NodeParams, rejectHtlc(add.id, add.channelId, add.amountMsat, htlcReceivedAt, trampolineReceivedAt_opt = None) } - private def rejectHtlc(htlcId: Long, channelId: ByteVector32, amount: MilliSatoshi, htlcReceivedAt: TimestampMilli, trampolineReceivedAt_opt: Option[TimestampMilli], failure: Option[FailureMessage] = None): Unit = { - val failureMessage = failure.getOrElse(IncorrectOrUnknownPaymentDetails(amount, nodeParams.currentBlockHeight)) + private def rejectHtlc(htlcId: Long, channelId: ByteVector32, amount: MilliSatoshi, htlcReceivedAt: TimestampMilli, trampolineReceivedAt_opt: Option[TimestampMilli], failure_opt: Option[FailureReason] = None): Unit = { + val failure = failure_opt.getOrElse(FailureReason.LocalFailure(IncorrectOrUnknownPaymentDetails(amount, nodeParams.currentBlockHeight))) val attribution = FailureAttributionData(htlcReceivedAt, trampolineReceivedAt_opt) - val cmd = CMD_FAIL_HTLC(htlcId, FailureReason.LocalFailure(failureMessage), Some(attribution), commit = true) + val cmd = CMD_FAIL_HTLC(htlcId, failure, Some(attribution), commit = true) PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) } - private def rejectPayment(upstream: Upstream.Hot.Trampoline, failure: Option[FailureMessage]): Unit = { - Metrics.recordPaymentRelayFailed(failure.map(_.getClass.getSimpleName).getOrElse("Unknown"), Tags.RelayType.Trampoline) - upstream.received.foreach(r => rejectHtlc(r.add.id, r.add.channelId, upstream.amountIn, r.receivedAt, Some(upstream.receivedAt), failure)) + private def rejectPayment(upstream: Upstream.Hot.Trampoline, failure: FailureReason, isLegacy: Boolean): Unit = { + val failure1 = failure match { + case failure: FailureReason.EncryptedDownstreamFailure => + Metrics.recordPaymentRelayFailed("Unknown", Tags.RelayType.Trampoline) + failure + case failure: FailureReason.LocalFailure => + Metrics.recordPaymentRelayFailed(failure.getClass.getSimpleName, Tags.RelayType.Trampoline) + failure + case failure: FailureReason.LocalTrampolineFailure => + Metrics.recordPaymentRelayFailed(failure.getClass.getSimpleName, Tags.RelayType.Trampoline) + if (isLegacy) { + // The payer won't be able to decrypt our trampoline failure: we use a legacy failure for backwards-compat. + FailureReason.LocalFailure(LegacyTrampolineFeeInsufficient()) + } else { + failure + } + } + upstream.received.foreach(r => rejectHtlc(r.add.id, r.add.channelId, upstream.amountIn, r.receivedAt, Some(upstream.receivedAt), Some(failure1))) } private def fulfillPayment(upstream: Upstream.Hot.Trampoline, paymentPreimage: ByteVector32, downstreamAttribution_opt: Option[ByteVector]): Unit = upstream.received.foreach(r => { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/OnTheFlyFunding.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/OnTheFlyFunding.scala index ed791b7f00..e9d208378c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/OnTheFlyFunding.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/OnTheFlyFunding.scala @@ -107,16 +107,20 @@ object OnTheFlyFunding { val failure = failure_opt match { case Some(f) => f match { case f: FailureReason.EncryptedDownstreamFailure => - // In the trampoline case, we currently ignore downstream failures: we should add dedicated failures to - // the BOLTs to better handle those cases. Sphinx.FailurePacket.decrypt(f.packet, f.attribution_opt, onionSharedSecrets).failure match { - case Left(Sphinx.CannotDecryptFailurePacket(_, _)) => - log.warning("couldn't decrypt downstream on-the-fly funding failure") + case Left(Sphinx.CannotDecryptFailurePacket(unwrapped, attribution_opt)) => + log.info("received encrypted on-the-fly funding failure") + // If we cannot decrypt the error, it is encrypted for the payer using the trampoline onion secrets. + // We unwrap the outer onion encryption and will relay the error upstream. + FailureReason.EncryptedDownstreamFailure(unwrapped, attribution_opt) case Right(f) => log.warning("downstream on-the-fly funding failure: {}", f.failureMessage.message) + // Otherwise, there was an issue with the way we forwarded the payment to the recipient. + // We ignore the specific downstream failure and return a temporary trampoline failure to the sender. + FailureReason.LocalTrampolineFailure(TemporaryTrampolineFailure()) } - FailureReason.LocalFailure(TemporaryNodeFailure()) case _: FailureReason.LocalFailure => f + case _: FailureReason.LocalTrampolineFailure => f } case None => FailureReason.LocalFailure(UnknownNextPeer()) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/PostRestartHtlcCleaner.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/PostRestartHtlcCleaner.scala index 8c51b41677..247104502c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/PostRestartHtlcCleaner.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/PostRestartHtlcCleaner.scala @@ -279,7 +279,8 @@ class PostRestartHtlcCleaner(nodeParams: NodeParams, register: ActorRef, initial Metrics.Resolved.withTag(Tags.Success, value = false).withTag(Metrics.Relayed, value = true).increment() // We don't bother decrypting the downstream failure to forward a more meaningful error upstream, it's // very likely that it won't be actionable anyway because of our node restart. - PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, CMD_FAIL_HTLC(htlcId, FailureReason.LocalFailure(TemporaryNodeFailure()), None, commit = true)) + val failure = FailureReason.LocalTrampolineFailure(TemporaryTrampolineFailure()) + PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, CMD_FAIL_HTLC(htlcId, failure, None, commit = true)) } } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala index aa564e0798..6e70f9f223 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala @@ -127,7 +127,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A case Some(attribution) => val unwrapped = Sphinx.Attribution.unwrap(attribution, d.sharedSecrets) if (unwrapped.holdTimes.nonEmpty) { - context.system.eventStream.publish(Router.ReportedHoldTimes(unwrapped.holdTimes)) + context.system.eventStream.publish(Router.ReportedHoldTimes(holdTimes = unwrapped.holdTimes, trampolineHoldTimes = Nil)) } unwrapped.remaining_opt case None => None @@ -198,7 +198,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A import d._ val htlcFailure = Sphinx.FailurePacket.decrypt(fail.reason, fail.attribution_opt, sharedSecrets) if (htlcFailure.holdTimes.nonEmpty) { - context.system.eventStream.publish(Router.ReportedHoldTimes(htlcFailure.holdTimes)) + context.system.eventStream.publish(Router.ReportedHoldTimes(holdTimes = htlcFailure.holdTimes, trampolineHoldTimes = Nil)) } ((htlcFailure.failure match { case success@Right(e) => diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala index e5cde7e99c..418027b492 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/TrampolinePaymentLifecycle.scala @@ -60,7 +60,7 @@ object TrampolinePaymentLifecycle { } private case class TrampolinePeerNotFound(trampolineNodeId: PublicKey) extends Command private case class CouldntAddHtlc(failure: Throwable) extends Command - private case class HtlcSettled(result: HtlcResult, part: PartialPayment, holdTimes: Seq[Sphinx.HoldTime]) extends Command + private case class HtlcSettled(result: HtlcResult, part: PartialPayment, holdTimes: Seq[Sphinx.HoldTime], trampolineHoldTimes: Seq[Sphinx.HoldTime]) extends Command private case class WrappedPeerChannels(channels: Seq[Peer.ChannelInfo]) extends Command // @formatter:on @@ -130,23 +130,28 @@ object TrampolinePaymentLifecycle { } case WrappedHtlcSettled(result) => result.result match { case fulfill: HtlcResult.Fulfill => - val holdTimes = fulfill match { + val (holdTimes, trampolineHoldTimes) = fulfill match { case HtlcResult.RemoteFulfill(updateFulfill) => updateFulfill.attribution_opt match { - case Some(attribution) => Sphinx.Attribution.unwrap(attribution, outerOnionSecrets).holdTimes - case None => Nil + case Some(attribution) => Sphinx.Attribution.unwrap(attribution, outerOnionSecrets) match { + case Sphinx.Attribution.UnwrappedAttribution(holdTimes, Some(remaining)) => (holdTimes, Sphinx.Attribution.unwrap(remaining, trampolineOnionSecrets).holdTimes) + case Sphinx.Attribution.UnwrappedAttribution(holdTimes, None) => (holdTimes, Nil) + } + case None => (Nil, Nil) } - case _: HtlcResult.OnChainFulfill => Nil + case _: HtlcResult.OnChainFulfill => (Nil, Nil) } - parent ! HtlcSettled(fulfill, part, holdTimes) + parent ! HtlcSettled(fulfill, part, holdTimes, trampolineHoldTimes) Behaviors.stopped case fail: HtlcResult.Fail => - val holdTimes = fail match { - case HtlcResult.RemoteFail(updateFail) => - Sphinx.FailurePacket.decrypt(updateFail.reason, updateFail.attribution_opt, outerOnionSecrets).holdTimes - case _ => Nil + val (holdTimes, trampolineHoldTimes) = fail match { + case HtlcResult.RemoteFail(updateFail) => Sphinx.FailurePacket.decrypt(updateFail.reason, updateFail.attribution_opt, outerOnionSecrets) match { + case Sphinx.HtlcFailure(holdTimes, Left(Sphinx.CannotDecryptFailurePacket(unwrapped, attribution_opt))) => (holdTimes, Sphinx.FailurePacket.decrypt(unwrapped, attribution_opt, trampolineOnionSecrets).holdTimes) + case Sphinx.HtlcFailure(holdTimes, Right(_: Sphinx.DecryptedFailurePacket)) => (holdTimes, Nil) + } + case _ => (Nil, Nil) } - parent ! HtlcSettled(fail, part, holdTimes) + parent ! HtlcSettled(fail, part, holdTimes, trampolineHoldTimes) Behaviors.stopped } } @@ -224,9 +229,9 @@ class TrampolinePaymentLifecycle private(nodeParams: NodeParams, cmd.replyTo ! PaymentFailed(cmd.paymentId, paymentHash, LocalFailure(totalAmount, Nil, failure) :: Nil) Behaviors.stopped } - case HtlcSettled(result: HtlcResult, part, holdTimes) => + case HtlcSettled(result: HtlcResult, part, holdTimes, trampolineHoldTimes) => if (holdTimes.nonEmpty) { - context.system.eventStream ! EventStream.Publish(Router.ReportedHoldTimes(holdTimes)) + context.system.eventStream ! EventStream.Publish(Router.ReportedHoldTimes(holdTimes, trampolineHoldTimes)) } result match { case fulfill: HtlcResult.Fulfill => diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala index 28b8e95b43..defcc13452 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala @@ -844,5 +844,10 @@ object Router { /** We have tried to relay this amount from this channel and it failed. */ case class ChannelCouldNotRelay(amount: MilliSatoshi, hop: ChannelHop) - case class ReportedHoldTimes(holdTimes: Seq[Sphinx.HoldTime]) + /** + * @param holdTimes hold time reported by intermediate nodes in a payment path. + * @param trampolineHoldTimes hold time report by trampoline nodes in a trampoline payment path. + */ + case class ReportedHoldTimes(holdTimes: Seq[Sphinx.HoldTime], trampolineHoldTimes: Seq[Sphinx.HoldTime]) + } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/CommandCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/CommandCodecs.scala index 0d056ac68d..254024074c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/CommandCodecs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/CommandCodecs.scala @@ -43,6 +43,7 @@ object CommandCodecs { { case FailureReason.EncryptedDownstreamFailure(packet, _) => Left(packet) case FailureReason.LocalFailure(f) => Right(f) + case FailureReason.LocalTrampolineFailure(f) => Right(f) } )) :: ("attribution_opt" | provide(Option.empty[FailureAttributionData])) :: @@ -92,6 +93,7 @@ object CommandCodecs { { case FailureReason.EncryptedDownstreamFailure(packet, _) => Left(packet) case FailureReason.LocalFailure(f) => Right(f) + case FailureReason.LocalTrampolineFailure(f) => Right(f) } )) :: ("attribution_opt" | provide(Option.empty[FailureAttributionData])) :: diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/FailureMessage.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/FailureMessage.scala index 563fcce0ae..6fd96a1884 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/FailureMessage.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/FailureMessage.scala @@ -21,7 +21,7 @@ import fr.acinq.eclair.crypto.{Mac32, Sphinx} import fr.acinq.eclair.wire.protocol.CommonCodecs._ import fr.acinq.eclair.wire.protocol.FailureMessageCodecs.failureMessageCodec import fr.acinq.eclair.wire.protocol.LightningMessageCodecs.{channelFlagsCodec, channelUpdateCodec, messageFlagsCodec, meteredLightningMessageCodec} -import fr.acinq.eclair.{BlockHeight, CltvExpiry, MilliSatoshi, MilliSatoshiLong, UInt64} +import fr.acinq.eclair.{BlockHeight, CltvExpiry, CltvExpiryDelta, MilliSatoshi, MilliSatoshiLong, UInt64} import scodec.bits.ByteVector import scodec.codecs._ import scodec.{Attempt, Codec, Err} @@ -38,7 +38,9 @@ object FailureReason { /** An encrypted failure coming from downstream which we should re-encrypt and forward upstream. */ case class EncryptedDownstreamFailure(packet: ByteVector, attribution_opt: Option[ByteVector]) extends FailureReason /** A local failure that should be encrypted for the node that created the payment onion. */ - case class LocalFailure(failure: FailureMessage) extends FailureReason + case class LocalFailure(failure: FailureMessage) extends FailureReason + /** A local failure that should be encrypted for the node that created the trampoline onion. */ + case class LocalTrampolineFailure(failure: FailureMessage) extends FailureReason } // @formatter:on @@ -73,17 +75,19 @@ case class RequiredChannelFeatureMissing(tlvs: TlvStream[FailureMessageTlv] = Tl case class UnknownNextPeer(tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends Perm { def message = "processing node does not know the next peer in the route" } case class AmountBelowMinimum(amount: MilliSatoshi, update_opt: Option[ChannelUpdate], tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends Update { def message = "payment amount was below the minimum required by the channel" } case class FeeInsufficient(amount: MilliSatoshi, update_opt: Option[ChannelUpdate], tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends Update { def message = "payment fee was below the minimum required by the channel" } -case class TrampolineFeeInsufficient(tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends Node { def message = "payment fee was below the minimum required by the trampoline node" } case class ChannelDisabled(messageFlags: ChannelUpdate.MessageFlags, channelFlags: ChannelUpdate.ChannelFlags, update_opt: Option[ChannelUpdate], tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends Update { def message = "channel is currently disabled" } case class IncorrectCltvExpiry(expiry: CltvExpiry, update_opt: Option[ChannelUpdate], tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends Update { def message = "payment expiry doesn't match the value in the onion" } case class IncorrectOrUnknownPaymentDetails(amount: MilliSatoshi, height: BlockHeight, tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends Perm { def message = "incorrect payment details or unknown payment hash" } case class ExpiryTooSoon(update_opt: Option[ChannelUpdate], tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends Update { def message = "payment expiry is too close to the current block height for safe handling by the relaying node" } -case class TrampolineExpiryTooSoon(tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends Node { def message = "payment expiry is too close to the current block height for safe handling by the relaying node" } case class FinalIncorrectCltvExpiry(expiry: CltvExpiry, tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends FailureMessage { def message = "payment expiry doesn't match the value in the onion" } case class FinalIncorrectHtlcAmount(amount: MilliSatoshi, tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends FailureMessage { def message = "payment amount is incorrect in the final htlc" } case class ExpiryTooFar(tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends FailureMessage { def message = "payment expiry is too far in the future" } case class InvalidOnionPayload(tag: UInt64, offset: Int, tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends Perm { def message = s"onion per-hop payload is invalid (tag=$tag)" } case class PaymentTimeout(tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends FailureMessage { def message = "the complete payment amount was not received within a reasonable time" } +case class TemporaryTrampolineFailure(tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends Node { def message = "the trampoline node was unable to relay the payment because of downstream temporary failures" } +case class LegacyTrampolineFeeInsufficient(tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends Node { def message = "payment fee was below the minimum required by the trampoline node" } +case class TrampolineFeeOrExpiryInsufficient(feeBase: MilliSatoshi, feeProportionalMillionths: Long, expiryDelta: CltvExpiryDelta, tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends Node { def message = "trampoline fees or expiry are insufficient to relay the payment" } +case class UnknownNextTrampoline(tlvs: TlvStream[FailureMessageTlv] = TlvStream.empty) extends Perm { def message = "the trampoline node was unable to find the next trampoline node" } /** * We allow remote nodes to send us unknown failure codes (e.g. deprecated failure codes). @@ -161,10 +165,10 @@ object FailureMessageCodecs { .typecase(PERM | 22, (("tag" | varint) :: ("offset" | uint16) :: ("tlvs" | failureTlvsCodec)).as[InvalidOnionPayload]) .typecase(23, failureTlvsCodec.as[PaymentTimeout]) .typecase(BADONION | PERM | 24, (sha256 :: failureTlvsCodec).as[InvalidOnionBlinding]) - // TODO: @t-bast: once fully spec-ed, these should probably include a NodeUpdate and use a different ID. - // We should update Phoenix and our nodes at the same time, or first update Phoenix to understand both new and old errors. - .typecase(NODE | 51, failureTlvsCodec.as[TrampolineFeeInsufficient]) - .typecase(NODE | 52, failureTlvsCodec.as[TrampolineExpiryTooSoon]), + .typecase(NODE | 25, failureTlvsCodec.as[TemporaryTrampolineFailure]) + .typecase(NODE | 26, (("feeBaseMsat" | millisatoshi32) :: ("feeProportionalMillionths" | uint32) :: ("cltvExpiryDelta" | cltvExpiryDelta) :: failureTlvsCodec).as[TrampolineFeeOrExpiryInsufficient]) + .typecase(PERM | 27, failureTlvsCodec.as[UnknownNextTrampoline]) + .typecase(NODE | 51, failureTlvsCodec.as[LegacyTrampolineFeeInsufficient]), fallback = unknownFailureMessageCodec.upcast[FailureMessage] ) @@ -174,10 +178,11 @@ object FailureMessageCodecs { val failureReasonCodec: Codec[FailureReason] = discriminated[FailureReason].by(uint8) // Order matters: latest codec comes first, then old codecs for backward compatibility + .typecase(3, variableSizeBytes(uint16, failureMessageCodec).as[FailureReason.LocalTrampolineFailure]) .typecase(2, encryptedDownstreamFailure) - .typecase(0, (varsizebinarydata :: provide[Option[ByteVector]](None)).as[FailureReason.EncryptedDownstreamFailure]) .typecase(1, variableSizeBytes(uint16, failureMessageCodec).as[FailureReason.LocalFailure]) - + .typecase(0, (varsizebinarydata :: provide[Option[ByteVector]](None)).as[FailureReason.EncryptedDownstreamFailure]) + private def failureOnionPayload(payloadAndPadLength: Int): Codec[FailureMessage] = Codec( encoder = f => variableSizeBytes(uint16, failureMessageCodec).encode(f).flatMap(bits => { val payloadLength = bits.bytes.length - 2 diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala index 9161a2b882..2db563db38 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala @@ -51,6 +51,7 @@ import fr.acinq.eclair.wire.protocol.OfferTypes.{Offer, OfferPaths} import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelUpdate, IncorrectOrUnknownPaymentDetails} import fr.acinq.eclair.{CltvExpiryDelta, EclairImpl, EncodedNodeId, Features, Kit, MilliSatoshiLong, ShortChannelId, TimestampMilli, randomBytes32, randomKey} import org.json4s.JsonAST.{JString, JValue} +import org.scalatest.Inside.inside import scodec.bits.{ByteVector, HexStringSyntax} import java.util.UUID @@ -177,7 +178,10 @@ class PaymentIntegrationSpec extends IntegrationSpec { assert(Crypto.sha256(ps.paymentPreimage) == invoice.paymentHash) eventListener.expectMsg(PaymentMetadataReceived(invoice.paymentHash, invoice.paymentMetadata.get)) - assert(holdTimesRecorder.expectMsgType[Router.ReportedHoldTimes].holdTimes.map(_.remoteNodeId) == Seq("B", "C", "D").map(nodes(_).nodeParams.nodeId)) + inside(holdTimesRecorder.expectMsgType[Router.ReportedHoldTimes]) { e => + assert(e.holdTimes.map(_.remoteNodeId) == Seq("B", "C", "D").map(nodes(_).nodeParams.nodeId)) + assert(e.trampolineHoldTimes.isEmpty) + } } test("send an HTLC A->D with an invalid expiry delta for B") { @@ -263,7 +267,10 @@ class PaymentIntegrationSpec extends IntegrationSpec { assert(failed.failures.size == 1) assert(failed.failures.head.asInstanceOf[RemoteFailure].e == DecryptedFailurePacket(nodes("D").nodeParams.nodeId, IncorrectOrUnknownPaymentDetails(amount, getBlockHeight()))) - assert(holdTimesRecorder.expectMsgType[Router.ReportedHoldTimes].holdTimes.map(_.remoteNodeId) == Seq("B", "C", "D").map(nodes(_).nodeParams.nodeId)) + inside(holdTimesRecorder.expectMsgType[Router.ReportedHoldTimes]) { e => + assert(e.holdTimes.map(_.remoteNodeId) == Seq("B", "C", "D").map(nodes(_).nodeParams.nodeId)) + assert(e.trampolineHoldTimes.isEmpty) + } } test("send an HTLC A->D with a lower amount than requested") { @@ -506,8 +513,14 @@ class PaymentIntegrationSpec extends IntegrationSpec { assert(relayed.amountIn - relayed.amountOut > 0.msat, relayed) assert(relayed.amountIn - relayed.amountOut < paymentSent.feesPaid, relayed) - assert(holdTimesRecorderG.expectMsgType[Router.ReportedHoldTimes].holdTimes.map(_.remoteNodeId) == Seq("C", "F").map(nodes(_).nodeParams.nodeId)) - assert(holdTimesRecorderB.expectMsgType[Router.ReportedHoldTimes].holdTimes.map(_.remoteNodeId) == Seq("G").map(nodes(_).nodeParams.nodeId)) + inside(holdTimesRecorderG.expectMsgType[Router.ReportedHoldTimes]) { e => + assert(e.holdTimes.map(_.remoteNodeId) == Seq("C", "F").map(nodes(_).nodeParams.nodeId)) + assert(e.trampolineHoldTimes.isEmpty) + } + inside(holdTimesRecorderB.expectMsgType[Router.ReportedHoldTimes]) { e => + assert(e.holdTimes.map(_.remoteNodeId) == Seq("G").map(nodes(_).nodeParams.nodeId)) + assert(e.trampolineHoldTimes.map(_.remoteNodeId) == Seq("G", "F").map(nodes(_).nodeParams.nodeId)) + } } test("send a trampoline payment D->B (via trampoline C)") { @@ -607,7 +620,10 @@ class PaymentIntegrationSpec extends IntegrationSpec { assert(paymentFailed.id == paymentId, paymentFailed) assert(paymentFailed.paymentHash == invoice.paymentHash, paymentFailed) - assert(holdTimesRecorder.expectMsgType[Router.ReportedHoldTimes].holdTimes.map(_.remoteNodeId) == Seq("C").map(nodes(_).nodeParams.nodeId)) + inside(holdTimesRecorder.expectMsgType[Router.ReportedHoldTimes]) { e => + assert(e.holdTimes.map(_.remoteNodeId) == Seq("C").map(nodes(_).nodeParams.nodeId)) + assert(e.trampolineHoldTimes.map(_.remoteNodeId) == Seq("C").map(nodes(_).nodeParams.nodeId)) + } } test("send a trampoline payment A->D (temporary remote failure at trampoline)") { @@ -628,8 +644,14 @@ class PaymentIntegrationSpec extends IntegrationSpec { assert(paymentFailed.id == paymentId, paymentFailed) assert(paymentFailed.paymentHash == invoice.paymentHash, paymentFailed) - assert(holdTimesRecorderB.expectMsgType[Router.ReportedHoldTimes].holdTimes.map(_.remoteNodeId) == Seq("C").map(nodes(_).nodeParams.nodeId)) - assert(holdTimesRecorderA.expectMsgType[Router.ReportedHoldTimes].holdTimes.map(_.remoteNodeId) == Seq("B").map(nodes(_).nodeParams.nodeId)) + inside(holdTimesRecorderB.expectMsgType[Router.ReportedHoldTimes]) { e => + assert(e.holdTimes.map(_.remoteNodeId) == Seq("C").map(nodes(_).nodeParams.nodeId)) + assert(e.trampolineHoldTimes.isEmpty) + } + inside(holdTimesRecorderA.expectMsgType[Router.ReportedHoldTimes]) { e => + assert(e.holdTimes.map(_.remoteNodeId) == Seq("B").map(nodes(_).nodeParams.nodeId)) + assert(e.trampolineHoldTimes.map(_.remoteNodeId) == Seq("B").map(nodes(_).nodeParams.nodeId)) + } } test("send a blinded payment B->D with many blinded routes") { @@ -816,8 +838,14 @@ class PaymentIntegrationSpec extends IntegrationSpec { val Some(IncomingBlindedPayment(_, _, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("D").nodeParams.db.payments.getIncomingPayment(paymentSent.paymentHash) assert(receivedAmount >= amount) - assert(holdTimesRecorderB.expectMsgType[Router.ReportedHoldTimes].holdTimes.map(_.remoteNodeId) == Seq("C").map(nodes(_).nodeParams.nodeId)) - assert(holdTimesRecorderA.expectMsgType[Router.ReportedHoldTimes].holdTimes.map(_.remoteNodeId) == Seq("B").map(nodes(_).nodeParams.nodeId)) + inside(holdTimesRecorderB.expectMsgType[Router.ReportedHoldTimes]) { e => + assert(e.holdTimes.map(_.remoteNodeId) == Seq("C").map(nodes(_).nodeParams.nodeId)) + assert(e.trampolineHoldTimes.isEmpty) + } + inside(holdTimesRecorderA.expectMsgType[Router.ReportedHoldTimes]) { e => + assert(e.holdTimes.map(_.remoteNodeId) == Seq("B").map(nodes(_).nodeParams.nodeId)) + assert(e.trampolineHoldTimes.map(_.remoteNodeId) == Seq("B", "C").map(nodes(_).nodeParams.nodeId)) + } } test("send a blinded payment D->A with trampoline (non-trampoline recipient)") { @@ -830,7 +858,8 @@ class PaymentIntegrationSpec extends IntegrationSpec { val offerHandler = TypedProbe[HandlerCommand]()(nodes("A").system.toTyped) nodes("A").offerManager ! RegisterOffer(offer, Some(offerKey), Some(pathId), offerHandler.ref) - val sender = TestProbe() + val (sender, holdTimesRecorder) = (TestProbe(), TestProbe()) + nodes("D").system.eventStream.subscribe(holdTimesRecorder.ref, classOf[Router.ReportedHoldTimes]) val dave = new EclairImpl(nodes("D")) dave.payOfferTrampoline(offer, amount, 1, nodes("C").nodeParams.nodeId, maxAttempts_opt = Some(1))(30 seconds).pipeTo(sender.ref) @@ -853,6 +882,11 @@ class PaymentIntegrationSpec extends IntegrationSpec { awaitCond(nodes("A").nodeParams.db.payments.getIncomingPayment(paymentSent.paymentHash).exists(_.status.isInstanceOf[IncomingPaymentStatus.Received])) val Some(IncomingBlindedPayment(_, _, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("A").nodeParams.db.payments.getIncomingPayment(paymentSent.paymentHash) assert(receivedAmount >= amount) + + inside(holdTimesRecorder.expectMsgType[Router.ReportedHoldTimes]) { e => + assert(e.holdTimes.map(_.remoteNodeId) == Seq("C").map(nodes(_).nodeParams.nodeId)) + assert(e.trampolineHoldTimes.map(_.remoteNodeId) == Seq("C").map(nodes(_).nodeParams.nodeId)) + } } test("send to compact route") { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala index d48b64e644..98e3afdf0d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala @@ -22,7 +22,7 @@ import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, Crypto} import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional} import fr.acinq.eclair.Features.{KeySend, _} import fr.acinq.eclair.TestConstants.Alice -import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FULFILL_HTLC, FailureAttributionData, FulfillAttributionData, Register} +import fr.acinq.eclair.channel._ import fr.acinq.eclair.db.{IncomingBlindedPayment, IncomingPaymentStatus, IncomingStandardPayment, PaymentType} import fr.acinq.eclair.payment.Bolt11Invoice.ExtraHop import fr.acinq.eclair.payment.PaymentReceived.PartialPayment @@ -30,7 +30,6 @@ import fr.acinq.eclair.payment.offer.OfferManager import fr.acinq.eclair.payment.receive.MultiPartHandler._ import fr.acinq.eclair.payment.receive.MultiPartPaymentFSM.HtlcPart import fr.acinq.eclair.payment.receive.{MultiPartPaymentFSM, PaymentHandler} -import fr.acinq.eclair.payment.relay.Relayer.RelayFees import fr.acinq.eclair.reputation.Reputation import fr.acinq.eclair.router.BlindedRouteCreation.aggregatePaymentInfo import fr.acinq.eclair.router.Router @@ -62,7 +61,8 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike val featuresWithMpp = Features[Feature]( VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, - BasicMultiPartPayment -> Optional + BasicMultiPartPayment -> Optional, + TrampolinePayment -> Optional, ) val featuresWithKeySend = Features[Feature]( @@ -75,6 +75,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, + TrampolinePayment -> Optional, RouteBlinding -> Optional, ) @@ -242,14 +243,14 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike assert(!invoice.features.hasFeature(TrampolinePayment)) } { - val handler = TestActorRef[PaymentHandler](PaymentHandler.props(Alice.nodeParams.copy(features = featuresWithMpp), TestProbe().ref, TestProbe().ref)) + val handler = TestActorRef[PaymentHandler](PaymentHandler.props(Alice.nodeParams.copy(features = featuresWithMpp.remove(TrampolinePayment)), TestProbe().ref, TestProbe().ref)) sender.send(handler, ReceiveStandardPayment(sender.ref, Some(42 msat), Left("1 coffee"))) val invoice = sender.expectMsgType[Bolt11Invoice] assert(invoice.features.hasFeature(BasicMultiPartPayment)) assert(!invoice.features.hasFeature(TrampolinePayment)) } { - val handler = TestActorRef[PaymentHandler](PaymentHandler.props(Alice.nodeParams.copy(features = featuresWithMpp.add(TrampolinePayment, Optional)), TestProbe().ref, TestProbe().ref)) + val handler = TestActorRef[PaymentHandler](PaymentHandler.props(Alice.nodeParams.copy(features = featuresWithMpp), TestProbe().ref, TestProbe().ref)) sender.send(handler, ReceiveStandardPayment(sender.ref, Some(42 msat), Left("1 coffee"))) val invoice = sender.expectMsgType[Bolt11Invoice] assert(invoice.features.hasFeature(BasicMultiPartPayment)) @@ -433,6 +434,22 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike assert(nodeParams.db.payments.getIncomingPayment(invoice.paymentHash).get.status == IncomingPaymentStatus.Pending) } + test("PaymentHandler should reject incoming trampoline payment with invalid payment secret") { f => + import f._ + + sender.send(handlerWithMpp, ReceiveStandardPayment(sender.ref, Some(1000 msat), Left("trampoline invalid payment secret"))) + val invoice = sender.expectMsgType[Bolt11Invoice] + assert(invoice.features.hasFeature(TrampolinePayment)) + + // Invalid payment secret. + val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, invoice.paymentHash, defaultExpiry, TestConstants.emptyOnionPacket, None, Reputation.maxEndorsement, None) + val trampolineOnion = TestConstants.emptyOnionPacket + sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, FinalPayload.Standard.createPayload(add.amountMsat, 1000 msat, add.cltvExpiry, invoice.paymentSecret.reverse, invoice.paymentMetadata, trampolineOnion_opt = Some(trampolineOnion)), TimestampMilli.now())) + val cmd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message + assert(cmd.reason == FailureReason.LocalTrampolineFailure(IncorrectOrUnknownPaymentDetails(1000 msat, nodeParams.currentBlockHeight))) + assert(nodeParams.db.payments.getIncomingPayment(invoice.paymentHash).get.status == IncomingPaymentStatus.Pending) + } + test("PaymentHandler should reject incoming blinded payment for Bolt 11 invoice") { f => import f._ diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala index 008d6ced38..bd3684db0f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentPacketSpec.scala @@ -1194,6 +1194,491 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { assert(decryptedFailure == failure) } + test("build htlc failure onion with attribution data (trampoline payment)") { + // Create a trampoline payment to e: + // .--> d --. + // / \ + // b -> c e + val invoiceFeatures = Features[Bolt11Feature](VariableLengthOnion -> Mandatory, PaymentSecret -> Mandatory, BasicMultiPartPayment -> Optional, Features.TrampolinePayment -> Optional) + val invoice = Bolt11Invoice(Block.RegtestGenesisBlock.hash, Some(finalAmount), paymentHash, priv_e.privateKey, Left("invoice"), CltvExpiryDelta(12), paymentSecret = paymentSecret, features = invoiceFeatures) + val payment = TrampolinePayment.buildOutgoingPayment(c, invoice, finalExpiry) + + val add_c = UpdateAddHtlc(randomBytes32(), 0, payment.trampolineAmount, paymentHash, payment.trampolineExpiry, payment.onion.packet, None, Reputation.maxEndorsement, None) + val Right(RelayToTrampolinePacket(_, _, payload_c, trampolinePacket_e, _)) = decrypt(add_c, priv_c.privateKey, Features.empty) + val (add_d, sharedSecrets_c) = { + // c finds a path c->d->e + val payloads = Seq( + NodePayload(d, PaymentOnion.IntermediatePayload.ChannelRelay.Standard(channelUpdate_de.shortChannelId, payload_c.amountToForward, payload_c.outgoingCltv)), + NodePayload(e, PaymentOnion.FinalPayload.Standard.createTrampolinePayload(payload_c.amountToForward, payload_c.amountToForward, payload_c.outgoingCltv, paymentSecret, trampolinePacket_e, None)) + ) + val onion_d = OutgoingPaymentPacket.buildOnion(payloads, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get + val add_d = UpdateAddHtlc(randomBytes32(), 0, payload_c.amountToForward + 500.msat, paymentHash, payload_c.outgoingCltv + CltvExpiryDelta(36), onion_d.packet, None, Reputation.maxEndorsement, None) + (add_d, onion_d.sharedSecrets) + } + val Right(ChannelRelayPacket(_, _, packet_e, _)) = decrypt(add_d, priv_d.privateKey, Features.empty) + val add_e = UpdateAddHtlc(randomBytes32(), 3, payload_c.amountToForward, paymentHash, payload_c.outgoingCltv, packet_e, None, Reputation.maxEndorsement, None) + val Right(FinalPacket(_, payload_e, _)) = decrypt(add_e, priv_e.privateKey, Features.empty) + assert(payload_e.isInstanceOf[FinalPayload.Standard]) + + // e returns a trampoline failure + val failure = IncorrectOrUnknownPaymentDetails(finalAmount, BlockHeight(currentBlockCount)) + val Right(fail_e: UpdateFailHtlc) = buildHtlcFailure(priv_e.privateKey, useAttributableFailures = true, CMD_FAIL_HTLC(add_e.id, FailureReason.LocalTrampolineFailure(failure), Some(FailureAttributionData(TimestampMilli(500), Some(TimestampMilli(520))))), add_e, now = TimestampMilli(550)) + assert(fail_e.id == add_e.id) + val Right(fail_d: UpdateFailHtlc) = buildHtlcFailure(priv_d.privateKey, useAttributableFailures = true, CMD_FAIL_HTLC(add_d.id, FailureReason.EncryptedDownstreamFailure(fail_e.reason, fail_e.attribution_opt), Some(FailureAttributionData(TimestampMilli(350), None))), add_d, now = TimestampMilli(560)) + assert(fail_d.id == add_d.id) + // c tries to decrypt the failure but cannot because it's encrypted for b, so she relays it upstream. + // c is however able to decrypt attribution data for the downstream path. + val failureDetails_c = Sphinx.FailurePacket.decrypt(fail_d.reason, fail_d.attribution_opt, sharedSecrets_c) + assert(failureDetails_c.holdTimes == Seq(HoldTime(200 millis, d), HoldTime(0 millis, e))) + assert(failureDetails_c.failure.isLeft) + val Sphinx.CannotDecryptFailurePacket(unwrapped_c, attribution_c) = failureDetails_c.failure.left.toOption.get + val Right(fail_b: UpdateFailHtlc) = buildHtlcFailure(priv_c.privateKey, useAttributableFailures = true, CMD_FAIL_HTLC(add_c.id, FailureReason.EncryptedDownstreamFailure(unwrapped_c, attribution_c), Some(FailureAttributionData(TimestampMilli(250), Some(TimestampMilli(280))))), add_c, now = TimestampMilli(600)) + // b decrypts the failure with the outer onion secrets *and* trampoline onion secrets + val failureDetails_b = Sphinx.FailurePacket.decrypt(fail_b.reason, fail_b.attribution_opt, payment.onion.sharedSecrets) + assert(failureDetails_b.holdTimes == Seq(HoldTime(300 millis, c))) + assert(failureDetails_b.failure.isLeft) + val Sphinx.CannotDecryptFailurePacket(unwrapped_b, attribution_b) = failureDetails_b.failure.left.toOption.get + val Sphinx.HtlcFailure(holdTimes, Right(Sphinx.DecryptedFailurePacket(failingNode, decryptedFailure))) = Sphinx.FailurePacket.decrypt(unwrapped_b, attribution_b, payment.trampolineOnion.sharedSecrets) + assert(holdTimes == List(Sphinx.HoldTime(300 millis, c), Sphinx.HoldTime(0 millis, e))) + assert(failingNode == e) + assert(decryptedFailure == failure) + } + + test("build htlc failure onion (blinded trampoline payment)") { + // Eve creates a trampoline blinded path to herself going through Carol and Dave. + // .-> Dave -. + // / \ + // Alice -> Bob -> Carol Eve + val path = { + val blindedPayloadEve = TlvStream[RouteBlindingEncryptedDataTlv](RouteBlindingEncryptedDataTlv.PathId(randomBytes32())) + val blindedPayloadDave = TlvStream[RouteBlindingEncryptedDataTlv]( + RouteBlindingEncryptedDataTlv.OutgoingNodeId(EncodedNodeId(e)), + RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(36), 750, 2_000 msat), + RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(850_000), 1 msat), + ) + val blindedPayloadCarol = TlvStream[RouteBlindingEncryptedDataTlv]( + RouteBlindingEncryptedDataTlv.OutgoingNodeId(EncodedNodeId(d)), + RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(64), 500, 3_000 msat), + RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(850_000), 1 msat), + ) + val blindedPayloads = Seq(blindedPayloadCarol, blindedPayloadDave, blindedPayloadEve).map { p => RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec.encode(p).require.bytes } + val blindedRouteDetails = Sphinx.RouteBlinding.create(randomKey(), Seq(c, d, e), blindedPayloads) + val paymentInfo = OfferTypes.PaymentInfo(10_000 msat, 0, CltvExpiryDelta(100), 1 msat, 500_000_000 msat, ByteVector.empty) + PaymentBlindedRoute(blindedRouteDetails.route, paymentInfo) + } + + // Alice creates a trampoline onion using Eve's blinded path. + val trampolineOnion = { + val payloadEve = PaymentOnion.FinalPayload.Blinded( + records = TlvStream( + OnionPaymentPayloadTlv.AmountToForward(150_000_000 msat), + OnionPaymentPayloadTlv.OutgoingCltv(CltvExpiry(800_000)), + OnionPaymentPayloadTlv.TotalAmount(150_000_000 msat), + OnionPaymentPayloadTlv.EncryptedRecipientData(path.route.encryptedPayloads.last), + ), + blindedRecords = TlvStream(RouteBlindingEncryptedDataTlv.PathId(randomBytes32())), + ) + val payloadDave = PaymentOnion.IntermediatePayload.NodeRelay.Blinded( + records = TlvStream(OnionPaymentPayloadTlv.EncryptedRecipientData(path.route.encryptedPayloads(1))), + paymentRelayData = BlindedRouteData.PaymentRelayData(TlvStream( + RouteBlindingEncryptedDataTlv.OutgoingNodeId(EncodedNodeId(e)), + RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(36), 750, 2_000 msat), + RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(850_000), 1 msat), + )), + nextPathKey = randomKey().publicKey + ) + val payloadCarol = PaymentOnion.IntermediatePayload.NodeRelay.Blinded( + records = TlvStream( + OnionPaymentPayloadTlv.EncryptedRecipientData(path.route.encryptedPayloads.head), + OnionPaymentPayloadTlv.PathKey(path.route.firstPathKey), + ), + paymentRelayData = BlindedRouteData.PaymentRelayData(TlvStream( + RouteBlindingEncryptedDataTlv.OutgoingNodeId(EncodedNodeId(d)), + RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(64), 500, 3_000 msat), + RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(850_000), 1 msat), + )), + nextPathKey = randomKey().publicKey + ) + OutgoingPaymentPacket.buildOnion(NodePayload(c, payloadCarol) :: NodePayload(path.route.blindedNodeIds(1), payloadDave) :: NodePayload(path.route.blindedNodeIds.last, payloadEve) :: Nil, paymentHash, None).toOption.get + } + + // Alice creates a payment onion for Carol (Alice -> Bob -> Carol). + val (htlcForBob, sharedSecretsAlice) = { + val payloadBob = PaymentOnion.IntermediatePayload.ChannelRelay.Standard(ShortChannelId(1105), 150_005_000 msat, CltvExpiry(800_100)) + val payloadCarol = PaymentOnion.TrampolineWithoutMppPayload.create(150_005_000 msat, CltvExpiry(800_100), trampolineOnion.packet) + val onion = OutgoingPaymentPacket.buildOnion(NodePayload(b, payloadBob) :: NodePayload(c, payloadCarol) :: Nil, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get + (UpdateAddHtlc(randomBytes32(), 1, 150_010_000 msat, paymentHash, CltvExpiry(800_150), onion.packet, None, Reputation.maxEndorsement, None), onion.sharedSecrets) + } + + // Bob decrypts the onion and relays to Carol. + val htlcForCarol = { + val Right(packetForBob: ChannelRelayPacket) = decrypt(htlcForBob, priv_b.privateKey, Features.empty) + UpdateAddHtlc(randomBytes32(), 1, packetForBob.amountToForward, paymentHash, packetForBob.outgoingCltv, packetForBob.nextPacket, None, Reputation.maxEndorsement, None) + } + + // Carol decrypts the onion and relays to Dave. + val (htlcForDave, sharedSecretsCarol) = { + val Right(RelayToBlindedTrampolinePacket(_, _, innerPayload, trampolineOnionForDave, _)) = decrypt(htlcForCarol, priv_c.privateKey, Features(Features.RouteBlinding -> FeatureSupport.Optional)) + assert(innerPayload.outgoing == Left(EncodedNodeId.WithPublicKey.Plain(d))) + val outgoingAmount = innerPayload.outgoingAmount(htlcForCarol.amountMsat) + val outgoingExpiry = innerPayload.outgoingExpiry(htlcForCarol.cltvExpiry) + val payloadDave = PaymentOnion.TrampolineWithoutMppPayload.create(outgoingAmount, outgoingExpiry, trampolineOnionForDave, Some(innerPayload.nextPathKey)) + val onion = OutgoingPaymentPacket.buildOnion(NodePayload(d, payloadDave) :: Nil, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get + (UpdateAddHtlc(randomBytes32(), 1, outgoingAmount, paymentHash, outgoingExpiry, onion.packet, None, Reputation.maxEndorsement, None), onion.sharedSecrets) + } + + // Dave decrypts the onion and relays to Eve. + val (htlcForEve, sharedSecretsDave) = { + val Right(RelayToBlindedTrampolinePacket(_, _, innerPayload, trampolineOnionForEve, _)) = decrypt(htlcForDave, priv_d.privateKey, Features(Features.RouteBlinding -> FeatureSupport.Optional)) + assert(innerPayload.outgoing == Left(EncodedNodeId.WithPublicKey.Plain(e))) + val outgoingAmount = innerPayload.outgoingAmount(htlcForDave.amountMsat) + val outgoingExpiry = innerPayload.outgoingExpiry(htlcForDave.cltvExpiry) + val payloadEve = PaymentOnion.TrampolineWithoutMppPayload.create(outgoingAmount, outgoingExpiry, trampolineOnionForEve, Some(innerPayload.nextPathKey)) + val onion = OutgoingPaymentPacket.buildOnion(NodePayload(e, payloadEve) :: Nil, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get + (UpdateAddHtlc(randomBytes32(), 1, outgoingAmount, paymentHash, outgoingExpiry, onion.packet, None, Reputation.maxEndorsement, None), onion.sharedSecrets) + } + + // Eve decrypts the onion and returns a non-blinded failure. + val failure = IncorrectOrUnknownPaymentDetails(150_000_000 msat, BlockHeight(800_000)) + val failedAt = TimestampMilli.now() + val failureForDave = { + val Right(FinalPacket(_, payload, _)) = decrypt(htlcForEve, priv_e.privateKey, Features(Features.RouteBlinding -> FeatureSupport.Optional)) + assert(payload.isInstanceOf[FinalPayload.Blinded]) + val Right(fail: UpdateFailHtlc) = buildHtlcFailure(priv_e.privateKey, useAttributableFailures = true, CMD_FAIL_HTLC(htlcForEve.id, FailureReason.LocalTrampolineFailure(failure), Some(FailureAttributionData(failedAt - 150.millis, Some(failedAt - 140.millis)))), htlcForEve, failedAt) + fail + } + + // Dave cannot decrypt the failure, so he forwards it to Carol. + val failureForCarol = { + val Left(Sphinx.CannotDecryptFailurePacket(unwrapped, attribution)) = Sphinx.FailurePacket.decrypt(failureForDave.reason, failureForDave.attribution_opt, sharedSecretsDave).failure + val Right(fail: UpdateFailHtlc) = buildHtlcFailure(priv_d.privateKey, useAttributableFailures = true, CMD_FAIL_HTLC(failureForDave.id, FailureReason.EncryptedDownstreamFailure(unwrapped, attribution), Some(FailureAttributionData(failedAt - 250.millis, Some(failedAt - 240.millis)))), htlcForDave, failedAt + 50.millis) + fail + } + + // Carol cannot decrypt the failure, so she forwards it to Bob. + val failureForBob = { + val Left(Sphinx.CannotDecryptFailurePacket(unwrapped, attribution)) = Sphinx.FailurePacket.decrypt(failureForCarol.reason, failureForCarol.attribution_opt, sharedSecretsCarol).failure + val Right(fail: UpdateFailHtlc) = buildHtlcFailure(priv_c.privateKey, useAttributableFailures = true, CMD_FAIL_HTLC(failureForCarol.id, FailureReason.EncryptedDownstreamFailure(unwrapped, attribution), Some(FailureAttributionData(failedAt - 400.millis, Some(failedAt - 450.millis)))), htlcForCarol, failedAt + 100.millis) + fail + } + + // Bob cannot decrypt the failure, so he forwards it to Alice. + val failureForAlice = { + val Right(fail: UpdateFailHtlc) = buildHtlcFailure(priv_b.privateKey, useAttributableFailures = true, CMD_FAIL_HTLC(failureForBob.id, FailureReason.EncryptedDownstreamFailure(failureForBob.reason, failureForBob.attribution_opt), Some(FailureAttributionData(failedAt - 550.millis, None))), htlcForBob, failedAt + 150.millis) + fail + } + + // Alice is able to decrypt the failure using the outer onion and trampoline onion shared secrets. + val Sphinx.HtlcFailure(holdTimes, Right(decrypted)) = Sphinx.FailurePacket.decrypt(failureForAlice.reason, failureForAlice.attribution_opt, sharedSecretsAlice ++ trampolineOnion.sharedSecrets) + assert(holdTimes == List(Sphinx.HoldTime(700 millis, b), Sphinx.HoldTime(500 millis, c), Sphinx.HoldTime(500 millis, c))) + assert(decrypted.failureMessage == failure) + assert(decrypted.originNode == path.route.blindedNodeIds.last) + } + + // See bolt04/trampoline-onion-error-test.json + test("build htlc failure onion (trampoline reference test vector)") { + // .-> Dave -. + // / \ + // Alice -> Bob -> Carol Eve + val paymentHash = ByteVector32.fromValidHex("4242424242424242424242424242424242424242424242424242424242424242") + val bob = PublicKey(hex"0324653eac434488002cc06bbfb7f10fe18991e35f9fe4302dbea6d2353dc0ab1c") + val carol = PublicKey(hex"027f31ebc5462c1fdce1b737ecff52d37d75dea43ce11c74d25aa297165faa2007") + val eve = PublicKey(hex"02edabbd16b41c8371b92ef2f04c1185b4f03b6dcd52ba9b78d9d7c89c8f221145") + + // Alice creates a trampoline onion Carol -> Eve. + val trampolineOnionForCarol = { + val paymentSecret = ByteVector32.fromValidHex("2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a") + val payloads = Seq( + NodePayload(carol, PaymentOnion.IntermediatePayload.NodeRelay.Standard(100_000_000 msat, CltvExpiry(800_000), eve)), + NodePayload(eve, PaymentOnion.FinalPayload.Standard.createPayload(100_000_000 msat, 100_000_000 msat, CltvExpiry(800_000), paymentSecret)), + ) + val sessionKey = PrivateKey(hex"0303030303030303030303030303030303030303030303030303030303030303") + OutgoingPaymentPacket.buildOnion(sessionKey, payloads, paymentHash, packetPayloadLength_opt = None).toOption.get + } + + // Alice wraps it into a payment onion Alice -> Bob -> Carol. + val (htlcForBob, sharedSecretsAlice) = { + val paymentSecret = ByteVector32.fromValidHex("2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b2b") + val payloads = Seq( + NodePayload(bob, PaymentOnion.IntermediatePayload.ChannelRelay.Standard(ShortChannelId.fromCoordinates("572330x7x1105").get, 100_005_000 msat, CltvExpiry(800_250))), + NodePayload(carol, PaymentOnion.FinalPayload.Standard.createTrampolinePayload(100_005_000 msat, 100_005_000 msat, CltvExpiry(800_250), paymentSecret, trampolineOnionForCarol.packet, None)), + ) + val sessionKey = PrivateKey(hex"0404040404040404040404040404040404040404040404040404040404040404") + val onion = OutgoingPaymentPacket.buildOnion(sessionKey, payloads, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get + (UpdateAddHtlc(randomBytes32(), 1, 100_006_000 msat, paymentHash, CltvExpiry(800_300), onion.packet, None, Reputation.maxEndorsement, None), onion.sharedSecrets) + } + + // Bob decrypts the payment onion and forwards it to Carol. + val htlcForCarol = { + val priv = PrivateKey(hex"4242424242424242424242424242424242424242424242424242424242424242") + val Right(ChannelRelayPacket(_, _, onionForCarol, _)) = decrypt(htlcForBob, priv, Features.empty) + UpdateAddHtlc(randomBytes32(), 1, 100_005_000 msat, paymentHash, CltvExpiry(800_250), onionForCarol, None, Reputation.maxEndorsement, None) + } + + // Carol decrypts the payment onion and the inner trampoline onion. + val trampolineOnionForEve = { + val priv = PrivateKey(hex"4343434343434343434343434343434343434343434343434343434343434343") + val Right(RelayToTrampolinePacket(_, _, _, trampolineOnionForEve, _)) = decrypt(htlcForCarol, priv, Features.empty) + trampolineOnionForEve + } + + // Carol wraps the trampoline onion for Eve into a payment Carol -> Dave -> Eve. + val (htlcForDave, sharedSecretsCarol) = { + val paymentSecret = ByteVector32.fromValidHex("2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c2c") + val dave = PublicKey(hex"032c0b7cf95324a07d05398b240174dc0c2be444d96b159aa6c7f7b1e668680991") + val eve = PublicKey(hex"02edabbd16b41c8371b92ef2f04c1185b4f03b6dcd52ba9b78d9d7c89c8f221145") + val payloads = Seq( + NodePayload(dave, PaymentOnion.IntermediatePayload.ChannelRelay.Standard(ShortChannelId.fromCoordinates("572330x42x1729").get, 100_000_000 msat, CltvExpiry(800_000))), + NodePayload(eve, PaymentOnion.FinalPayload.Standard.createTrampolinePayload(100_000_000 msat, 100_000_000 msat, CltvExpiry(800_000), paymentSecret, trampolineOnionForEve, None)), + ) + val sessionKey = PrivateKey(hex"0505050505050505050505050505050505050505050505050505050505050505") + val onion = OutgoingPaymentPacket.buildOnion(sessionKey, payloads, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get + (UpdateAddHtlc(randomBytes32(), 1, 100_001_000 msat, paymentHash, CltvExpiry(800_100), onion.packet, None, Reputation.maxEndorsement, None), onion.sharedSecrets) + } + + // Dave decrypts the payment onion and forwards it to Eve. + val htlcForEve = { + val priv = PrivateKey(hex"4444444444444444444444444444444444444444444444444444444444444444") + val Right(ChannelRelayPacket(_, _, onionForEve, _)) = decrypt(htlcForDave, priv, Features.empty) + UpdateAddHtlc(randomBytes32(), 1, 100_000_000 msat, paymentHash, CltvExpiry(800_000), onionForEve, None, Reputation.maxEndorsement, None) + } + + // Eve returns an encrypted failure. + val failedAt = TimestampMilli.now() + val failureForDave = { + val priv = PrivateKey(hex"4545454545454545454545454545454545454545454545454545454545454545") + val failure = IncorrectOrUnknownPaymentDetails(100_000_000 msat, BlockHeight(800_000)) + val Right(fail: UpdateFailHtlc) = buildHtlcFailure(priv, useAttributableFailures = true, CMD_FAIL_HTLC(htlcForEve.id, FailureReason.LocalTrampolineFailure(failure), Some(FailureAttributionData(failedAt, Some(failedAt)))), htlcForEve, failedAt + 1.millis) + assert(fail.reason == hex"8d332db2f2192d54ccf1717318e231f5fa80853036e86f7a6d6ac60a88452efeeef9d37d69bbb954bb5868d2c364a3eec1a5647b6f1a166e90f0682882679e8121bc2eed0d750cfef58de6e455920dbb5def6244cb8b2b47bc3f19dc903d9f2c578733aae4e943a1c7ce9893687848d40d2cac174905e99c1b65e4d637583b93c231c593d9752fc6b0265f647cbf708efb2de632f8df4c8253ad56dd13552ff1704ac7f1ed7c1c6450995a7e4c820f55bd1ae4330e7e822f18b2d5cbbe578d5708fbec0474ec2a0501c6100cf644c634c166e7501c7d70723be5defe8b0fd200d7ed64e2b09260867864890558351af9178c867a6cb59e83185ba4e92c64e19a6d057af2856c7b9aeb66a9c30d6f672de9a556d4020aabb388af96e644fa34d8bef34262") + fail + } + + // Dave forwards the failure to Carol. + val failureForCarol = { + val priv = PrivateKey(hex"4444444444444444444444444444444444444444444444444444444444444444") + val Right(fail: UpdateFailHtlc) = buildHtlcFailure(priv, useAttributableFailures = true, CMD_FAIL_HTLC(failureForDave.id, FailureReason.EncryptedDownstreamFailure(failureForDave.reason, failureForDave.attribution_opt), Some(FailureAttributionData(failedAt - 150.millis, None))), htlcForDave, failedAt + 10.millis) + assert(fail.reason == hex"59048eb5ae82a97991db9e51d94efbeb91035627d3aa9608e7079509ec7dceb26bfa9346c7bb4aa33b42edb0d7d29f2e962afd03c5220a2597662bceb3ce0ef43090ddd89dd61b51a5c35bda4b19a74939b538b45680c25430fa3f024a8b16736179f3f583538ba80625aafc6e363a4a4f5055ad780dfe93e0086b26206db463eeed12c77b4df58b3df3f8e977061722f0007329aecaf7f0853bd46ee3afbbc26969f723f61f02a4000b8a59d572d9adfc147b6b53e19a49289714ffb8b64bd5d8f9a57854bd775c8cc488c108031cf4a447ef3f1e011b35d16050b6e91a4de63b85e6167ce602363df02d10cdc29ac52f59a788f16ec2fe65c6717346f054c0e42b1ab87547d047277d901a7e081291a040bdebbc2fa578b7aa996261c2396237255c54") + fail + } + + // Carol cannot decrypt the failure, so she forwards it to Bob. + val failureForBob = { + val priv = PrivateKey(hex"4343434343434343434343434343434343434343434343434343434343434343") + assert(sharedSecretsCarol.map(_.secret) == Seq(ByteVector32.fromValidHex("ba32b1206434f6ded793257f34e497d4ba67f66e026a108860cc65dd9a6c8989"), ByteVector32.fromValidHex("e79cf33c343ca26b7048f029ad3c70f0dae0e3061eef960b5677696ffd297f54"))) + val Left(Sphinx.CannotDecryptFailurePacket(unwrapped, attribution_opt)) = Sphinx.FailurePacket.decrypt(failureForCarol.reason, failureForCarol.attribution_opt, sharedSecretsCarol).failure + assert(unwrapped == hex"af2704cc847cc1e1bd33242f6a123a5139015910d813c18f9c7cfda520519cabaa2bfcdb3e43e30adefd8471e095949ab5d39e1b3b875f48f8162c9740fa837ca72404a0eccaca2b521a46cb5d6cc0d1dcc8a15670efe05d8ba8b22f08b2315b796dbd713b55b7ed639f5fbeff3b0dcb87cd36b44b56530fcb2c752a98497241611ff09c03b6853a7eae890723899057bf3ab77d1638a287178c370305f54ade00af7e5c4fbb838aed9ac86c24fd70c5f716e5cdefdf8c3037bc85b467251b0644a01f3c428744ed09263dd07b6671960775425f9025f290af6e0a53cc16976107ba13200b13703fa5f09c5b4edca19853e9528feef13b4e62633c97e4f417c9a098c2ce7125c29c3196cd92b5240d31ecf2d22197cd26aa005c88598d0ab9fd5ea0e77e") + val Right(fail: UpdateFailHtlc) = buildHtlcFailure(priv, useAttributableFailures = true, CMD_FAIL_HTLC(failureForCarol.id, FailureReason.EncryptedDownstreamFailure(unwrapped, attribution_opt), Some(FailureAttributionData(failedAt - 320.millis, Some(failedAt - 320.millis)))), htlcForCarol, failedAt + 20.millis) + assert(fail.reason == hex"d3ab773d52436535f3f85a1204afc975319b28e7ab953c6ee81d61aa0fc77885abea0f0cbb44a6729b4916f490994dcb180d6cb0fdfe2c50ee0f96ef4d24f82fdb488817e7d099362351df2ded93992d175d1fbcf04ffc1c3f4d1e797c81c652f4f648a03950ed43002b75f23c734de26a33060320a436214f7c2fe71eb3e34c0eabe9eada713b667e882a99667bf6327353447dbd0115c5e0ca74524fb6c4eb2839c7db82804644306f534878c163b0e7f2398592ba3b7155d1db259447c56de0420bb81d9a0a33150e4316818cd66b5d74fb3327e769b83c865aea6e77493bf0a977417b8a52328b5927aec7bd14764a7a304ef15b5349ceb606ada2babc45c1f1b5ed0b9e3b2181a61611b93a6f40f360e508d3a42dcacf690bda01da0086f603c382") + fail + } + + // Bob forwards the failure to Alice. + val failureForAlice = { + val priv = PrivateKey(hex"4242424242424242424242424242424242424242424242424242424242424242") + val Right(fail: UpdateFailHtlc) = buildHtlcFailure(priv, useAttributableFailures = true, CMD_FAIL_HTLC(failureForBob.id, FailureReason.EncryptedDownstreamFailure(failureForBob.reason, failureForBob.attribution_opt), Some(FailureAttributionData(failedAt - 410.millis, None))), htlcForBob, failedAt + 30.millis) + assert(fail.reason == hex"f8941a320b8fde4ad7b9b920c69cbf334114737497d93059d77e591eaa78d6334d3e2aeefcb0cc83402eaaf91d07d695cd895d9cad1018abdaf7d2a49d7657b1612729db7f393f0bb62b25afaaaa326d72a9214666025385033f2ec4605dcf1507467b5726d806da180ea224a7d8631cd31b0bdd08eead8bfe14fc8c7475e17768b1321b54dd4294aecc96da391efe0ca5bd267a45ee085c85a60cf9a9ac152fa4795fff8700a3ea4f848817f5e6943e855ab2e86f6929c9e885d8b20c49b14d2512c59ed21f10bd38691110b0d82c00d9fa48a20f10c7550358724c6e8e2b966e56a0aadf458695b273768062fa7c6e60eb72d4cdc67bf525c194e4a17fdcaa0e9d80480b586bf113f14eea530b6728a1c53fe5cee092e24a90f21f4b764015e7ed5e23") + fail + } + + // Alice decrypts the failure message. + assert(sharedSecretsAlice.map(_.secret) == Seq(ByteVector32.fromValidHex("4bb46ded4fb0cae53a11db2e3ed5239ad083e2df6b9a98122268414ad525a868"), ByteVector32.fromValidHex("4546a66fb391bab8f165ffe444bf5d90192d1fad950434033183c64f08b6b091"))) + val Sphinx.HtlcFailure(holdTimes, Left(Sphinx.CannotDecryptFailurePacket(trampolineFailure, trampolineAttribution))) = Sphinx.FailurePacket.decrypt(failureForAlice.reason, failureForAlice.attribution_opt, sharedSecretsAlice) + assert(holdTimes == List(Sphinx.HoldTime(400 millis, bob), Sphinx.HoldTime(300 millis, carol))) + assert(trampolineFailure == hex"1ecb52bae42f744559016d17fb74c65e0a252ee89e4fbcd977a6f9b89dda92d5169dca498cde6fd2b33108433ec4243e95f90be7286e4d8ac011fd572c1a5e2e534a374ae2cdad3906e9eafe73be334d40b4796a77b5550742e759590ad4f2af5d32d701a1bc87101b87eed4460bc288970fc905faa3b122fb2f93e5e430f8744da9918e93db0ae9987abb5fbe5740127d0ce48e022e85a988ba84e5390bc3f3e4026b841109489a21d0e7050815fd069d50ff1222a48708badffa2904de8786d00200e59c2f0f6a7f38a1ac2d11cbf1eded8414de55ba516af6c43c339ad02d363b60f91258d2596f6cc329666e825cc830996b4e0726ab85e69bfc52e6e7f91b8fbfb2a3e3b69cf20677138e6aaeb262a463ca9448b27eeacafbe52bee5bca0796ef69") + assert(trampolineOnionForCarol.sharedSecrets.map(_.secret) == Seq(ByteVector32.fromValidHex("cf4ca0186dc2c2ea3f8e5b0999418151a6c61339ee09fef4c4804cd2c60fb359"), ByteVector32.fromValidHex("7bd32e41e242e1bb33e9bbfbff62b51249332a7c86d814dd4c8945b9c3bc9950"))) + val Sphinx.HtlcFailure(trampolineHoldTimes, Right(f)) = Sphinx.FailurePacket.decrypt(trampolineFailure, trampolineAttribution, trampolineOnionForCarol.sharedSecrets) + assert(trampolineHoldTimes == List(Sphinx.HoldTime(300 millis, carol), Sphinx.HoldTime(0 millis, eve))) + assert(f.originNode == eve) + assert(f.failureMessage == IncorrectOrUnknownPaymentDetails(100_000_000 msat, BlockHeight(800_000))) + } + + // See bolt04/trampoline-onion-error-test.json + test("build htlc failure onion (trampoline reference test vector, blinded payment)") { + val preimage = ByteVector32.fromValidHex("85bf91e47ab9f1bccdb43bce6aaf80d3bcf81d7bc6ebdfe515be3650e71c197d") + val paymentHash = Crypto.sha256(preimage) + val bob = PublicKey(hex"0324653eac434488002cc06bbfb7f10fe18991e35f9fe4302dbea6d2353dc0ab1c") + val carol = PublicKey(hex"027f31ebc5462c1fdce1b737ecff52d37d75dea43ce11c74d25aa297165faa2007") + val dave = PublicKey(hex"032c0b7cf95324a07d05398b240174dc0c2be444d96b159aa6c7f7b1e668680991") + val evePriv = PrivateKey(hex"4545454545454545454545454545454545454545454545454545454545454545") + val eve = evePriv.publicKey + + // Eve creates a blinded path to herself going through Carol and Dave. + val pathId = hex"c110b6b8bf6e404439f81b0d7b506346c9f2dee3004fa2b79a2f67fa48a4cb12" + val path = { + val blindedPayloadEve = TlvStream[RouteBlindingEncryptedDataTlv](RouteBlindingEncryptedDataTlv.PathId(pathId)) + val blindedPayloadDave = TlvStream[RouteBlindingEncryptedDataTlv]( + RouteBlindingEncryptedDataTlv.OutgoingNodeId(EncodedNodeId(eve)), + RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(30), 0, 2_000 msat), + RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(850_000), 1 msat), + ) + val blindedPayloadCarol = TlvStream[RouteBlindingEncryptedDataTlv]( + RouteBlindingEncryptedDataTlv.OutgoingNodeId(EncodedNodeId(dave)), + RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(70), 0, 3_000 msat), + RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(850_000), 1 msat), + ) + assert(RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec.encode(blindedPayloadCarol).require.bytes == hex"0x0421032c0b7cf95324a07d05398b240174dc0c2be444d96b159aa6c7f7b1e668680991 0a080046000000000bb80c0500 0cf85001") + assert(RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec.encode(blindedPayloadDave).require.bytes == hex"0x042102edabbd16b41c8371b92ef2f04c1185b4f03b6dcd52ba9b78d9d7c89c8f221145 0a08001e0000000007d00c0500 0cf85001") + assert(RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec.encode(blindedPayloadEve).require.bytes == hex"0620c110b6b8bf6e404439f81b0d7b506346c9f2dee3004fa2b79a2f67fa48a4cb12") + val sessionKey = PrivateKey(hex"c775094205ef9fd01baf02e3509d35aef5d41072aeceac99ccc18f7ec98fbf9a") + val blindedRouteDetails = Sphinx.RouteBlinding.create(sessionKey, Seq(carol, dave, eve), Seq(blindedPayloadCarol, blindedPayloadDave, blindedPayloadEve).map { p => + RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec.encode(p).require.bytes + }) + assert(EncodedNodeId(carol) == blindedRouteDetails.route.firstNodeId) + assert(PublicKey(hex"02fdbcd6421a898889635964b708e351a2da7623e4f72b0c500b8c2b26b519fcbc") == blindedRouteDetails.lastPathKey) + assert(PublicKey(hex"021ec9467be2cae24056472eec967aca614b6ce5639793bf3c367efbdadec505fc") == blindedRouteDetails.route.firstPathKey) + val blindedNodes = Seq( + PublicKey(hex"0291973446349ea220d41e5dcbc66e4a635d37de56de000dc4161996d07db4d721"), + PublicKey(hex"02bd4c23c41d52f27215752ffae592752ce779e2c47330ac3e124465544ff8c124"), + PublicKey(hex"03af94235d0a0da23a86b005fdbdaaf60e7c7430119e71b68d364c41e6c242084f"), + ) + assert(blindedNodes == blindedRouteDetails.route.blindedNodeIds) + val encryptedPayloads = Seq( + hex"ca6bbd4376ccc3bdf31cb3e31e4a6b707a25f387a84b90b47b44703834b578ebbe78fc50625084d3aed99f4801ed353beb522c60d7821234ff8b51ee2a36dd561745ad06", + hex"6e7178bd22210c97d06ba273c565f499528b81cfb77f4cc42a430648944c722dc552ca0cc7e7e402d59187b2f6c529a5a35f6221acae703ca133011ecb4536d81ce99958", + hex"fb22a19839151cb8d4523696dc6f6e69a8e20ec34a54b788adc253d7292acc9833bc9dde604518e86b86bc6b099cacd9ed17", + ) + assert(encryptedPayloads == blindedRouteDetails.route.encryptedPayloads) + val paymentInfo = OfferTypes.PaymentInfo(5_000 msat, 0, CltvExpiryDelta(100), 1 msat, 500_000_000 msat, ByteVector.empty) + PaymentBlindedRoute(blindedRouteDetails.route, paymentInfo) + } + + // Alice creates a trampoline onion using Eve's blinded path. + val trampolineOnion = { + assert(EncodedNodeId(carol) == path.route.firstNodeId) + val payloadEve = PaymentOnion.FinalPayload.Blinded( + records = TlvStream( + OnionPaymentPayloadTlv.AmountToForward(150_000_000 msat), + OnionPaymentPayloadTlv.OutgoingCltv(CltvExpiry(800_000)), + OnionPaymentPayloadTlv.TotalAmount(150_000_000 msat), + OnionPaymentPayloadTlv.EncryptedRecipientData(path.route.encryptedPayloads.last), + ), + blindedRecords = TlvStream(RouteBlindingEncryptedDataTlv.PathId(pathId)), + ) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadEve.records).require.bytes == hex"45 020408f0d180 04030c3500 0a32fb22a19839151cb8d4523696dc6f6e69a8e20ec34a54b788adc253d7292acc9833bc9dde604518e86b86bc6b099cacd9ed17 120408f0d180") + val payloadDave = PaymentOnion.IntermediatePayload.NodeRelay.Blinded( + records = TlvStream( + OnionPaymentPayloadTlv.EncryptedRecipientData(path.route.encryptedPayloads(1)), + ), + paymentRelayData = BlindedRouteData.PaymentRelayData(TlvStream( + RouteBlindingEncryptedDataTlv.OutgoingNodeId(EncodedNodeId(eve)), + RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(30), 0, 2_000 msat), + RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(850_000), 1 msat), + )), + nextPathKey = randomKey().publicKey + ) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadDave.records).require.bytes == hex"46 0a446e7178bd22210c97d06ba273c565f499528b81cfb77f4cc42a430648944c722dc552ca0cc7e7e402d59187b2f6c529a5a35f6221acae703ca133011ecb4536d81ce99958") + val payloadCarol = PaymentOnion.IntermediatePayload.NodeRelay.Blinded( + records = TlvStream( + OnionPaymentPayloadTlv.EncryptedRecipientData(path.route.encryptedPayloads.head), + OnionPaymentPayloadTlv.PathKey(path.route.firstPathKey), + ), + paymentRelayData = BlindedRouteData.PaymentRelayData(TlvStream( + RouteBlindingEncryptedDataTlv.OutgoingNodeId(EncodedNodeId(eve)), + RouteBlindingEncryptedDataTlv.PaymentRelay(CltvExpiryDelta(70), 0, 3_000 msat), + RouteBlindingEncryptedDataTlv.PaymentConstraints(CltvExpiry(850_000), 1 msat), + )), + nextPathKey = randomKey().publicKey + ) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadCarol.records).require.bytes == hex"69 0a44ca6bbd4376ccc3bdf31cb3e31e4a6b707a25f387a84b90b47b44703834b578ebbe78fc50625084d3aed99f4801ed353beb522c60d7821234ff8b51ee2a36dd561745ad06 0c21021ec9467be2cae24056472eec967aca614b6ce5639793bf3c367efbdadec505fc") + val sessionKey = PrivateKey(hex"b88c99e501646c71afbea99afba7201069159298e84d327f4663fe76c3246d6c") + val trampolineOnion = OutgoingPaymentPacket.buildOnion(sessionKey, NodePayload(carol, payloadCarol) :: NodePayload(path.route.blindedNodeIds(1), payloadDave) :: NodePayload(path.route.blindedNodeIds.last, payloadEve) :: Nil, paymentHash, None).toOption.get + val encoded = OnionRoutingCodecs.onionRoutingPacketCodec(trampolineOnion.packet.payload.length.toInt).encode(trampolineOnion.packet).require.bytes + assert(encoded == hex"0003775903fcf5d90e9a667d39b7077ac05d6af2887046bae7db7be95d21e2e43cda0ca84ff61e9235e79fcf66e76110f6b0d1c3880ea594fca459db9e7f45700cd068ed3c57a1e08861fd64ff3422847b87f41398f607c50fe731d68283e57f512db8f7c7323d10a67669e6c73f074175465a1777fc0f0fb1d95754b6fd4e118d331a6a2204f8ad8599965ea11ba3a62de6ee557e4e5bf94439eae57f317153e051aef57a61a80d94004a9b15fbb146e0de23c08263021b9915babfe63bf25289dc4a118f7f4ae54034a423ca0d40898775adf8bd7d4c8bb99a355e8543195d7eae5b697a49a7d8886395fa9f9b3783ae76604048553817bf1aa74d6d95d081757fee83bb7cc24c4d57365aac18011881d9e690affb96c9d867a7644ee63f9828d2f942dfa227004fab8eb110f55f7ab3652fe7e57ce6ece0c7edb6601d16f4c60b985d971d16a4c89c6cb7c303e04c9df926397ebd40da168ad0daefb0d834dec4b14869376d82261d51f0eb63a2890cf0571ec0489bb61cb7dbf2dfdb12047492f0248064c121027b37b8bfba04d0e6cdb3c6727f989b54") + trampolineOnion + } + + // Alice creates a payment onion for Carol (Alice -> Bob -> Carol). + val (onionForBob, sharedSecretsAlice) = { + val sessionKey = PrivateKey(hex"ba1560fc54cf1813db7f522ccd2dd7479b27bae31a0016748a49d795be7f00c2") + val payloadBob = PaymentOnion.IntermediatePayload.ChannelRelay.Standard(ShortChannelId.fromCoordinates("572330x42x2821").get, 150_005_000 msat, CltvExpiry(800_100)) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadBob.records).require.bytes == hex"15 020408f0e508 04030c3564 060808bbaa00002a0b05") + val payloadCarol = PaymentOnion.TrampolineWithoutMppPayload.create(150_005_000 msat, CltvExpiry(800_100), trampolineOnion.packet) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadCarol.records).require.bytes == hex"fd01a8 020408f0e508 04030c3564 14fd01990003775903fcf5d90e9a667d39b7077ac05d6af2887046bae7db7be95d21e2e43cda0ca84ff61e9235e79fcf66e76110f6b0d1c3880ea594fca459db9e7f45700cd068ed3c57a1e08861fd64ff3422847b87f41398f607c50fe731d68283e57f512db8f7c7323d10a67669e6c73f074175465a1777fc0f0fb1d95754b6fd4e118d331a6a2204f8ad8599965ea11ba3a62de6ee557e4e5bf94439eae57f317153e051aef57a61a80d94004a9b15fbb146e0de23c08263021b9915babfe63bf25289dc4a118f7f4ae54034a423ca0d40898775adf8bd7d4c8bb99a355e8543195d7eae5b697a49a7d8886395fa9f9b3783ae76604048553817bf1aa74d6d95d081757fee83bb7cc24c4d57365aac18011881d9e690affb96c9d867a7644ee63f9828d2f942dfa227004fab8eb110f55f7ab3652fe7e57ce6ece0c7edb6601d16f4c60b985d971d16a4c89c6cb7c303e04c9df926397ebd40da168ad0daefb0d834dec4b14869376d82261d51f0eb63a2890cf0571ec0489bb61cb7dbf2dfdb12047492f0248064c121027b37b8bfba04d0e6cdb3c6727f989b54") + val onionForBob = OutgoingPaymentPacket.buildOnion(sessionKey, NodePayload(bob, payloadBob) :: NodePayload(carol, payloadCarol) :: Nil, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get + val encoded = PaymentOnionCodecs.paymentOnionPacketCodec.encode(onionForBob.packet).require.bytes + assert(encoded == hex"00024a8829a84a737a3fc6895779624ef7f356199aa9370284126c5d8dd609b0ededf2389838c342759c38ebc1d83652bc317ce75244c3e5147a8b778a294bec8438c9a2fe3707931775ba350aea1e5b2b9961db32cd5afccc68bbe74730866ded045d7f1c427b9fc523272cdc9a767691961bb79a8ac4fe676c0ff93e10074c6cc692400237a7faa43325bd3662161cbe9f8d9f7b4a55c3188534798030bdccfb269e4c6ecb314a066535b76faf83924618fcb95a6aa84d8d22195493b2d705c2472dacf61a2fc564e410da53423846455bd102eec8d372a08ae105b234bf35a8debccca0932fad6ea76b2e6478114c2add5c2aa4f7427a231fb60eb5fa9b7c62b626eeb9b87e90071aebbfca9e6512e04379f149b9d6f0ba2cdaffc4aba69caa46522eebf9e5d5b864a1e9c9132d3dd51032e66462308ace1a591c2266593985196126dcbb653c476ff2132da4173c3f50b42c73e578345ea6a9a673c3f810dc7c1159dd675da6acdf14256543eca500ecf9fe1cd70813b5c4f6d048afd2846fb542a89613423292f56ae1f7a427cd2ac2ef5ad30e3a347487c57d0fd7ff517288720a86e3718d955e4e1b81786721f5c3b6f7e8ff2bedfc295dfb1a4b2731c955c386177e08c85e9449a6f03b599da0efb14fd04eb9911210e3ae46d4a507d624baedf6d6979ee2114b12ab2ed74c14ab246425a4222e005e10273c2b39a5d8c615141403b05354c1812c79a63c7af91cfa8723969097d94f31a1495da02e6262b955f2577b1d8e9fceca90be0c93e61f05c94683f642917ee2599a42f13a00024dc81fbdfb35ad437805edbed0453f9f31643c6af0db46950f9290c2a5a0839baf53afe33c310fb3fd542127b087997eb62a8ad9b5a04f4bce859f11a31224a5535678a625ff628c8ac87b533cc606321dfc553d3caf5e68c1e587c0e80248a09b406555c2233dd35cf9d803c9f9cdb1cc65de2f6c42bbadf875e647541ec052570d3fa66e98ceee34060599bd18a98bd004f7d7cfc80bc5fdfbd182eff1471180b3fb3de0dca7af2cb1ac56473a78caec24e98904b78dd23ca81de5de0c2f5f63c1f9b5911d4191a47860b0ef65412cd76b354357e4a686b571fd9f2575c4ce18fd5a4a151faea0368815a2efd504ad27be5cd3077ed0c165db4e9c70bf554ae90fd2625b361d85e030f65fae4fa62affb0457d74072c125e1c1d1172d6081124cbf11045fad8d7a366d5fa3c534a89ab8c96c56dadca8a7efc6b12f077955db19dd4f662515e9eed9eb547df5df0fbf391fcdb3031ad83b594f12c35e01a4467f1cefbb16e097292eaabe073fbbd02ddcec0cca5c6169e352236b78021532d058e0d4ccf503a73840e7cef1209d8b10e62dfc2e1e8a86327e90dc39534a4f34b5fcebdcdd70a50b2689aad5fe0c77515f5efaf33d38e82e8a708d08bac92f31e2d3cdf65278599984741cfd630111c78bf6f6d4766b103247a46b0d5fc7ebeffc92484f5493a5023f6fd5d45a2d11906f72730fe98ba83948a92eca2671a35d843f652a085799118804e9cb5af10d9f598fb9cde9dd0f0da64f8ba6cb5acd5fee9dcdf7cb9f25e878f1f5e1eec874f598a0641352cb9b9bd201421476150c7f14d61f95ed8db27c9e2380915efa7c994625a207f09f40331f79ee889e8abd519bd28e78eaed5261d75c5cde8d67223bec4b4b83009a84f3e7fd7453a80ac4a4c0cad6913b84fdbc94a612ca3abcde92f0df8da1e7414565f3aa24d8e2a82c9e1d09c927aaedd5992fc3abc2393f64a72351237a871a00ad58b1378851cbe8489cb149ff71fe1a1cf063e97751c8c1360675925c1aae9ca78288c7fe081fc6de52bf09e7ff0aa58eedc910661c74fa812a2ec43368a53263f11ccd916ed6a7e06f356fffdd5dcca2e12ae7fe559eb1067a33b44") + (onionForBob.packet, onionForBob.sharedSecrets) + } + + // Bob decrypts the onion and relays to Carol. + val onionForCarol = { + val bobPriv = PrivateKey(hex"4242424242424242424242424242424242424242424242424242424242424242") + val add = UpdateAddHtlc(randomBytes32(), 1, 150_010_000 msat, paymentHash, CltvExpiry(800_150), onionForBob, None, Reputation.maxEndorsement, None) + val Right(packetForBob: ChannelRelayPacket) = decrypt(add, bobPriv, Features.empty) + assert(packetForBob.payload.outgoing.contains(ShortChannelId.fromCoordinates("572330x42x2821").get)) + assert(packetForBob.amountToForward == 150_005_000.msat) + assert(packetForBob.outgoingCltv == CltvExpiry(800_100)) + val encoded = PaymentOnionCodecs.paymentOnionPacketCodec.encode(packetForBob.nextPacket).require.bytes + assert(encoded == hex"00030147117e68955d3033a3dc6334599ee4d01cd141e0a2ee77361665510a14eb337bf9125b50b71732f89f6eb42d27437efbb005319f70d11f4560749fbb2b43b600170bb1d5338c179db895a4bf817538ff1dd512c18fd4c59831c97fc9681c01a3a424f1ec27d447dae3650e47f9c1bb011ecc76483e91c0341455ee2ee30b05ae271e52fbc6086bea913eda6ba1b74f31e1dadd59c4a13886b907d353f7bd2785b8f204013aeef68d6de7340d2380452a5ef70f8c7542611d4bd8c66c3df00988240c0eed8f75ad554f0031e8a4cef9d73d7407ecb4a343d3ac1a9a68eeaa2194e245c66181726756fac2c30ad06c59f21a11167aa2750500c49657897544ee91b7aa0ddb2ef4fa23fc502926d4ce9b751c08668914be4fc558efd1d634ffee02ce5c00f10926696babcf70a530c27a4efb8e26dc6b7813ece23d64263dcd39a173f4259fffe19bca10689c671ba3903cd408ee792e87be524feaa38405ec7a99b9e7704edbc0c2410eeaee04268384e5baa78cea60b662efef7892fde2e8694db29a2f4f30e163ee951a348d7688bcbd4cd6a327ee36dfb3a200f3a04787234c971ec96df0493b9637c36f757e0cdc5f65a09e5c9d0b8c36313b89ecad900e5d7b443fb8b7c03d0e1edd06b6c0955c62641333fd00f339ed81ef3159125e6563b21d4852cc70bbb159ef06363b6ed5025ac1eca1f35d379be937837db850748090178a848057899379d611484f10139edc02e3b3d2177c0d430496bc78328b41655da444298c6efe022818b1a685846cb4396d9328a6e1ce63538f35736f67da52def0b0c2c68836a3282c4e52ecd8ea88bcec7f17d4c60dc4678cc09b70997f871809766f3b7149235abc0b0ef4db40c4dd3e37b383238ed2841354ba5664f6378c5c642013c70d4b666359ea6440dccf37be08c0c7ab7026658411aebe43a6f05b1dbf925acc4aeda16d3653b59abd4c2b83125fe71951b8d782c97f5fafa0b04ab9f32b69e3b6abf80ca02453eceb7e0147ffd8ff69ad6239e46a3cfa9fa8cdb9ac951e14b037cf117a323bc022baf93ec24146208516945585623d23e32868cfef47aefcae5ec8fdf45f3fda1a0562dd6d1e8dfdbacf90c16a94f0586dc2def77b1ce386d2e8c23182a9d05aa4e7f954e3cdba07742500eeb48a049f461383eb6431ed4f0cbe7abed1063c67723e94f50c264a056d8fe2f75270b55baa05afec063bee72d923bd59e3bbbe5a051713abbd832424838916b638b3b2e1d78483f7e4c1135968b81b5a270f048fed9ca304c292c8551a625d364a76722a53c4cc10ea2fcde378c86fd16fc7515b38e13cb9e754a4ca93ddec33febf4bdea92f4613d3c035cf7e17c876953f09b2f7d28d52c2db8bfa58be4daf4a646a96358c6f3fef0da19b3dfb07990b1bcdba62a5750eac5b0df938f447a05bf26ea9acf8d89115ef998ad4b2bd57397caae6f52825fe7d39fed886399e25cd80a502d6c5e402c283a04fc8e212ec0805d48ec3213aa065a7346fc6d05897098cdcded7e89c5a5df675c7c085186b497dff317d85fbde9ca6d8c15d0e74a337c233b24eb647d007a52bd1ae9957c1e5085f1b7db1d56b2711b8124f2ff62cacd223fc542d65ae93b4fedb5d2fa691664bb1d6f63469ef34cb0ffa1a3de7a0b68140baa042eee5e07d964163b43c751c057f04d30c260d37dae8dcdae19dcb333e7bd75b49452fb52ca2f4d05911cd03b3703ef90779252752c57ad82a9199bf3fc8c400300ba8b7327efdf9f03d9723f4ba9b2dd4c001fbeeb70d127c7e32ac3c1b73f6644bb4e3314ab121ac59f2387acbdbd6ca6bc6d4b5cc777d3f9affbc04caaac72da3c56b25af3fd12bcff88c8964bdedad9c3ebd38d0083e82c40d10d5fb4b20e0fd8907c7a9f67a3fb9784b35103b99bccf0a8b") + packetForBob.nextPacket + } + + // Carol decrypts the onion: she is the introduction node of the blinded path, and should relay to Dave. + val onionForDave = { + val carolPriv = PrivateKey(hex"4343434343434343434343434343434343434343434343434343434343434343") + val add = UpdateAddHtlc(randomBytes32(), 1, 150_005_000 msat, paymentHash, CltvExpiry(800_100), onionForCarol, None, Reputation.maxEndorsement, None) + val Right(RelayToBlindedTrampolinePacket(_, payload, innerPayload, trampolineOnionForDave, _)) = decrypt(add, carolPriv, Features.empty) + assert(payload.amount == 150_005_000.msat) + assert(payload.expiry == CltvExpiry(800_100)) + assert(innerPayload.outgoingAmount(add.amountMsat) == 150_002_000.msat) + assert(innerPayload.outgoingExpiry(add.cltvExpiry) == CltvExpiry(800_030)) + assert(innerPayload.outgoing == Left(EncodedNodeId.WithPublicKey.Plain(dave))) + val sessionKey = PrivateKey(hex"1a7c672a778795bf93d30f2b643258bb9536b6395caad1db75d717be3633acc5") + val payloadDave = PaymentOnion.TrampolineWithoutMppPayload.create(150_002_000 msat, CltvExpiry(800_030), trampolineOnionForDave, Some(innerPayload.nextPathKey)) + assert(PaymentOnionCodecs.perHopPayloadCodec.encode(payloadDave.records).require.bytes == hex"fd01cb 020408f0d950 04030c351e 0c21038277801bcd52897eb6c2568a474494fa493b4e1d615823064c21cad45bf41c30 14fd0199000209bf6fd592367ef37d5caf1041863c7b7a4c78e35fa6006359d4b09a6d078526466d008baeef3dcf2dfb7d5ff14d08b7ce6d3fd8f01a6243317d032fb49e94464db2ce7ec6f45e10b77ac2625896085c180abb3ed3b169caeffd3d892bdf0588802bb69e4034aa8c6591d188876490ee7584b084570e798bcd8b795740f23a8654a36e4322cb348895aee722ff625e83acb744b60012c32cc94fc91e8d23321cbc089dabc72f717daaf128867b435e008efe677288fc498a08bb5f58ddab853b2ffca1a5e82cdb34e7fef791b41401a8a579f04d2636e34c61dd3a17000842983074c6d4b90e15e4fa8533380224ec4ffa5a61e115e24043d49917eca869d9d902610c4389d5eb98a99107aea3d5a883eb3198c2266c823ee595341a24d4953fc7d58660182368467aa66be662bd1205dfd2ee4476176ca5eadfb9a49a1e547c509fa8d92b147a4692d4379efd0c458b35d929984e8a09ce0bc1177bcf51fff29f44535d9abbe097fe171ee7684dc7c0f9c2c92e332bda4c7c32319cf2743beaad05387bdbd5577bdd694d8ad93f128109c2b93b915b3d") + val onionForDave = OutgoingPaymentPacket.buildOnion(sessionKey, NodePayload(dave, payloadDave) :: Nil, paymentHash, Some(PaymentOnionCodecs.paymentOnionPayloadLength)).toOption.get.packet + val encoded = PaymentOnionCodecs.paymentOnionPacketCodec.encode(onionForDave).require.bytes + assert(encoded == hex"0002352e39e9e03e58469a9b32c70af97e8aca844eef1d125f98d1d03a3386954d7bb9e1e225cdcfc31d3054eb49aa245dc9db2e3c59d5c6ce785f7bcb6cbe6469869bf2f9f7e4fdc0e35f2db322a876337e54cab7d488f2a8e5f1f0e8fe9203aae82efdc2c8b77f28a10a76fe7376915873143274f17dcf90d80373d5728668887e631e2155de5c19fe99180cd4b9ebecf937d9fb3b3e7ce351d4d9b05126a65cc93ee40f71b0680668b6ca6a0052f13a5631b1ed191bdb0c2cc9d443fd1f16cd5fcd667191bb372e1dc34d5678d0a0c7587441ede631b035255e40d292ec2a1ebf1f769a629396eb66f1661aa0953425c58c4dd45489450bb9cb4642b3e183e47437bed11ac6e159de51b5a53ef62ac3911c1ff9aeea5181d851dd1aeed59a92a7e434695706523e13d06cba7c09af83be9eab4378ac1507bc0ec4ca6785fee13445c6202ec9abdebc3aabc01a45bf49ec4139168fad9eff03b08e31d63da12e6be181d855a578912f62d9a0e999513a9bcb2d3a104656c0451f04f196d3c2eaf74f95a295828e5e73dfcce61f0fe6471a24303602339ee4b3cceaa20e97e703b4c080f4868e9a9b7cb4bb606576ef3008531c9cfc05dd901a5205802e1bc3104a47fe617eef57cf656e5ec8afc556c2a74a6e8980319e422f9429062a1c06377339cc86806e99d2cadfaf19770b2ec6585dd5b4f31c010cd089804644c3ef484c1a030f5248bf7feed39362d92dc08723277b952845f0208ac1f44e779661d9bbbab37bace8b32882c5ca4064b2a63740f0d9119732b49e9979ab23b22697da3430a59e39b7cf86ecb52da78f9f8df9be7a071e47de47fc8657180476a6e9e8f86b29b9b44f63c11fece6bf7c2a24cb6e9035a73964be967abf8faafa34b40566277021dab1b390d71ea654f7288d233551fcdcfcbcfa5e50f65a11346424dfae23a2b1e017c55bcedee4841ce14c4f87e1ff32f3e88d410c13681d555cb5d5b6c2072d781de21c33dc087ba462b5e9d62ff17d94579d4c2e90549d8949a070394eccfeda33c329bcb27a5697459791b165ae85d758bcc47c69bd19f52e1508c3b77dd2bbef4119c003d39a7f6a23b8645999c3164b28cb46685b2a0bfbeea52f38a5b71f1899f50ec9baf1248bb3956f334a96bbcc60e547a1b12137768bb4e4ccec0dcafb409fac6419aff6e5ea39ae1c78d94253fc933444b669dd9cbfc0e4bd2c80c9a2b63b7c5963ffa5d87ba69896fd5f925805adfff4fb8563555fb5fd3bfe40f4b9275655d38640e8e4ca9f0874ad432ddcd726bf413914aface266dae9c66fe641e118dbb7fb418306e9ef274b4713fbd5a301e347f37a11a47302b3e4846078fac76abb3846c1891fa93c4051d3d5f7028e10b7b9856fc3bd48fd44a2b7b01ce6f3d0d5aa27695bcdced1b8ec899a30623d6ef97c6d77879cb9edd57f2f2f520d087dd723129684daec75021e326d8a833e005522aa44544bd08af7cb2d716827ba327f60309a05c53141b8b66608c4caf762b01fc88f0f59d18f82df0950ba9ef1412be8074a708409a612971c62e41bbccbac62514342d95774a5cf23fda4cd4168dd93b30ff5f9ca676542c9304d2f3be9d2643c57be2eecc823b5521854579e734b61165907b0b729a92e2bab80954fe063954a58314d406534b5ae04c11b067ede1b190c994623c66ab0a376773325d28ce904fddbbf83f755a6176464c48f9a84901ef6f662800dbab121eae95b2b4d54b640aeb0db272b8c1f43a610edce98db371435924026b601ae0ae1329ae18c192443f5d0c133a506a235e922bb687f8c5aadc7684008c229d84035b4e44d34fc7598a3f63ea4de01038806e9e9b0bdb41f33783b9f4d41915b85c5af4f76c7d2135905232de6f88ea226bd2548ee25167ce61a6aeef82ea94422ed91") + onionForDave + } + + // Dave decrypts the onion and blinded path data and sends a failure back to Carol. + val failedAt = TimestampMilli.now() + val failureForCarol = { + val davePriv = PrivateKey(hex"4444444444444444444444444444444444444444444444444444444444444444") + val add = UpdateAddHtlc(randomBytes32(), 3, 150_002_000 msat, paymentHash, CltvExpiry(800_030), onionForDave, None, Reputation.maxEndorsement, None) + val Right(RelayToBlindedTrampolinePacket(_, payload, innerPayload, _, _)) = decrypt(add, davePriv, Features(Features.RouteBlinding -> FeatureSupport.Optional)) + assert(payload.amount == 150_002_000.msat) + assert(payload.expiry == CltvExpiry(800_030)) + assert(innerPayload.outgoing == Left(EncodedNodeId.WithPublicKey.Plain(eve))) + assert(innerPayload.outgoingAmount(add.amountMsat) == 150_000_000.msat) + assert(innerPayload.outgoingExpiry(add.cltvExpiry) == CltvExpiry(800_000)) + + // Nodes after the introduction node must send `update_fail_malformed_htlc` messages. + val failure = InvalidOnionBlinding(Sphinx.hash(payload.records.get[OnionPaymentPayloadTlv.TrampolineOnion].get.packet)) + UpdateFailMalformedHtlc(add.channelId, add.id, failure.onionHash, failure.code) + } + + // Carol is the introduction node, so she creates a normal failure for Alice, wrapped inside a failure for Bob. + val failureForBob = { + val carolPriv = PrivateKey(hex"4343434343434343434343434343434343434343434343434343434343434343") + val add = UpdateAddHtlc(randomBytes32(), 1, 150_005_000 msat, paymentHash, CltvExpiry(800_100), onionForCarol, None, Reputation.maxEndorsement, None) + val Right(fail: UpdateFailHtlc) = buildHtlcFailure(carolPriv, useAttributableFailures = true, CMD_FAIL_HTLC(add.id, FailureReason.LocalTrampolineFailure(TemporaryTrampolineFailure()), Some(FailureAttributionData(failedAt - 130.millis, Some(failedAt - 120.millis)))), add, failedAt + 15.millis) + assert(fail.reason == hex"7b5d085fc545742096fabdcc1d37a1c62b88080f3c6ab8f4abd4f0fbd35639e36625db968c12816c072f374d79e5c642a6419aa13c12abb69537581ad0afe96f41cc2789d91b25ce04d46fffbeeb6ef639f44f9a2188f21a64cb836167267b54b5b3611ab77efd5f052f76295966c41d0b6d8a1c0fce72eece41e67156e649de0c849d96be186f5d57790409f918b45cd052d3bd3972cd4f0edc990ac13d5d54ecf2667888e989bf5a034b32a0693d01a0d9465007d5395343cc1a61129bcf0cbb67eff959f73350853c039219a9e356999389c1604f7728e3d66a239ddf1cf12b12f6a6d6810065955c8128c43e85d8290f499f420f308cc1a8ae3acb4a1aa268dbfeb3b17c17ca50b8fadce5f38e1519881bd29ac2823043f35891de23e156e40ac0b2") + fail + } + + // Bob forwards the failure to Alice. + val failureForAlice = { + val bobPriv = PrivateKey(hex"4242424242424242424242424242424242424242424242424242424242424242") + val add = UpdateAddHtlc(randomBytes32(), 1, 150_010_000 msat, paymentHash, CltvExpiry(800_150), onionForBob, None, Reputation.maxEndorsement, None) + val Right(fail: UpdateFailHtlc) = buildHtlcFailure(bobPriv, useAttributableFailures = true, CMD_FAIL_HTLC(add.id, FailureReason.EncryptedDownstreamFailure(failureForBob.reason, failureForBob.attribution_opt), Some(FailureAttributionData(failedAt - 150.millis, None))), add, failedAt + 25.millis) + assert(fail.reason == hex"c572fa6d2b2ab9dd05f361b15acc20327b7067758155ee3361b71ae52dbd2b819b54a39e339a4138998d07ed2d84967df06eeb9283b617694eaba71f934036b68a826f3d1a7b47e01a34828f34717959dd2c8aaff91cfcdbb46f7c6485279580c3229bb4c73bcde667850620ec09443540a4edc9ab126bb17de6ccc85c9e0d04979e65ecc9c84a4bb8b301c21159882699aa8d50ca420286b6977f467567d1614f5172e6a591a26aa7cf2e29f929b7b31303b1c75036a91710fd5499f487829d8d4912d1dd103a3d25b5cfeca0902bdc6c2cc58283fad5066192b52e3300c88d741cda42108f886f61644a84b21d3a06b9a5c7b493096a8095d67fecc9b48a32f989a31771a8d1cbd36cbe7d2e3f1e33b15aa68ca6eae03895094aa27d1549d8ed8e31b2") + fail + } + + // Alice decrypts the failure message. + assert(sharedSecretsAlice.map(_.secret) == Seq(ByteVector32.fromValidHex("21940511853dd5bd14ffc350149b04ba88df752e31ebe5b2ad039bdc1ce160e4"), ByteVector32.fromValidHex("21ea283167d589794e1bb1702ec6254647a043cd1a535559f0ba2e2e280e5a9f"))) + val Sphinx.HtlcFailure(holdTimes, Left(Sphinx.CannotDecryptFailurePacket(trampolineFailure, trampolineAttribution))) = Sphinx.FailurePacket.decrypt(failureForAlice.reason, failureForAlice.attribution_opt, sharedSecretsAlice) + assert(holdTimes == List(Sphinx.HoldTime(100 millis, bob), Sphinx.HoldTime(100 millis, carol))) + assert(trampolineFailure == hex"ab449799be03533fc503942d59c85c95b42d05307e2b382f46147a4c23c9b1dcf4f03baab8a1ca8467fa5dd26953a2c47679f7b9568742cf1b43ed52b6786236dea7bb4e14a0c5fdd530c35f3e60b455fd3081b16ed97298dc944d0f1d9c97e09b28eb9c2de05f84a1416d41f2117285121e2a9dcc9a78988f8b402ae4311354e46b9f92fd095beb3b1a74f38b272bc98b3eeff5b582e08bf8a5fc4afb3e0f7771b09c5c9e8c46baf71adeff8fddb7a5a53a4ace9ef32b23460520dc7759e8c4a7ae7ce1a92432e0d77b7c5a0b437541ba0f55ccfb3fce1a17a13ed57e6137e2dc7e100dc87ac02a34ac676d735192fc698da48461620dbf2a83361b8892d9863c2e357659275dd243c036cd62f81124f2683405413acbe9f450283ba60fe3169cc783f4") + assert(trampolineOnion.sharedSecrets.map(_.secret) == Seq(ByteVector32.fromValidHex("10d608eee2dcc6a00f1c08fa14659bd88531a608be3d68c8cf5988874a2e589b"), ByteVector32.fromValidHex("746a293d921da6db1a5564b204a5a9f06c4ca0cf5540593319ffa511e6b411f0"), ByteVector32.fromValidHex("b98befc45213d74357027e292510d9ba8000ba07d659c2af58854a818e3d436f"))) + val Sphinx.HtlcFailure(trampolineHoldTimes, Right(f)) = Sphinx.FailurePacket.decrypt(trampolineFailure, trampolineAttribution, trampolineOnion.sharedSecrets) + assert(trampolineHoldTimes == List(Sphinx.HoldTime(100 millis, carol))) + assert(f.originNode == carol) + assert(f.failureMessage == TemporaryTrampolineFailure()) + } + } object PaymentPacketSpec { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PostRestartHtlcCleanerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PostRestartHtlcCleanerSpec.scala index fe15c42619..3f3826730d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PostRestartHtlcCleanerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PostRestartHtlcCleanerSpec.scala @@ -591,7 +591,7 @@ class PostRestartHtlcCleanerSpec extends TestKitBaseClass with FixtureAnyFunSuit sender.send(relayer, buildForwardFail(testCase.downstream_1_1, testCase.upstream_1)) val fails = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]] :: register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]] :: Nil assert(fails.toSet == testCase.upstream_1.originHtlcs.map { - case Upstream.Cold.Channel(channelId, htlcId, _) => Register.Forward(null, channelId, CMD_FAIL_HTLC(htlcId, FailureReason.LocalFailure(TemporaryNodeFailure()), None, commit = true)) + case Upstream.Cold.Channel(channelId, htlcId, _) => Register.Forward(null, channelId, CMD_FAIL_HTLC(htlcId, FailureReason.LocalTrampolineFailure(TemporaryTrampolineFailure()), None, commit = true)) }.toSet) sender.send(relayer, buildForwardFail(testCase.downstream_1_1, testCase.upstream_1)) @@ -603,7 +603,7 @@ class PostRestartHtlcCleanerSpec extends TestKitBaseClass with FixtureAnyFunSuit sender.send(relayer, buildForwardFail(testCase.downstream_2_3, testCase.upstream_2)) register.expectMsg(testCase.upstream_2.originHtlcs.map { - case Upstream.Cold.Channel(channelId, htlcId, _) => Register.Forward(null, channelId, CMD_FAIL_HTLC(htlcId, FailureReason.LocalFailure(TemporaryNodeFailure()), None, commit = true)) + case Upstream.Cold.Channel(channelId, htlcId, _) => Register.Forward(null, channelId, CMD_FAIL_HTLC(htlcId, FailureReason.LocalTrampolineFailure(TemporaryTrampolineFailure()), None, commit = true)) }.head) register.expectNoMessage(100 millis) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala index b571a8dc24..8b56680722 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala @@ -50,7 +50,7 @@ import fr.acinq.eclair.wire.protocol.PaymentOnion.{FinalPayload, IntermediatePay import fr.acinq.eclair.wire.protocol.RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec import fr.acinq.eclair.wire.protocol.RouteBlindingEncryptedDataTlv.{AllowedFeatures, PathId, PaymentConstraints} import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{Alias, BlockHeight, Bolt11Feature, Bolt12Feature, CltvExpiry, CltvExpiryDelta, EncodedNodeId, FeatureSupport, Features, MilliSatoshi, MilliSatoshiLong, NodeParams, RealShortChannelId, ShortChannelId, TestConstants, TimestampMilli, UInt64, randomBytes32, randomKey} +import fr.acinq.eclair.{Alias, BlockHeight, Bolt11Feature, Bolt12Feature, CltvExpiry, CltvExpiryDelta, EncodedNodeId, FeatureSupport, Features, MilliSatoshi, MilliSatoshiLong, NodeParams, RealShortChannelId, ShortChannelId, TestConstants, TimestampMilli, UInt64, randomBytes, randomBytes32, randomKey} import org.scalatest.funsuite.FixtureAnyFunSuiteLike import org.scalatest.{Outcome, Tag} import scodec.bits.{ByteVector, HexStringSyntax} @@ -325,7 +325,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) - assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalFailure(TrampolineExpiryTooSoon()), Some(FailureAttributionData(p.receivedAt, Some(p.receivedAt))), commit = true)) + assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalTrampolineFailure(TrampolineFeeOrExpiryInsufficient(1_096_000 msat, 60, CltvExpiryDelta(288))), Some(FailureAttributionData(p.receivedAt, Some(p.receivedAt))), commit = true)) register.expectNoMessage(100 millis) } @@ -341,7 +341,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) - assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalFailure(TrampolineExpiryTooSoon()), Some(FailureAttributionData(p.receivedAt, Some(p.receivedAt))), commit = true)) + assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalTrampolineFailure(TrampolineFeeOrExpiryInsufficient(1_096_000 msat, 60, CltvExpiryDelta(288))), Some(FailureAttributionData(p.receivedAt, Some(p.receivedAt))), commit = true)) register.expectNoMessage(100 millis) } @@ -362,7 +362,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl p.foreach { p => val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) - assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalFailure(TrampolineExpiryTooSoon()), Some(FailureAttributionData(p.receivedAt, Some(TimestampMilli(20)))), commit = true)) + assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalTrampolineFailure(TrampolineFeeOrExpiryInsufficient(1_096_000 msat, 60, CltvExpiryDelta(288))), Some(FailureAttributionData(p.receivedAt, Some(TimestampMilli(20)))), commit = true)) } register.expectNoMessage(100 millis) @@ -414,7 +414,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) - assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalFailure(TrampolineFeeInsufficient()), Some(FailureAttributionData(p.receivedAt, Some(p.receivedAt))), commit = true)) + assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalTrampolineFailure(TrampolineFeeOrExpiryInsufficient(1_096_000 msat, 60, CltvExpiryDelta(288))), Some(FailureAttributionData(p.receivedAt, Some(p.receivedAt))), commit = true)) register.expectNoMessage(100 millis) } @@ -432,7 +432,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl p.foreach { p => val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) - assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalFailure(TrampolineFeeInsufficient()), Some(FailureAttributionData(p.receivedAt, Some(TimestampMilli(486)))), commit = true)) + assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalTrampolineFailure(TrampolineFeeOrExpiryInsufficient(1_096_000 msat, 60, CltvExpiryDelta(288))), Some(FailureAttributionData(p.receivedAt, Some(TimestampMilli(486)))), commit = true)) } register.expectNoMessage(100 millis) @@ -447,7 +447,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) - assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalFailure(InvalidOnionPayload(UInt64(2), 0)), Some(FailureAttributionData(p.receivedAt, Some(p.receivedAt))), commit = true)) + assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalTrampolineFailure(InvalidOnionPayload(UInt64(2), 0)), Some(FailureAttributionData(p.receivedAt, Some(p.receivedAt))), commit = true)) register.expectNoMessage(100 millis) } @@ -465,7 +465,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl p.foreach { p => val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) - assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalFailure(InvalidOnionPayload(UInt64(2), 0)), Some(FailureAttributionData(p.receivedAt, Some(TimestampMilli(9)))), commit = true)) + assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalTrampolineFailure(InvalidOnionPayload(UInt64(2), 0)), Some(FailureAttributionData(p.receivedAt, Some(TimestampMilli(9)))), commit = true)) } register.expectNoMessage(100 millis) @@ -490,7 +490,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl incomingMultiPart.foreach { p => val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) - assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalFailure(TrampolineFeeInsufficient()), Some(FailureAttributionData(p.receivedAt, Some(incomingMultiPart.last.receivedAt))), commit = true)) + assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalTrampolineFailure(TrampolineFeeOrExpiryInsufficient(2_740_000 msat, 150, CltvExpiryDelta(720))), Some(FailureAttributionData(p.receivedAt, Some(incomingMultiPart.last.receivedAt))), commit = true)) } register.expectNoMessage(100 millis) @@ -518,7 +518,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl incoming.foreach { p => val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) - assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalFailure(TemporaryNodeFailure()), Some(FailureAttributionData(p.receivedAt, Some(incoming.last.receivedAt))), commit = true)) + assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalTrampolineFailure(TemporaryTrampolineFailure()), Some(FailureAttributionData(p.receivedAt, Some(incoming.last.receivedAt))), commit = true)) } register.expectNoMessage(100 millis) @@ -544,7 +544,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl incomingMultiPart.foreach { p => val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) - assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalFailure(TrampolineFeeInsufficient()), Some(FailureAttributionData(p.receivedAt, Some(incomingMultiPart.last.receivedAt))), commit = true)) + assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalTrampolineFailure(TrampolineFeeOrExpiryInsufficient(2_740_000 msat, 150, CltvExpiryDelta(720))), Some(FailureAttributionData(p.receivedAt, Some(incomingMultiPart.last.receivedAt))), commit = true)) } register.expectNoMessage(100 millis) @@ -563,13 +563,15 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl val payFSM = mockPayFSM.expectMessageType[akka.actor.ActorRef] router.expectMessageType[RouteRequest] - val failures = RemoteFailure(outgoingAmount, Nil, Sphinx.DecryptedFailurePacket(outgoingNodeId, FinalIncorrectHtlcAmount(42 msat))) :: UnreadableRemoteFailure(outgoingAmount, Nil, Sphinx.CannotDecryptFailurePacket(ByteVector.empty, None), Nil) :: Nil + // Encrypted trampoline failure created by the recipient for the payer. + val encryptedFailure = Sphinx.CannotDecryptFailurePacket(randomBytes(292), None) + val failures = RemoteFailure(outgoingAmount, Nil, Sphinx.DecryptedFailurePacket(outgoingNodeId, PaymentTimeout())) :: UnreadableRemoteFailure(outgoingAmount, Nil, encryptedFailure, Nil) :: Nil payFSM ! PaymentFailed(relayId, incomingMultiPart.head.add.paymentHash, failures) incomingMultiPart.foreach { p => val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) - assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalFailure(FinalIncorrectHtlcAmount(42 msat)), Some(FailureAttributionData(p.receivedAt, Some(incomingMultiPart.last.receivedAt))), commit = true)) + assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.EncryptedDownstreamFailure(encryptedFailure.unwrapped, encryptedFailure.attribution_opt), Some(FailureAttributionData(p.receivedAt, Some(incomingMultiPart.last.receivedAt))), commit = true)) } register.expectNoMessage(100 millis) @@ -789,7 +791,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl incomingMultiPart.foreach { p => val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) - assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalFailure(TemporaryNodeFailure()), Some(FailureAttributionData(p.receivedAt, Some(incomingMultiPart.last.receivedAt))), commit = true)) + assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalTrampolineFailure(TemporaryTrampolineFailure()), Some(FailureAttributionData(p.receivedAt, Some(incomingMultiPart.last.receivedAt))), commit = true)) } parent.expectMessageType[NodeRelayer.RelayComplete] } @@ -1016,7 +1018,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl incomingPayments.foreach { p => val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) - assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalFailure(UnknownNextPeer()), Some(FailureAttributionData(p.receivedAt, Some(incomingMultiPart.last.receivedAt))), commit = true)) + assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalTrampolineFailure(UnknownNextPeer()), Some(FailureAttributionData(p.receivedAt, Some(incomingMultiPart.last.receivedAt))), commit = true)) } } @@ -1086,7 +1088,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl incomingPayments.foreach { p => val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) - assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalFailure(UnknownNextPeer()), Some(FailureAttributionData(p.receivedAt, Some(incomingMultiPart.last.receivedAt))), commit = true)) + assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalTrampolineFailure(UnknownNextPeer()), Some(FailureAttributionData(p.receivedAt, Some(incomingMultiPart.last.receivedAt))), commit = true)) } } @@ -1147,7 +1149,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl incomingPayments.foreach { p => val fwd = register.expectMessageType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == p.add.channelId) - assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalFailure(UnknownNextPeer()), Some(FailureAttributionData(p.receivedAt, Some(incomingMultiPart.last.receivedAt))), commit = true)) + assert(fwd.message == CMD_FAIL_HTLC(p.add.id, FailureReason.LocalTrampolineFailure(UnknownNextPeer()), Some(FailureAttributionData(p.receivedAt, Some(incomingMultiPart.last.receivedAt))), commit = true)) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/OnTheFlyFundingSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/OnTheFlyFundingSpec.scala index 00569b880e..32d5a52b8b 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/OnTheFlyFundingSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/OnTheFlyFundingSpec.scala @@ -263,7 +263,10 @@ class OnTheFlyFundingSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike { val fwd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == add.channelId) assert(fwd.message.id == add.id) - assert(fwd.message.reason == FailureReason.LocalFailure(TemporaryNodeFailure())) + assert(fwd.message.reason.isInstanceOf[FailureReason.EncryptedDownstreamFailure]) + // This is a trampoline payment: we unwrap the failure packet before forwarding upstream. + val Left(Sphinx.CannotDecryptFailurePacket(expected, _)) = Sphinx.FailurePacket.decrypt(fail4.reason, attribution_opt = None, onionSharedSecrets).failure + assert(fwd.message.reason.asInstanceOf[FailureReason.EncryptedDownstreamFailure].packet == expected) }) val fail5 = WillFailHtlc(willAdd5.id, paymentHash, randomBytes(292)) @@ -272,7 +275,10 @@ class OnTheFlyFundingSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike { val fwd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]] assert(fwd.channelId == add.channelId) assert(fwd.message.id == add.id) - assert(fwd.message.reason == FailureReason.LocalFailure(TemporaryNodeFailure())) + assert(fwd.message.reason.isInstanceOf[FailureReason.EncryptedDownstreamFailure]) + // This is a trampoline payment: we unwrap the failure packet before forwarding upstream. + val Left(Sphinx.CannotDecryptFailurePacket(expected, _)) = Sphinx.FailurePacket.decrypt(fail5.reason, attribution_opt = None, onionSharedSecrets).failure + assert(fwd.message.reason.asInstanceOf[FailureReason.EncryptedDownstreamFailure].packet == expected) }) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/CommandCodecsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/CommandCodecsSpec.scala index 586e188da7..43abda7fef 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/CommandCodecsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/CommandCodecsSpec.scala @@ -39,6 +39,7 @@ class CommandCodecsSpec extends AnyFunSuite { CMD_FAIL_HTLC(253, FailureReason.LocalFailure(TemporaryNodeFailure()), Some(FailureAttributionData(TimestampMilli(123), None))) -> hex"0007 00000000000000fd 01 0002 2002 ff 000000000000007b 00", CMD_FAIL_HTLC(253, FailureReason.LocalFailure(TemporaryNodeFailure()), Some(FailureAttributionData(TimestampMilli(123), Some(TimestampMilli(125))))) -> hex"0007 00000000000000fd 01 0002 2002 ff 000000000000007b ff 000000000000007d", CMD_FAIL_HTLC(253, FailureReason.LocalFailure(TemporaryNodeFailure(TlvStream(Set.empty[FailureMessageTlv], Set(GenericTlv(UInt64(17), hex"deadbeef"))))), Some(FailureAttributionData(TimestampMilli(456), None))) -> hex"0007 00000000000000fd 01 0008 2002 1104deadbeef ff 00000000000001c8 00", + CMD_FAIL_HTLC(253, FailureReason.LocalTrampolineFailure(TemporaryTrampolineFailure()), Some(FailureAttributionData(TimestampMilli(123), None))) -> hex"0007 00000000000000fd 03 0002 2019 ff 000000000000007b 00", CMD_FAIL_MALFORMED_HTLC(7984, ByteVector32(hex"17cc093e177c7a7fcaa9e96ab407146c8886546a5690f945c98ac20c4ab3b4f3"), FailureMessageCodecs.BADONION) -> hex"0002 0000000000001f30 17cc093e177c7a7fcaa9e96ab407146c8886546a5690f945c98ac20c4ab3b4f38000", ) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/FailureMessageCodecsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/FailureMessageCodecsSpec.scala index a92d1fa695..2a6d7b601d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/FailureMessageCodecsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/FailureMessageCodecsSpec.scala @@ -69,8 +69,9 @@ class FailureMessageCodecsSpec extends AnyFunSuite { ExpiryTooFar() -> hex"0015", InvalidOnionPayload(UInt64(561), 1105) -> hex"4016 fd0231 0451", PaymentTimeout() -> hex"0017", - TrampolineFeeInsufficient() -> hex"2033", - TrampolineExpiryTooSoon() -> hex"2034", + TemporaryTrampolineFailure() -> hex"2019", + TrampolineFeeOrExpiryInsufficient(100 msat, 50, CltvExpiryDelta(36)) -> hex"201a 00000064 00000032 0024", + UnknownNextTrampoline() -> hex"401b", ) testCases.foreach { case (msg, bin) => val encoded = failureMessageCodec.encode(msg).require