From c53b52c4bb3ac3ea182293e43f00b92d7c65d513 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Mon, 13 Nov 2023 13:41:10 +0100 Subject: [PATCH] resolve compact paths in OfferPayment --- .../main/scala/fr/acinq/eclair/Eclair.scala | 7 +- .../main/scala/fr/acinq/eclair/Setup.scala | 2 +- .../acinq/eclair/payment/Bolt12Invoice.scala | 3 + .../acinq/eclair/payment/send/Autoprobe.scala | 2 +- .../eclair/payment/send/OfferPayment.scala | 73 ++++++- .../payment/send/PaymentInitiator.scala | 204 +++++------------- .../acinq/eclair/payment/send/Recipient.scala | 17 +- .../integration/ChannelIntegrationSpec.scala | 6 +- .../integration/PaymentIntegrationSpec.scala | 40 ++-- .../PerformanceIntegrationSpec.scala | 2 +- .../basic/fixtures/MinimalNodeFixture.scala | 6 +- .../basic/payment/OfferPaymentSpec.scala | 8 +- .../eclair/payment/PaymentInitiatorSpec.scala | 120 +++-------- .../eclair/payment/PaymentPacketSpec.scala | 8 +- .../payment/send/OfferPaymentSpec.scala | 66 +++++- .../acinq/eclair/router/BaseRouterSpec.scala | 7 +- 16 files changed, 256 insertions(+), 315 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala index 4015457914..86c0ec0162 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala @@ -427,7 +427,8 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging { } else { val recipientAmount = recipientAmount_opt.getOrElse(invoice.amount_opt.getOrElse(route.amount)) val trampoline_opt = trampolineFees_opt.map(fees => TrampolineAttempt(trampolineSecret_opt.getOrElse(randomBytes32()), fees, trampolineExpiryDelta_opt.get)) - appKit.paymentInitiator.toTyped.ask(replyTo => SendPaymentToRoute(replyTo.toClassic, recipientAmount, invoice, route, externalId_opt, parentId_opt, trampoline_opt)) + val sendPayment = SendPaymentToRoute(recipientAmount, invoice, Nil, route, externalId_opt, parentId_opt, trampoline_opt) + (appKit.paymentInitiator ? sendPayment).mapTo[SendPaymentToRouteResponse] } } @@ -441,7 +442,7 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging { externalId_opt match { case Some(externalId) if externalId.length > externalIdMaxLength => Left(new IllegalArgumentException(s"externalId is too long: cannot exceed $externalIdMaxLength characters")) case _ if invoice.isExpired() => Left(new IllegalArgumentException("invoice has expired")) - case _ => Right(SendPaymentToNode(ActorRef.noSender, amount, invoice, maxAttempts, externalId_opt, routeParams = routeParams)) + case _ => Right(SendPaymentToNode(ActorRef.noSender, amount, invoice, Nil, maxAttempts, externalId_opt, routeParams = routeParams)) } case Left(t) => Left(t) } @@ -701,7 +702,7 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging { case Left(t) => return Future.failed(t) } val sendPaymentConfig = OfferPayment.SendPaymentConfig(externalId_opt, connectDirectly, maxAttempts_opt.getOrElse(appKit.nodeParams.maxPaymentAttempts), routeParams, blocking) - val offerPayment = appKit.system.spawnAnonymous(OfferPayment(appKit.nodeParams, appKit.postman, appKit.paymentInitiator)) + val offerPayment = appKit.system.spawnAnonymous(OfferPayment(appKit.nodeParams, appKit.postman, appKit.router, appKit.paymentInitiator)) offerPayment.ask((ref: typed.ActorRef[Any]) => OfferPayment.PayOffer(ref.toClassic, offer, amount, quantity, sendPaymentConfig)).flatMap { case f: OfferPayment.Failure => Future.failed(new Exception(f.toString)) case x => Future.successful(x) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala index c2ef0fe926..d04a5105f5 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala @@ -371,7 +371,7 @@ class Setup(val datadir: File, _ = switchboard ! Switchboard.Init(channels) clientSpawner = system.actorOf(SimpleSupervisor.props(ClientSpawner.props(nodeParams.keyPair, nodeParams.socksProxy_opt, nodeParams.peerConnectionConf, switchboard, router), "client-spawner", SupervisorStrategy.Restart)) server = system.actorOf(SimpleSupervisor.props(Server.props(nodeParams.keyPair, nodeParams.peerConnectionConf, switchboard, router, serverBindingAddress, Some(tcpBound)), "server", SupervisorStrategy.Restart)) - paymentInitiator = system.actorOf(SimpleSupervisor.props(PaymentInitiator.props(nodeParams, PaymentInitiator.SimplePaymentFactory(nodeParams, router, register), router), "payment-initiator", SupervisorStrategy.Restart)) + paymentInitiator = system.actorOf(SimpleSupervisor.props(PaymentInitiator.props(nodeParams, PaymentInitiator.SimplePaymentFactory(nodeParams, router, register)), "payment-initiator", SupervisorStrategy.Restart)) _ = for (i <- 0 until config.getInt("autoprobe-count")) yield system.actorOf(SimpleSupervisor.props(Autoprobe.props(nodeParams, router, paymentInitiator), s"payment-autoprobe-$i", SupervisorStrategy.Restart)) balanceActor = system.spawn(BalanceActor(nodeParams.db, bitcoinClient, channelsListener, nodeParams.balanceCheckInterval), name = "balance-actor") diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt12Invoice.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt12Invoice.scala index 0307ab5d11..03c4ab715d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt12Invoice.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt12Invoice.scala @@ -20,6 +20,7 @@ import fr.acinq.bitcoin.Bech32 import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Crypto} import fr.acinq.eclair.crypto.Sphinx +import fr.acinq.eclair.crypto.Sphinx.RouteBlinding.BlindedRoute import fr.acinq.eclair.wire.protocol.OfferTypes._ import fr.acinq.eclair.wire.protocol.OnionRoutingCodecs.{InvalidTlvPayload, MissingRequiredTlv} import fr.acinq.eclair.wire.protocol.{GenericTlv, OfferCodecs, OfferTypes, TlvStream} @@ -88,6 +89,8 @@ case class Bolt12Invoice(records: TlvStream[InvoiceTlv]) extends Invoice { case class PaymentBlindedRoute(route: BlindedContactInfo, paymentInfo: PaymentInfo) +case class ResolvedPaymentBlindedRoute(route: BlindedRoute, paymentInfo: PaymentInfo) + object Bolt12Invoice { val hrp = "lni" val signatureTag: ByteVector = ByteVector(("lightning" + "invoice" + "signature").getBytes) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Autoprobe.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Autoprobe.scala index 7f5c060af7..a30526e43f 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Autoprobe.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/Autoprobe.scala @@ -68,7 +68,7 @@ class Autoprobe(nodeParams: NodeParams, router: ActorRef, paymentInitiator: Acto ByteVector.empty) log.info(s"sending payment probe to node=$targetNodeId payment_hash=${fakeInvoice.paymentHash}") val routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams - paymentInitiator ! PaymentInitiator.SendPaymentToNode(self, PAYMENT_AMOUNT_MSAT, fakeInvoice, maxAttempts = 1, routeParams = routeParams) + paymentInitiator ! PaymentInitiator.SendPaymentToNode(self, PAYMENT_AMOUNT_MSAT, fakeInvoice, Nil, maxAttempts = 1, routeParams = routeParams) case None => log.info(s"could not find a destination, re-scheduling") scheduleProbe() diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/OfferPayment.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/OfferPayment.scala index fd2afc4afd..213300a582 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/OfferPayment.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/OfferPayment.scala @@ -20,15 +20,18 @@ import akka.actor.typed.Behavior import akka.actor.typed.scaladsl.{ActorContext, Behaviors} import akka.actor.{ActorRef, typed} import fr.acinq.bitcoin.scalacompat.ByteVector32 -import fr.acinq.bitcoin.scalacompat.Crypto.PrivateKey +import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} +import fr.acinq.eclair.crypto.Sphinx.RouteBlinding.BlindedRoute import fr.acinq.eclair.message.Postman.{OnionMessageResponse, SendMessage} import fr.acinq.eclair.message.{OnionMessages, Postman} import fr.acinq.eclair.payment.send.PaymentInitiator.SendPaymentToNode +import fr.acinq.eclair.payment.{PaymentBlindedRoute, ResolvedPaymentBlindedRoute} +import fr.acinq.eclair.router.Router import fr.acinq.eclair.router.Router.RouteParams import fr.acinq.eclair.wire.protocol.MessageOnion.{FinalPayload, InvoicePayload} -import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, Offer} +import fr.acinq.eclair.wire.protocol.OfferTypes.{BlindedPath, CompactBlindedPath, InvoiceRequest, Offer, PaymentInfo} import fr.acinq.eclair.wire.protocol.{OnionMessagePayloadTlv, TlvStream} -import fr.acinq.eclair.{Features, InvoiceFeature, MilliSatoshi, NodeParams, TimestampSecond, randomKey} +import fr.acinq.eclair.{Features, InvoiceFeature, MilliSatoshi, NodeParams, RealShortChannelId, TimestampSecond, randomKey} object OfferPayment { sealed trait Failure @@ -49,6 +52,10 @@ object OfferPayment { override def toString: String = s"Invalid invoice response: $response, invoice request: $request" } + case class UnknownShortChannelIds(scids: Seq[RealShortChannelId]) extends Failure { + override def toString: String = s"Unknown short channel ids: $scids" + } + sealed trait Command case class PayOffer(replyTo: ActorRef, @@ -59,6 +66,8 @@ object OfferPayment { case class WrappedMessageResponse(response: OnionMessageResponse) extends Command + private case class WrappedNodeId(nodeId_opt: Option[PublicKey]) extends Command + case class SendPaymentConfig(externalId_opt: Option[String], connectDirectly: Boolean, maxAttempts: Int, @@ -67,6 +76,7 @@ object OfferPayment { def apply(nodeParams: NodeParams, postman: typed.ActorRef[Postman.Command], + router: ActorRef, paymentInitiator: ActorRef): Behavior[Command] = { Behaviors.setup(context => Behaviors.receiveMessagePartial { @@ -89,13 +99,14 @@ object OfferPayment { } else { val payerKey = randomKey() val request = InvoiceRequest(offer, amount, quantity, nodeParams.features.bolt12Features(), payerKey, nodeParams.chainHash) - sendInvoiceRequest(nodeParams, postman, paymentInitiator, context, request, payerKey, replyTo, 0, sendPaymentConfig) + sendInvoiceRequest(nodeParams, postman, router, paymentInitiator, context, request, payerKey, replyTo, 0, sendPaymentConfig) } }) } def sendInvoiceRequest(nodeParams: NodeParams, postman: typed.ActorRef[Postman.Command], + router: ActorRef, paymentInitiator: ActorRef, context: ActorContext[Command], request: InvoiceRequest, @@ -107,11 +118,12 @@ object OfferPayment { val messageContent = TlvStream[OnionMessagePayloadTlv](OnionMessagePayloadTlv.InvoiceRequest(request.records)) val routingStrategy = if (sendPaymentConfig.connectDirectly) OnionMessages.RoutingStrategy.connectDirectly else OnionMessages.RoutingStrategy.FindRoute postman ! SendMessage(contactInfo, routingStrategy, messageContent, expectsReply = true, context.messageAdapter(WrappedMessageResponse)) - waitForInvoice(nodeParams, postman, paymentInitiator, context, request, payerKey, replyTo, attemptNumber + 1, sendPaymentConfig) + waitForInvoice(nodeParams, postman, router, paymentInitiator, context, request, payerKey, replyTo, attemptNumber + 1, sendPaymentConfig) } def waitForInvoice(nodeParams: NodeParams, postman: typed.ActorRef[Postman.Command], + router: ActorRef, paymentInitiator: ActorRef, context: ActorContext[Command], request: InvoiceRequest, @@ -121,20 +133,63 @@ object OfferPayment { sendPaymentConfig: SendPaymentConfig): Behavior[Command] = { Behaviors.receiveMessagePartial { case WrappedMessageResponse(Postman.Response(payload: InvoicePayload)) if payload.invoice.validateFor(request).isRight => - val recipientAmount = payload.invoice.amount - paymentInitiator ! SendPaymentToNode(replyTo, recipientAmount, payload.invoice, maxAttempts = sendPaymentConfig.maxAttempts, externalId = sendPaymentConfig.externalId_opt, routeParams = sendPaymentConfig.routeParams, payerKey_opt = Some(payerKey), blockUntilComplete = sendPaymentConfig.blocking) - Behaviors.stopped + val sendPaymentToNode = SendPaymentToNode(replyTo, payload.invoice.amount, payload.invoice, Nil, maxAttempts = sendPaymentConfig.maxAttempts, externalId = sendPaymentConfig.externalId_opt, routeParams = sendPaymentConfig.routeParams, payerKey_opt = Some(payerKey), blockUntilComplete = sendPaymentConfig.blocking) + val scids = payload.invoice.blindedPaths.collect { case PaymentBlindedRoute(CompactBlindedPath(scdidDir, _, _), _) => scdidDir.scid } + resolve(context, paymentInitiator, router, sendPaymentToNode, payload.invoice.blindedPaths, Nil, scids) case WrappedMessageResponse(Postman.Response(payload)) => // We've received a response but it is not an invoice as we expected or it is an invalid invoice. replyTo ! InvalidInvoiceResponse(request, payload) Behaviors.stopped case WrappedMessageResponse(Postman.NoReply) if attemptNumber < nodeParams.onionMessageConfig.maxAttempts => // We didn't get a response, let's retry. - sendInvoiceRequest(nodeParams, postman, paymentInitiator, context, request, payerKey, replyTo, attemptNumber, sendPaymentConfig) + sendInvoiceRequest(nodeParams, postman, router, paymentInitiator, context, request, payerKey, replyTo, attemptNumber, sendPaymentConfig) case WrappedMessageResponse(_) => // We can't reach the offer node or the offer node can't reach us. replyTo ! NoInvoiceResponse Behaviors.stopped } } + + def resolve(context: ActorContext[Command], + paymentInitiator: ActorRef, + router: ActorRef, + sendPaymentToNode: SendPaymentToNode, + toResolve: Seq[PaymentBlindedRoute], + resolved: Seq[ResolvedPaymentBlindedRoute], + scids: Seq[RealShortChannelId]): Behavior[Command] = { + if (toResolve.isEmpty) { + if (resolved.isEmpty) { + // No route could be resolved + sendPaymentToNode.replyTo ! UnknownShortChannelIds(scids) + } else { + paymentInitiator ! sendPaymentToNode.copy(resolvedPaths = resolved) + } + Behaviors.stopped + } else { + toResolve.head match { + case PaymentBlindedRoute(BlindedPath(route), paymentInfo) => + resolve(context, paymentInitiator, router, sendPaymentToNode, toResolve.tail, resolved :+ ResolvedPaymentBlindedRoute(route, paymentInfo), scids) + case PaymentBlindedRoute(route: CompactBlindedPath, paymentInfo) => + router ! Router.GetNodeId(context.messageAdapter(WrappedNodeId), route.introductionNode.scid, route.introductionNode.isNode1) + waitForNodeId(context, paymentInitiator, router, sendPaymentToNode, route, paymentInfo, toResolve.tail, resolved, scids) + } + } + } + + def waitForNodeId(context: ActorContext[Command], + paymentInitiator: ActorRef, + router: ActorRef, + sendPaymentToNode: SendPaymentToNode, + compactRoute: CompactBlindedPath, + paymentInfo: PaymentInfo, + toResolve: Seq[PaymentBlindedRoute], + resolved: Seq[ResolvedPaymentBlindedRoute], + scids: Seq[RealShortChannelId]): Behavior[Command] = + Behaviors.receiveMessagePartial { + case WrappedNodeId(None) => + resolve(context, paymentInitiator, router, sendPaymentToNode, toResolve, resolved, scids) + case WrappedNodeId(Some(nodeId)) => + val resolvedPaymentBlindedRoute = ResolvedPaymentBlindedRoute(BlindedRoute(nodeId, compactRoute.blindingKey, compactRoute.blindedNodes), paymentInfo) + resolve(context, paymentInitiator, router, sendPaymentToNode, toResolve, resolved :+ resolvedPaymentBlindedRoute, scids) + } } \ No newline at end of file diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentInitiator.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentInitiator.scala index 9204a4489e..032c5184ea 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentInitiator.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentInitiator.scala @@ -16,7 +16,6 @@ package fr.acinq.eclair.payment.send -import akka.actor.typed.scaladsl.adapter._ import akka.actor.{Actor, ActorContext, ActorLogging, ActorRef, Props} import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto} @@ -37,7 +36,7 @@ import scala.util.{Failure, Success, Try} /** * Created by PM on 29/08/2016. */ -class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentInitiator.MultiPartPaymentFactory, router: ActorRef) extends Actor with ActorLogging { +class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentInitiator.MultiPartPaymentFactory) extends Actor with ActorLogging { import PaymentInitiator._ @@ -46,30 +45,27 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn def main(pending: Map[UUID, PendingPayment]): Receive = { case r: SendPaymentToNode => val replyTo = if (r.replyTo == ActorRef.noSender) sender() else r.replyTo - val paymentCfg = r.paymentConfig() - if (!r.blockUntilComplete && r.paymentConfig_opt.isEmpty) { + val paymentId = UUID.randomUUID() + if (!r.blockUntilComplete) { // Immediately return the paymentId - replyTo ! paymentCfg.id + replyTo ! paymentId } + val paymentCfg = SendPaymentConfig(paymentId, paymentId, r.externalId, r.paymentHash, r.invoice.nodeId, Upstream.Local(paymentId), Some(r.invoice), r.payerKey_opt, storeInDb = true, publishEvent = true, recordPathFindingMetrics = true) val finalExpiry = r.finalExpiry(nodeParams) - (r.invoice match { - case invoice: Bolt11Invoice => Some(ClearRecipient(invoice, r.recipientAmount, finalExpiry, r.userCustomTlvs)) - case invoice: Bolt12Invoice => BlindedRecipient(invoice, r.recipientAmount, finalExpiry, r.userCustomTlvs) - }) match { - case Some(recipient) => - if (!nodeParams.features.invoiceFeatures().areSupported(recipient.features)) { - replyTo ! PaymentFailed(paymentCfg.id, r.paymentHash, LocalFailure(r.recipientAmount, Nil, UnsupportedFeatures(recipient.features)) :: Nil) - } else if (Features.canUseFeature(nodeParams.features.invoiceFeatures(), recipient.features, Features.BasicMultiPartPayment)) { - val fsm = outgoingPaymentFactory.spawnOutgoingMultiPartPayment(context, paymentCfg, publishPreimage = !r.blockUntilComplete) - fsm ! MultiPartPaymentLifecycle.SendMultiPartPayment(self, recipient, r.maxAttempts, r.routeParams) - context become main(pending + (paymentCfg.id -> PendingPaymentToNode(replyTo, r))) - } else { - val fsm = outgoingPaymentFactory.spawnOutgoingPayment(context, paymentCfg) - fsm ! PaymentLifecycle.SendPaymentToNode(self, recipient, r.maxAttempts, r.routeParams) - context become main(pending + (paymentCfg.id -> PendingPaymentToNode(replyTo, r))) - } - case None => - context.spawnAnonymous(CompactRoutesResolver(self, router, r.copy(replyTo = replyTo, paymentConfig_opt = Some(paymentCfg)), paymentCfg.id)) + val recipient = r.invoice match { + case invoice: Bolt11Invoice => ClearRecipient(invoice, r.recipientAmount, finalExpiry, r.userCustomTlvs) + case invoice: Bolt12Invoice => BlindedRecipient(invoice, r.resolvedPaths, r.recipientAmount, finalExpiry, r.userCustomTlvs) + } + if (!nodeParams.features.invoiceFeatures().areSupported(recipient.features)) { + replyTo ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, UnsupportedFeatures(recipient.features)) :: Nil) + } else if (Features.canUseFeature(nodeParams.features.invoiceFeatures(), recipient.features, Features.BasicMultiPartPayment)) { + val fsm = outgoingPaymentFactory.spawnOutgoingMultiPartPayment(context, paymentCfg, publishPreimage = !r.blockUntilComplete) + fsm ! MultiPartPaymentLifecycle.SendMultiPartPayment(self, recipient, r.maxAttempts, r.routeParams) + context become main(pending + (paymentId -> PendingPaymentToNode(replyTo, r))) + } else { + val fsm = outgoingPaymentFactory.spawnOutgoingPayment(context, paymentCfg) + fsm ! PaymentLifecycle.SendPaymentToNode(self, recipient, r.maxAttempts, r.routeParams) + context become main(pending + (paymentId -> PendingPaymentToNode(replyTo, r))) } case r: SendSpontaneousPayment => @@ -84,58 +80,57 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn case r: SendTrampolinePayment => val paymentId = UUID.randomUUID() - r.replyTo ! paymentId + sender() ! paymentId r.trampolineAttempts match { case Nil => - r.replyTo ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, TrampolineFeesMissing) :: Nil) + sender() ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, TrampolineFeesMissing) :: Nil) case _ if !r.invoice.features.hasFeature(Features.TrampolinePaymentPrototype) && r.invoice.amount_opt.isEmpty => - r.replyTo ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, TrampolineLegacyAmountLessInvoice) :: Nil) + sender() ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, TrampolineLegacyAmountLessInvoice) :: Nil) case (trampolineFees, trampolineExpiryDelta) :: remainingAttempts => log.info(s"sending trampoline payment with trampoline fees=$trampolineFees and expiry delta=$trampolineExpiryDelta") sendTrampolinePayment(paymentId, r, trampolineFees, trampolineExpiryDelta) match { case Success(_) => - context become main(pending + (paymentId -> PendingTrampolinePayment(r.replyTo, remainingAttempts, r))) + context become main(pending + (paymentId -> PendingTrampolinePayment(sender(), remainingAttempts, r))) case Failure(t) => log.warning("cannot send outgoing trampoline payment: {}", t.getMessage) - r.replyTo ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, t) :: Nil) + sender() ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, t) :: Nil) } } case r: SendPaymentToRoute => - val paymentCfg = r.paymentConfig() + val paymentId = UUID.randomUUID() + val parentPaymentId = r.parentId.getOrElse(UUID.randomUUID()) r.trampoline_opt match { case _ if !nodeParams.features.invoiceFeatures().areSupported(r.invoice.features) => - r.replyTo ! PaymentFailed(paymentCfg.id, r.paymentHash, LocalFailure(r.recipientAmount, Nil, UnsupportedFeatures(r.invoice.features)) :: Nil) + sender() ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, UnsupportedFeatures(r.invoice.features)) :: Nil) case Some(trampolineAttempt) => val trampolineNodeId = r.route.targetNodeId log.info(s"sending trampoline payment to ${r.recipientNodeId} with trampoline=$trampolineNodeId, trampoline fees=${trampolineAttempt.fees}, expiry delta=${trampolineAttempt.cltvExpiryDelta}") val trampolineHop = NodeHop(trampolineNodeId, r.recipientNodeId, trampolineAttempt.cltvExpiryDelta, trampolineAttempt.fees) buildTrampolineRecipient(r, trampolineHop) match { case Success(recipient) => - r.replyTo ! SendPaymentToRouteResponse(paymentCfg.id, paymentCfg.parentId, Some(recipient.trampolinePaymentSecret)) + sender() ! SendPaymentToRouteResponse(paymentId, parentPaymentId, Some(recipient.trampolinePaymentSecret)) + val paymentCfg = SendPaymentConfig(paymentId, parentPaymentId, r.externalId, r.paymentHash, r.recipientNodeId, Upstream.Local(paymentId), Some(r.invoice), None, storeInDb = true, publishEvent = true, recordPathFindingMetrics = false) val payFsm = outgoingPaymentFactory.spawnOutgoingPayment(context, paymentCfg) payFsm ! PaymentLifecycle.SendPaymentToRoute(self, Left(r.route), recipient) - context become main(pending + (paymentCfg.id -> PendingPaymentToRoute(r.replyTo, r))) + context become main(pending + (paymentId -> PendingPaymentToRoute(sender(), r))) case Failure(t) => log.warning("cannot send outgoing trampoline payment: {}", t.getMessage) - r.replyTo ! PaymentFailed(paymentCfg.id, r.paymentHash, LocalFailure(r.recipientAmount, Nil, t) :: Nil) + sender() ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, t) :: Nil) } case None => - r.replyTo ! SendPaymentToRouteResponse(paymentCfg.id, paymentCfg.parentId, None) + sender() ! SendPaymentToRouteResponse(paymentId, parentPaymentId, None) + val paymentCfg = SendPaymentConfig(paymentId, parentPaymentId, r.externalId, r.paymentHash, r.recipientNodeId, Upstream.Local(paymentId), Some(r.invoice), None, storeInDb = true, publishEvent = true, recordPathFindingMetrics = false) val finalExpiry = r.finalExpiry(nodeParams) - (r.invoice match { - case invoice: Bolt11Invoice => Some(ClearRecipient(invoice, r.recipientAmount, finalExpiry, Set.empty)) - case invoice: Bolt12Invoice => BlindedRecipient(invoice, r.recipientAmount, finalExpiry, Set.empty) - }) match { - case Some(recipient) => - val payFsm = outgoingPaymentFactory.spawnOutgoingPayment(context, paymentCfg) - payFsm ! PaymentLifecycle.SendPaymentToRoute(self, Left(r.route), recipient) - context become main(pending + (paymentCfg.id -> PendingPaymentToRoute(r.replyTo, r))) - case None => - context.spawnAnonymous(CompactRoutesResolver(self, router, r.copy(paymentConfig_opt = Some(paymentCfg)), paymentCfg.id)) + val recipient = r.invoice match { + case invoice: Bolt11Invoice => ClearRecipient(invoice, r.recipientAmount, finalExpiry, Set.empty) + case invoice: Bolt12Invoice => BlindedRecipient(invoice, r.resolvedPaths, r.recipientAmount, finalExpiry, Set.empty) } + val payFsm = outgoingPaymentFactory.spawnOutgoingPayment(context, paymentCfg) + payFsm ! PaymentLifecycle.SendPaymentToRoute(self, Left(r.route), recipient) + context become main(pending + (paymentId -> PendingPaymentToRoute(sender(), r))) case _ => - r.replyTo ! PaymentFailed(paymentCfg.id, r.paymentHash, LocalFailure(r.recipientAmount, Nil, TrampolineMultiNodeNotSupported) :: Nil) + sender() ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, TrampolineMultiNodeNotSupported) :: Nil) } case pf: PaymentFailed => pending.get(pf.id).foreach { @@ -256,7 +251,7 @@ object PaymentInitiator { } } - def props(nodeParams: NodeParams, outgoingPaymentFactory: MultiPartPaymentFactory, router: ActorRef) = Props(new PaymentInitiator(nodeParams, outgoingPaymentFactory, router)) + def props(nodeParams: NodeParams, outgoingPaymentFactory: MultiPartPaymentFactory) = Props(new PaymentInitiator(nodeParams, outgoingPaymentFactory)) // @formatter:off sealed trait PendingPayment { @@ -278,7 +273,6 @@ object PaymentInitiator { sealed trait SendRequestedPayment { // @formatter:off - def replyTo: ActorRef def recipientAmount: MilliSatoshi def invoice: Invoice def recipientNodeId: PublicKey = invoice.nodeId @@ -306,8 +300,7 @@ object PaymentInitiator { * msat and cltv of 144, and retry with 15 msat and 288 in case an error occurs. * @param routeParams (optional) parameters to fine-tune the routing algorithm. */ - case class SendTrampolinePayment(replyTo: ActorRef, - recipientAmount: MilliSatoshi, + case class SendTrampolinePayment(recipientAmount: MilliSatoshi, invoice: Invoice, trampolineNodeId: PublicKey, trampolineAttempts: Seq[(MilliSatoshi, CltvExpiryDelta)], @@ -315,30 +308,25 @@ object PaymentInitiator { /** * @param recipientAmount amount that should be received by the final recipient (usually from a Bolt 11 invoice). - * @param invoice invoice to pay that may have been modified from the original invoice. + * @param invoice invoice. + * @param resolvedPaths when using a Bolt 12 invoice, list of payment paths to reach the recipient. * @param maxAttempts maximum number of retries. * @param externalId (optional) externally-controlled identifier (to reconcile between application DB and eclair DB). * @param routeParams (optional) parameters to fine-tune the routing algorithm. * @param payerKey_opt (optional) private key associated with the invoice request when paying an offer. * @param userCustomTlvs (optional) user-defined custom tlvs that will be added to the onion sent to the target node. * @param blockUntilComplete (optional) if true, wait until the payment completes before returning a result. - * @param paymentConfig_opt (optional) config for this payment. */ case class SendPaymentToNode(replyTo: ActorRef, recipientAmount: MilliSatoshi, invoice: Invoice, + resolvedPaths: Seq[ResolvedPaymentBlindedRoute], maxAttempts: Int, externalId: Option[String] = None, routeParams: RouteParams, payerKey_opt: Option[PrivateKey] = None, userCustomTlvs: Set[GenericTlv] = Set.empty, - blockUntilComplete: Boolean = false, - paymentConfig_opt: Option[SendPaymentConfig] = None) extends SendRequestedPayment { - def paymentConfig(): SendPaymentConfig = paymentConfig_opt.getOrElse { - val paymentId = UUID.randomUUID() - SendPaymentConfig(paymentId, paymentId, externalId, paymentHash, invoice.nodeId, Upstream.Local(paymentId), Some(invoice), payerKey_opt, storeInDb = true, publishEvent = true, recordPathFindingMetrics = true) - } - } + blockUntilComplete: Boolean = false) extends SendRequestedPayment /** * @param recipientAmount amount that should be received by the final recipient. @@ -384,7 +372,8 @@ object PaymentInitiator { * * @param recipientAmount amount that should be received by the final recipient (usually from a Bolt 11 invoice). * This amount may be split between multiple requests if using MPP. - * @param invoice invoice to pay that may have been modified from the original invoice. + * @param invoice Bolt 11 invoice. + * @param resolvedPaths when using a Bolt 12 invoice, list of payment paths to reach the recipient. * @param route route to use to reach either the final recipient or the trampoline node. * @param externalId (optional) externally-controlled identifier (to reconcile between application DB and eclair DB). * @param parentId id of the whole payment. When manually sending a multi-part payment, you need to make @@ -392,22 +381,14 @@ object PaymentInitiator { * be generated that can be used for the remaining partial payments. * @param trampoline_opt if trampoline is used, this field must be provided. When manually sending a multi-part * payment, you need to make sure all partial payments share the same values. - * @param paymentConfig_opt (optional) config for this payment. */ - case class SendPaymentToRoute(replyTo: ActorRef, - recipientAmount: MilliSatoshi, + case class SendPaymentToRoute(recipientAmount: MilliSatoshi, invoice: Invoice, + resolvedPaths: Seq[ResolvedPaymentBlindedRoute], route: PredefinedRoute, externalId: Option[String], parentId: Option[UUID], - trampoline_opt: Option[TrampolineAttempt], - paymentConfig_opt: Option[SendPaymentConfig] = None) extends SendRequestedPayment { - def paymentConfig(): SendPaymentConfig = paymentConfig_opt.getOrElse { - val paymentId = UUID.randomUUID() - val parentPaymentId = parentId.getOrElse(UUID.randomUUID()) - SendPaymentConfig(paymentId, parentPaymentId, externalId, paymentHash, recipientNodeId, Upstream.Local(paymentId), Some(invoice), None, storeInDb = true, publishEvent = true, recordPathFindingMetrics = false) - } - } + trampoline_opt: Option[TrampolineAttempt]) extends SendRequestedPayment /** * @param paymentId id of the outgoing payment (mapped to a single outgoing HTLC). @@ -429,7 +410,7 @@ object PaymentInitiator { * @param paymentHash payment hash. * @param recipientNodeId id of the final recipient. * @param upstream information about the payment origin (to link upstream to downstream when relaying a payment). - * @param invoice Invoice. + * @param invoice Bolt 11 invoice. * @param storeInDb whether to store data in the payments DB (e.g. when we're relaying a trampoline payment, we * don't want to store in the DB). * @param publishEvent whether to publish a [[fr.acinq.eclair.payment.PaymentEvent]] on success/failure (e.g. for @@ -456,85 +437,4 @@ object PaymentInitiator { def createPaymentSent(recipient: Recipient, preimage: ByteVector32, parts: Seq[PaymentSent.PartialPayment]) = PaymentSent(parentId, paymentHash, preimage, recipient.totalAmount, recipient.nodeId, parts) } - object CompactRoutesResolver { - - import akka.actor.typed.Behavior - import akka.actor.typed.scaladsl.{ActorContext, Behaviors} - import fr.acinq.eclair.crypto.Sphinx.RouteBlinding.BlindedRoute - import fr.acinq.eclair.router.Router - import fr.acinq.eclair.wire.protocol.OfferTypes.{BlindedPath, CompactBlindedPath, PaymentInfo, InvoicePaths, InvoiceBlindedPay} - - sealed trait Command - - private case class WrappedNodeId(nodeId_opt: Option[PublicKey]) extends Command - - def apply(paymentInitiator: ActorRef, router: ActorRef, sendRequestedPayment: SendRequestedPayment, paymentId: UUID): Behavior[Command] = - Behaviors.setup { context => - sendRequestedPayment.invoice match { - case invoice: Bolt12Invoice => - resolve(context, paymentInitiator, router, sendRequestedPayment, paymentId, invoice, invoice.blindedPaths, Nil) - case _ => - // This should never happen as there is no need to resolve anything for other types of invoices. - sendRequestedPayment.replyTo ! PaymentFailed(paymentId, sendRequestedPayment.paymentHash, LocalFailure(sendRequestedPayment.recipientAmount, Nil, RouteNotFound) :: Nil) - Behaviors.stopped - } - } - - def resolve(context: ActorContext[Command], - paymentInitiator: ActorRef, - router: ActorRef, - sendRequestedPayment: SendRequestedPayment, - paymentId: UUID, - invoice: Bolt12Invoice, - toResolve: Seq[PaymentBlindedRoute], - resolved: Seq[PaymentBlindedRoute]): Behavior[Command] = { - if (toResolve.isEmpty) { - if (resolved.isEmpty) { - // No route could be resolved - sendRequestedPayment.replyTo ! PaymentFailed(paymentId, sendRequestedPayment.paymentHash, LocalFailure(sendRequestedPayment.recipientAmount, Nil, RouteNotFound) :: Nil) - } else { - val tlvs = invoice.records.records.map { - case InvoicePaths(_) => InvoicePaths(resolved.map(_.route)) - case InvoiceBlindedPay(_) => InvoiceBlindedPay(resolved.map(_.paymentInfo)) - case x => x - } - val resolvedInvoice = Bolt12Invoice(invoice.records.copy(records = tlvs)) - val sendToResolved = sendRequestedPayment match { - case s: SendTrampolinePayment => s.copy(invoice = resolvedInvoice) - case s: SendPaymentToNode => s.copy(invoice = resolvedInvoice) - case s: SendPaymentToRoute => s.copy(invoice = resolvedInvoice) - } - paymentInitiator ! sendToResolved - } - Behaviors.stopped - } else { - toResolve.head match { - case paymentBlindedRoute@PaymentBlindedRoute(_: BlindedPath, _) => - resolve(context, paymentInitiator, router, sendRequestedPayment, paymentId, invoice, toResolve.tail, resolved :+ paymentBlindedRoute) - case PaymentBlindedRoute(route: CompactBlindedPath, paymentInfo) => - router ! Router.GetNodeId(context.messageAdapter(WrappedNodeId), route.introductionNode.scid, route.introductionNode.isNode1) - waitForNodeId(context, paymentInitiator, router, sendRequestedPayment, paymentId, invoice, route, paymentInfo, toResolve.tail, resolved) - } - } - } - - def waitForNodeId(context: ActorContext[Command], - paymentInitiator: ActorRef, - router: ActorRef, - sendRequestedPayment: SendRequestedPayment, - paymentId: UUID, - invoice: Bolt12Invoice, - compactRoute: CompactBlindedPath, - paymentInfo: PaymentInfo, - toResolve: Seq[PaymentBlindedRoute], - resolved: Seq[PaymentBlindedRoute]): Behavior[Command] = - Behaviors.receiveMessage { - case WrappedNodeId(None) => - resolve(context, paymentInitiator, router, sendRequestedPayment, paymentId, invoice, toResolve, resolved) - case WrappedNodeId(Some(nodeId)) => - val paymentBlindedRoute = PaymentBlindedRoute(BlindedPath(BlindedRoute(nodeId, compactRoute.blindingKey, compactRoute.blindedNodes)), paymentInfo) - resolve(context, paymentInitiator, router, sendRequestedPayment, paymentId, invoice, toResolve, resolved :+ paymentBlindedRoute) - } - } - } 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 abda773d99..3eb95ef5df 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 @@ -21,10 +21,10 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.eclair.crypto.Sphinx import fr.acinq.eclair.payment.Invoice.ExtraEdge import fr.acinq.eclair.payment.OutgoingPaymentPacket._ -import fr.acinq.eclair.payment.{Bolt11Invoice, Bolt12Invoice, OutgoingPaymentPacket} +import fr.acinq.eclair.payment.{Bolt11Invoice, Bolt12Invoice, OutgoingPaymentPacket, ResolvedPaymentBlindedRoute} import fr.acinq.eclair.router.Router._ import fr.acinq.eclair.wire.protocol.PaymentOnion.{FinalPayload, IntermediatePayload, OutgoingBlindedPerHopPayload} -import fr.acinq.eclair.wire.protocol.{GenericTlv, OfferTypes, OnionRoutingPacket, PaymentOnionCodecs} +import fr.acinq.eclair.wire.protocol.{GenericTlv, OnionRoutingPacket, PaymentOnionCodecs} import fr.acinq.eclair.{CltvExpiry, Features, InvoiceFeature, MilliSatoshi, MilliSatoshiLong, ShortChannelId} import scodec.bits.ByteVector @@ -122,7 +122,7 @@ case class BlindedRecipient(nodeId: PublicKey, totalAmount: MilliSatoshi, expiry: CltvExpiry, blindedHops: Seq[BlindedHop], - customTlvs: Set[GenericTlv] = Set.empty) extends Recipient { + customTlvs: Set[GenericTlv]) extends Recipient { require(blindedHops.nonEmpty, "blinded routes must be provided") override val extraEdges = blindedHops.map { h => @@ -166,18 +166,15 @@ case class BlindedRecipient(nodeId: PublicKey, } object BlindedRecipient { - def apply(invoice: Bolt12Invoice, totalAmount: MilliSatoshi, expiry: CltvExpiry, customTlvs: Set[GenericTlv]): Option[BlindedRecipient] = { - val blindedHops = invoice.blindedPaths.map( + def apply(invoice: Bolt12Invoice, paths: Seq[ResolvedPaymentBlindedRoute], totalAmount: MilliSatoshi, expiry: CltvExpiry, customTlvs: Set[GenericTlv]): BlindedRecipient = { + val blindedHops = paths.map( path => { // We don't know the scids of channels inside the blinded route, but it's useful to have an ID to refer to a // given edge in the graph, so we create a dummy one for the duration of the payment attempt. val dummyId = ShortChannelId.generateLocalAlias() - path.route match { - case OfferTypes.BlindedPath(route) => BlindedHop(dummyId, route, path.paymentInfo) - case _: OfferTypes.CompactBlindedPath => return None - } + BlindedHop(dummyId, path.route, path.paymentInfo) }) - Some(BlindedRecipient(invoice.nodeId, invoice.features, totalAmount, expiry, blindedHops, customTlvs)) + BlindedRecipient(invoice.nodeId, invoice.features, totalAmount, expiry, blindedHops, customTlvs) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/ChannelIntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/ChannelIntegrationSpec.scala index 4976c2797f..a4ffde4a09 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/ChannelIntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/ChannelIntegrationSpec.scala @@ -143,7 +143,7 @@ abstract class ChannelIntegrationSpec extends IntegrationSpec { val paymentHash = Crypto.sha256(preimage) // A sends a payment to F val paymentSender = TestProbe() - val paymentReq = SendPaymentToNode(paymentSender.ref, 100000000 msat, Bolt11Invoice(Block.RegtestGenesisBlock.hash, None, paymentHash, nodes("F").nodeParams.privateKey, Left("test"), finalCltvExpiryDelta), maxAttempts = 1, routeParams = integrationTestRouteParams) + val paymentReq = SendPaymentToNode(paymentSender.ref, 100000000 msat, Bolt11Invoice(Block.RegtestGenesisBlock.hash, None, paymentHash, nodes("F").nodeParams.privateKey, Left("test"), finalCltvExpiryDelta), Nil, maxAttempts = 1, routeParams = integrationTestRouteParams) paymentSender.send(nodes("A").paymentInitiator, paymentReq) val paymentId = paymentSender.expectMsgType[UUID] // F gets the htlc @@ -381,7 +381,7 @@ abstract class ChannelIntegrationSpec extends IntegrationSpec { def send(amountMsat: MilliSatoshi, paymentHandler: ActorRef, paymentInitiator: ActorRef): UUID = { sender.send(paymentHandler, ReceiveStandardPayment(sender.ref.toTyped, Some(amountMsat), Left("1 coffee"))) val invoice = sender.expectMsgType[Bolt11Invoice] - val sendReq = SendPaymentToNode(sender.ref, amountMsat, invoice, maxAttempts = 1, routeParams = integrationTestRouteParams) + val sendReq = SendPaymentToNode(sender.ref, amountMsat, invoice, Nil, maxAttempts = 1, routeParams = integrationTestRouteParams) sender.send(paymentInitiator, sendReq) sender.expectMsgType[UUID] } @@ -703,7 +703,7 @@ abstract class AnchorChannelIntegrationSpec extends ChannelIntegrationSpec { val invoice = sender.expectMsgType[Bolt11Invoice] // then we make the actual payment - sender.send(nodes("C").paymentInitiator, SendPaymentToNode(sender.ref, amountMsat, invoice, maxAttempts = 1, routeParams = integrationTestRouteParams)) + sender.send(nodes("C").paymentInitiator, SendPaymentToNode(sender.ref, amountMsat, invoice, Nil, maxAttempts = 1, routeParams = integrationTestRouteParams)) val paymentId = sender.expectMsgType[UUID] val ps = sender.expectMsgType[PaymentSent](60 seconds) assert(ps.id == paymentId) 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 ac08bb4ec2..2cff4c6e57 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 @@ -167,7 +167,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { assert(invoice.paymentMetadata.nonEmpty) // then we make the actual payment - sender.send(nodes("A").paymentInitiator, SendPaymentToNode(sender.ref, amountMsat, invoice, routeParams = integrationTestRouteParams, maxAttempts = 1)) + sender.send(nodes("A").paymentInitiator, SendPaymentToNode(sender.ref, amountMsat, invoice, Nil, routeParams = integrationTestRouteParams, maxAttempts = 1)) val paymentId = sender.expectMsgType[UUID] val ps = sender.expectMsgType[PaymentSent] assert(ps.id == paymentId) @@ -193,7 +193,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { sender.send(nodes("D").paymentHandler, ReceiveStandardPayment(sender.ref, Some(amountMsat), Left("1 coffee"))) val invoice = sender.expectMsgType[Bolt11Invoice] // then we make the actual payment, do not randomize the route to make sure we route through node B - val sendReq = SendPaymentToNode(sender.ref, amountMsat, invoice, routeParams = integrationTestRouteParams, maxAttempts = 5) + val sendReq = SendPaymentToNode(sender.ref, amountMsat, invoice, Nil, routeParams = integrationTestRouteParams, maxAttempts = 5) sender.send(nodes("A").paymentInitiator, sendReq) // A will receive an error from B that include the updated channel update, then will retry the payment val paymentId = sender.expectMsgType[UUID] @@ -234,7 +234,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { sender.send(nodes("D").paymentHandler, ReceiveStandardPayment(sender.ref, Some(amountMsat), Left("1 coffee"))) val invoice = sender.expectMsgType[Bolt11Invoice] // then we make the payment (B-C has a smaller capacity than A-B and C-D) - val sendReq = SendPaymentToNode(sender.ref, amountMsat, invoice, routeParams = integrationTestRouteParams, maxAttempts = 5) + val sendReq = SendPaymentToNode(sender.ref, amountMsat, invoice, Nil, routeParams = integrationTestRouteParams, maxAttempts = 5) sender.send(nodes("A").paymentInitiator, sendReq) // A will first receive an error from C, then retry and route around C: A->B->E->C->D sender.expectMsgType[UUID] @@ -245,7 +245,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { val sender = TestProbe() val amount = 100000000 msat val unknownInvoice = Bolt11Invoice(Block.RegtestGenesisBlock.hash, Some(amount), randomBytes32(), nodes("D").nodeParams.privateKey, Left("test"), finalCltvExpiryDelta) - val invoice = SendPaymentToNode(sender.ref, amount, unknownInvoice, routeParams = integrationTestRouteParams, maxAttempts = 5) + val invoice = SendPaymentToNode(sender.ref, amount, unknownInvoice, Nil, routeParams = integrationTestRouteParams, maxAttempts = 5) sender.send(nodes("A").paymentInitiator, invoice) // A will receive an error from D and won't retry @@ -265,7 +265,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { val invoice = sender.expectMsgType[Bolt11Invoice] // A send payment of only 1 mBTC - val sendReq = SendPaymentToNode(sender.ref, 100000000 msat, invoice, routeParams = integrationTestRouteParams, maxAttempts = 5) + val sendReq = SendPaymentToNode(sender.ref, 100000000 msat, invoice, Nil, routeParams = integrationTestRouteParams, maxAttempts = 5) sender.send(nodes("A").paymentInitiator, sendReq) // A will first receive an IncorrectPaymentAmount error from D @@ -285,7 +285,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { val invoice = sender.expectMsgType[Bolt11Invoice] // A send payment of 6 mBTC - val sendReq = SendPaymentToNode(sender.ref, 600000000 msat, invoice, routeParams = integrationTestRouteParams, maxAttempts = 5) + val sendReq = SendPaymentToNode(sender.ref, 600000000 msat, invoice, Nil, routeParams = integrationTestRouteParams, maxAttempts = 5) sender.send(nodes("A").paymentInitiator, sendReq) // A will first receive an IncorrectPaymentAmount error from D @@ -305,7 +305,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { val invoice = sender.expectMsgType[Bolt11Invoice] // A send payment of 3 mBTC, more than asked but it should still be accepted - val sendReq = SendPaymentToNode(sender.ref, 300000000 msat, invoice, routeParams = integrationTestRouteParams, maxAttempts = 5) + val sendReq = SendPaymentToNode(sender.ref, 300000000 msat, invoice, Nil, routeParams = integrationTestRouteParams, maxAttempts = 5) sender.send(nodes("A").paymentInitiator, sendReq) sender.expectMsgType[UUID] } @@ -318,7 +318,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { sender.send(nodes("D").paymentHandler, ReceiveStandardPayment(sender.ref, Some(amountMsat), Left("1 payment"))) val invoice = sender.expectMsgType[Bolt11Invoice] - val sendReq = SendPaymentToNode(sender.ref, amountMsat, invoice, routeParams = integrationTestRouteParams, maxAttempts = 5) + val sendReq = SendPaymentToNode(sender.ref, amountMsat, invoice, Nil, routeParams = integrationTestRouteParams, maxAttempts = 5) sender.send(nodes("A").paymentInitiator, sendReq) sender.expectMsgType[UUID] sender.expectMsgType[PaymentSent] // the payment FSM will also reply to the sender after the payment is completed @@ -333,7 +333,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { val invoice = sender.expectMsgType[Bolt11Invoice] // the payment is requesting to use a capacity-optimized route which will select node G even though it's a bit more expensive - sender.send(nodes("A").paymentInitiator, SendPaymentToNode(sender.ref, amountMsat, invoice, maxAttempts = 1, routeParams = integrationTestRouteParams.copy(heuristics = Left(WeightRatios(0, 0, 0, 1, RelayFees(0 msat, 0)))))) + sender.send(nodes("A").paymentInitiator, SendPaymentToNode(sender.ref, amountMsat, invoice, Nil, maxAttempts = 1, routeParams = integrationTestRouteParams.copy(heuristics = Left(WeightRatios(0, 0, 0, 1, RelayFees(0 msat, 0)))))) sender.expectMsgType[UUID] val ps = sender.expectMsgType[PaymentSent] ps.parts.foreach(part => assert(part.route.getOrElse(Nil).exists(_.nodeId == nodes("G").nodeParams.nodeId))) @@ -347,7 +347,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { val invoice = sender.expectMsgType[Bolt11Invoice] assert(invoice.features.hasFeature(Features.BasicMultiPartPayment)) - sender.send(nodes("B").paymentInitiator, SendPaymentToNode(sender.ref, amount, invoice, maxAttempts = 5, routeParams = integrationTestRouteParams)) + sender.send(nodes("B").paymentInitiator, SendPaymentToNode(sender.ref, amount, invoice, Nil, maxAttempts = 5, routeParams = integrationTestRouteParams)) val paymentId = sender.expectMsgType[UUID] val paymentSent = sender.expectMsgType[PaymentSent](max = 30 seconds) assert(paymentSent.id == paymentId, paymentSent) @@ -390,7 +390,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { val canSend = sender.expectMsgType[Relayer.OutgoingChannels].channels.map(_.commitments.availableBalanceForSend).sum assert(canSend > amount) - sender.send(nodes("B").paymentInitiator, SendPaymentToNode(sender.ref, amount, invoice, maxAttempts = 1, routeParams = integrationTestRouteParams)) + sender.send(nodes("B").paymentInitiator, SendPaymentToNode(sender.ref, amount, invoice, Nil, maxAttempts = 1, routeParams = integrationTestRouteParams)) val paymentId = sender.expectMsgType[UUID] val paymentFailed = sender.expectMsgType[PaymentFailed](max = 30 seconds) assert(paymentFailed.id == paymentId, paymentFailed) @@ -413,7 +413,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { val invoice = sender.expectMsgType[Bolt11Invoice] assert(invoice.features.hasFeature(Features.BasicMultiPartPayment)) - sender.send(nodes("D").paymentInitiator, SendPaymentToNode(sender.ref, amount, invoice, maxAttempts = 3, routeParams = integrationTestRouteParams)) + sender.send(nodes("D").paymentInitiator, SendPaymentToNode(sender.ref, amount, invoice, Nil, maxAttempts = 3, routeParams = integrationTestRouteParams)) val paymentId = sender.expectMsgType[UUID] val paymentSent = sender.expectMsgType[PaymentSent](max = 30 seconds) assert(paymentSent.id == paymentId, paymentSent) @@ -445,7 +445,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { val canSend = sender.expectMsgType[Relayer.OutgoingChannels].channels.map(_.commitments.availableBalanceForSend).sum assert(canSend < amount) - sender.send(nodes("D").paymentInitiator, SendPaymentToNode(sender.ref, amount, invoice, maxAttempts = 1, routeParams = integrationTestRouteParams)) + sender.send(nodes("D").paymentInitiator, SendPaymentToNode(sender.ref, amount, invoice, Nil, maxAttempts = 1, routeParams = integrationTestRouteParams)) val paymentId = sender.expectMsgType[UUID] val paymentFailed = sender.expectMsgType[PaymentFailed](max = 30 seconds) assert(paymentFailed.id == paymentId, paymentFailed) @@ -473,7 +473,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { // The first attempt should fail, but the second one should succeed. val attempts = (1210000 msat, CltvExpiryDelta(42)) :: (1210100 msat, CltvExpiryDelta(288)) :: Nil - val payment = SendTrampolinePayment(sender.ref, amount, invoice, nodes("G").nodeParams.nodeId, attempts, routeParams = integrationTestRouteParams) + val payment = SendTrampolinePayment(amount, invoice, nodes("G").nodeParams.nodeId, attempts, routeParams = integrationTestRouteParams) sender.send(nodes("B").paymentInitiator, payment) val paymentId = sender.expectMsgType[UUID] val paymentSent = sender.expectMsgType[PaymentSent](max = 30 seconds) @@ -520,7 +520,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { // C -> B which would have a fee of 501000 if it could route the whole payment // C -> G -> B which would have a fee of 757061 if it was used to route the whole payment // The actual fee needed will be between these two values. - val payment = SendTrampolinePayment(sender.ref, amount, invoice, nodes("C").nodeParams.nodeId, Seq((750000 msat, CltvExpiryDelta(288))), routeParams = integrationTestRouteParams) + val payment = SendTrampolinePayment(amount, invoice, nodes("C").nodeParams.nodeId, Seq((750000 msat, CltvExpiryDelta(288))), routeParams = integrationTestRouteParams) sender.send(nodes("D").paymentInitiator, payment) val paymentId = sender.expectMsgType[UUID] val paymentSent = sender.expectMsgType[PaymentSent](max = 30 seconds) @@ -573,7 +573,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { assert(!invoice.features.hasFeature(Features.TrampolinePaymentPrototype)) assert(invoice.paymentMetadata.nonEmpty) - val payment = SendTrampolinePayment(sender.ref, amount, invoice, nodes("C").nodeParams.nodeId, Seq((1500000 msat, CltvExpiryDelta(432))), routeParams = integrationTestRouteParams) + val payment = SendTrampolinePayment(amount, invoice, nodes("C").nodeParams.nodeId, Seq((1500000 msat, CltvExpiryDelta(432))), routeParams = integrationTestRouteParams) sender.send(nodes("F").paymentInitiator, payment) val paymentId = sender.expectMsgType[UUID] val paymentSent = sender.expectMsgType[PaymentSent](max = 30 seconds) @@ -610,7 +610,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { // We put most of the capacity C <-> D on D's side. sender.send(nodes("D").paymentHandler, ReceiveStandardPayment(sender.ref, Some(8000000000L msat), Left("plz send everything"))) val pr1 = sender.expectMsgType[Bolt11Invoice] - sender.send(nodes("C").paymentInitiator, SendPaymentToNode(sender.ref, 8000000000L msat, pr1, maxAttempts = 3, routeParams = integrationTestRouteParams)) + sender.send(nodes("C").paymentInitiator, SendPaymentToNode(sender.ref, 8000000000L msat, pr1, Nil, maxAttempts = 3, routeParams = integrationTestRouteParams)) sender.expectMsgType[UUID] sender.expectMsgType[PaymentSent](max = 30 seconds) @@ -621,7 +621,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { assert(invoice.features.hasFeature(Features.BasicMultiPartPayment)) assert(invoice.features.hasFeature(Features.TrampolinePaymentPrototype)) - val payment = SendTrampolinePayment(sender.ref, amount, invoice, nodes("C").nodeParams.nodeId, Seq((250000 msat, CltvExpiryDelta(288))), routeParams = integrationTestRouteParams) + val payment = SendTrampolinePayment(amount, invoice, nodes("C").nodeParams.nodeId, Seq((250000 msat, CltvExpiryDelta(288))), routeParams = integrationTestRouteParams) sender.send(nodes("B").paymentInitiator, payment) val paymentId = sender.expectMsgType[UUID] val paymentFailed = sender.expectMsgType[PaymentFailed](max = 30 seconds) @@ -642,7 +642,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { assert(invoice.features.hasFeature(Features.BasicMultiPartPayment)) assert(invoice.features.hasFeature(Features.TrampolinePaymentPrototype)) - val payment = SendTrampolinePayment(sender.ref, amount, invoice, nodes("B").nodeParams.nodeId, Seq((450000 msat, CltvExpiryDelta(288))), routeParams = integrationTestRouteParams) + val payment = SendTrampolinePayment(amount, invoice, nodes("B").nodeParams.nodeId, Seq((450000 msat, CltvExpiryDelta(288))), routeParams = integrationTestRouteParams) sender.send(nodes("A").paymentInitiator, payment) val paymentId = sender.expectMsgType[UUID] val paymentFailed = sender.expectMsgType[PaymentFailed](max = 30 seconds) @@ -663,7 +663,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { assert(invoice.features.hasFeature(Features.BasicMultiPartPayment)) assert(invoice.features.hasFeature(Features.TrampolinePaymentPrototype)) - val payment = SendTrampolinePayment(sender.ref, amount, invoice, nodes("C").nodeParams.nodeId, Seq((500000 msat, CltvExpiryDelta(288))), routeParams = integrationTestRouteParams) + val payment = SendTrampolinePayment(amount, invoice, nodes("C").nodeParams.nodeId, Seq((500000 msat, CltvExpiryDelta(288))), routeParams = integrationTestRouteParams) sender.send(nodes("A").paymentInitiator, payment) val paymentId = sender.expectMsgType[UUID] val paymentSent = sender.expectMsgType[PaymentSent](max = 30 seconds) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PerformanceIntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PerformanceIntegrationSpec.scala index 2bcae8c93e..4329f251ae 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PerformanceIntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PerformanceIntegrationSpec.scala @@ -87,7 +87,7 @@ class PerformanceIntegrationSpec extends IntegrationSpec { sender.send(nodes("B").paymentHandler, ReceiveStandardPayment(sender.ref.toTyped, Some(amountMsat), Left("1 coffee"))) val pr = sender.expectMsgType[Bolt11Invoice] // then we make the actual payment - sender.send(nodes("A").paymentInitiator, PaymentInitiator.SendPaymentToNode(sender.ref, amountMsat, pr, routeParams = integrationTestRouteParams, maxAttempts = 1)) + sender.send(nodes("A").paymentInitiator, PaymentInitiator.SendPaymentToNode(sender.ref, amountMsat, pr, Nil, routeParams = integrationTestRouteParams, maxAttempts = 1)) val paymentId = sender.expectMsgType[UUID] sender.expectMsgType[PreimageReceived] val ps = sender.expectMsgType[PaymentSent] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala index dadd736db4..76c642cc1f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala @@ -103,7 +103,7 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat val peerFactory = Switchboard.SimplePeerFactory(nodeParams, wallet, channelFactory, pendingChannelsRateLimiter, register) val switchboard = system.actorOf(Switchboard.props(nodeParams, peerFactory), "switchboard") val paymentFactory = PaymentInitiator.SimplePaymentFactory(nodeParams, router, register) - val paymentInitiator = system.actorOf(PaymentInitiator.props(nodeParams, paymentFactory, router), "payment-initiator") + val paymentInitiator = system.actorOf(PaymentInitiator.props(nodeParams, paymentFactory), "payment-initiator") val channels = nodeParams.db.channels.listLocalChannels() val postman = system.spawn(Behaviors.supervise(Postman(nodeParams, switchboard, router.toTyped, register, offerManager)).onFailure(typed.SupervisorStrategy.restart), name = "postman") switchboard ! Switchboard.Init(channels) @@ -340,11 +340,11 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat } } - def sendPayment(node1: MinimalNodeFixture, amount: MilliSatoshi, invoice: Invoice)(implicit system: ActorSystem): Either[PaymentFailed, PaymentSent] = { + def sendPayment(node1: MinimalNodeFixture, amount: MilliSatoshi, invoice: Bolt11Invoice)(implicit system: ActorSystem): Either[PaymentFailed, PaymentSent] = { val sender = TestProbe("sender") val routeParams = node1.nodeParams.routerConf.pathFindingExperimentConf.experiments.values.head.getDefaultRouteParams - sender.send(node1.paymentInitiator, PaymentInitiator.SendPaymentToNode(sender.ref, amount, invoice, maxAttempts = 1, routeParams = routeParams, blockUntilComplete = true)) + sender.send(node1.paymentInitiator, PaymentInitiator.SendPaymentToNode(sender.ref, amount, invoice, Nil, maxAttempts = 1, routeParams = routeParams, blockUntilComplete = true)) sender.expectMsgType[PaymentEvent] match { case e: PaymentSent => Right(e) case e: PaymentFailed => Left(e) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala index 85a54b51f7..e45c3c0c75 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/payment/OfferPaymentSpec.scala @@ -128,7 +128,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val offer = Offer(None, "test", recipient.nodeId, Features.empty, recipient.nodeParams.chainHash) val handler = recipient.system.spawnAnonymous(offerHandler(amount, routes)) recipient.offerManager ! OfferManager.RegisterOffer(offer, recipient.nodeParams.privateKey, None, handler) - val offerPayment = payer.system.spawnAnonymous(OfferPayment(payer.nodeParams, payer.postman, payer.paymentInitiator)) + val offerPayment = payer.system.spawnAnonymous(OfferPayment(payer.nodeParams, payer.postman, payer.router, payer.paymentInitiator)) val sendPaymentConfig = OfferPayment.SendPaymentConfig(None, connectDirectly = false, maxAttempts = 1, payer.routeParams, blocking = true) offerPayment ! OfferPayment.PayOffer(sender.ref, offer, amount, 1, sendPaymentConfig) (offer, sender.expectMsgType[PaymentEvent]) @@ -148,7 +148,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val offer = Offer(None, "test", recipientKey.publicKey, Features.empty, recipient.nodeParams.chainHash, additionalTlvs = Set(OfferPaths(offerPaths))) val handler = recipient.system.spawnAnonymous(offerHandler(amount, routes)) recipient.offerManager ! OfferManager.RegisterOffer(offer, recipientKey, Some(pathId), handler) - val offerPayment = payer.system.spawnAnonymous(OfferPayment(payer.nodeParams, payer.postman, payer.paymentInitiator)) + val offerPayment = payer.system.spawnAnonymous(OfferPayment(payer.nodeParams, payer.postman, payer.router, payer.paymentInitiator)) val sendPaymentConfig = OfferPayment.SendPaymentConfig(None, connectDirectly = false, maxAttempts = 1, payer.routeParams, blocking = true) offerPayment ! OfferPayment.PayOffer(sender.ref, offer, amount, 1, sendPaymentConfig) (offer, sender.expectMsgType[PaymentEvent]) @@ -162,7 +162,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val offer = Offer(None, "test", recipient.nodeId, Features.empty, recipient.nodeParams.chainHash) val handler = recipient.system.spawnAnonymous(offerHandler(recipientAmount, routes)) recipient.offerManager ! OfferManager.RegisterOffer(offer, recipient.nodeParams.privateKey, None, handler) - val offerPayment = payer.system.spawnAnonymous(OfferPayment(payer.nodeParams, payer.postman, paymentInterceptor.ref)) + val offerPayment = payer.system.spawnAnonymous(OfferPayment(payer.nodeParams, payer.postman, payer.router, paymentInterceptor.ref)) val sendPaymentConfig = OfferPayment.SendPaymentConfig(None, connectDirectly = false, maxAttempts = 1, payer.routeParams, blocking = true) offerPayment ! OfferPayment.PayOffer(sender.ref, offer, recipientAmount, 1, sendPaymentConfig) // We intercept the payment and modify it to use a different amount. @@ -372,7 +372,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val receivingRoute = ReceivingRoute(Seq(bob.nodeId, carol.nodeId), maxFinalExpiryDelta) val handler = carol.system.spawnAnonymous(offerHandler(amount, Seq(receivingRoute))) carol.offerManager ! OfferManager.RegisterOffer(compactOffer, recipientKey, Some(pathId), handler) - val offerPayment = alice.system.spawnAnonymous(OfferPayment(alice.nodeParams, alice.postman, alice.paymentInitiator)) + val offerPayment = alice.system.spawnAnonymous(OfferPayment(alice.nodeParams, alice.postman, alice.router, alice.paymentInitiator)) val sendPaymentConfig = OfferPayment.SendPaymentConfig(None, connectDirectly = false, maxAttempts = 1, alice.routeParams, blocking = true) offerPayment ! OfferPayment.PayOffer(probe.ref, compactOffer, amount, 1, sendPaymentConfig) val payment = verifyPaymentSuccess(compactOffer, amount, probe.expectMsgType[PaymentEvent]) 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 27218a53c2..6509d5350c 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 @@ -34,10 +34,10 @@ import fr.acinq.eclair.payment.send.PaymentError.UnsupportedFeatures import fr.acinq.eclair.payment.send.PaymentInitiator._ import fr.acinq.eclair.payment.send._ import fr.acinq.eclair.router.Router._ -import fr.acinq.eclair.router.{BlindedRouteCreation, RouteNotFound, Router} -import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, Offer} +import fr.acinq.eclair.router.{BlindedRouteCreation, RouteNotFound} +import fr.acinq.eclair.wire.protocol.OfferTypes.{BlindedPath, InvoiceRequest, Offer} import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{Bolt11Feature, Bolt12Feature, CltvExpiry, CltvExpiryDelta, Feature, Features, MilliSatoshiLong, NodeParams, PaymentFinalExpiryConf, RealShortChannelId, TestConstants, TestKitBaseClass, TimestampSecond, UnknownFeature, randomBytes32, randomKey} +import fr.acinq.eclair.{Bolt11Feature, Bolt12Feature, CltvExpiry, CltvExpiryDelta, Feature, Features, MilliSatoshiLong, NodeParams, PaymentFinalExpiryConf, TestConstants, TestKitBaseClass, TimestampSecond, UnknownFeature, randomBytes32, randomKey} import org.scalatest.funsuite.FixtureAnyFunSuiteLike import org.scalatest.{Outcome, Tag} import scodec.bits.{ByteVector, HexStringSyntax} @@ -57,7 +57,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike val RandomizeFinalExpiry = "random_final_expiry" } - case class FixtureParam(nodeParams: NodeParams, initiator: TestActorRef[PaymentInitiator], payFsm: TestProbe, multiPartPayFsm: TestProbe, sender: TestProbe, eventListener: TestProbe, router: TestProbe) + case class FixtureParam(nodeParams: NodeParams, initiator: TestActorRef[PaymentInitiator], payFsm: TestProbe, multiPartPayFsm: TestProbe, sender: TestProbe, eventListener: TestProbe) val featuresWithoutMpp: Features[Bolt11Feature] = Features( VariableLengthOnion -> Mandatory, @@ -114,17 +114,16 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike val nodeParams = TestConstants.Alice.nodeParams.copy(features = features.unscoped(), paymentFinalExpiry = paymentFinalExpiry) val (sender, payFsm, multiPartPayFsm) = (TestProbe(), TestProbe(), TestProbe()) val eventListener = TestProbe() - val router = TestProbe() system.eventStream.subscribe(eventListener.ref, classOf[PaymentEvent]) - val initiator = TestActorRef(new PaymentInitiator(nodeParams, FakePaymentFactory(payFsm, multiPartPayFsm), router.ref)) - withFixture(test.toNoArgTest(FixtureParam(nodeParams, initiator, payFsm, multiPartPayFsm, sender, eventListener, router))) + val initiator = TestActorRef(new PaymentInitiator(nodeParams, FakePaymentFactory(payFsm, multiPartPayFsm))) + withFixture(test.toNoArgTest(FixtureParam(nodeParams, initiator, payFsm, multiPartPayFsm, sender, eventListener))) } test("forward payment with user custom tlv records") { f => import f._ val customRecords = Set(GenericTlv(500L, hex"01020304"), GenericTlv(501L, hex"d34db33f")) val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, None, paymentHash, priv_c.privateKey, Left("test"), Channel.MIN_CLTV_EXPIRY_DELTA) - val req = SendPaymentToNode(sender.ref, finalAmount, invoice, 1, userCustomTlvs = customRecords, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendPaymentToNode(sender.ref, finalAmount, invoice, Nil, 1, userCustomTlvs = customRecords, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) sender.expectMsgType[UUID] payFsm.expectMsgType[SendPaymentConfig] @@ -167,7 +166,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike Bolt11Invoice.InvoiceFeatures(invoiceFeatures) ) val invoice = Bolt11Invoice("lnbc", Some(finalAmount), TimestampSecond.now(), randomKey().publicKey, taggedFields, ByteVector.empty) - val req = SendPaymentToNode(sender.ref, finalAmount + 100.msat, invoice, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendPaymentToNode(sender.ref, finalAmount + 100.msat, invoice, Nil, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) val id = sender.expectMsgType[UUID] val fail = sender.expectMsgType[PaymentFailed] @@ -182,7 +181,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike val finalExpiryDelta = CltvExpiryDelta(36) val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some invoice"), finalExpiryDelta) val route = PredefinedNodeRoute(finalAmount, Seq(a, b, c)) - val request = SendPaymentToRoute(sender.ref, finalAmount, invoice, route, None, None, None) + val request = SendPaymentToRoute(finalAmount, invoice, Nil, route, None, None, None) sender.send(initiator, request) val payment = sender.expectMsgType[SendPaymentToRouteResponse] payFsm.expectMsg(SendPaymentConfig(payment.paymentId, payment.parentId, None, paymentHash, c, Upstream.Local(payment.paymentId), Some(invoice), None, storeInDb = true, publishEvent = true, recordPathFindingMetrics = false)) @@ -206,7 +205,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike import f._ val finalExpiryDelta = CltvExpiryDelta(24) val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some MPP invoice"), finalExpiryDelta, features = featuresWithoutRouteBlinding) - val req = SendPaymentToNode(sender.ref, finalAmount, invoice, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendPaymentToNode(sender.ref, finalAmount, invoice, Nil, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) assert(req.finalExpiry(nodeParams) == (finalExpiryDelta + 1).toCltvExpiry(nodeParams.currentBlockHeight)) sender.send(initiator, req) val id = sender.expectMsgType[UUID] @@ -230,7 +229,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike test("forward multi-part payment") { f => import f._ val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some invoice"), CltvExpiryDelta(18), features = featuresWithoutRouteBlinding) - val req = SendPaymentToNode(sender.ref, finalAmount + 100.msat, invoice, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendPaymentToNode(sender.ref, finalAmount + 100.msat, invoice, Nil, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) val id = sender.expectMsgType[UUID] multiPartPayFsm.expectMsg(SendPaymentConfig(id, id, None, paymentHash, c, Upstream.Local(id), Some(invoice), None, storeInDb = true, publishEvent = true, recordPathFindingMetrics = true)) @@ -254,7 +253,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike import f._ val invoiceFinalExpiryDelta = CltvExpiryDelta(6) val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some invoice"), invoiceFinalExpiryDelta, features = featuresWithoutRouteBlinding) - val req = SendPaymentToNode(sender.ref, finalAmount, invoice, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendPaymentToNode(sender.ref, finalAmount, invoice, Nil, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) val id = sender.expectMsgType[UUID] multiPartPayFsm.expectMsg(SendPaymentConfig(id, id, None, paymentHash, c, Upstream.Local(id), Some(invoice), None, storeInDb = true, publishEvent = true, recordPathFindingMetrics = true)) @@ -268,7 +267,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike import f._ val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some invoice"), CltvExpiryDelta(18), features = featuresWithoutRouteBlinding) val route = PredefinedChannelRoute(finalAmount / 2, c, Seq(channelUpdate_ab.shortChannelId, channelUpdate_bc.shortChannelId)) - val req = SendPaymentToRoute(sender.ref, finalAmount, invoice, route, None, None, None) + val req = SendPaymentToRoute(finalAmount, invoice, Nil, route, None, None, None) sender.send(initiator, req) val payment = sender.expectMsgType[SendPaymentToRouteResponse] payFsm.expectMsg(SendPaymentConfig(payment.paymentId, payment.parentId, None, paymentHash, c, Upstream.Local(payment.paymentId), Some(invoice), None, storeInDb = true, publishEvent = true, recordPathFindingMetrics = false)) @@ -306,7 +305,8 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike import f._ val payerKey = randomKey() val invoice = createBolt12Invoice(Features.empty, payerKey) - val req = SendPaymentToNode(sender.ref, finalAmount, invoice, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams, payerKey_opt = Some(payerKey)) + val resolvedPaths = invoice.blindedPaths.map(path => ResolvedPaymentBlindedRoute(path.route.asInstanceOf[BlindedPath].route, path.paymentInfo)) + val req = SendPaymentToNode(sender.ref, finalAmount, invoice, resolvedPaths, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams, payerKey_opt = Some(payerKey)) sender.send(initiator, req) val id = sender.expectMsgType[UUID] payFsm.expectMsg(SendPaymentConfig(id, id, None, paymentHash, invoice.nodeId, Upstream.Local(id), Some(invoice), Some(payerKey), storeInDb = true, publishEvent = true, recordPathFindingMetrics = true)) @@ -336,7 +336,8 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike import f._ val payerKey = randomKey() val invoice = createBolt12Invoice(Features(BasicMultiPartPayment -> Optional), payerKey) - val req = SendPaymentToNode(sender.ref, finalAmount, invoice, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams, payerKey_opt = Some(payerKey)) + val resolvedPaths = invoice.blindedPaths.map(path => ResolvedPaymentBlindedRoute(path.route.asInstanceOf[BlindedPath].route, path.paymentInfo)) + val req = SendPaymentToNode(sender.ref, finalAmount, invoice, resolvedPaths, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams, payerKey_opt = Some(payerKey)) sender.send(initiator, req) val id = sender.expectMsgType[UUID] multiPartPayFsm.expectMsg(SendPaymentConfig(id, id, None, paymentHash, invoice.nodeId, Upstream.Local(id), Some(invoice), Some(payerKey), storeInDb = true, publishEvent = true, recordPathFindingMetrics = true)) @@ -364,7 +365,8 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike test("reject blinded payment when route blinding deactivated", Tag(Tags.DisableRouteBlinding)) { f => import f._ val invoice = createBolt12Invoice(Features(BasicMultiPartPayment -> Optional), randomKey()) - val req = SendPaymentToNode(sender.ref, finalAmount, invoice, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val resolvedPaths = invoice.blindedPaths.map(path => ResolvedPaymentBlindedRoute(path.route.asInstanceOf[BlindedPath].route, path.paymentInfo)) + val req = SendPaymentToNode(sender.ref, finalAmount, invoice, resolvedPaths, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) val id = sender.expectMsgType[UUID] val fail = sender.expectMsgType[PaymentFailed] @@ -372,82 +374,12 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike assert(fail.failures == LocalFailure(finalAmount, Nil, UnsupportedFeatures(invoice.features)) :: Nil) } - test("forward blinded payment to compact route") { f => - import f._ - val payerKey = randomKey() - val offer = Offer(None, "Bolt12 is compact", e, Features.empty, Block.RegtestGenesisBlock.hash) - val invoiceRequest = InvoiceRequest(offer, finalAmount, 1, Features.empty, randomKey(), Block.RegtestGenesisBlock.hash) - val blindedRoute = BlindedRouteCreation.createBlindedRouteWithoutHops(e, hex"2a2a2a2a", 1 msat, CltvExpiry(500_000)).route - val compactRoute = OfferTypes.CompactBlindedPath(OfferTypes.ShortChannelIdDir(isNode1 = false, RealShortChannelId(987654)), blindedRoute.blindingKey, blindedRoute.blindedNodes) - val paymentInfo = OfferTypes.PaymentInfo(1_000 msat, 0, CltvExpiryDelta(24), 0 msat, finalAmount, Features.empty) - val invoice = Bolt12Invoice(invoiceRequest, paymentPreimage, priv_e.privateKey, 300 seconds, Features.empty, Seq(PaymentBlindedRoute(compactRoute, paymentInfo))) - val req = SendPaymentToNode(sender.ref, finalAmount, invoice, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams, payerKey_opt = Some(payerKey)) - sender.send(initiator, req) - val id = sender.expectMsgType[UUID] - val getNodeId = router.expectMsgType[Router.GetNodeId] - assert(!getNodeId.isNode1) - assert(getNodeId.shortChannelId == RealShortChannelId(987654)) - getNodeId.replyTo ! Some(a) - payFsm.expectMsg(SendPaymentConfig(id, id, None, paymentHash, invoice.nodeId, Upstream.Local(id), Some(invoice), Some(payerKey), storeInDb = true, publishEvent = true, recordPathFindingMetrics = true)) - val payment = payFsm.expectMsgType[PaymentLifecycle.SendPaymentToNode] - assert(payment.amount == finalAmount) - assert(payment.recipient.nodeId == invoice.nodeId) - assert(payment.recipient.totalAmount == finalAmount) - assert(payment.recipient.extraEdges.length == 1) - val extraEdge = payment.recipient.extraEdges.head - assert(extraEdge.sourceNodeId == a) - assert(payment.recipient.expiry == req.finalExpiry(nodeParams)) - assert(payment.recipient.isInstanceOf[BlindedRecipient]) - - sender.send(initiator, GetPayment(PaymentIdentifier.PaymentUUID(id))) - val pendingById = sender.expectMsgType[PaymentIsPending] - assert(pendingById.paymentId == id) - assert(pendingById.paymentHash == invoice.paymentHash) - assert(pendingById.pending.asInstanceOf[PendingPaymentToNode].sender == sender.ref) - val r = pendingById.pending.asInstanceOf[PendingPaymentToNode].request - assert(r.copy(invoice = req.invoice, paymentConfig_opt = None) == req) - assert(r.paymentConfig_opt.get.invoice.contains(req.invoice)) - - sender.send(initiator, GetPayment(PaymentIdentifier.PaymentHash(invoice.paymentHash))) - sender.expectMsg(pendingById) - - val pf = PaymentFailed(id, invoice.paymentHash, Nil) - payFsm.send(initiator, pf) - sender.expectMsg(pf) - eventListener.expectNoMessage(100 millis) - - sender.send(initiator, GetPayment(PaymentIdentifier.PaymentUUID(id))) - sender.expectMsg(NoPendingPayment(PaymentIdentifier.PaymentUUID(id))) - } - - test("reject payment to unknown compact route") { f => - import f._ - val payerKey = randomKey() - val offer = Offer(None, "Bolt12 is compact", e, Features.empty, Block.RegtestGenesisBlock.hash) - val invoiceRequest = InvoiceRequest(offer, finalAmount, 1, Features.empty, randomKey(), Block.RegtestGenesisBlock.hash) - val blindedRoute = BlindedRouteCreation.createBlindedRouteWithoutHops(e, hex"2a2a2a2a", 1 msat, CltvExpiry(500_000)).route - val compactRoute = OfferTypes.CompactBlindedPath(OfferTypes.ShortChannelIdDir(isNode1 = true, RealShortChannelId(654321)), blindedRoute.blindingKey, blindedRoute.blindedNodes) - val paymentInfo = OfferTypes.PaymentInfo(1_000 msat, 0, CltvExpiryDelta(24), 0 msat, finalAmount, Features.empty) - val invoice = Bolt12Invoice(invoiceRequest, paymentPreimage, priv_e.privateKey, 300 seconds, Features.empty, Seq(PaymentBlindedRoute(compactRoute, paymentInfo))) - val req = SendPaymentToNode(sender.ref, finalAmount, invoice, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams, payerKey_opt = Some(payerKey)) - sender.send(initiator, req) - val id = sender.expectMsgType[UUID] - val getNodeId = router.expectMsgType[Router.GetNodeId] - assert(getNodeId.isNode1) - assert(getNodeId.shortChannelId == RealShortChannelId(654321)) - getNodeId.replyTo ! None - - val fail = sender.expectMsgType[PaymentFailed] - assert(fail.id == id) - assert(fail.failures == LocalFailure(finalAmount, Nil, RouteNotFound) :: Nil) - } - test("forward trampoline payment") { f => import f._ val ignoredRoutingHints = List(List(ExtraHop(b, channelUpdate_bc.shortChannelId, feeBase = 10 msat, feeProportionalMillionths = 1, cltvExpiryDelta = CltvExpiryDelta(12)))) val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some phoenix invoice"), CltvExpiryDelta(9), features = featuresWithTrampoline, extraHops = ignoredRoutingHints) val trampolineFees = 21_000 msat - val req = SendTrampolinePayment(sender.ref, finalAmount, invoice, b, Seq((trampolineFees, CltvExpiryDelta(12))), nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendTrampolinePayment(finalAmount, invoice, b, Seq((trampolineFees, CltvExpiryDelta(12))), nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) val id = sender.expectMsgType[UUID] multiPartPayFsm.expectMsgType[SendPaymentConfig] @@ -474,7 +406,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike import f._ val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some wallet invoice"), CltvExpiryDelta(9)) val trampolineFees = 21_000 msat - val req = SendTrampolinePayment(sender.ref, finalAmount, invoice, b, Seq((trampolineFees, CltvExpiryDelta(12))), routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendTrampolinePayment(finalAmount, invoice, b, Seq((trampolineFees, CltvExpiryDelta(12))), routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) sender.expectMsgType[UUID] multiPartPayFsm.expectMsgType[SendPaymentConfig] @@ -498,7 +430,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike val routingHints = List(List(Bolt11Invoice.ExtraHop(b, channelUpdate_bc.shortChannelId, 10 msat, 100, CltvExpiryDelta(144)))) val invoice = Bolt11Invoice(Block.RegtestGenesisBlock.hash, None, paymentHash, priv_a.privateKey, Left("#abittooreckless"), CltvExpiryDelta(18), None, None, routingHints, features = featuresWithoutRouteBlinding) val trampolineFees = 21_000 msat - val req = SendTrampolinePayment(sender.ref, finalAmount, invoice, b, Seq((trampolineFees, CltvExpiryDelta(12))), routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendTrampolinePayment(finalAmount, invoice, b, Seq((trampolineFees, CltvExpiryDelta(12))), routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) val id = sender.expectMsgType[UUID] val fail = sender.expectMsgType[PaymentFailed] @@ -513,7 +445,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike import f._ val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some phoenix invoice"), CltvExpiryDelta(18), features = featuresWithTrampoline) val trampolineAttempts = (21_000 msat, CltvExpiryDelta(12)) :: (25_000 msat, CltvExpiryDelta(24)) :: Nil - val req = SendTrampolinePayment(sender.ref, finalAmount, invoice, b, trampolineAttempts, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendTrampolinePayment(finalAmount, invoice, b, trampolineAttempts, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) val id = sender.expectMsgType[UUID] val cfg = multiPartPayFsm.expectMsgType[SendPaymentConfig] @@ -550,7 +482,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike import f._ val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some phoenix invoice"), CltvExpiryDelta(18), features = featuresWithTrampoline) val trampolineAttempts = (21_000 msat, CltvExpiryDelta(12)) :: (25_000 msat, CltvExpiryDelta(24)) :: Nil - val req = SendTrampolinePayment(sender.ref, finalAmount, invoice, b, trampolineAttempts, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendTrampolinePayment(finalAmount, invoice, b, trampolineAttempts, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) sender.expectMsgType[UUID] val cfg = multiPartPayFsm.expectMsgType[SendPaymentConfig] @@ -581,7 +513,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike import f._ val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some phoenix invoice"), CltvExpiryDelta(18), features = featuresWithTrampoline) val trampolineAttempts = (21_000 msat, CltvExpiryDelta(12)) :: (25_000 msat, CltvExpiryDelta(24)) :: Nil - val req = SendTrampolinePayment(sender.ref, finalAmount, invoice, b, trampolineAttempts, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendTrampolinePayment(finalAmount, invoice, b, trampolineAttempts, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) sender.expectMsgType[UUID] @@ -609,7 +541,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike val invoice = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(finalAmount), paymentHash, priv_c.privateKey, Left("Some invoice"), CltvExpiryDelta(18)) val trampolineAttempt = TrampolineAttempt(randomBytes32(), 100 msat, CltvExpiryDelta(144)) val route = PredefinedNodeRoute(finalAmount + trampolineAttempt.fees, Seq(a, b)) - val req = SendPaymentToRoute(sender.ref, finalAmount, invoice, route, None, None, Some(trampolineAttempt)) + val req = SendPaymentToRoute(finalAmount, invoice, Nil, route, None, None, Some(trampolineAttempt)) sender.send(initiator, req) val payment = sender.expectMsgType[SendPaymentToRouteResponse] assert(payment.trampolineSecret.contains(trampolineAttempt.paymentSecret)) 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 bdd6edca8c..5893a57361 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 @@ -32,7 +32,7 @@ import fr.acinq.eclair.router.BlindedRouteCreation import fr.acinq.eclair.router.Router.{NodeHop, Route} import fr.acinq.eclair.transactions.Transactions import fr.acinq.eclair.transactions.Transactions.InputInfo -import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, Offer, PaymentInfo} +import fr.acinq.eclair.wire.protocol.OfferTypes.{BlindedPath, InvoiceRequest, Offer, PaymentInfo} import fr.acinq.eclair.wire.protocol.OnionPaymentPayloadTlv.{AmountToForward, OutgoingCltv, PaymentData} import fr.acinq.eclair.wire.protocol.PaymentOnion.{FinalPayload, IntermediatePayload} import fr.acinq.eclair.wire.protocol._ @@ -220,7 +220,8 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { val blindedRoute = OfferTypes.BlindedPath(BlindedRouteCreation.createBlindedRouteWithoutHops(c, hex"deadbeef", 1 msat, CltvExpiry(500_000)).route) val paymentInfo = PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 1 msat, amount_bc, Features.empty) val invoice = Bolt12Invoice(invoiceRequest, paymentPreimage, recipientKey, 300 seconds, features, Seq(PaymentBlindedRoute(blindedRoute, paymentInfo))) - val Some(recipient) = BlindedRecipient(invoice, amount_bc, expiry_bc, Set.empty) + val resolvedPaths = invoice.blindedPaths.map(path => ResolvedPaymentBlindedRoute(path.route.asInstanceOf[BlindedPath].route, path.paymentInfo)) + val recipient = BlindedRecipient(invoice, resolvedPaths, amount_bc, expiry_bc, Set.empty) val hops = Seq(channelHopFromUpdate(a, b, channelUpdate_ab), channelHopFromUpdate(b, c, channelUpdate_bc)) val Right(payment) = buildOutgoingPayment(ActorRef.noSender, priv_a.privateKey, Upstream.Local(UUID.randomUUID()), paymentHash, Route(amount_bc, hops, Some(recipient.blindedHops.head)), recipient) assert(payment.outgoingChannel == channelUpdate_ab.shortChannelId) @@ -471,7 +472,8 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { val blindedRoute = OfferTypes.BlindedPath(tmpBlindedRoute.copy(blindedNodes = tmpBlindedRoute.blindedNodes.reverse)) val paymentInfo = OfferTypes.PaymentInfo(fee_b, 0, channelUpdate_bc.cltvExpiryDelta, 0 msat, amount_bc, Features.empty) val invoice = Bolt12Invoice(invoiceRequest, paymentPreimage, priv_c.privateKey, 300 seconds, features, Seq(PaymentBlindedRoute(blindedRoute, paymentInfo))) - val Some(recipient) = BlindedRecipient(invoice, amount_bc, expiry_bc, Set.empty) + val resolvedPaths = invoice.blindedPaths.map(path => ResolvedPaymentBlindedRoute(path.route.asInstanceOf[BlindedPath].route, path.paymentInfo)) + val recipient = BlindedRecipient(invoice, resolvedPaths, amount_bc, expiry_bc, Set.empty) val route = Route(amount_bc, Seq(channelHopFromUpdate(a, b, channelUpdate_ab)), Some(recipient.blindedHops.head)) (route, recipient) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/send/OfferPaymentSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/send/OfferPaymentSpec.scala index 9dfd689c33..100474c164 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/send/OfferPaymentSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/send/OfferPaymentSpec.scala @@ -19,21 +19,20 @@ package fr.acinq.eclair.payment.send import akka.actor.ActorSystem import akka.actor.testkit.typed.scaladsl.{ScalaTestWithActorTestKit, TestProbe => TypedProbe} import akka.actor.typed.ActorRef -import akka.actor.typed.scaladsl.adapter._ import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory import fr.acinq.eclair.crypto.Sphinx.RouteBlinding -import fr.acinq.eclair.message.OnionMessages.Recipient import fr.acinq.eclair.message.OnionMessages.RoutingStrategy.FindRoute import fr.acinq.eclair.message.Postman import fr.acinq.eclair.payment.send.OfferPayment._ import fr.acinq.eclair.payment.send.PaymentInitiator.SendPaymentToNode import fr.acinq.eclair.payment.{Bolt12Invoice, PaymentBlindedRoute} +import fr.acinq.eclair.router.Router import fr.acinq.eclair.router.Router.RouteParams import fr.acinq.eclair.wire.protocol.MessageOnion.InvoicePayload -import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, Offer, PaymentInfo} +import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, Offer, PaymentInfo, ShortChannelIdDir} import fr.acinq.eclair.wire.protocol.{OfferTypes, OnionMessagePayloadTlv, TlvStream} -import fr.acinq.eclair.{CltvExpiryDelta, Features, MilliSatoshiLong, NodeParams, TestConstants, randomBytes32, randomKey} +import fr.acinq.eclair.{CltvExpiryDelta, Features, MilliSatoshiLong, NodeParams, RealShortChannelId, TestConstants, randomBytes, randomBytes32, randomKey} import org.scalatest.Outcome import org.scalatest.funsuite.FixtureAnyFunSuiteLike import scodec.bits.HexStringSyntax @@ -42,16 +41,17 @@ import scala.concurrent.duration.DurationInt class OfferPaymentSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("application")) with FixtureAnyFunSuiteLike { - case class FixtureParam(offerPayment: ActorRef[Command], nodeParams: NodeParams, postman: TypedProbe[Postman.Command], paymentInitiator: TestProbe, routeParams: RouteParams) + case class FixtureParam(offerPayment: ActorRef[Command], nodeParams: NodeParams, postman: TypedProbe[Postman.Command], router: TestProbe, paymentInitiator: TestProbe, routeParams: RouteParams) override def withFixture(test: OneArgTest): Outcome = { val nodeParams = TestConstants.Alice.nodeParams val postman = TypedProbe[Postman.Command]("postman") - val paymentInitiator = TestProbe("paymentInitiator")(system.toClassic) - val offerPayment = testKit.spawn(OfferPayment(nodeParams, postman.ref, paymentInitiator.ref)) + val router = TestProbe("router") + val paymentInitiator = TestProbe("paymentInitiator") + val offerPayment = testKit.spawn(OfferPayment(nodeParams, postman.ref, router.ref, paymentInitiator.ref)) val routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams try { - withFixture(test.toNoArgTest(FixtureParam(offerPayment, nodeParams, postman, paymentInitiator, routeParams))) + withFixture(test.toNoArgTest(FixtureParam(offerPayment, nodeParams, postman, router, paymentInitiator, routeParams))) } finally { testKit.stop(offerPayment) } @@ -130,4 +130,54 @@ class OfferPaymentSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app TypedProbe().expectTerminated(offerPayment) } + + test("resolve compact paths") { f => + import f._ + + val probe = TestProbe() + val merchantKey = randomKey() + + val offer = Offer(None, "offer", merchantKey.publicKey, Features.empty, nodeParams.chainHash) + offerPayment ! PayOffer(probe.ref, offer, 40_000_000 msat, 1, SendPaymentConfig(None, connectDirectly = false, 1, routeParams, blocking = false)) + val Postman.SendMessage(OfferTypes.RecipientNodeId(recipientId), FindRoute, message, expectsReply, replyTo) = postman.expectMessageType[Postman.SendMessage] + assert(recipientId == merchantKey.publicKey) + assert(message.get[OnionMessagePayloadTlv.InvoiceRequest].nonEmpty) + assert(expectsReply) + val Right(invoiceRequest) = InvoiceRequest.validate(message.get[OnionMessagePayloadTlv.InvoiceRequest].get.tlvs) + + val preimage = randomBytes32() + val blindedRoutes = Seq.fill(6)(RouteBlinding.create(randomKey(), Seq.fill(3)(randomKey().publicKey), Seq.fill(3)(randomBytes(10))).route) + val paymentRoutes = Seq( + PaymentBlindedRoute(OfferTypes.BlindedPath(blindedRoutes(0)), PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 1_000_000_000 msat, Features.empty)), + PaymentBlindedRoute(OfferTypes.CompactBlindedPath(ShortChannelIdDir(isNode1 = true, RealShortChannelId(11111)), blindedRoutes(1).blindingKey, blindedRoutes(1).blindedNodes), PaymentInfo(1 msat, 11, CltvExpiryDelta(111), 0 msat, 1_000_000_000 msat, Features.empty)), + PaymentBlindedRoute(OfferTypes.BlindedPath(blindedRoutes(2)), PaymentInfo(2 msat, 22, CltvExpiryDelta(222), 0 msat, 1_000_000_000 msat, Features.empty)), + PaymentBlindedRoute(OfferTypes.CompactBlindedPath(ShortChannelIdDir(isNode1 = false, RealShortChannelId(33333)), blindedRoutes(3).blindingKey, blindedRoutes(3).blindedNodes), PaymentInfo(3 msat, 33, CltvExpiryDelta(333), 0 msat, 1_000_000_000 msat, Features.empty)), + PaymentBlindedRoute(OfferTypes.CompactBlindedPath(ShortChannelIdDir(isNode1 = false, RealShortChannelId(44444)), blindedRoutes(4).blindingKey, blindedRoutes(4).blindedNodes), PaymentInfo(4 msat, 44, CltvExpiryDelta(444), 0 msat, 1_000_000_000 msat, Features.empty)), + PaymentBlindedRoute(OfferTypes.BlindedPath(blindedRoutes(5)), PaymentInfo(5 msat, 55, CltvExpiryDelta(555), 0 msat, 1_000_000_000 msat, Features.empty)), + ) + val invoice = Bolt12Invoice(invoiceRequest, preimage, merchantKey, 1 minute, Features.empty, paymentRoutes) + replyTo ! Postman.Response(InvoicePayload(TlvStream(OnionMessagePayloadTlv.Invoice(invoice.records)), TlvStream.empty)) + + val getNode1 = router.expectMsgType[Router.GetNodeId] + assert(getNode1.isNode1) + assert(getNode1.shortChannelId == RealShortChannelId(11111)) + getNode1.replyTo ! Some(blindedRoutes(1).introductionNodeId) + + val getNode3 = router.expectMsgType[Router.GetNodeId] + assert(!getNode3.isNode1) + assert(getNode3.shortChannelId == RealShortChannelId(33333)) + getNode3.replyTo ! None + + val getNode4 = router.expectMsgType[Router.GetNodeId] + assert(!getNode4.isNode1) + assert(getNode4.shortChannelId == RealShortChannelId(44444)) + getNode4.replyTo ! Some(blindedRoutes(4).introductionNodeId) + + val send = paymentInitiator.expectMsgType[SendPaymentToNode] + assert(send.invoice == invoice) + assert(send.resolvedPaths.map(_.route) == Seq(blindedRoutes(0), blindedRoutes(1), blindedRoutes(2), blindedRoutes(4), blindedRoutes(5))) + assert(send.resolvedPaths.map(_.paymentInfo.feeBase) == Seq(0 msat, 1 msat, 2 msat, 4 msat, 5 msat)) + + TypedProbe().expectTerminated(offerPayment) + } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala index 081630b6e6..d69212665b 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala @@ -31,12 +31,12 @@ import fr.acinq.eclair.crypto.TransportHandler import fr.acinq.eclair.crypto.keymanager.{LocalChannelKeyManager, LocalNodeKeyManager} import fr.acinq.eclair.io.Peer.PeerRoutingMessage import fr.acinq.eclair.payment.send.BlindedRecipient -import fr.acinq.eclair.payment.{Bolt12Invoice, PaymentBlindedRoute} +import fr.acinq.eclair.payment.{Bolt12Invoice, PaymentBlindedRoute, ResolvedPaymentBlindedRoute} import fr.acinq.eclair.router.Announcements._ import fr.acinq.eclair.router.BaseRouterSpec.channelAnnouncement import fr.acinq.eclair.router.Router._ import fr.acinq.eclair.transactions.Scripts -import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, Offer} +import fr.acinq.eclair.wire.protocol.OfferTypes.{BlindedPath, InvoiceRequest, Offer} import fr.acinq.eclair.wire.protocol._ import org.scalatest.Outcome import org.scalatest.funsuite.FixtureAnyFunSuiteLike @@ -275,7 +275,8 @@ object BaseRouterSpec { PaymentBlindedRoute(blindedRoute, paymentInfo) }) val invoice = Bolt12Invoice(invoiceRequest, preimage, recipientKey, 300 seconds, features, blindedRoutes) - val Some(recipient) = BlindedRecipient(invoice, amount, expiry, Set.empty) + val resolvedPaths = invoice.blindedPaths.map(path => ResolvedPaymentBlindedRoute(path.route.asInstanceOf[BlindedPath].route, path.paymentInfo)) + val recipient = BlindedRecipient(invoice, resolvedPaths, amount, expiry, Set.empty) (invoice, recipient) }