From ccdde635692d8e70991ba676fe6a304b778db3fc Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Mon, 11 Sep 2023 11:08:00 +0200 Subject: [PATCH 01/12] Add support for sciddir_or_pubkey Offers (https://github.com/lightning/bolts/pull/798) allow nodes to be identified using either the public key or a pair channel id and direction. The goal is to save bytes as channel id and direction only use 9 bytes instead of 33 for a public key. --- .../main/scala/fr/acinq/eclair/Eclair.scala | 8 +-- .../acinq/eclair/json/JsonSerializers.scala | 12 +++- .../acinq/eclair/message/OnionMessages.scala | 8 ++- .../fr/acinq/eclair/message/Postman.scala | 72 ++++++++++++------- .../acinq/eclair/payment/Bolt12Invoice.scala | 2 +- .../eclair/payment/offer/OfferManager.scala | 11 +-- .../payment/receive/MultiPartHandler.scala | 2 +- .../eclair/payment/send/OfferPayment.scala | 10 +-- .../scala/fr/acinq/eclair/router/Router.scala | 12 +++- .../eclair/wire/protocol/OfferCodecs.scala | 32 +++++++-- .../eclair/wire/protocol/OfferTypes.scala | 19 +++-- 11 files changed, 126 insertions(+), 62 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 82bbe64d9d..4e15cfd42e 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala @@ -663,15 +663,15 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging { userCustomContent: ByteVector)(implicit timeout: Timeout): Future[SendOnionMessageResponse] = { TlvCodecs.tlvStream(MessageOnionCodecs.onionTlvCodec).decode(userCustomContent.bits) match { case Attempt.Successful(DecodeResult(userTlvs, _)) => - val destination = recipient match { - case Left(key) => OnionMessages.Recipient(key, None) - case Right(route) => OnionMessages.BlindedPath(route) + val contactInfo = recipient match { + case Left(key) => OfferTypes.RecipientNodeId(key) + case Right(route) => OfferTypes.BlindedPath(route) } val routingStrategy = intermediateNodes_opt match { case Some(intermediateNodes) => OnionMessages.RoutingStrategy.UseRoute(intermediateNodes) case None => OnionMessages.RoutingStrategy.FindRoute } - appKit.postman.ask(ref => Postman.SendMessage(destination, routingStrategy, userTlvs, expectsReply, ref)).map { + appKit.postman.ask(ref => Postman.SendMessage(contactInfo, routingStrategy, userTlvs, expectsReply, ref)).map { case Postman.Response(payload) => SendOnionMessageResponse(sent = true, None, Some(SendOnionMessageResponsePayload(payload.records))) case Postman.NoReply => SendOnionMessageResponse(sent = true, Some("No response"), None) case Postman.MessageSent => SendOnionMessageResponse(sent = true, None, None) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala index dccf8fbaf6..c093cdd575 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala @@ -440,9 +440,17 @@ object InvoiceSerializer extends MinimalSerializer({ UnknownFeatureSerializer )), JField("blindedPaths", JArray(p.blindedPaths.map(path => { + val introductionNode = path.route match { + case OfferTypes.BlindedPath(route) => route.introductionNodeId.toString + case OfferTypes.CompactBlindedPath(shortIdDir, _, _) => s"${if (shortIdDir.isNode1) '0' else '1'}x${shortIdDir.scid.toString}" + } + val blindedNodes = path.route match { + case OfferTypes.BlindedPath(route) => route.blindedNodes + case OfferTypes.CompactBlindedPath(_, _, nodes) => nodes + } JObject(List( - JField("introductionNodeId", JString(path.route.introductionNodeId.toString())), - JField("blindedNodeIds", JArray(path.route.blindedNodes.map(n => JString(n.blindedPublicKey.toString())).toList)) + JField("introductionNodeId", JString(introductionNode)), + JField("blindedNodeIds", JArray(blindedNodes.map(n => JString(n.blindedPublicKey.toString)).toList)) )) }).toList)), JField("createdAt", JLong(p.createdAt.toLong)), diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/message/OnionMessages.scala b/eclair-core/src/main/scala/fr/acinq/eclair/message/OnionMessages.scala index 397cc46a66..6db3d2eba7 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/message/OnionMessages.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/message/OnionMessages.scala @@ -54,8 +54,12 @@ object OnionMessages { } // @formatter:off - sealed trait Destination - case class BlindedPath(route: Sphinx.RouteBlinding.BlindedRoute) extends Destination + sealed trait Destination { + def nodeId: PublicKey + } + case class BlindedPath(route: Sphinx.RouteBlinding.BlindedRoute) extends Destination { + override def nodeId: PublicKey = route.introductionNodeId + } case class Recipient(nodeId: PublicKey, pathId: Option[ByteVector], padding: Option[ByteVector] = None, customTlvs: Set[GenericTlv] = Set.empty) extends Destination // @formatter:on diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/message/Postman.scala b/eclair-core/src/main/scala/fr/acinq/eclair/message/Postman.scala index 268ee2beb6..2f6842d958 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/message/Postman.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/message/Postman.scala @@ -22,15 +22,16 @@ import akka.actor.typed.scaladsl.{ActorContext, Behaviors} import akka.actor.typed.{ActorRef, Behavior} import fr.acinq.bitcoin.scalacompat.ByteVector32 import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey +import fr.acinq.eclair.crypto.Sphinx.RouteBlinding.BlindedRoute import fr.acinq.eclair.io.MessageRelay -import fr.acinq.eclair.io.MessageRelay.RelayPolicy import fr.acinq.eclair.message.OnionMessages.{Destination, RoutingStrategy} import fr.acinq.eclair.payment.offer.OfferManager import fr.acinq.eclair.router.Router import fr.acinq.eclair.router.Router.{MessageRoute, MessageRouteNotFound, MessageRouteResponse} import fr.acinq.eclair.wire.protocol.MessageOnion.{FinalPayload, InvoiceRequestPayload} -import fr.acinq.eclair.wire.protocol.{OnionMessage, OnionMessagePayloadTlv, TlvStream} -import fr.acinq.eclair.{NodeParams, ShortChannelId, randomBytes32, randomKey} +import fr.acinq.eclair.wire.protocol.OfferTypes.{CompactBlindedPath, ContactInfo} +import fr.acinq.eclair.wire.protocol.{OfferTypes, OnionMessagePayloadTlv, TlvStream} +import fr.acinq.eclair.{NodeParams, randomBytes32, randomKey} import scala.collection.mutable @@ -40,13 +41,13 @@ object Postman { /** * Builds a message packet and send it to the destination using the provided path. * - * @param destination Recipient of the message + * @param contactInfo Recipient of the message * @param routingStrategy How to reach the destination (recipient or blinded path introduction node). * @param message Content of the message to send * @param expectsReply Whether the message expects a reply * @param replyTo Actor to send the status and reply to */ - case class SendMessage(destination: Destination, + case class SendMessage(contactInfo: ContactInfo, routingStrategy: RoutingStrategy, message: TlvStream[OnionMessagePayloadTlv], expectsReply: Boolean, @@ -63,7 +64,7 @@ object Postman { case class MessageFailed(reason: String) extends MessageStatus // @formatter:on - def apply(nodeParams: NodeParams, switchboard: akka.actor.ActorRef, router: ActorRef[Router.MessageRouteRequest], register: akka.actor.ActorRef, offerManager: typed.ActorRef[OfferManager.RequestInvoice]): Behavior[Command] = { + def apply(nodeParams: NodeParams, switchboard: akka.actor.ActorRef, router: ActorRef[Router.PostmanRequest], register: akka.actor.ActorRef, offerManager: typed.ActorRef[OfferManager.RequestInvoice]): Behavior[Command] = { Behaviors.setup(context => { context.system.eventStream ! EventStream.Subscribe(context.messageAdapter[OnionMessages.ReceiveMessage](r => WrappedMessage(r.finalPayload))) @@ -110,31 +111,32 @@ object SendingMessage { case object SendMessage extends Command private case class SendingStatus(status: MessageRelay.Status) extends Command private case class WrappedMessageRouteResponse(response: MessageRouteResponse) extends Command + private case class WrappedNodeIdResponse(nodeId_opt: Option[PublicKey]) extends Command // @formatter:on def apply(nodeParams: NodeParams, - router: ActorRef[Router.MessageRouteRequest], + router: ActorRef[Router.PostmanRequest], postman: ActorRef[Postman.Command], switchboard: akka.actor.ActorRef, register: akka.actor.ActorRef, - destination: Destination, + contactInfo: ContactInfo, message: TlvStream[OnionMessagePayloadTlv], routingStrategy: RoutingStrategy, expectsReply: Boolean, replyTo: ActorRef[Postman.OnionMessageResponse]): Behavior[Command] = { Behaviors.setup(context => { - val actor = new SendingMessage(nodeParams, router, postman, switchboard, register, destination, message, routingStrategy, expectsReply, replyTo, context) + val actor = new SendingMessage(nodeParams, router, postman, switchboard, register, contactInfo, message, routingStrategy, expectsReply, replyTo, context) actor.start() }) } } private class SendingMessage(nodeParams: NodeParams, - router: ActorRef[Router.MessageRouteRequest], + router: ActorRef[Router.PostmanRequest], postman: ActorRef[Postman.Command], switchboard: akka.actor.ActorRef, register: akka.actor.ActorRef, - destination: Destination, + contactInfo: ContactInfo, message: TlvStream[OnionMessagePayloadTlv], routingStrategy: RoutingStrategy, expectsReply: Boolean, @@ -146,27 +148,43 @@ private class SendingMessage(nodeParams: NodeParams, def start(): Behavior[Command] = { Behaviors.receiveMessagePartial { case SendMessage => - val targetNodeId = destination match { - case OnionMessages.BlindedPath(route) => route.introductionNodeId - case OnionMessages.Recipient(nodeId, _, _, _) => nodeId - } - routingStrategy match { - case RoutingStrategy.UseRoute(intermediateNodes) => sendToRoute(intermediateNodes, targetNodeId) - case RoutingStrategy.FindRoute if targetNodeId == nodeParams.nodeId => - context.self ! WrappedMessageRouteResponse(MessageRoute(Nil, targetNodeId)) - waitForRouteFromRouter() - case RoutingStrategy.FindRoute => - router ! Router.MessageRouteRequest(context.messageAdapter(WrappedMessageRouteResponse), nodeParams.nodeId, targetNodeId, Set.empty) - waitForRouteFromRouter() + contactInfo match { + case compact: OfferTypes.CompactBlindedPath => + router ! Router.GetNodeId(context.messageAdapter(WrappedNodeIdResponse), compact.introductionNode.scid, compact.introductionNode.isNode1) + waitForNodeId(compact) + case OfferTypes.BlindedPath(route) => sendToDestination(OnionMessages.BlindedPath(route)) + case OfferTypes.RecipientNodeId(nodeId) => sendToDestination(OnionMessages.Recipient(nodeId, None)) } } } - private def waitForRouteFromRouter(): Behavior[Command] = { + private def waitForNodeId(compactBlindedPath: CompactBlindedPath): Behavior[Command] = { + Behaviors.receiveMessagePartial { + case WrappedNodeIdResponse(None) => + replyTo ! Postman.MessageFailed("Unknown target") + Behaviors.stopped + case WrappedNodeIdResponse(Some(nodeId)) => + sendToDestination(OnionMessages.BlindedPath(BlindedRoute(nodeId, compactBlindedPath.blindingKey, compactBlindedPath.blindedNodes))) + } + } + + private def sendToDestination(destination: Destination): Behavior[Command] = { + routingStrategy match { + case RoutingStrategy.UseRoute(intermediateNodes) => sendToRoute(intermediateNodes, destination) + case RoutingStrategy.FindRoute if destination.nodeId == nodeParams.nodeId => + context.self ! WrappedMessageRouteResponse(MessageRoute(Nil, destination.nodeId)) + waitForRouteFromRouter(destination) + case RoutingStrategy.FindRoute => + router ! Router.MessageRouteRequest(context.messageAdapter(WrappedMessageRouteResponse), nodeParams.nodeId, destination.nodeId, Set.empty) + waitForRouteFromRouter(destination) + } + } + + private def waitForRouteFromRouter(destination: Destination): Behavior[Command] = { Behaviors.receiveMessagePartial { case WrappedMessageRouteResponse(MessageRoute(intermediateNodes, targetNodeId)) => context.log.debug("Found route: {}", (intermediateNodes :+ targetNodeId).mkString(" -> ")) - sendToRoute(intermediateNodes, targetNodeId) + sendToRoute(intermediateNodes, destination) case WrappedMessageRouteResponse(MessageRouteNotFound(targetNodeId)) => context.log.debug("No route found to {}", targetNodeId) replyTo ! Postman.MessageFailed("No route found") @@ -174,12 +192,12 @@ private class SendingMessage(nodeParams: NodeParams, } } - private def sendToRoute(intermediateNodes: Seq[PublicKey], targetNodeId: PublicKey): Behavior[Command] = { + private def sendToRoute(intermediateNodes: Seq[PublicKey], destination: Destination): Behavior[Command] = { val messageId = randomBytes32() val replyRoute = if (expectsReply) { val numHopsToAdd = 0.max(nodeParams.onionMessageConfig.minIntermediateHops - intermediateNodes.length - 1) - val intermediateHops = (Seq(targetNodeId) ++ intermediateNodes.reverse ++ Seq.fill(numHopsToAdd)(nodeParams.nodeId)).map(OnionMessages.IntermediateNode(_)) + val intermediateHops = (Seq(destination.nodeId) ++ intermediateNodes.reverse ++ Seq.fill(numHopsToAdd)(nodeParams.nodeId)).map(OnionMessages.IntermediateNode(_)) val lastHop = OnionMessages.Recipient(nodeParams.nodeId, Some(messageId)) Some(OnionMessages.buildRoute(randomKey(), intermediateHops, lastHop)) } else { 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 0d96f5b244..0307ab5d11 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 @@ -86,7 +86,7 @@ case class Bolt12Invoice(records: TlvStream[InvoiceTlv]) extends Invoice { } -case class PaymentBlindedRoute(route: Sphinx.RouteBlinding.BlindedRoute, paymentInfo: PaymentInfo) +case class PaymentBlindedRoute(route: BlindedContactInfo, paymentInfo: PaymentInfo) object Bolt12Invoice { val hrp = "lni" diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala index 4c2d511c6c..4800b00afe 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/offer/OfferManager.scala @@ -20,6 +20,7 @@ import akka.actor.typed.scaladsl.{ActorContext, Behaviors} import akka.actor.typed.{ActorRef, Behavior} import fr.acinq.bitcoin.scalacompat.Crypto.PrivateKey import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto} +import fr.acinq.eclair.crypto.Sphinx.RouteBlinding import fr.acinq.eclair.db.{IncomingBlindedPayment, IncomingPaymentStatus, PaymentType} import fr.acinq.eclair.message.{OnionMessages, Postman} import fr.acinq.eclair.payment.MinimalBolt12Invoice @@ -106,7 +107,7 @@ object OfferManager { case RequestInvoice(messagePayload, postman) => registeredOffers.get(messagePayload.invoiceRequest.offer.offerId) match { case Some(registered) if registered.pathId_opt.map(_.bytes) == messagePayload.pathId_opt && messagePayload.invoiceRequest.isValid => - val child = context.spawnAnonymous(InvoiceRequestActor(nodeParams, messagePayload.invoiceRequest, registered.handler, registered.nodeKey, router, OnionMessages.BlindedPath(messagePayload.replyPath), postman)) + val child = context.spawnAnonymous(InvoiceRequestActor(nodeParams, messagePayload.invoiceRequest, registered.handler, registered.nodeKey, router, messagePayload.replyPath, postman)) child ! InvoiceRequestActor.RequestInvoice case _ => context.log.debug("offer {} is not registered or invoice request is invalid", messagePayload.invoiceRequest.offer.offerId) } @@ -167,7 +168,7 @@ object OfferManager { offerHandler: ActorRef[HandleInvoiceRequest], nodeKey: PrivateKey, router: akka.actor.ActorRef, - pathToSender: OnionMessages.Destination, + pathToSender: RouteBlinding.BlindedRoute, postman: ActorRef[Postman.SendMessage]): Behavior[Command] = { Behaviors.setup { context => Behaviors.withMdc(Logs.mdc(category_opt = Some(Logs.LogCategory.PAYMENT))) { @@ -184,13 +185,13 @@ object OfferManager { invoiceRequest: InvoiceRequest, nodeKey: PrivateKey, router: akka.actor.ActorRef, - pathToSender: OnionMessages.Destination, + pathToSender: RouteBlinding.BlindedRoute, postman: ActorRef[Postman.SendMessage], context: ActorContext[Command]) { def waitForHandler(): Behavior[Command] = { Behaviors.receiveMessagePartial { case RejectRequest(error) => - postman ! Postman.SendMessage(pathToSender, OnionMessages.RoutingStrategy.FindRoute, TlvStream(OnionMessagePayloadTlv.InvoiceError(TlvStream(OfferTypes.Error(error)))), expectsReply = false, context.messageAdapter[Postman.OnionMessageResponse](WrappedOnionMessageResponse)) + postman ! Postman.SendMessage(OfferTypes.BlindedPath(pathToSender), OnionMessages.RoutingStrategy.FindRoute, TlvStream(OnionMessagePayloadTlv.InvoiceError(TlvStream(OfferTypes.Error(error)))), expectsReply = false, context.messageAdapter[Postman.OnionMessageResponse](WrappedOnionMessageResponse)) waitForSent() case ApproveRequest(amount, routes, pluginData_opt, additionalTlvs, customTlvs) => val preimage = randomBytes32() @@ -208,7 +209,7 @@ object OfferManager { case WrappedInvoiceResponse(invoiceResponse) => invoiceResponse match { case CreateInvoiceActor.InvoiceCreated(invoice) => - postman ! Postman.SendMessage(pathToSender, OnionMessages.RoutingStrategy.FindRoute, TlvStream(OnionMessagePayloadTlv.Invoice(invoice.records)), expectsReply = false, context.messageAdapter[Postman.OnionMessageResponse](WrappedOnionMessageResponse)) + postman ! Postman.SendMessage(OfferTypes.BlindedPath(pathToSender), OnionMessages.RoutingStrategy.FindRoute, TlvStream(OnionMessagePayloadTlv.Invoice(invoice.records)), expectsReply = false, context.messageAdapter[Postman.OnionMessageResponse](WrappedOnionMessageResponse)) waitForSent() case f: CreateInvoiceActor.InvoiceCreationFailed => context.log.debug("invoice creation failed: {}", f.message) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala index 472156ecad..a7a3930bbf 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala @@ -381,7 +381,7 @@ object MultiPartHandler { } })).map(paths => { val invoiceFeatures = nodeParams.features.bolt12Features() - val invoice = Bolt12Invoice(r.invoiceRequest, r.paymentPreimage, r.nodeKey, nodeParams.invoiceExpiry, invoiceFeatures, paths.map { case (blindedRoute, paymentInfo, _) => PaymentBlindedRoute(blindedRoute.route, paymentInfo) }, r.additionalTlvs, r.customTlvs) + val invoice = Bolt12Invoice(r.invoiceRequest, r.paymentPreimage, r.nodeKey, nodeParams.invoiceExpiry, invoiceFeatures, paths.map { case (blindedRoute, paymentInfo, _) => PaymentBlindedRoute(OfferTypes.BlindedPath(blindedRoute.route), paymentInfo) }, r.additionalTlvs, r.customTlvs) log.debug("generated invoice={} for offer={}", invoice.toString, r.invoiceRequest.offer.toString) invoice }))(WrappedInvoiceResult) 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 0a0ab233c0..fd2afc4afd 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 @@ -103,16 +103,10 @@ object OfferPayment { replyTo: ActorRef, attemptNumber: Int, sendPaymentConfig: SendPaymentConfig): Behavior[Command] = { - val destination = request.offer.contactInfo match { - case Left(blindedRoutes) => - val blindedRoute = blindedRoutes(attemptNumber % blindedRoutes.length) - OnionMessages.BlindedPath(blindedRoute) - case Right(nodeId) => - OnionMessages.Recipient(nodeId, None, None) - } + val contactInfo = request.offer.contactInfos(attemptNumber % request.offer.contactInfos.length) val messageContent = TlvStream[OnionMessagePayloadTlv](OnionMessagePayloadTlv.InvoiceRequest(request.records)) val routingStrategy = if (sendPaymentConfig.connectDirectly) OnionMessages.RoutingStrategy.connectDirectly else OnionMessages.RoutingStrategy.FindRoute - postman ! SendMessage(destination, routingStrategy, messageContent, expectsReply = true, context.messageAdapter(WrappedMessageResponse)) + postman ! SendMessage(contactInfo, routingStrategy, messageContent, expectsReply = true, context.messageAdapter(WrappedMessageResponse)) waitForInvoice(nodeParams, postman, paymentInitiator, context, request, payerKey, replyTo, attemptNumber + 1, sendPaymentConfig) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala index c3111c8833..304bb6285c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala @@ -243,6 +243,10 @@ class Router(val nodeParams: NodeParams, watcher: typed.ActorRef[ZmqWatcher.Comm case Event(r: MessageRouteRequest, d) => stay() using RouteCalculation.handleMessageRouteRequest(d, nodeParams.currentBlockHeight, r, nodeParams.routerConf.messageRouteParams) + case Event(GetNodeId(replyTo, shortChannelId, isNode1), d) => + replyTo ! d.channels.get(shortChannelId).map(channel => if (isNode1) channel.nodeId1 else channel.nodeId2) + stay() + // Warning: order matters here, this must be the first match for HasChainHash messages ! case Event(PeerRoutingMessage(_, _, routingMessage: HasChainHash), _) if routingMessage.chainHash != nodeParams.chainHash => sender() ! TransportHandler.ReadAck(routingMessage) @@ -589,10 +593,16 @@ object Router { extraEdges: Seq[ExtraEdge] = Nil, paymentContext: Option[PaymentContext] = None) + sealed trait PostmanRequest + case class MessageRouteRequest(replyTo: typed.ActorRef[MessageRouteResponse], source: PublicKey, target: PublicKey, - ignoredNodes: Set[PublicKey]) + ignoredNodes: Set[PublicKey]) extends PostmanRequest + + case class GetNodeId(replyTo: typed.ActorRef[Option[PublicKey]], + shortChannelId: RealShortChannelId, + isNode1: Boolean) extends PostmanRequest // @formatter:off sealed trait MessageRouteResponse { def target: PublicKey } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferCodecs.scala index d3079fec97..420778c72f 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferCodecs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferCodecs.scala @@ -22,7 +22,7 @@ import fr.acinq.eclair.wire.protocol.CommonCodecs._ import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequestChain, InvoiceRequestPayerNote, InvoiceRequestQuantity, _} import fr.acinq.eclair.wire.protocol.TlvCodecs.{tlvField, tmillisatoshi, tu32, tu64overflow} import fr.acinq.eclair.{TimestampSecond, UInt64} -import scodec.Codec +import scodec.{Attempt, Codec, Err} import scodec.codecs._ object OfferCodecs { @@ -46,12 +46,34 @@ object OfferCodecs { private val blindedNodesCodec: Codec[Seq[BlindedNode]] = listOfN(uint8, blindedNodeCodec).xmap(_.toSeq, _.toList) - private val pathCodec: Codec[BlindedRoute] = + private val blindedPathCodec: Codec[BlindedPath] = (("firstNodeId" | publicKey) :: ("blinding" | publicKey) :: - ("path" | blindedNodesCodec)).as[BlindedRoute] + ("path" | blindedNodesCodec)).as[BlindedRoute].as[BlindedPath] - private val offerPaths: Codec[OfferPaths] = tlvField(list(pathCodec).xmap[Seq[BlindedRoute]](_.toSeq, _.toList)) + private val isNode1: Codec[Boolean] = uint8.narrow( + n => if (n == 0) Attempt.Successful(true) else if (n == 1) Attempt.Successful(false) else Attempt.Failure(new Err.MatchingDiscriminatorNotFound(n)), + b => if (b) 0 else 1 + ) + + private val shortChannelIdDirCodec: Codec[ShortChannelIdDir] = + (("isNode1" | isNode1) :: + ("scid" | realshortchannelid)).as[ShortChannelIdDir] + + private val compactBlindedPathCodec: Codec[CompactBlindedPath] = + (("introductionNode" | shortChannelIdDirCodec) :: + ("blinding" | publicKey) :: + ("path" | blindedNodesCodec)).as[CompactBlindedPath] + + private val pathCodec: Codec[BlindedContactInfo] = fallback(blindedPathCodec, compactBlindedPathCodec).xmap({ + case Left(path) => path + case Right(compact) => compact + }, { + case path: BlindedPath => Left(path) + case compact: CompactBlindedPath => Right(compact) + }) + + private val offerPaths: Codec[OfferPaths] = tlvField(list(pathCodec).xmap[Seq[BlindedContactInfo]](_.toSeq, _.toList)) private val offerIssuer: Codec[OfferIssuer] = tlvField(utf8) @@ -114,7 +136,7 @@ object OfferCodecs { .typecase(UInt64(240), signature) ).complete - private val invoicePaths: Codec[InvoicePaths] = tlvField(list(pathCodec).xmap[Seq[BlindedRoute]](_.toSeq, _.toList)) + private val invoicePaths: Codec[InvoicePaths] = tlvField(list(pathCodec).xmap[Seq[BlindedContactInfo]](_.toSeq, _.toList)) private val paymentInfo: Codec[PaymentInfo] = (("fee_base_msat" | millisatoshi32) :: diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferTypes.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferTypes.scala index d359396327..4f6c183bb6 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferTypes.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferTypes.scala @@ -19,11 +19,11 @@ package fr.acinq.eclair.wire.protocol import fr.acinq.bitcoin.Bech32 import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey, XonlyPublicKey} import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, ByteVector64, Crypto, LexicographicalOrdering} -import fr.acinq.eclair.crypto.Sphinx.RouteBlinding.BlindedRoute +import fr.acinq.eclair.crypto.Sphinx.RouteBlinding.{BlindedNode, BlindedRoute} import fr.acinq.eclair.wire.protocol.CommonCodecs.varint import fr.acinq.eclair.wire.protocol.OnionRoutingCodecs.{ForbiddenTlv, InvalidTlvPayload, MissingRequiredTlv} import fr.acinq.eclair.wire.protocol.TlvCodecs.genericTlv -import fr.acinq.eclair.{Bolt12Feature, CltvExpiryDelta, Feature, Features, MilliSatoshi, TimestampSecond, UInt64, nodeFee, randomBytes32} +import fr.acinq.eclair.{Bolt12Feature, CltvExpiryDelta, Feature, Features, MilliSatoshi, RealShortChannelId, ShortChannelId, TimestampSecond, UInt64, nodeFee, randomBytes32} import scodec.Codec import scodec.bits.ByteVector import scodec.codecs.vector @@ -35,6 +35,13 @@ import scala.util.{Failure, Try} * see https://github.com/lightning/bolts/blob/master/12-offer-encoding.md */ object OfferTypes { + case class ShortChannelIdDir(isNode1: Boolean, scid: RealShortChannelId) + + sealed trait ContactInfo + sealed trait BlindedContactInfo extends ContactInfo + case class BlindedPath(route: BlindedRoute) extends BlindedContactInfo + case class CompactBlindedPath(introductionNode: ShortChannelIdDir, blindingKey: PublicKey, blindedNodes: Seq[BlindedNode]) extends BlindedContactInfo + case class RecipientNodeId(nodeId: PublicKey) extends ContactInfo sealed trait Bolt12Tlv extends Tlv @@ -84,7 +91,7 @@ object OfferTypes { /** * Paths that can be used to retrieve an invoice. */ - case class OfferPaths(paths: Seq[BlindedRoute]) extends OfferTlv + case class OfferPaths(paths: Seq[BlindedContactInfo]) extends OfferTlv /** * Name of the offer creator. @@ -144,7 +151,7 @@ object OfferTypes { /** * Payment paths to send the payment to. */ - case class InvoicePaths(paths: Seq[BlindedRoute]) extends InvoiceTlv + case class InvoicePaths(paths: Seq[BlindedContactInfo]) extends InvoiceTlv case class PaymentInfo(feeBase: MilliSatoshi, feeProportionalMillionths: Long, @@ -228,12 +235,12 @@ object OfferTypes { val description: String = records.get[OfferDescription].get.description val features: Features[Bolt12Feature] = records.get[OfferFeatures].map(_.features.bolt12Features()).getOrElse(Features.empty) val expiry: Option[TimestampSecond] = records.get[OfferAbsoluteExpiry].map(_.absoluteExpiry) - private val paths: Option[Seq[BlindedRoute]] = records.get[OfferPaths].map(_.paths) + private val paths: Option[Seq[BlindedContactInfo]] = records.get[OfferPaths].map(_.paths) val issuer: Option[String] = records.get[OfferIssuer].map(_.issuer) val quantityMax: Option[Long] = records.get[OfferQuantityMax].map(_.max).map { q => if (q == 0) Long.MaxValue else q } val nodeId: PublicKey = records.get[OfferNodeId].map(_.publicKey).get - val contactInfo: Either[Seq[BlindedRoute], PublicKey] = paths.map(Left(_)).getOrElse(Right(nodeId)) + val contactInfos: Seq[ContactInfo] = paths.getOrElse(Seq(RecipientNodeId(nodeId))) def encode(): String = { val data = OfferCodecs.offerTlvCodec.encode(records).require.bytes From bd4c4962c3c8458d22fb9698d136fda0f49aa4e1 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Mon, 23 Oct 2023 16:44:49 +0200 Subject: [PATCH 02/12] Send payment to sciddir_or_pubkey --- .../acinq/eclair/payment/Bolt11Invoice.scala | 2 +- .../fr/acinq/eclair/payment/Invoice.scala | 3 +- .../acinq/eclair/payment/PaymentPacket.scala | 2 +- .../payment/receive/MultiPartHandler.scala | 4 +- .../payment/send/PaymentLifecycle.scala | 8 ++-- .../acinq/eclair/payment/send/Recipient.scala | 41 +++++++++++-------- .../scala/fr/acinq/eclair/router/Graph.scala | 19 ++++++--- .../eclair/router/RouteCalculation.scala | 31 ++++++++------ .../ChannelStateTestsHelperMethods.scala | 5 ++- .../fr/acinq/eclair/db/PaymentsDbSpec.scala | 2 +- .../integration/PaymentIntegrationSpec.scala | 12 +++--- .../basic/payment/OfferPaymentSpec.scala | 8 ++-- .../fr/acinq/eclair/message/PostmanSpec.scala | 17 ++++---- .../eclair/payment/Bolt12InvoiceSpec.scala | 2 +- .../eclair/payment/MultiPartHandlerSpec.scala | 22 +++++----- .../MultiPartPaymentLifecycleSpec.scala | 22 +++++----- .../eclair/payment/PaymentInitiatorSpec.scala | 2 +- .../eclair/payment/PaymentLifecycleSpec.scala | 8 ++-- .../eclair/payment/PaymentPacketSpec.scala | 20 +++++---- .../payment/offer/OfferManagerSpec.scala | 2 +- .../payment/relay/NodeRelayerSpec.scala | 4 +- .../payment/send/OfferPaymentSpec.scala | 12 +++--- .../eclair/router/BalanceEstimateSpec.scala | 2 +- .../acinq/eclair/router/BaseRouterSpec.scala | 5 ++- .../fr/acinq/eclair/router/RouterSpec.scala | 14 +++---- .../protocol/MessageOnionCodecsSpec.scala | 6 +-- 26 files changed, 153 insertions(+), 122 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt11Invoice.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt11Invoice.scala index 027f217f9e..f2e401f026 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt11Invoice.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt11Invoice.scala @@ -559,7 +559,7 @@ object Bolt11Invoice { val nextNodeIds = extraRoute.map(_.nodeId).drop(1) :+ targetNodeId extraRoute.zip(nextNodeIds).map { case (extraHop, nextNodeId) => - Invoice.ExtraEdge(extraHop.nodeId, nextNodeId, extraHop.shortChannelId, extraHop.feeBase, extraHop.feeProportionalMillionths, extraHop.cltvExpiryDelta, 1 msat, None) + Invoice.ExtraEdge(Right(extraHop.nodeId), nextNodeId, extraHop.shortChannelId, extraHop.feeBase, extraHop.feeProportionalMillionths, extraHop.cltvExpiryDelta, 1 msat, None) } } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Invoice.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Invoice.scala index b477e34330..c55fc5d879 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Invoice.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Invoice.scala @@ -20,6 +20,7 @@ import fr.acinq.bitcoin.scalacompat.ByteVector32 import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.eclair.payment.relay.Relayer import fr.acinq.eclair.wire.protocol.ChannelUpdate +import fr.acinq.eclair.wire.protocol.OfferTypes.ShortChannelIdDir import fr.acinq.eclair.{CltvExpiryDelta, Features, InvoiceFeature, MilliSatoshi, ShortChannelId, TimestampSecond} import scala.concurrent.duration.FiniteDuration @@ -41,7 +42,7 @@ trait Invoice { object Invoice { /** An extra edge that can be used to pay a given invoice and may not be part of the public graph. */ - case class ExtraEdge(sourceNodeId: PublicKey, + case class ExtraEdge(sourceNodeId: Either[ShortChannelIdDir, PublicKey], targetNodeId: PublicKey, shortChannelId: ShortChannelId, feeBase: MilliSatoshi, diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala index 2ff77b5998..0a59fef0f4 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala @@ -235,7 +235,7 @@ object OutgoingPaymentPacket { case class CannotDecryptBlindedRoute(message: String) extends OutgoingPaymentError { override def getMessage: String = message } case class InvalidRouteRecipient(expected: PublicKey, actual: PublicKey) extends OutgoingPaymentError { override def getMessage: String = s"expected route to $expected, got route to $actual" } case class MissingTrampolineHop(trampolineNodeId: PublicKey) extends OutgoingPaymentError { override def getMessage: String = s"expected route to trampoline node $trampolineNodeId" } - case class MissingBlindedHop(introductionNodeIds: Set[PublicKey]) extends OutgoingPaymentError { override def getMessage: String = s"expected blinded route using one of the following introduction nodes: ${introductionNodeIds.mkString(", ")}" } + case object MissingBlindedHop extends OutgoingPaymentError { override def getMessage: String = s"expected blinded route as final hop" } case object EmptyRoute extends OutgoingPaymentError { override def getMessage: String = "route cannot be empty" } sealed trait Upstream diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala index a7a3930bbf..56ef89f6cb 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala @@ -359,8 +359,8 @@ object MultiPartHandler { context.pipeToSelf(Future.sequence(r.routes.map(route => { val dummyHops = route.dummyHops.map(h => { // We don't want to restrict HTLC size in dummy hops, so we use htlc_minimum_msat = 1 msat and htlc_maximum_msat = None. - val edge = Invoice.ExtraEdge(nodeParams.nodeId, nodeParams.nodeId, ShortChannelId.toSelf, h.feeBase, h.feeProportionalMillionths, h.cltvExpiryDelta, htlcMinimum = 1 msat, htlcMaximum_opt = None) - ChannelHop(edge.shortChannelId, edge.sourceNodeId, edge.targetNodeId, HopRelayParams.FromHint(edge)) + val edge = Invoice.ExtraEdge(Right(nodeParams.nodeId), nodeParams.nodeId, ShortChannelId.toSelf, h.feeBase, h.feeProportionalMillionths, h.cltvExpiryDelta, htlcMinimum = 1 msat, htlcMaximum_opt = None) + ChannelHop(edge.shortChannelId, nodeParams.nodeId, edge.targetNodeId, HopRelayParams.FromHint(edge)) }) if (route.nodes.length == 1) { val blindedRoute = if (dummyHops.isEmpty) { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala index 381ce075e1..39bb0687fd 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala @@ -300,7 +300,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A log.info("received an update for a routing hint (shortChannelId={} nodeId={} enabled={} update={})", failure.update.shortChannelId, nodeId, failure.update.channelFlags.isEnabled, failure.update) if (failure.update.channelFlags.isEnabled) { data.recipient.extraEdges.map { - case edge: ExtraEdge if edge.sourceNodeId == nodeId && edge.targetNodeId == hop.nextNodeId => edge.update(failure.update) + case edge: ExtraEdge if edge.sourceNodeId == Right(nodeId) && edge.targetNodeId == hop.nextNodeId => edge.update(failure.update) case edge: ExtraEdge => edge } } else { @@ -308,10 +308,10 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A // contain channel flags to indicate that it's disabled // we want the exclusion to be router-wide so that sister payments in the case of MPP are aware the channel is faulty data.recipient.extraEdges - .find(edge => edge.sourceNodeId == nodeId && edge.targetNodeId == hop.nextNodeId) - .foreach(edge => router ! ExcludeChannel(ChannelDesc(edge.shortChannelId, edge.sourceNodeId, edge.targetNodeId), Some(nodeParams.routerConf.channelExcludeDuration))) + .find(edge => edge.sourceNodeId == Right(nodeId) && edge.targetNodeId == hop.nextNodeId) + .foreach(edge => router ! ExcludeChannel(ChannelDesc(edge.shortChannelId, nodeId, edge.targetNodeId), Some(nodeParams.routerConf.channelExcludeDuration))) // we remove this edge for our next payment attempt - data.recipient.extraEdges.filterNot(edge => edge.sourceNodeId == nodeId && edge.targetNodeId == hop.nextNodeId) + data.recipient.extraEdges.filterNot(edge => edge.sourceNodeId == Right(nodeId) && edge.targetNodeId == hop.nextNodeId) } } case None => 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 293fa10178..63d79dd70e 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,11 +21,11 @@ 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, PaymentBlindedRoute} import fr.acinq.eclair.router.Router._ import fr.acinq.eclair.wire.protocol.PaymentOnion.{FinalPayload, IntermediatePayload, OutgoingBlindedPerHopPayload} -import fr.acinq.eclair.wire.protocol.{GenericTlv, OnionRoutingPacket, PaymentOnionCodecs} -import fr.acinq.eclair.{CltvExpiry, Features, InvoiceFeature, MilliSatoshi, MilliSatoshiLong, ShortChannelId} +import fr.acinq.eclair.wire.protocol.{GenericTlv, OfferTypes, OnionRoutingPacket, PaymentOnionCodecs} +import fr.acinq.eclair.{Alias, CltvExpiry, Features, InvoiceFeature, MilliSatoshi, MilliSatoshiLong, ShortChannelId} import scodec.bits.ByteVector /** @@ -121,18 +121,30 @@ case class BlindedRecipient(nodeId: PublicKey, features: Features[InvoiceFeature], totalAmount: MilliSatoshi, expiry: CltvExpiry, - blindedHops: Seq[BlindedHop], + blindedPaths: Map[Alias, PaymentBlindedRoute], customTlvs: Set[GenericTlv] = Set.empty) extends Recipient { - require(blindedHops.nonEmpty, "blinded routes must be provided") + require(blindedPaths.nonEmpty, "blinded routes must be provided") - override val extraEdges = blindedHops.map { h => - ExtraEdge(h.route.introductionNodeId, nodeId, h.dummyId, h.paymentInfo.feeBase, h.paymentInfo.feeProportionalMillionths, h.paymentInfo.cltvExpiryDelta, h.paymentInfo.minHtlc, Some(h.paymentInfo.maxHtlc)) - } + override val extraEdges = blindedPaths.map { case (scid, path) => + val introductionNodeId = path.route match { + case OfferTypes.BlindedPath(route) => Right(route.introductionNodeId) + case OfferTypes.CompactBlindedPath(introductionNode, _, _) => Left(introductionNode) + } + ExtraEdge( + introductionNodeId, + nodeId, + scid, + path.paymentInfo.feeBase, + path.paymentInfo.feeProportionalMillionths, + path.paymentInfo.cltvExpiryDelta, + path.paymentInfo.minHtlc, + Some(path.paymentInfo.maxHtlc)) + }.toSeq private def validateRoute(route: Route): Either[OutgoingPaymentError, BlindedHop] = { route.finalHop_opt match { case Some(blindedHop: BlindedHop) => Right(blindedHop) - case _ => Left(MissingBlindedHop(blindedHops.map(_.route.introductionNodeId).toSet)) + case _ => Left(MissingBlindedHop) } } @@ -167,14 +179,7 @@ case class BlindedRecipient(nodeId: PublicKey, object BlindedRecipient { def apply(invoice: Bolt12Invoice, totalAmount: MilliSatoshi, expiry: CltvExpiry, customTlvs: Set[GenericTlv]): BlindedRecipient = { - val blindedHops = invoice.blindedPaths.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() - BlindedHop(dummyId, path.route, path.paymentInfo) - }) - BlindedRecipient(invoice.nodeId, invoice.features, totalAmount, expiry, blindedHops, customTlvs) + BlindedRecipient(invoice.nodeId, invoice.features, totalAmount, expiry, invoice.blindedPaths.map((ShortChannelId.generateLocalAlias(), _)).toMap, customTlvs) } } @@ -201,7 +206,7 @@ case class ClearTrampolineRecipient(invoice: Bolt11Invoice, override val nodeId = invoice.nodeId override val features = invoice.features - override val extraEdges = Seq(ExtraEdge(trampolineNodeId, nodeId, ShortChannelId.generateLocalAlias(), trampolineFee, 0, trampolineHop.cltvExpiryDelta, 1 msat, None)) + override val extraEdges = Seq(ExtraEdge(Right(trampolineNodeId), nodeId, ShortChannelId.generateLocalAlias(), trampolineFee, 0, trampolineHop.cltvExpiryDelta, 1 msat, None)) private def validateRoute(route: Route): Either[OutgoingPaymentError, NodeHop] = { route.finalHop_opt match { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala index 89af943807..32fbb90869 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala @@ -21,11 +21,12 @@ import fr.acinq.bitcoin.scalacompat.{Btc, BtcDouble, MilliBtc, Satoshi} import fr.acinq.eclair._ import fr.acinq.eclair.payment.Invoice import fr.acinq.eclair.payment.relay.Relayer.RelayFees -import fr.acinq.eclair.router.Graph.GraphStructure.{GraphEdge, DirectedGraph} +import fr.acinq.eclair.router.Graph.GraphStructure.{DirectedGraph, GraphEdge} import fr.acinq.eclair.router.Router._ import fr.acinq.eclair.wire.protocol.{ChannelUpdate, NodeAnnouncement} import scala.annotation.tailrec +import scala.collection.immutable.SortedMap import scala.collection.mutable object Graph { @@ -612,15 +613,23 @@ object Graph { balance_opt = pc.getBalanceSameSideAs(u) ) - def apply(e: Invoice.ExtraEdge): GraphEdge = { + def fromExtraEdge(e: Invoice.ExtraEdge, publicChannels: SortedMap[RealShortChannelId, PublicChannel]): Option[GraphEdge] = { val maxBtc = 21e6.btc - GraphEdge( - desc = ChannelDesc(e.shortChannelId, e.sourceNodeId, e.targetNodeId), + val sourceNodeId = e.sourceNodeId match { + case Left(scidDir) => publicChannels.get(scidDir.scid) match { + case Some(pc) if scidDir.isNode1 => pc.nodeId1 + case Some(pc) => pc.nodeId2 + case None => return None + } + case Right(publicKey) => publicKey + } + Some(GraphEdge( + desc = ChannelDesc(e.shortChannelId, sourceNodeId, e.targetNodeId), params = HopRelayParams.FromHint(e), // Routing hints don't include the channel's capacity, so we assume it's big enough. capacity = maxBtc.toSatoshi, balance_opt = None, - ) + )) } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala index b0b1e4c4c9..be54356e94 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala @@ -22,16 +22,18 @@ import com.softwaremill.quicklens.ModifyPimp import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.eclair.Logs.LogCategory import fr.acinq.eclair._ -import fr.acinq.eclair.message.SendingMessage +import fr.acinq.eclair.crypto.Sphinx.RouteBlinding import fr.acinq.eclair.payment.send._ import fr.acinq.eclair.router.Graph.GraphStructure.DirectedGraph.graphEdgeToHop import fr.acinq.eclair.router.Graph.GraphStructure.{DirectedGraph, GraphEdge} import fr.acinq.eclair.router.Graph.{InfiniteLoop, MessagePath, NegativeProbability, RichWeight} import fr.acinq.eclair.router.Monitoring.{Metrics, Tags} import fr.acinq.eclair.router.Router._ +import fr.acinq.eclair.wire.protocol.OfferTypes import kamon.tag.TagSet import scala.annotation.tailrec +import scala.collection.immutable.SortedMap import scala.collection.mutable import scala.util.{Failure, Random, Success, Try} @@ -66,7 +68,7 @@ object RouteCalculation { paymentHash_opt = fr.paymentContext.map(_.paymentHash))) { implicit val sender: ActorRef = ctx.self // necessary to preserve origin when sending messages to other actors - val extraEdges = fr.extraEdges.map(GraphEdge(_)) + val extraEdges = fr.extraEdges.flatMap(GraphEdge.fromExtraEdge(_, d.channels)) val g = extraEdges.foldLeft(d.graphWithBalances.graph) { case (g: DirectedGraph, e: GraphEdge) => g.addEdge(e) } fr.route match { @@ -129,7 +131,7 @@ object RouteCalculation { * * The routes found must then be post-processed by calling [[addFinalHop]]. */ - private def computeTarget(r: RouteRequest, ignoredEdges: Set[ChannelDesc]): (PublicKey, MilliSatoshi, MilliSatoshi, Set[GraphEdge]) = { + private def computeTarget(r: RouteRequest, ignoredEdges: Set[ChannelDesc], publicChannels: SortedMap[RealShortChannelId, PublicChannel]): (PublicKey, MilliSatoshi, MilliSatoshi, Set[GraphEdge]) = { val pendingAmount = r.pendingPayments.map(_.amount).sum val totalMaxFee = r.routeParams.getMaxFee(r.target.totalAmount) val pendingChannelFee = r.pendingPayments.map(_.channelFee(r.routeParams.includeLocalChannelCost)).sum @@ -139,8 +141,8 @@ object RouteCalculation { val amountToSend = recipient.totalAmount - pendingAmount val maxFee = totalMaxFee - pendingChannelFee val extraEdges = recipient.extraEdges - .filter(_.sourceNodeId != r.source) // we ignore routing hints for our own channels, we have more accurate information - .map(GraphEdge(_)) + .flatMap(GraphEdge.fromExtraEdge(_, publicChannels)) + .filterNot(e => (e.desc.a == r.source) || (e.desc.b == r.source)) // we ignore routing hints for our own channels, we have more accurate information .filterNot(e => ignoredEdges.contains(e.desc)) .toSet (targetNodeId, amountToSend, maxFee, extraEdges) @@ -156,7 +158,7 @@ object RouteCalculation { .map(_.copy(targetNodeId = targetNodeId)) .filterNot(e => ignoredEdges.exists(_.shortChannelId == e.shortChannelId)) // For blinded routes, the maximum htlc field is used to indicate the maximum amount that can be sent through the route. - .map(e => GraphEdge(e).copy(balance_opt = e.htlcMaximum_opt)) + .flatMap(e => GraphEdge.fromExtraEdge(e, publicChannels).map(_.copy(balance_opt = e.htlcMaximum_opt))) .toSet val amountToSend = recipient.totalAmount - pendingAmount // When we are the introduction node and includeLocalChannelCost is false, we cannot easily remove the fee for @@ -182,11 +184,16 @@ object RouteCalculation { case _: SpontaneousRecipient => Some(route) case recipient: ClearTrampolineRecipient => Some(route.copy(finalHop_opt = Some(recipient.trampolineHop))) case recipient: BlindedRecipient => - route.hops.lastOption.flatMap { - hop => recipient.blindedHops.find(_.dummyId == hop.shortChannelId) - }.map { - blindedHop => Route(route.amount, route.hops.dropRight(1), Some(blindedHop)) - } + route.hops.lastOption.flatMap(lastHop =>{ + val alias = Alias(lastHop.shortChannelId.toLong) + recipient.blindedPaths.get(alias).map(path => { + val blindedRoute = path.route match { + case OfferTypes.BlindedPath(blindedRoute) => blindedRoute + case OfferTypes.CompactBlindedPath(_, blindingKey, blindedNodes) => RouteBlinding.BlindedRoute(lastHop.nodeId, blindingKey, blindedNodes) + } + Route(route.amount, route.hops.dropRight(1), Some(BlindedHop(alias, blindedRoute, path.paymentInfo))) + }) + }) } }) } @@ -200,7 +207,7 @@ object RouteCalculation { implicit val sender: ActorRef = ctx.self // necessary to preserve origin when sending messages to other actors val ignoredEdges = r.ignore.channels ++ d.excludedChannels.keySet - val (targetNodeId, amountToSend, maxFee, extraEdges) = computeTarget(r, ignoredEdges) + val (targetNodeId, amountToSend, maxFee, extraEdges) = computeTarget(r, ignoredEdges, d.channels) val routesToFind = if (r.routeParams.randomize) DEFAULT_ROUTES_COUNT else 1 log.info(s"finding routes ${r.source}->$targetNodeId with assistedChannels={} ignoreNodes={} ignoreChannels={} excludedChannels={}", extraEdges.map(_.desc.shortChannelId).mkString(","), r.ignore.nodes.map(_.value).mkString(","), r.ignore.channels.mkString(","), d.excludedChannels.mkString(",")) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala index 142e4a2a45..e5157c2873 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala @@ -388,8 +388,9 @@ trait ChannelStateTestsBase extends Assertions with Eventually { } def makeSingleHopRoute(amount: MilliSatoshi, destination: PublicKey): Route = { - val dummyParams = HopRelayParams.FromHint(Invoice.ExtraEdge(randomKey().publicKey, destination, ShortChannelId(0), 0 msat, 0, CltvExpiryDelta(0), 0 msat, None)) - Route(amount, Seq(ChannelHop(ShortChannelId(0), dummyParams.extraHop.sourceNodeId, dummyParams.extraHop.targetNodeId, dummyParams)), None) + val dummyNodeId = randomKey().publicKey + val dummyParams = HopRelayParams.FromHint(Invoice.ExtraEdge(Right(dummyNodeId), destination, ShortChannelId(0), 0 msat, 0, CltvExpiryDelta(0), 0 msat, None)) + Route(amount, Seq(ChannelHop(ShortChannelId(0), dummyNodeId, dummyParams.extraHop.targetNodeId, dummyParams)), None) } def addHtlc(amount: MilliSatoshi, s: TestFSMRef[ChannelState, ChannelData, Channel], r: TestFSMRef[ChannelState, ChannelData, Channel], s2r: TestProbe, r2s: TestProbe): (ByteVector32, UpdateAddHtlc) = { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala index 8589fbd137..81d731d2a1 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala @@ -801,7 +801,7 @@ object PaymentsDbSpec { def createBolt12Invoice(amount: MilliSatoshi, payerKey: PrivateKey, recipientKey: PrivateKey, preimage: ByteVector32): Bolt12Invoice = { val offer = Offer(Some(amount), "some offer", recipientKey.publicKey, Features.empty, Block.TestnetGenesisBlock.hash) val invoiceRequest = InvoiceRequest(offer, 789 msat, 1, Features.empty, payerKey, Block.TestnetGenesisBlock.hash) - val dummyRoute = PaymentBlindedRoute(RouteBlinding.create(randomKey(), Seq(randomKey().publicKey), Seq(randomBytes(100))).route, PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 0 msat, Features.empty)) + val dummyRoute = PaymentBlindedRoute(BlindedPath(RouteBlinding.create(randomKey(), Seq(randomKey().publicKey), Seq(randomBytes(100))).route), PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 0 msat, Features.empty)) Bolt12Invoice(invoiceRequest, preimage, recipientKey, 1 hour, Features.empty, Seq(dummyRoute)) } } 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 e0ef583e04..36ed539858 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 @@ -44,7 +44,7 @@ import fr.acinq.eclair.router.Graph.WeightRatios import fr.acinq.eclair.router.Router.{GossipDecision, PublicChannel} import fr.acinq.eclair.router.{Announcements, AnnouncementsBatchValidationSpec, Router} import fr.acinq.eclair.wire.protocol.OfferTypes.{Offer, OfferPaths} -import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelUpdate, IncorrectOrUnknownPaymentDetails} +import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelUpdate, IncorrectOrUnknownPaymentDetails, OfferTypes} import fr.acinq.eclair.{CltvExpiryDelta, EclairImpl, Features, Kit, MilliSatoshiLong, ShortChannelId, TimestampMilli, randomBytes32, randomKey} import org.json4s.JsonAST.{JString, JValue} import scodec.bits.{ByteVector, HexStringSyntax} @@ -692,9 +692,9 @@ class PaymentIntegrationSpec extends IntegrationSpec { val chain = nodes("D").nodeParams.chainHash val pathId = randomBytes32() val offerPaths = Seq( - buildRoute(randomKey(), Seq(IntermediateNode(nodes("G").nodeParams.nodeId), IntermediateNode(nodes("C").nodeParams.nodeId)), Recipient(nodes("D").nodeParams.nodeId, Some(pathId))), - buildRoute(randomKey(), Seq(IntermediateNode(nodes("B").nodeParams.nodeId), IntermediateNode(nodes("C").nodeParams.nodeId)), Recipient(nodes("D").nodeParams.nodeId, Some(pathId))), - buildRoute(randomKey(), Seq(IntermediateNode(nodes("E").nodeParams.nodeId), IntermediateNode(nodes("C").nodeParams.nodeId)), Recipient(nodes("D").nodeParams.nodeId, Some(pathId))) + OfferTypes.BlindedPath(buildRoute(randomKey(), Seq(IntermediateNode(nodes("G").nodeParams.nodeId), IntermediateNode(nodes("C").nodeParams.nodeId)), Recipient(nodes("D").nodeParams.nodeId, Some(pathId)))), + OfferTypes.BlindedPath(buildRoute(randomKey(), Seq(IntermediateNode(nodes("B").nodeParams.nodeId), IntermediateNode(nodes("C").nodeParams.nodeId)), Recipient(nodes("D").nodeParams.nodeId, Some(pathId)))), + OfferTypes.BlindedPath(buildRoute(randomKey(), Seq(IntermediateNode(nodes("E").nodeParams.nodeId), IntermediateNode(nodes("C").nodeParams.nodeId)), Recipient(nodes("D").nodeParams.nodeId, Some(pathId)))) ) val offer = Offer(Some(amount), "test offer", recipientKey.publicKey, nodes("D").nodeParams.features.bolt12Features(), chain, additionalTlvs = Set(OfferPaths(offerPaths))) val offerHandler = TypedProbe[HandlerCommand]()(nodes("D").system.toTyped) @@ -764,7 +764,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { val amount = 50_000_000 msat val chain = nodes("A").nodeParams.chainHash val pathId = randomBytes32() - val offerPath = buildRoute(randomKey(), Seq(IntermediateNode(nodes("A").nodeParams.nodeId), IntermediateNode(nodes("A").nodeParams.nodeId)), Recipient(nodes("A").nodeParams.nodeId, Some(pathId))) + val offerPath = OfferTypes.BlindedPath(buildRoute(randomKey(), Seq(IntermediateNode(nodes("A").nodeParams.nodeId), IntermediateNode(nodes("A").nodeParams.nodeId)), Recipient(nodes("A").nodeParams.nodeId, Some(pathId)))) val offer = Offer(Some(amount), "test offer", recipientKey.publicKey, nodes("A").nodeParams.features.bolt12Features(), chain, additionalTlvs = Set(OfferPaths(Seq(offerPath)))) val offerHandler = TypedProbe[HandlerCommand]()(nodes("A").system.toTyped) nodes("A").offerManager ! RegisterOffer(offer, recipientKey, Some(pathId), offerHandler.ref) @@ -798,7 +798,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { val amount = 10_000_000 msat val chain = nodes("C").nodeParams.chainHash val pathId = randomBytes32() - val offerPath = buildRoute(randomKey(), Seq(IntermediateNode(nodes("B").nodeParams.nodeId), IntermediateNode(nodes("C").nodeParams.nodeId)), Recipient(nodes("C").nodeParams.nodeId, Some(pathId))) + val offerPath = OfferTypes.BlindedPath(buildRoute(randomKey(), Seq(IntermediateNode(nodes("B").nodeParams.nodeId), IntermediateNode(nodes("C").nodeParams.nodeId)), Recipient(nodes("C").nodeParams.nodeId, Some(pathId)))) val offer = Offer(Some(amount), "tricky test offer", recipientKey.publicKey, nodes("C").nodeParams.features.bolt12Features(), chain, additionalTlvs = Set(OfferPaths(Seq(offerPath)))) val offerHandler = TypedProbe[HandlerCommand]()(nodes("C").system.toTyped) nodes("C").offerManager ! RegisterOffer(offer, recipientKey, Some(pathId), offerHandler.ref) 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 fb57e6b07c..85a54b51f7 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 @@ -40,6 +40,8 @@ import fr.acinq.eclair.testutils.FixtureSpec import fr.acinq.eclair.wire.protocol.OfferTypes.{Offer, OfferPaths} import fr.acinq.eclair.wire.protocol.{IncorrectOrUnknownPaymentDetails, InvalidOnionBlinding} import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, Features, MilliSatoshi, MilliSatoshiLong, ShortChannelId, randomBytes32, randomKey} +import fr.acinq.eclair.wire.protocol.{IncorrectOrUnknownPaymentDetails, InvalidOnionBlinding, OfferTypes} +import fr.acinq.eclair.{CltvExpiryDelta, Features, MilliSatoshi, MilliSatoshiLong, randomBytes32, randomKey} import org.scalatest.concurrent.IntegrationPatience import org.scalatest.{Tag, TestData} import scodec.bits.HexStringSyntax @@ -141,7 +143,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val offerPaths = routes.map(route => { val ourNodeId = route.nodes.last val intermediateNodes = route.nodes.dropRight(1).map(IntermediateNode(_)) ++ route.dummyHops.map(_ => IntermediateNode(ourNodeId)) - buildRoute(randomKey(), intermediateNodes, Recipient(ourNodeId, Some(pathId))) + OfferTypes.BlindedPath(buildRoute(randomKey(), intermediateNodes, Recipient(ourNodeId, Some(pathId)))) }) val offer = Offer(None, "test", recipientKey.publicKey, Features.empty, recipient.nodeParams.chainHash, additionalTlvs = Set(OfferPaths(offerPaths))) val handler = recipient.system.spawnAnonymous(offerHandler(amount, routes)) @@ -360,10 +362,10 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience { val recipientKey = randomKey() val pathId = randomBytes32() - val blindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(bob.nodeId), IntermediateNode(carol.nodeId)), Recipient(carol.nodeId, Some(pathId))) + val blindedRoute = OfferTypes.BlindedPath(buildRoute(randomKey(), Seq(IntermediateNode(bob.nodeId), IntermediateNode(carol.nodeId)), Recipient(carol.nodeId, Some(pathId)))) val offer = Offer(None, "test", recipientKey.publicKey, Features.empty, carol.nodeParams.chainHash, additionalTlvs = Set(OfferPaths(Seq(blindedRoute)))) val scid_bc = getPeerChannels(bob, carol.nodeId).head.data.asInstanceOf[DATA_NORMAL].shortIds.real.toOption.get - val compactBlindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(bob.nodeId, Some(scid_bc)), IntermediateNode(carol.nodeId, Some(ShortChannelId.toSelf))), Recipient(carol.nodeId, Some(pathId))) + val compactBlindedRoute = OfferTypes.BlindedPath(buildRoute(randomKey(), Seq(IntermediateNode(bob.nodeId, Some(scid_bc)), IntermediateNode(carol.nodeId, Some(ShortChannelId.toSelf))), Recipient(carol.nodeId, Some(pathId)))) val compactOffer = Offer(None, "test", recipientKey.publicKey, Features.empty, carol.nodeParams.chainHash, additionalTlvs = Set(OfferPaths(Seq(compactBlindedRoute)))) assert(compactOffer.toString.length < offer.toString.length) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala index e9c48d4c41..0134d2e0ab 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala @@ -30,6 +30,7 @@ import fr.acinq.eclair.message.OnionMessages.RoutingStrategy.FindRoute import fr.acinq.eclair.message.OnionMessages.{BlindedPath, IntermediateNode, ReceiveMessage, Recipient, buildMessage, buildRoute} import fr.acinq.eclair.message.Postman._ import fr.acinq.eclair.payment.offer.OfferManager.RequestInvoice +import fr.acinq.eclair.router.Router import fr.acinq.eclair.router.Router.{MessageRoute, MessageRouteRequest} import fr.acinq.eclair.wire.protocol.OnionMessagePayloadTlv.{InvoiceRequest, ReplyPath} import fr.acinq.eclair.wire.protocol.RouteBlindingEncryptedDataTlv.PathId @@ -41,14 +42,14 @@ import scodec.bits.HexStringSyntax class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("application")) with FixtureAnyFunSuiteLike { - case class FixtureParam(postman: ActorRef[Command], nodeParams: NodeParams, messageSender: TestProbe[OnionMessageResponse], switchboard: TestProbe[Any], offerManager: TestProbe[RequestInvoice], router: TestProbe[MessageRouteRequest]) + case class FixtureParam(postman: ActorRef[Command], nodeParams: NodeParams, messageSender: TestProbe[OnionMessageResponse], switchboard: TestProbe[Any], offerManager: TestProbe[RequestInvoice], router: TestProbe[Router.PostmanRequest]) override def withFixture(test: OneArgTest): Outcome = { val nodeParams = TestConstants.Alice.nodeParams val messageSender = TestProbe[OnionMessageResponse]("messageSender") val switchboard = TestProbe[Any]("switchboard") val offerManager = TestProbe[RequestInvoice]("offerManager") - val router = TestProbe[MessageRouteRequest]("router") + val router = TestProbe[Router.PostmanRequest]("router") val register = TestProbe[Any]("register") val postman = testKit.spawn(Postman(nodeParams, switchboard.ref.toClassic, router.ref, register.ref.toClassic, offerManager.ref)) try { @@ -72,7 +73,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat val recipientKey = randomKey() - postman ! SendMessage(Recipient(recipientKey.publicKey, None), FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = true, messageSender.ref) + postman ! SendMessage(OfferTypes.RecipientNodeId(recipientKey.publicKey), FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = true, messageSender.ref) val MessageRouteRequest(waitingForRoute, source, target, _) = router.expectMessageType[MessageRouteRequest] assert(source == nodeParams.nodeId) @@ -100,7 +101,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat val recipientKey = randomKey() - postman ! SendMessage(Recipient(recipientKey.publicKey, None), FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = true, messageSender.ref) + postman ! SendMessage(OfferTypes.RecipientNodeId(recipientKey.publicKey), FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = true, messageSender.ref) val MessageRouteRequest(waitingForRoute, source, target, _) = router.expectMessageType[MessageRouteRequest] assert(source == nodeParams.nodeId) @@ -119,7 +120,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat val recipientKey = randomKey() - postman ! SendMessage(Recipient(recipientKey.publicKey, None), FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = true, messageSender.ref) + postman ! SendMessage(OfferTypes.RecipientNodeId(recipientKey.publicKey), FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = true, messageSender.ref) val MessageRouteRequest(waitingForRoute, source, target, _) = router.expectMessageType[MessageRouteRequest] assert(source == nodeParams.nodeId) @@ -146,7 +147,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat val recipientKey = randomKey() - postman ! SendMessage(Recipient(recipientKey.publicKey, None), FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = false, messageSender.ref) + postman ! SendMessage(OfferTypes.RecipientNodeId(recipientKey.publicKey), FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = false, messageSender.ref) val MessageRouteRequest(waitingForRoute, source, target, _) = router.expectMessageType[MessageRouteRequest] assert(source == nodeParams.nodeId) @@ -169,7 +170,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat val recipientKey = randomKey() val blindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(nodeParams.nodeId)), Recipient(recipientKey.publicKey, None)) - postman ! SendMessage(BlindedPath(blindedRoute), FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = false, messageSender.ref) + postman ! SendMessage(OfferTypes.BlindedPath(blindedRoute), FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = false, messageSender.ref) val Peer.RelayOnionMessage(messageId, message, Some(replyTo)) = expectRelayToConnected(switchboard, recipientKey.publicKey) replyTo ! Sent(messageId) @@ -199,7 +200,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat val (a, b, c, d) = (randomKey(), randomKey(), randomKey(), randomKey()) - postman ! SendMessage(Recipient(d.publicKey, None), FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(11), hex"012345"))), expectsReply = true, messageSender.ref) + postman ! SendMessage(OfferTypes.RecipientNodeId(d.publicKey), FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(11), hex"012345"))), expectsReply = true, messageSender.ref) val MessageRouteRequest(waitingForRoute, source, target, _) = router.expectMessageType[MessageRouteRequest] assert(source == nodeParams.nodeId) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt12InvoiceSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt12InvoiceSpec.scala index 34c19258a0..99d6d9c3c2 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt12InvoiceSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt12InvoiceSpec.scala @@ -51,7 +51,7 @@ class Bolt12InvoiceSpec extends AnyFunSuite { def createPaymentBlindedRoute(nodeId: PublicKey, sessionKey: PrivateKey = randomKey(), pathId: ByteVector = randomBytes32()): PaymentBlindedRoute = { val selfPayload = blindedRouteDataCodec.encode(TlvStream(PathId(pathId), PaymentConstraints(CltvExpiry(1234567), 0 msat), AllowedFeatures(Features.empty))).require.bytes - PaymentBlindedRoute(Sphinx.RouteBlinding.create(sessionKey, Seq(nodeId), Seq(selfPayload)).route, PaymentInfo(1 msat, 2, CltvExpiryDelta(3), 4 msat, 5 msat, Features.empty)) + PaymentBlindedRoute(OfferTypes.BlindedPath(Sphinx.RouteBlinding.create(sessionKey, Seq(nodeId), Seq(selfPayload)).route), PaymentInfo(1 msat, 2, CltvExpiryDelta(3), 4 msat, 5 msat, Features.empty)) } test("check invoice signature") { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala index b8dcf1c545..bce0ec6195 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala @@ -271,9 +271,9 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike val invoiceReq = InvoiceRequest(offer, 25_000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash) val router = TestProbe() val (a, b, c, d) = (randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, nodeParams.nodeId) - val hop_ab = Router.ChannelHop(ShortChannelId(1), a, b, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(a, b, ShortChannelId(1), 1000 msat, 0, CltvExpiryDelta(100), 1 msat, None))) - val hop_bd = Router.ChannelHop(ShortChannelId(2), b, d, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(b, d, ShortChannelId(2), 800 msat, 0, CltvExpiryDelta(50), 1 msat, None))) - val hop_cd = Router.ChannelHop(ShortChannelId(3), c, d, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(c, d, ShortChannelId(3), 0 msat, 0, CltvExpiryDelta(75), 1 msat, None))) + val hop_ab = Router.ChannelHop(ShortChannelId(1), a, b, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(Right(a), b, ShortChannelId(1), 1000 msat, 0, CltvExpiryDelta(100), 1 msat, None))) + val hop_bd = Router.ChannelHop(ShortChannelId(2), b, d, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(Right(b), d, ShortChannelId(2), 800 msat, 0, CltvExpiryDelta(50), 1 msat, None))) + val hop_cd = Router.ChannelHop(ShortChannelId(3), c, d, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(Right(c), d, ShortChannelId(3), 0 msat, 0, CltvExpiryDelta(75), 1 msat, None))) val receivingRoutes = Seq( ReceivingRoute(Seq(a, b, d), CltvExpiryDelta(100), Seq(DummyBlindedHop(150 msat, 0, CltvExpiryDelta(25)))), ReceivingRoute(Seq(c, d), CltvExpiryDelta(50), Seq(DummyBlindedHop(250 msat, 0, CltvExpiryDelta(10)), DummyBlindedHop(150 msat, 0, CltvExpiryDelta(80)))), @@ -294,19 +294,19 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike assert(invoice.description == Left("a blinded coffee please")) assert(invoice.invoiceRequest.offer == offer) assert(invoice.blindedPaths.length == 3) - assert(invoice.blindedPaths(0).route.blindedNodeIds.length == 4) - assert(invoice.blindedPaths(0).route.introductionNodeId == a) + assert(invoice.blindedPaths(0).route.asInstanceOf[OfferTypes.BlindedPath].route.blindedNodeIds.length == 4) + assert(invoice.blindedPaths(0).route.asInstanceOf[OfferTypes.BlindedPath].route.introductionNodeId == a) assert(invoice.blindedPaths(0).paymentInfo == PaymentInfo(1950 msat, 0, CltvExpiryDelta(193), 1 msat, 25_000 msat, Features.empty)) - assert(invoice.blindedPaths(1).route.blindedNodeIds.length == 4) - assert(invoice.blindedPaths(1).route.introductionNodeId == c) + assert(invoice.blindedPaths(1).route.asInstanceOf[OfferTypes.BlindedPath].route.blindedNodeIds.length == 4) + assert(invoice.blindedPaths(1).route.asInstanceOf[OfferTypes.BlindedPath].route.introductionNodeId == c) assert(invoice.blindedPaths(1).paymentInfo == PaymentInfo(400 msat, 0, CltvExpiryDelta(183), 1 msat, 25_000 msat, Features.empty)) - assert(invoice.blindedPaths(2).route.blindedNodeIds.length == 1) - assert(invoice.blindedPaths(2).route.introductionNodeId == d) + assert(invoice.blindedPaths(2).route.asInstanceOf[OfferTypes.BlindedPath].route.blindedNodeIds.length == 1) + assert(invoice.blindedPaths(2).route.asInstanceOf[OfferTypes.BlindedPath].route.introductionNodeId == d) assert(invoice.blindedPaths(2).paymentInfo == PaymentInfo(0 msat, 0, CltvExpiryDelta(18), 0 msat, 25_000 msat, Features.empty)) // Offer invoices shouldn't be stored in the DB until we receive a payment for it. assert(nodeParams.db.payments.getIncomingPayment(invoice.paymentHash).isEmpty) // Check that all non-final encrypted payloads for blinded routes have the same length. - assert(invoice.blindedPaths.flatMap(_.route.encryptedPayloads.dropRight(1)).map(_.length).toSet.size == 1) + assert(invoice.blindedPaths.flatMap(_.route.asInstanceOf[OfferTypes.BlindedPath].route.encryptedPayloads.dropRight(1)).map(_.length).toSet.size == 1) } test("Invoice generation with route blinding should fail when router returns an error") { f => @@ -317,7 +317,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike val invoiceReq = InvoiceRequest(offer, 25_000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash) val router = TestProbe() val (a, b, c) = (randomKey().publicKey, randomKey().publicKey, nodeParams.nodeId) - val hop_ac = Router.ChannelHop(ShortChannelId(1), a, c, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(a, c, ShortChannelId(1), 100 msat, 0, CltvExpiryDelta(50), 1 msat, None))) + val hop_ac = Router.ChannelHop(ShortChannelId(1), a, c, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(Right(a), c, ShortChannelId(1), 100 msat, 0, CltvExpiryDelta(50), 1 msat, None))) val receivingRoutes = Seq( ReceivingRoute(Seq(a, c), CltvExpiryDelta(100)), ReceivingRoute(Seq(b, c), CltvExpiryDelta(100)), diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala index ee5da5b8ff..d2dbba681e 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala @@ -351,7 +351,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS import f._ // The B -> E channel is private and provided in the invoice routing hints. - val extraEdge = ExtraEdge(b, e, hop_be.shortChannelId, hop_be.params.relayFees.feeBase, hop_be.params.relayFees.feeProportionalMillionths, hop_be.params.cltvExpiryDelta, hop_be.params.htlcMinimum, hop_be.params.htlcMaximum_opt) + val extraEdge = ExtraEdge(Right(b), e, hop_be.shortChannelId, hop_be.params.relayFees.feeBase, hop_be.params.relayFees.feeProportionalMillionths, hop_be.params.cltvExpiryDelta, hop_be.params.htlcMinimum, hop_be.params.htlcMaximum_opt) val recipient = ClearRecipient(e, Features.empty, finalAmount, expiry, randomBytes32(), Seq(extraEdge)) val payment = SendMultiPartPayment(sender.ref, recipient, 3, routeParams) sender.send(payFsm, payment) @@ -374,7 +374,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS import f._ // The B -> E channel is private and provided in the invoice routing hints. - val extraEdge = ExtraEdge(b, e, hop_be.shortChannelId, hop_be.params.relayFees.feeBase, hop_be.params.relayFees.feeProportionalMillionths, hop_be.params.cltvExpiryDelta, hop_be.params.htlcMinimum, hop_be.params.htlcMaximum_opt) + val extraEdge = ExtraEdge(Right(b), e, hop_be.shortChannelId, hop_be.params.relayFees.feeBase, hop_be.params.relayFees.feeProportionalMillionths, hop_be.params.cltvExpiryDelta, hop_be.params.htlcMinimum, hop_be.params.htlcMaximum_opt) val recipient = ClearRecipient(e, Features.empty, finalAmount, expiry, randomBytes32(), Seq(extraEdge)) val payment = SendMultiPartPayment(sender.ref, recipient, 3, routeParams) sender.send(payFsm, payment) @@ -417,9 +417,9 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS test("update routing hints") { () => val recipient = ClearRecipient(e, Features.empty, finalAmount, expiry, randomBytes32(), Seq( - ExtraEdge(a, b, ShortChannelId(1), 10 msat, 0, CltvExpiryDelta(12), 1 msat, None), - ExtraEdge(b, c, ShortChannelId(2), 0 msat, 100, CltvExpiryDelta(24), 1 msat, None), - ExtraEdge(a, c, ShortChannelId(3), 1 msat, 10, CltvExpiryDelta(144), 1 msat, None) + ExtraEdge(Right(a), b, ShortChannelId(1), 10 msat, 0, CltvExpiryDelta(12), 1 msat, None), + ExtraEdge(Right(b), c, ShortChannelId(2), 0 msat, 100, CltvExpiryDelta(24), 1 msat, None), + ExtraEdge(Right(a), c, ShortChannelId(3), 1 msat, 10, CltvExpiryDelta(144), 1 msat, None) )) def makeChannelUpdate(shortChannelId: ShortChannelId, feeBase: MilliSatoshi, feeProportional: Long, cltvExpiryDelta: CltvExpiryDelta): ChannelUpdate = { @@ -433,9 +433,9 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS UnreadableRemoteFailure(finalAmount, Nil) ) val extraEdges1 = Seq( - ExtraEdge(a, b, ShortChannelId(1), 10 msat, 0, CltvExpiryDelta(12), 1 msat, None), - ExtraEdge(b, c, ShortChannelId(2), 15 msat, 150, CltvExpiryDelta(48), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)), - ExtraEdge(a, c, ShortChannelId(3), 1 msat, 10, CltvExpiryDelta(144), 1 msat, None) + ExtraEdge(Right(a), b, ShortChannelId(1), 10 msat, 0, CltvExpiryDelta(12), 1 msat, None), + ExtraEdge(Right(b), c, ShortChannelId(2), 15 msat, 150, CltvExpiryDelta(48), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)), + ExtraEdge(Right(a), c, ShortChannelId(3), 1 msat, 10, CltvExpiryDelta(144), 1 msat, None) ) assert(extraEdges1.zip(PaymentFailure.updateExtraEdges(failures, recipient).extraEdges).forall { case (e1, e2) => e1 == e2 }) } @@ -447,9 +447,9 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS RemoteFailure(finalAmount, Nil, Sphinx.DecryptedFailurePacket(a, FeeInsufficient(100 msat, makeChannelUpdate(ShortChannelId(1), 23 msat, 23, CltvExpiryDelta(23))))), ) val extraEdges1 = Seq( - ExtraEdge(a, b, ShortChannelId(1), 23 msat, 23, CltvExpiryDelta(23), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)), - ExtraEdge(b, c, ShortChannelId(2), 21 msat, 21, CltvExpiryDelta(21), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)), - ExtraEdge(a, c, ShortChannelId(3), 22 msat, 22, CltvExpiryDelta(22), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)) + ExtraEdge(Right(a), b, ShortChannelId(1), 23 msat, 23, CltvExpiryDelta(23), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)), + ExtraEdge(Right(b), c, ShortChannelId(2), 21 msat, 21, CltvExpiryDelta(21), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)), + ExtraEdge(Right(a), c, ShortChannelId(3), 22 msat, 22, CltvExpiryDelta(22), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)) ) assert(extraEdges1.zip(PaymentFailure.updateExtraEdges(failures, recipient).extraEdges).forall { case (e1, e2) => e1 == e2 }) } 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 f2201cade2..d3e4c91188 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 @@ -296,7 +296,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike def createBolt12Invoice(features: Features[Bolt12Feature], payerKey: PrivateKey): Bolt12Invoice = { val offer = Offer(None, "Bolt12 r0cks", e, features, Block.RegtestGenesisBlock.hash) val invoiceRequest = InvoiceRequest(offer, finalAmount, 1, features, randomKey(), Block.RegtestGenesisBlock.hash) - val blindedRoute = BlindedRouteCreation.createBlindedRouteWithoutHops(e, hex"2a2a2a2a", 1 msat, CltvExpiry(500_000)).route + val blindedRoute = OfferTypes.BlindedPath(BlindedRouteCreation.createBlindedRouteWithoutHops(e, hex"2a2a2a2a", 1 msat, CltvExpiry(500_000)).route) val paymentInfo = OfferTypes.PaymentInfo(1_000 msat, 0, CltvExpiryDelta(24), 0 msat, finalAmount, Features.empty) Bolt12Invoice(invoiceRequest, paymentPreimage, priv_e.privateKey, 300 seconds, features, Seq(PaymentBlindedRoute(blindedRoute, paymentInfo))) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala index a598b0e583..1a5c309f35 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala @@ -197,7 +197,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { val recipientNodeId = randomKey().publicKey val route = PredefinedNodeRoute(defaultAmountMsat, Seq(a, b, c, recipientNodeId)) - val extraEdges = Seq(ExtraEdge(c, recipientNodeId, ShortChannelId(561), 1 msat, 100, CltvExpiryDelta(144), 1 msat, None)) + val extraEdges = Seq(ExtraEdge(Right(c), recipientNodeId, ShortChannelId(561), 1 msat, 100, CltvExpiryDelta(144), 1 msat, None)) val recipient = ClearRecipient(recipientNodeId, Features.empty, defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret, extraEdges) val request = SendPaymentToRoute(sender.ref, Left(route), recipient) @@ -594,8 +594,8 @@ class PaymentLifecycleSpec extends BaseRouterSpec { // we build an assisted route for channel bc and cd val recipient = ClearRecipient(d, Features.empty, defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret, Seq( - ExtraEdge(b, c, scid_bc, update_bc.feeBaseMsat, update_bc.feeProportionalMillionths, update_bc.cltvExpiryDelta, 1 msat, None), - ExtraEdge(c, d, scid_cd, update_cd.feeBaseMsat, update_cd.feeProportionalMillionths, update_cd.cltvExpiryDelta, 1 msat, None) + ExtraEdge(Right(b), c, scid_bc, update_bc.feeBaseMsat, update_bc.feeProportionalMillionths, update_bc.cltvExpiryDelta, 1 msat, None), + ExtraEdge(Right(c), d, scid_cd, update_cd.feeBaseMsat, update_cd.feeProportionalMillionths, update_cd.cltvExpiryDelta, 1 msat, None) )) val request = SendPaymentToNode(sender.ref, recipient, 5, defaultRouteParams) sender.send(paymentFSM, request) @@ -638,7 +638,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { // we build an assisted route for channel cd val recipient = ClearRecipient(d, Features.empty, defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret, Seq( - ExtraEdge(c, d, scid_cd, update_cd.feeBaseMsat, update_cd.feeProportionalMillionths, update_cd.cltvExpiryDelta, 1 msat, None) + ExtraEdge(Right(c), d, scid_cd, update_cd.feeBaseMsat, update_cd.feeProportionalMillionths, update_cd.cltvExpiryDelta, 1 msat, None) )) val request = SendPaymentToNode(sender.ref, recipient, 1, defaultRouteParams) sender.send(paymentFSM, request) 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 804e5d1d4f..a1d90a0974 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 @@ -28,7 +28,7 @@ import fr.acinq.eclair.payment.IncomingPaymentPacket.{ChannelRelayPacket, FinalP import fr.acinq.eclair.payment.OutgoingPaymentPacket._ import fr.acinq.eclair.payment.send.{BlindedRecipient, ClearRecipient, ClearTrampolineRecipient} import fr.acinq.eclair.router.BaseRouterSpec.{blindedRouteFromHops, channelHopFromUpdate} -import fr.acinq.eclair.router.BlindedRouteCreation +import fr.acinq.eclair.router.{BlindedRouteCreation, Router} import fr.acinq.eclair.router.Router.{NodeHop, Route} import fr.acinq.eclair.transactions.Transactions import fr.acinq.eclair.transactions.Transactions.InputInfo @@ -161,7 +161,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { test("build outgoing blinded payment") { val (invoice, route, recipient) = longBlindedHops(hex"deadbeef") assert(recipient.extraEdges.length == 1) - assert(recipient.extraEdges.head.sourceNodeId == c) + assert(recipient.extraEdges.head.sourceNodeId == Right(c)) assert(recipient.extraEdges.head.targetNodeId == invoice.nodeId) val Right(payment) = buildOutgoingPayment(ActorRef.noSender, priv_a.privateKey, Upstream.Local(UUID.randomUUID()), paymentHash, route, recipient) assert(payment.outgoingChannel == channelUpdate_ab.shortChannelId) @@ -217,12 +217,14 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { val features = Features[Bolt12Feature](BasicMultiPartPayment -> Optional) val offer = Offer(None, "Bolt12 r0cks", recipientKey.publicKey, features, Block.RegtestGenesisBlock.hash) val invoiceRequest = InvoiceRequest(offer, amount_bc, 1, features, randomKey(), Block.RegtestGenesisBlock.hash) - val blindedRoute = BlindedRouteCreation.createBlindedRouteWithoutHops(c, hex"deadbeef", 1 msat, CltvExpiry(500_000)).route + 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 recipient = BlindedRecipient(invoice, 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) + val (alias, blindedPath) = recipient.blindedPaths.head + val finalHop = Router.BlindedHop(alias, blindedPath.route.asInstanceOf[OfferTypes.BlindedPath].route, blindedPath.paymentInfo) + val Right(payment) = buildOutgoingPayment(ActorRef.noSender, priv_a.privateKey, Upstream.Local(UUID.randomUUID()), paymentHash, Route(amount_bc, hops, Some(finalHop)), recipient) assert(payment.outgoingChannel == channelUpdate_ab.shortChannelId) assert(payment.cmd.amount == amount_ab) assert(payment.cmd.cltvExpiry == expiry_ab) @@ -250,7 +252,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { test("build outgoing blinded payment starting at our node") { val (route, recipient) = singleBlindedHop(hex"123456") assert(recipient.extraEdges.length == 1) - assert(recipient.extraEdges.head.sourceNodeId == a) + assert(recipient.extraEdges.head.sourceNodeId == Right(a)) val Right(payment) = buildOutgoingPayment(ActorRef.noSender, priv_a.privateKey, Upstream.Local(UUID.randomUUID()), paymentHash, route, recipient) assert(payment.outgoingChannel == channelUpdate_ab.shortChannelId) assert(payment.cmd.amount == finalAmount) @@ -418,7 +420,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { assert(buildOutgoingPayment(ActorRef.noSender, priv_a.privateKey, Upstream.Local(UUID.randomUUID()), paymentHash, route, recipient).isRight) val routeMissingBlindedHop = route.copy(finalHop_opt = None) val Left(failure) = buildOutgoingPayment(ActorRef.noSender, priv_a.privateKey, Upstream.Local(UUID.randomUUID()), paymentHash, routeMissingBlindedHop, recipient) - assert(failure == MissingBlindedHop(Set(c))) + assert(failure == MissingBlindedHop) } test("fail to decrypt when the onion is invalid") { @@ -468,11 +470,13 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { val invoiceRequest = InvoiceRequest(offer, amount_bc, 1, features, randomKey(), Block.RegtestGenesisBlock.hash) // We send the wrong blinded payload to the introduction node. val tmpBlindedRoute = BlindedRouteCreation.createBlindedRouteFromHops(Seq(channelHopFromUpdate(b, c, channelUpdate_bc)), hex"deadbeef", 1 msat, CltvExpiry(500_000)).route - val blindedRoute = tmpBlindedRoute.copy(blindedNodes = tmpBlindedRoute.blindedNodes.reverse) + 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 recipient = BlindedRecipient(invoice, amount_bc, expiry_bc, Set.empty) - val route = Route(amount_bc, Seq(channelHopFromUpdate(a, b, channelUpdate_ab)), Some(recipient.blindedHops.head)) + val (alias, blindedPath) = recipient.blindedPaths.head + val finalHop = Router.BlindedHop(alias, blindedPath.route.asInstanceOf[OfferTypes.BlindedPath].route, blindedPath.paymentInfo) + val route = Route(amount_bc, Seq(channelHopFromUpdate(a, b, channelUpdate_ab)), Some(finalHop)) (route, recipient) } val Right(payment) = buildOutgoingPayment(ActorRef.noSender, priv_a.privateKey, Upstream.Local(UUID.randomUUID()), paymentHash, route, recipient) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/offer/OfferManagerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/offer/OfferManagerSpec.scala index bf273c2503..9f47005c96 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/offer/OfferManagerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/offer/OfferManagerSpec.scala @@ -88,7 +88,7 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app import f._ assert(invoice.blindedPaths.length == 1) - val blindedPath = invoice.blindedPaths.head.route + val blindedPath = invoice.blindedPaths.head.route.asInstanceOf[OfferTypes.BlindedPath].route val Right(RouteBlindingDecryptedData(encryptedDataTlvs, _)) = RouteBlindingEncryptedDataCodecs.decode(nodeParams.privateKey, blindedPath.blindingKey, blindedPath.encryptedPayloads.head) val paymentTlvs = TlvStream[OnionPaymentPayloadTlv]( OnionPaymentPayloadTlv.AmountToForward(invoice.amount), diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala index 19b88cf48d..71c7ab83ab 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala @@ -733,7 +733,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl assert(outgoingPayment.recipient.nodeId == outgoingNodeId) assert(outgoingPayment.recipient.totalAmount == outgoingAmount) assert(outgoingPayment.recipient.expiry == outgoingExpiry) - assert(outgoingPayment.recipient.extraEdges.head == ExtraEdge(hints.head.nodeId, outgoingNodeId, ShortChannelId(42), 10 msat, 1, CltvExpiryDelta(12), 1 msat, None)) + assert(outgoingPayment.recipient.extraEdges.head == ExtraEdge(Right(hints.head.nodeId), outgoingNodeId, ShortChannelId(42), 10 msat, 1, CltvExpiryDelta(12), 1 msat, None)) assert(outgoingPayment.recipient.isInstanceOf[ClearRecipient]) val recipient = outgoingPayment.recipient.asInstanceOf[ClearRecipient] assert(recipient.nextTrampolineOnion_opt.isEmpty) @@ -777,7 +777,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl assert(outgoingPayment.recipient.nodeId == outgoingNodeId) assert(outgoingPayment.amount == outgoingAmount) assert(outgoingPayment.recipient.expiry == outgoingExpiry) - assert(outgoingPayment.recipient.extraEdges.head == ExtraEdge(hints.head.nodeId, outgoingNodeId, ShortChannelId(42), 10 msat, 1, CltvExpiryDelta(12), 1 msat, None)) + assert(outgoingPayment.recipient.extraEdges.head == ExtraEdge(Right(hints.head.nodeId), outgoingNodeId, ShortChannelId(42), 10 msat, 1, CltvExpiryDelta(12), 1 msat, None)) assert(outgoingPayment.recipient.isInstanceOf[ClearRecipient]) val recipient = outgoingPayment.recipient.asInstanceOf[ClearRecipient] assert(recipient.nextTrampolineOnion_opt.isEmpty) 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 a6eb63626b..9dfd689c33 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 @@ -32,7 +32,7 @@ import fr.acinq.eclair.payment.{Bolt12Invoice, PaymentBlindedRoute} 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.{OnionMessagePayloadTlv, TlvStream} +import fr.acinq.eclair.wire.protocol.{OfferTypes, OnionMessagePayloadTlv, TlvStream} import fr.acinq.eclair.{CltvExpiryDelta, Features, MilliSatoshiLong, NodeParams, TestConstants, randomBytes32, randomKey} import org.scalatest.Outcome import org.scalatest.funsuite.FixtureAnyFunSuiteLike @@ -67,14 +67,14 @@ class OfferPaymentSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app val offer = Offer(None, "amountless 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(Recipient(recipientId, _, _, _), FindRoute, message, expectsReply, replyTo) = postman.expectMessageType[Postman.SendMessage] + 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 paymentRoute = PaymentBlindedRoute(RouteBlinding.create(randomKey(), Seq(merchantKey.publicKey), Seq(hex"7777")).route, PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 1_000_000_000 msat, Features.empty)) + val paymentRoute = PaymentBlindedRoute(OfferTypes.BlindedPath(RouteBlinding.create(randomKey(), Seq(merchantKey.publicKey), Seq(hex"7777")).route), PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 1_000_000_000 msat, Features.empty)) val invoice = Bolt12Invoice(invoiceRequest, preimage, merchantKey, 1 minute, Features.empty, Seq(paymentRoute)) replyTo ! Postman.Response(InvoicePayload(TlvStream(OnionMessagePayloadTlv.Invoice(invoice.records)), TlvStream.empty)) val send = paymentInitiator.expectMsgType[SendPaymentToNode] @@ -92,7 +92,7 @@ class OfferPaymentSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app val offer = Offer(None, "amountless 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)) for (_ <- 1 to nodeParams.onionMessageConfig.maxAttempts) { - val Postman.SendMessage(Recipient(recipientId, _, _, _), FindRoute, message, expectsReply, replyTo) = postman.expectMessageType[Postman.SendMessage] + 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) @@ -114,14 +114,14 @@ class OfferPaymentSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app val offer = Offer(None, "amountless 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(Recipient(recipientId, _, _, _), FindRoute, message, expectsReply, replyTo) = postman.expectMessageType[Postman.SendMessage] + 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 paymentRoute = PaymentBlindedRoute(RouteBlinding.create(randomKey(), Seq(merchantKey.publicKey), Seq(hex"7777")).route, PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 1_000_000_000 msat, Features.empty)) + val paymentRoute = PaymentBlindedRoute(OfferTypes.BlindedPath(RouteBlinding.create(randomKey(), Seq(merchantKey.publicKey), Seq(hex"7777")).route), PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 1_000_000_000 msat, Features.empty)) val invoice = Bolt12Invoice(invoiceRequest, preimage, randomKey(), 1 minute, Features.empty, Seq(paymentRoute)) replyTo ! Postman.Response(InvoicePayload(TlvStream(OnionMessagePayloadTlv.Invoice(invoice.records)), TlvStream.empty)) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/BalanceEstimateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/BalanceEstimateSpec.scala index 9784f93345..3469afb1e5 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/BalanceEstimateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/BalanceEstimateSpec.scala @@ -38,7 +38,7 @@ class BalanceEstimateSpec extends AnyFunSuite { def makeEdge(nodeId1: PublicKey, nodeId2: PublicKey, channelId: Long, capacity: Satoshi): GraphEdge = GraphEdge( ChannelDesc(ShortChannelId(channelId), nodeId1, nodeId2), - HopRelayParams.FromHint(Invoice.ExtraEdge(nodeId1, nodeId2, ShortChannelId(channelId), 0 msat, 0, CltvExpiryDelta(0), 0 msat, None)), + HopRelayParams.FromHint(Invoice.ExtraEdge(Right(nodeId1), nodeId2, ShortChannelId(channelId), 0 msat, 0, CltvExpiryDelta(0), 0 msat, None)), capacity, None) def makeEdge(channelId: Long, capacity: Satoshi): GraphEdge = 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 e1f716739b..35575ab3d9 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 @@ -254,7 +254,8 @@ object BaseRouterSpec { preimage: ByteVector32 = randomBytes32(), pathId: ByteVector = randomBytes(32)): (Bolt12Invoice, BlindedHop, BlindedRecipient) = { val (invoice, recipient) = blindedRoutesFromPaths(amount, expiry, Seq(hops), routeExpiry, preimage, pathId) - (invoice, recipient.blindedHops.head, recipient) + val (alias, blindedPath) = recipient.blindedPaths.head + (invoice, BlindedHop(alias, blindedPath.route.asInstanceOf[OfferTypes.BlindedPath].route, blindedPath.paymentInfo), recipient) } def blindedRoutesFromPaths(amount: MilliSatoshi, @@ -270,7 +271,7 @@ object BaseRouterSpec { val offer = Offer(None, "Bolt12 r0cks", recipientKey.publicKey, features, Block.RegtestGenesisBlock.hash) val invoiceRequest = InvoiceRequest(offer, amount, 1, features, randomKey(), Block.RegtestGenesisBlock.hash) val blindedRoutes = paths.map(hops => { - val blindedRoute = BlindedRouteCreation.createBlindedRouteFromHops(hops, pathId, 1 msat, routeExpiry).route + val blindedRoute = OfferTypes.BlindedPath(BlindedRouteCreation.createBlindedRouteFromHops(hops, pathId, 1 msat, routeExpiry).route) val paymentInfo = BlindedRouteCreation.aggregatePaymentInfo(amount, hops, Channel.MIN_CLTV_EXPIRY_DELTA) PaymentBlindedRoute(blindedRoute, paymentInfo) }) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala index a8c82aa9be..67b106b944 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala @@ -541,8 +541,8 @@ class RouterSpec extends BaseRouterSpec { val sender = TestProbe() val r = randomKey().publicKey val hopsToRecipient = Seq( - ChannelHop(ShortChannelId(10000), b, r, HopRelayParams.FromHint(ExtraEdge(b, r, ShortChannelId(10000), 800 msat, 0, CltvExpiryDelta(36), 1 msat, Some(400_000 msat)))) :: Nil, - ChannelHop(ShortChannelId(10001), c, r, HopRelayParams.FromHint(ExtraEdge(c, r, ShortChannelId(10001), 500 msat, 0, CltvExpiryDelta(36), 1 msat, Some(400_000 msat)))) :: Nil, + ChannelHop(ShortChannelId(10000), b, r, HopRelayParams.FromHint(ExtraEdge(Right(b), r, ShortChannelId(10000), 800 msat, 0, CltvExpiryDelta(36), 1 msat, Some(400_000 msat)))) :: Nil, + ChannelHop(ShortChannelId(10001), c, r, HopRelayParams.FromHint(ExtraEdge(Right(c), r, ShortChannelId(10001), 500 msat, 0, CltvExpiryDelta(36), 1 msat, Some(400_000 msat)))) :: Nil, ) { @@ -551,14 +551,14 @@ class RouterSpec extends BaseRouterSpec { sender.send(router, RouteRequest(a, recipient, DEFAULT_ROUTE_PARAMS, allowMultiPart = true)) val routes = sender.expectMsgType[RouteResponse].routes assert(routes.length == 2) - assert(routes.flatMap(_.finalHop_opt) == recipient.blindedHops) + assert(routes.flatMap(_.finalHop_opt) == recipient.blindedPaths.map{case (alias, blindedPath) => BlindedHop(alias, blindedPath.route.asInstanceOf[OfferTypes.BlindedPath].route, blindedPath.paymentInfo)}) assert(routes.map(route => route2NodeIds(route)).toSet == Set(Seq(a, b), Seq(a, b, c))) assert(routes.map(route => route.blindedFee + route.channelFee(false)).toSet == Set(510 msat, 800 msat)) } { // One blinded route is ignored, we use the other one: val (_, recipient) = blindedRoutesFromPaths(300_000 msat, DEFAULT_EXPIRY, hopsToRecipient, DEFAULT_EXPIRY) - val ignored = Ignore(Set.empty, Set(ChannelDesc(recipient.extraEdges.last.shortChannelId, recipient.extraEdges.last.sourceNodeId, recipient.extraEdges.last.targetNodeId))) + val ignored = Ignore(Set.empty, Set(ChannelDesc(recipient.extraEdges.last.shortChannelId, recipient.extraEdges.last.sourceNodeId.toOption.get, recipient.extraEdges.last.targetNodeId))) sender.send(router, RouteRequest(a, recipient, DEFAULT_ROUTE_PARAMS, ignore = ignored)) val routes = sender.expectMsgType[RouteResponse].routes assert(routes.length == 1) @@ -569,7 +569,7 @@ class RouterSpec extends BaseRouterSpec { { // One blinded route is ignored, the other one doesn't have enough capacity: val (_, recipient) = blindedRoutesFromPaths(500_000 msat, DEFAULT_EXPIRY, hopsToRecipient, DEFAULT_EXPIRY) - val ignored = Ignore(Set.empty, Set(ChannelDesc(recipient.extraEdges.last.shortChannelId, recipient.extraEdges.last.sourceNodeId, recipient.extraEdges.last.targetNodeId))) + val ignored = Ignore(Set.empty, Set(ChannelDesc(recipient.extraEdges.last.shortChannelId, recipient.extraEdges.last.sourceNodeId.toOption.get, recipient.extraEdges.last.targetNodeId))) sender.send(router, RouteRequest(a, recipient, DEFAULT_ROUTE_PARAMS, allowMultiPart = true, ignore = ignored)) sender.expectMsg(Failure(RouteNotFound)) } @@ -834,7 +834,7 @@ class RouterSpec extends BaseRouterSpec { { val amount = 10_000.msat - val invoiceRoutingHint = Invoice.ExtraEdge(b, targetNodeId, RealShortChannelId(BlockHeight(420000), 516, 1105), 10 msat, 150, CltvExpiryDelta(96), 1 msat, None) + val invoiceRoutingHint = Invoice.ExtraEdge(Right(b), targetNodeId, RealShortChannelId(BlockHeight(420000), 516, 1105), 10 msat, 150, CltvExpiryDelta(96), 1 msat, None) val preComputedRoute = PredefinedChannelRoute(amount, targetNodeId, Seq(scid_ab, invoiceRoutingHint.shortChannelId)) // the amount affects the way we estimate the channel capacity of the hinted channel assert(amount < RoutingHeuristics.CAPACITY_CHANNEL_LOW) @@ -850,7 +850,7 @@ class RouterSpec extends BaseRouterSpec { } { val amount = RoutingHeuristics.CAPACITY_CHANNEL_LOW * 2 - val invoiceRoutingHint = Invoice.ExtraEdge(h, targetNodeId, RealShortChannelId(BlockHeight(420000), 516, 1105), 10 msat, 150, CltvExpiryDelta(96), 1 msat, None) + val invoiceRoutingHint = Invoice.ExtraEdge(Right(h), targetNodeId, RealShortChannelId(BlockHeight(420000), 516, 1105), 10 msat, 150, CltvExpiryDelta(96), 1 msat, None) val preComputedRoute = PredefinedChannelRoute(amount, targetNodeId, Seq(scid_ag_private, scid_gh, invoiceRoutingHint.shortChannelId)) // the amount affects the way we estimate the channel capacity of the hinted channel assert(amount > RoutingHeuristics.CAPACITY_CHANNEL_LOW) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/MessageOnionCodecsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/MessageOnionCodecsSpec.scala index db27a01ddd..58b5d65e27 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/MessageOnionCodecsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/MessageOnionCodecsSpec.scala @@ -95,7 +95,7 @@ class MessageOnionCodecsSpec extends AnyFunSuiteLike { val payerKey = randomKey() val request = OfferTypes.InvoiceRequest(offer, 100_000 msat, 1, Features.empty, payerKey, Block.LivenetGenesisBlock.hash) val selfPayload = blindedRouteDataCodec.encode(TlvStream(PathId(randomBytes32()), PaymentConstraints(CltvExpiry(1234567), 0 msat), AllowedFeatures(Features.empty))).require.bytes - val route = PaymentBlindedRoute(Sphinx.RouteBlinding.create(randomKey(), Seq(nodeKey.publicKey), Seq(selfPayload)).route, PaymentInfo(1 msat, 2, CltvExpiryDelta(3), 4 msat, 5 msat, Features.empty)) + val route = PaymentBlindedRoute(OfferTypes.BlindedPath(Sphinx.RouteBlinding.create(randomKey(), Seq(nodeKey.publicKey), Seq(selfPayload)).route), PaymentInfo(1 msat, 2, CltvExpiryDelta(3), 4 msat, 5 msat, Features.empty)) val invoice = Bolt12Invoice(request, randomBytes32(), nodeKey, 300 seconds, Features.empty, Seq(route)) val testCasesInvalid = Seq[TlvStream[OnionMessagePayloadTlv]]( @@ -106,7 +106,7 @@ class MessageOnionCodecsSpec extends AnyFunSuiteLike { // Invoice and unknown TLV. TlvStream(Set[OnionMessagePayloadTlv](EncryptedData(hex""), Invoice(invoice.records)), Set(GenericTlv(UInt64(1), hex""))), // Invoice and ReplyPath. - TlvStream(EncryptedData(hex""), Invoice(invoice.records), ReplyPath(route.route)), + TlvStream(EncryptedData(hex""), Invoice(invoice.records), ReplyPath(route.route.asInstanceOf[OfferTypes.BlindedPath].route)), // Invoice and InvoiceError. TlvStream(EncryptedData(hex""), Invoice(invoice.records), InvoiceError(TlvStream(OfferTypes.Error("")))), // InvoiceRequest without ReplyPath. @@ -118,7 +118,7 @@ class MessageOnionCodecsSpec extends AnyFunSuiteLike { assert(finalPayload.isInstanceOf[InvalidResponsePayload]) } - val Right(invoiceRequestPayload) = FinalPayload.validate(TlvStream(EncryptedData(hex""), InvoiceRequest(request.records), ReplyPath(route.route)), TlvStream.empty) + val Right(invoiceRequestPayload) = FinalPayload.validate(TlvStream(EncryptedData(hex""), InvoiceRequest(request.records), ReplyPath(route.route.asInstanceOf[OfferTypes.BlindedPath].route)), TlvStream.empty) assert(invoiceRequestPayload.isInstanceOf[InvoiceRequestPayload]) val Right(invoicePayload) = FinalPayload.validate(TlvStream(EncryptedData(hex""), Invoice(invoice.records)), TlvStream.empty) From 69e320847c09df6df2f28f338f0e760c2efefa4a Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Thu, 26 Oct 2023 14:31:06 +0200 Subject: [PATCH 03/12] Test --- .../fr/acinq/eclair/message/PostmanSpec.scala | 57 ++++++++++++++++++- 1 file changed, 56 insertions(+), 1 deletion(-) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala index 0134d2e0ab..34f02e4db1 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala @@ -35,7 +35,7 @@ import fr.acinq.eclair.router.Router.{MessageRoute, MessageRouteRequest} import fr.acinq.eclair.wire.protocol.OnionMessagePayloadTlv.{InvoiceRequest, ReplyPath} import fr.acinq.eclair.wire.protocol.RouteBlindingEncryptedDataTlv.PathId import fr.acinq.eclair.wire.protocol.{GenericTlv, MessageOnion, OfferTypes, OnionMessagePayloadTlv, TlvStream} -import fr.acinq.eclair.{Features, MilliSatoshiLong, NodeParams, TestConstants, UInt64, randomKey} +import fr.acinq.eclair.{Features, MilliSatoshiLong, NodeParams, RealShortChannelId, TestConstants, UInt64, randomKey} import org.scalatest.Outcome import org.scalatest.funsuite.FixtureAnyFunSuiteLike import scodec.bits.HexStringSyntax @@ -236,4 +236,59 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat postman ! WrappedMessage(replyPayload) assert(replyPayload.records.unknown == Set(GenericTlv(UInt64(13), hex"6789"))) } + + test("send to compact route") { f => + import f._ + + val recipientKey = randomKey() + + val route = buildRoute(randomKey(), Seq(), Recipient(recipientKey.publicKey, None)) + val compactRoute = OfferTypes.CompactBlindedPath(OfferTypes.ShortChannelIdDir(isNode1 = false, RealShortChannelId(1234)), route.blindingKey, route.blindedNodes) + postman ! SendMessage(compactRoute, FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = false, messageSender.ref) + + val getNodeId = router.expectMessageType[Router.GetNodeId] + assert(!getNodeId.isNode1) + assert(getNodeId.shortChannelId == RealShortChannelId(1234)) + getNodeId.replyTo ! Some(recipientKey.publicKey) + + val MessageRouteRequest(waitingForRoute, source, target, _) = router.expectMessageType[MessageRouteRequest] + assert(source == nodeParams.nodeId) + assert(target == recipientKey.publicKey) + waitingForRoute ! MessageRoute(Seq.empty, target) + + val RelayMessage(messageId, _, nextNodeId, message, _, Some(replyTo)) = switchboard.expectMessageType[RelayMessage] + assert(nextNodeId == recipientKey.publicKey) + replyTo ! Sent(messageId) + val ReceiveMessage(finalPayload) = OnionMessages.process(recipientKey, message) + assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd"))) + assert(finalPayload.records.get[ReplyPath].isEmpty) + + messageSender.expectMessage(MessageSent) + messageSender.expectNoMessage() + } + + test("send to compact route that starts at ourselves") { f => + import f._ + + val recipientKey = randomKey() + + val route = buildRoute(randomKey(), Seq(IntermediateNode(nodeParams.nodeId)), Recipient(recipientKey.publicKey, None)) + val compactRoute = OfferTypes.CompactBlindedPath(OfferTypes.ShortChannelIdDir(isNode1 = true, RealShortChannelId(1234)), route.blindingKey, route.blindedNodes) + postman ! SendMessage(compactRoute, FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = false, messageSender.ref) + + val getNodeId = router.expectMessageType[Router.GetNodeId] + assert(getNodeId.isNode1) + assert(getNodeId.shortChannelId == RealShortChannelId(1234)) + getNodeId.replyTo ! Some(nodeParams.nodeId) + + val RelayMessage(messageId, _, nextNodeId, message, _, Some(replyTo)) = switchboard.expectMessageType[RelayMessage] + assert(nextNodeId == recipientKey.publicKey) + replyTo ! Sent(messageId) + val ReceiveMessage(finalPayload) = OnionMessages.process(recipientKey, message) + assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd"))) + assert(finalPayload.records.get[ReplyPath].isEmpty) + + messageSender.expectMessage(MessageSent) + messageSender.expectNoMessage() + } } From 075c1cec2cb0143b0d055a845ad6ce4787c58078 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Thu, 26 Oct 2023 17:08:51 +0200 Subject: [PATCH 04/12] Allow generating invoice with compact blinded route --- .../payment/receive/MultiPartHandler.scala | 18 +++++--- .../integration/PaymentIntegrationSpec.scala | 42 ++++++++++++++++++- 2 files changed, 54 insertions(+), 6 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala index 56ef89f6cb..82fee0f555 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala @@ -36,7 +36,7 @@ import fr.acinq.eclair.payment.offer.OfferManager import fr.acinq.eclair.router.BlindedRouteCreation.{aggregatePaymentInfo, createBlindedRouteFromHops, createBlindedRouteWithoutHops} import fr.acinq.eclair.router.Router import fr.acinq.eclair.router.Router.{ChannelHop, HopRelayParams} -import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, InvoiceTlv} +import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, InvoiceTlv, ShortChannelIdDir} import fr.acinq.eclair.wire.protocol.PaymentOnion.FinalPayload import fr.acinq.eclair.wire.protocol._ import fr.acinq.eclair.{Bolt11Feature, CltvExpiryDelta, FeatureSupport, Features, Logs, MilliSatoshi, MilliSatoshiLong, NodeParams, ShortChannelId, TimestampMilli, randomBytes32} @@ -276,7 +276,7 @@ object MultiPartHandler { * @param maxFinalExpiryDelta maximum expiry delta that senders can use: the route expiry will be computed based on this value. * @param dummyHops (optional) dummy hops to add to the blinded route. */ - case class ReceivingRoute(nodes: Seq[PublicKey], maxFinalExpiryDelta: CltvExpiryDelta, dummyHops: Seq[DummyBlindedHop] = Nil) + case class ReceivingRoute(nodes: Seq[PublicKey], maxFinalExpiryDelta: CltvExpiryDelta, dummyHops: Seq[DummyBlindedHop] = Nil, shortChannelIdDir_opt: Option[ShortChannelIdDir] = None) /** * Use this message to create a Bolt 12 invoice to receive a payment for a given offer. @@ -368,20 +368,28 @@ object MultiPartHandler { } else { createBlindedRouteFromHops(dummyHops, r.pathId, nodeParams.channelConf.htlcMinimum, route.maxFinalExpiryDelta.toCltvExpiry(nodeParams.currentBlockHeight)) } + val contactInfo = route.shortChannelIdDir_opt match { + case Some(shortChannelIdDir) => OfferTypes.CompactBlindedPath(shortChannelIdDir, blindedRoute.route.blindingKey, blindedRoute.route.blindedNodes) + case None => OfferTypes.BlindedPath(blindedRoute.route) + } val paymentInfo = aggregatePaymentInfo(r.amount, dummyHops, nodeParams.channelConf.minFinalExpiryDelta) - Future.successful((blindedRoute, paymentInfo, r.pathId)) + Future.successful(PaymentBlindedRoute(contactInfo, paymentInfo)) } else { implicit val timeout: Timeout = 10.seconds r.router.ask(Router.FinalizeRoute(Router.PredefinedNodeRoute(r.amount, route.nodes))).mapTo[Router.RouteResponse].map(routeResponse => { val clearRoute = routeResponse.routes.head val blindedRoute = createBlindedRouteFromHops(clearRoute.hops ++ dummyHops, r.pathId, nodeParams.channelConf.htlcMinimum, route.maxFinalExpiryDelta.toCltvExpiry(nodeParams.currentBlockHeight)) + val contactInfo = route.shortChannelIdDir_opt match { + case Some(shortChannelIdDir) => OfferTypes.CompactBlindedPath(shortChannelIdDir, blindedRoute.route.blindingKey, blindedRoute.route.blindedNodes) + case None => OfferTypes.BlindedPath(blindedRoute.route) + } val paymentInfo = aggregatePaymentInfo(r.amount, clearRoute.hops ++ dummyHops, nodeParams.channelConf.minFinalExpiryDelta) - (blindedRoute, paymentInfo, r.pathId) + PaymentBlindedRoute(contactInfo, paymentInfo) }) } })).map(paths => { val invoiceFeatures = nodeParams.features.bolt12Features() - val invoice = Bolt12Invoice(r.invoiceRequest, r.paymentPreimage, r.nodeKey, nodeParams.invoiceExpiry, invoiceFeatures, paths.map { case (blindedRoute, paymentInfo, _) => PaymentBlindedRoute(OfferTypes.BlindedPath(blindedRoute.route), paymentInfo) }, r.additionalTlvs, r.customTlvs) + val invoice = Bolt12Invoice(r.invoiceRequest, r.paymentPreimage, r.nodeKey, nodeParams.invoiceExpiry, invoiceFeatures, paths, r.additionalTlvs, r.customTlvs) log.debug("generated invoice={} for offer={}", invoice.toString, r.invoiceRequest.offer.toString) invoice }))(WrappedInvoiceResult) 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 36ed539858..2d432ac40e 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 @@ -43,7 +43,7 @@ import fr.acinq.eclair.payment.send.PaymentInitiator.{SendPaymentToNode, SendTra import fr.acinq.eclair.router.Graph.WeightRatios import fr.acinq.eclair.router.Router.{GossipDecision, PublicChannel} import fr.acinq.eclair.router.{Announcements, AnnouncementsBatchValidationSpec, Router} -import fr.acinq.eclair.wire.protocol.OfferTypes.{Offer, OfferPaths} +import fr.acinq.eclair.wire.protocol.OfferTypes.{CompactBlindedPath, Offer, OfferPaths, ShortChannelIdDir} import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelUpdate, IncorrectOrUnknownPaymentDetails, OfferTypes} import fr.acinq.eclair.{CltvExpiryDelta, EclairImpl, Features, Kit, MilliSatoshiLong, ShortChannelId, TimestampMilli, randomBytes32, randomKey} import org.json4s.JsonAST.{JString, JValue} @@ -827,6 +827,46 @@ class PaymentIntegrationSpec extends IntegrationSpec { assert(receivedAmount >= amount) } + test("send to compact route") { + val recipientKey = randomKey() + val amount = 10_000_000 msat + val chain = nodes("C").nodeParams.chainHash + val pathId = randomBytes32() + val offerPath = OfferTypes.BlindedPath(buildRoute(randomKey(), Seq(IntermediateNode(nodes("B").nodeParams.nodeId), IntermediateNode(nodes("C").nodeParams.nodeId)), Recipient(nodes("C").nodeParams.nodeId, Some(pathId)))) + val offer = Offer(Some(amount), "test offer", recipientKey.publicKey, nodes("C").nodeParams.features.bolt12Features(), chain, additionalTlvs = Set(OfferPaths(Seq(offerPath)))) + val offerHandler = TypedProbe[HandlerCommand]()(nodes("C").system.toTyped) + nodes("C").offerManager ! RegisterOffer(offer, recipientKey, Some(pathId), offerHandler.ref) + + val sender = TestProbe() + val alice = new EclairImpl(nodes("A")) + alice.payOfferBlocking(offer, amount, 1, maxAttempts_opt = Some(3))(30 seconds).pipeTo(sender.ref) + + val handleInvoiceRequest = offerHandler.expectMessageType[HandleInvoiceRequest] + val probe = TestProbe() + probe.send(nodes("C").router, Router.GetChannels) + val b = nodes("B").nodeParams.nodeId + val channelWithB = probe.expectMsgType[Iterable[ChannelAnnouncement]].find(ann => ann.nodeId1 == b || ann.nodeId2 == b).get + val receivingRoutes = Seq( + ReceivingRoute(Seq(nodes("B").nodeParams.nodeId, nodes("C").nodeParams.nodeId), CltvExpiryDelta(555), Seq(DummyBlindedHop(55 msat, 55, CltvExpiryDelta(55))), Some(ShortChannelIdDir(channelWithB.nodeId1 == b, channelWithB.shortChannelId))) + ) + handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, receivingRoutes, pluginData_opt = Some(hex"eff0")) + + val handlePayment = offerHandler.expectMessageType[HandlePayment] + assert(handlePayment.offerId == offer.offerId) + assert(handlePayment.pluginData_opt.contains(hex"eff0")) + handlePayment.replyTo ! PaymentActor.AcceptPayment() + + val paymentSent = sender.expectMsgType[PaymentSent] + assert(paymentSent.recipientAmount == amount, paymentSent) + assert(paymentSent.feesPaid >= 0.msat, paymentSent) + val Some(invoice: Bolt12Invoice) = nodes("A").nodeParams.db.payments.listOutgoingPaymentsToOffer(offer.offerId).head.invoice + assert(invoice.blindedPaths.forall(_.route.isInstanceOf[CompactBlindedPath])) + + awaitCond(nodes("C").nodeParams.db.payments.getIncomingPayment(paymentSent.paymentHash).exists(_.status.isInstanceOf[IncomingPaymentStatus.Received])) + val Some(IncomingBlindedPayment(_, _, _, _, IncomingPaymentStatus.Received(receivedAmount, _))) = nodes("C").nodeParams.db.payments.getIncomingPayment(paymentSent.paymentHash) + assert(receivedAmount >= amount) + } + test("generate and validate lots of channels") { val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) // we simulate fake channels by publishing a funding tx and sending announcement messages to a node at random From 686c16fa714bb652505df1370f3d5498fe6ee178 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Fri, 3 Nov 2023 14:25:41 +0100 Subject: [PATCH 05/12] Fix merge --- .../fr/acinq/eclair/message/PostmanSpec.scala | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala index 34f02e4db1..9956d6864d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala @@ -25,6 +25,7 @@ import fr.acinq.bitcoin.scalacompat.Block import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.eclair.crypto.Sphinx.RouteBlinding.{BlindedNode, BlindedRoute} import fr.acinq.eclair.io.MessageRelay.{Disconnected, Sent} +import fr.acinq.eclair.io.PeerConnection.ConnectionResult import fr.acinq.eclair.io.{Peer, PeerConnection} import fr.acinq.eclair.message.OnionMessages.RoutingStrategy.FindRoute import fr.acinq.eclair.message.OnionMessages.{BlindedPath, IntermediateNode, ReceiveMessage, Recipient, buildMessage, buildRoute} @@ -256,9 +257,13 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat assert(target == recipientKey.publicKey) waitingForRoute ! MessageRoute(Seq.empty, target) - val RelayMessage(messageId, _, nextNodeId, message, _, Some(replyTo)) = switchboard.expectMessageType[RelayMessage] + val Peer.Connect(nextNodeId, _, replyConnectedTo, _) = switchboard.expectMessageType[Peer.Connect] assert(nextNodeId == recipientKey.publicKey) - replyTo ! Sent(messageId) + val peerConnection = TestProbe[Any]("peerConnection") + val peer = TestProbe[Any]("peer") + replyConnectedTo ! ConnectionResult.Connected(peerConnection.ref.toClassic, peer.ref.toClassic) + val Peer.RelayOnionMessage(messageId, message, Some(replySentTo)) = peer.expectMessageType[Peer.RelayOnionMessage] + replySentTo ! Sent(messageId) val ReceiveMessage(finalPayload) = OnionMessages.process(recipientKey, message) assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd"))) assert(finalPayload.records.get[ReplyPath].isEmpty) @@ -281,9 +286,13 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat assert(getNodeId.shortChannelId == RealShortChannelId(1234)) getNodeId.replyTo ! Some(nodeParams.nodeId) - val RelayMessage(messageId, _, nextNodeId, message, _, Some(replyTo)) = switchboard.expectMessageType[RelayMessage] + val Peer.Connect(nextNodeId, _, replyConnectedTo, _) = switchboard.expectMessageType[Peer.Connect] assert(nextNodeId == recipientKey.publicKey) - replyTo ! Sent(messageId) + val peerConnection = TestProbe[Any]("peerConnection") + val peer = TestProbe[Any]("peer") + replyConnectedTo ! ConnectionResult.Connected(peerConnection.ref.toClassic, peer.ref.toClassic) + val Peer.RelayOnionMessage(messageId, message, Some(replySentTo)) = peer.expectMessageType[Peer.RelayOnionMessage] + replySentTo ! Sent(messageId) val ReceiveMessage(finalPayload) = OnionMessages.process(recipientKey, message) assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd"))) assert(finalPayload.records.get[ReplyPath].isEmpty) From dfe5653ccb92ec116cd647dfc95a52703766089e Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Thu, 9 Nov 2023 16:31:13 +0100 Subject: [PATCH 06/12] Resolve compact routes in payment initator --- .../main/scala/fr/acinq/eclair/Eclair.scala | 3 +- .../main/scala/fr/acinq/eclair/Setup.scala | 2 +- .../acinq/eclair/payment/Bolt11Invoice.scala | 2 +- .../fr/acinq/eclair/payment/Invoice.scala | 3 +- .../acinq/eclair/payment/PaymentPacket.scala | 2 +- .../payment/receive/MultiPartHandler.scala | 4 +- .../payment/send/PaymentInitiator.scala | 196 ++++++++++++++---- .../payment/send/PaymentLifecycle.scala | 8 +- .../acinq/eclair/payment/send/Recipient.scala | 44 ++-- .../scala/fr/acinq/eclair/router/Graph.scala | 19 +- .../eclair/router/RouteCalculation.scala | 31 ++- .../fr/acinq/eclair/EclairImplSpec.scala | 8 +- .../ChannelStateTestsHelperMethods.scala | 5 +- .../integration/PaymentIntegrationSpec.scala | 12 +- .../basic/fixtures/MinimalNodeFixture.scala | 2 +- .../eclair/payment/MultiPartHandlerSpec.scala | 8 +- .../MultiPartPaymentLifecycleSpec.scala | 22 +- .../eclair/payment/PaymentInitiatorSpec.scala | 25 +-- .../eclair/payment/PaymentLifecycleSpec.scala | 8 +- .../eclair/payment/PaymentPacketSpec.scala | 20 +- .../payment/relay/NodeRelayerSpec.scala | 4 +- .../eclair/router/BalanceEstimateSpec.scala | 2 +- .../acinq/eclair/router/BaseRouterSpec.scala | 5 +- .../fr/acinq/eclair/router/RouterSpec.scala | 14 +- 24 files changed, 267 insertions(+), 182 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 4e15cfd42e..4015457914 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala @@ -427,8 +427,7 @@ 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)) - val sendPayment = SendPaymentToRoute(recipientAmount, invoice, route, externalId_opt, parentId_opt, trampoline_opt) - (appKit.paymentInitiator ? sendPayment).mapTo[SendPaymentToRouteResponse] + appKit.paymentInitiator.toTyped.ask(replyTo => SendPaymentToRoute(replyTo.toClassic, recipientAmount, invoice, route, externalId_opt, parentId_opt, trampoline_opt)) } } 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 d04a5105f5..c2ef0fe926 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)), "payment-initiator", SupervisorStrategy.Restart)) + paymentInitiator = system.actorOf(SimpleSupervisor.props(PaymentInitiator.props(nodeParams, PaymentInitiator.SimplePaymentFactory(nodeParams, router, register), router), "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/Bolt11Invoice.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt11Invoice.scala index f2e401f026..027f217f9e 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt11Invoice.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Bolt11Invoice.scala @@ -559,7 +559,7 @@ object Bolt11Invoice { val nextNodeIds = extraRoute.map(_.nodeId).drop(1) :+ targetNodeId extraRoute.zip(nextNodeIds).map { case (extraHop, nextNodeId) => - Invoice.ExtraEdge(Right(extraHop.nodeId), nextNodeId, extraHop.shortChannelId, extraHop.feeBase, extraHop.feeProportionalMillionths, extraHop.cltvExpiryDelta, 1 msat, None) + Invoice.ExtraEdge(extraHop.nodeId, nextNodeId, extraHop.shortChannelId, extraHop.feeBase, extraHop.feeProportionalMillionths, extraHop.cltvExpiryDelta, 1 msat, None) } } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Invoice.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Invoice.scala index c55fc5d879..b477e34330 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Invoice.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Invoice.scala @@ -20,7 +20,6 @@ import fr.acinq.bitcoin.scalacompat.ByteVector32 import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.eclair.payment.relay.Relayer import fr.acinq.eclair.wire.protocol.ChannelUpdate -import fr.acinq.eclair.wire.protocol.OfferTypes.ShortChannelIdDir import fr.acinq.eclair.{CltvExpiryDelta, Features, InvoiceFeature, MilliSatoshi, ShortChannelId, TimestampSecond} import scala.concurrent.duration.FiniteDuration @@ -42,7 +41,7 @@ trait Invoice { object Invoice { /** An extra edge that can be used to pay a given invoice and may not be part of the public graph. */ - case class ExtraEdge(sourceNodeId: Either[ShortChannelIdDir, PublicKey], + case class ExtraEdge(sourceNodeId: PublicKey, targetNodeId: PublicKey, shortChannelId: ShortChannelId, feeBase: MilliSatoshi, diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala index 0a59fef0f4..2ff77b5998 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentPacket.scala @@ -235,7 +235,7 @@ object OutgoingPaymentPacket { case class CannotDecryptBlindedRoute(message: String) extends OutgoingPaymentError { override def getMessage: String = message } case class InvalidRouteRecipient(expected: PublicKey, actual: PublicKey) extends OutgoingPaymentError { override def getMessage: String = s"expected route to $expected, got route to $actual" } case class MissingTrampolineHop(trampolineNodeId: PublicKey) extends OutgoingPaymentError { override def getMessage: String = s"expected route to trampoline node $trampolineNodeId" } - case object MissingBlindedHop extends OutgoingPaymentError { override def getMessage: String = s"expected blinded route as final hop" } + case class MissingBlindedHop(introductionNodeIds: Set[PublicKey]) extends OutgoingPaymentError { override def getMessage: String = s"expected blinded route using one of the following introduction nodes: ${introductionNodeIds.mkString(", ")}" } case object EmptyRoute extends OutgoingPaymentError { override def getMessage: String = "route cannot be empty" } sealed trait Upstream diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala index 82fee0f555..a1c1e262e4 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala @@ -359,8 +359,8 @@ object MultiPartHandler { context.pipeToSelf(Future.sequence(r.routes.map(route => { val dummyHops = route.dummyHops.map(h => { // We don't want to restrict HTLC size in dummy hops, so we use htlc_minimum_msat = 1 msat and htlc_maximum_msat = None. - val edge = Invoice.ExtraEdge(Right(nodeParams.nodeId), nodeParams.nodeId, ShortChannelId.toSelf, h.feeBase, h.feeProportionalMillionths, h.cltvExpiryDelta, htlcMinimum = 1 msat, htlcMaximum_opt = None) - ChannelHop(edge.shortChannelId, nodeParams.nodeId, edge.targetNodeId, HopRelayParams.FromHint(edge)) + val edge = Invoice.ExtraEdge(nodeParams.nodeId, nodeParams.nodeId, ShortChannelId.toSelf, h.feeBase, h.feeProportionalMillionths, h.cltvExpiryDelta, htlcMinimum = 1 msat, htlcMaximum_opt = None) + ChannelHop(edge.shortChannelId, edge.sourceNodeId, edge.targetNodeId, HopRelayParams.FromHint(edge)) }) if (route.nodes.length == 1) { val blindedRoute = if (dummyHops.isEmpty) { 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 8d8eb71041..515c986b08 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,6 +16,7 @@ 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} @@ -36,7 +37,7 @@ import scala.util.{Failure, Success, Try} /** * Created by PM on 29/08/2016. */ -class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentInitiator.MultiPartPaymentFactory) extends Actor with ActorLogging { +class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentInitiator.MultiPartPaymentFactory, router: ActorRef) extends Actor with ActorLogging { import PaymentInitiator._ @@ -45,27 +46,30 @@ 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 paymentId = UUID.randomUUID() + val paymentCfg = r.paymentConfig() if (!r.blockUntilComplete) { // Immediately return the paymentId - replyTo ! paymentId + replyTo ! paymentCfg.id } - 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) - val recipient = r.invoice match { - case invoice: Bolt11Invoice => ClearRecipient(invoice, r.recipientAmount, finalExpiry, r.userCustomTlvs) + (r.invoice match { + case invoice: Bolt11Invoice => Some(ClearRecipient(invoice, r.recipientAmount, finalExpiry, r.userCustomTlvs)) case invoice: Bolt12Invoice => BlindedRecipient(invoice, 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))) + }) 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)) } case r: SendSpontaneousPayment => @@ -80,57 +84,58 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn case r: SendTrampolinePayment => val paymentId = UUID.randomUUID() - sender() ! paymentId + r.replyTo ! paymentId r.trampolineAttempts match { case Nil => - sender() ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, TrampolineFeesMissing) :: Nil) + r.replyTo ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, TrampolineFeesMissing) :: Nil) case _ if !r.invoice.features.hasFeature(Features.TrampolinePaymentPrototype) && r.invoice.amount_opt.isEmpty => - sender() ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, TrampolineLegacyAmountLessInvoice) :: Nil) + r.replyTo ! 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(sender(), remainingAttempts, r))) + context become main(pending + (paymentId -> PendingTrampolinePayment(r.replyTo, remainingAttempts, r))) case Failure(t) => log.warning("cannot send outgoing trampoline payment: {}", t.getMessage) - sender() ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, t) :: Nil) + r.replyTo ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, t) :: Nil) } } case r: SendPaymentToRoute => - val paymentId = UUID.randomUUID() - val parentPaymentId = r.parentId.getOrElse(UUID.randomUUID()) + val paymentCfg = r.paymentConfig() r.trampoline_opt match { case _ if !nodeParams.features.invoiceFeatures().areSupported(r.invoice.features) => - sender() ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, UnsupportedFeatures(r.invoice.features)) :: Nil) + r.replyTo ! PaymentFailed(paymentCfg.id, 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) => - 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) + r.replyTo ! SendPaymentToRouteResponse(paymentCfg.id, paymentCfg.parentId, Some(recipient.trampolinePaymentSecret)) val payFsm = outgoingPaymentFactory.spawnOutgoingPayment(context, paymentCfg) payFsm ! PaymentLifecycle.SendPaymentToRoute(self, Left(r.route), recipient) - context become main(pending + (paymentId -> PendingPaymentToRoute(sender(), r))) + context become main(pending + (paymentCfg.id -> PendingPaymentToRoute(r.replyTo, r))) case Failure(t) => log.warning("cannot send outgoing trampoline payment: {}", t.getMessage) - sender() ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, t) :: Nil) + r.replyTo ! PaymentFailed(paymentCfg.id, r.paymentHash, LocalFailure(r.recipientAmount, Nil, t) :: Nil) } case 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) + r.replyTo ! SendPaymentToRouteResponse(paymentCfg.id, paymentCfg.parentId, None) val finalExpiry = r.finalExpiry(nodeParams) - val recipient = r.invoice match { - case invoice: Bolt11Invoice => ClearRecipient(invoice, r.recipientAmount, finalExpiry, Set.empty) + (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 payFsm = outgoingPaymentFactory.spawnOutgoingPayment(context, paymentCfg) - payFsm ! PaymentLifecycle.SendPaymentToRoute(self, Left(r.route), recipient) - context become main(pending + (paymentId -> PendingPaymentToRoute(sender(), r))) case _ => - sender() ! PaymentFailed(paymentId, r.paymentHash, LocalFailure(r.recipientAmount, Nil, TrampolineMultiNodeNotSupported) :: Nil) + r.replyTo ! PaymentFailed(paymentCfg.id, r.paymentHash, LocalFailure(r.recipientAmount, Nil, TrampolineMultiNodeNotSupported) :: Nil) } case pf: PaymentFailed => pending.get(pf.id).foreach { @@ -187,7 +192,7 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn case PaymentIdentifier.PaymentUUID(paymentId) => pending.get(paymentId).map(pp => (paymentId, pp)) case PaymentIdentifier.PaymentHash(paymentHash) => pending.collectFirst { case (paymentId, pp) if pp.paymentHash == paymentHash => (paymentId, pp) } case PaymentIdentifier.OfferId(offerId) => pending.collectFirst { - case (paymentId, pp@PendingPaymentToNode(_, SendPaymentToNode(_, _, invoice: Bolt12Invoice, _, _, _, _, _, _))) if invoice.invoiceRequest.offer.offerId == offerId => + case (paymentId, pp@PendingPaymentToNode(_, SendPaymentToNode(_, _, invoice: Bolt12Invoice, _, _, _, _, _, _, _))) if invoice.invoiceRequest.offer.offerId == offerId => (paymentId, pp) } } @@ -251,7 +256,7 @@ object PaymentInitiator { } } - def props(nodeParams: NodeParams, outgoingPaymentFactory: MultiPartPaymentFactory) = Props(new PaymentInitiator(nodeParams, outgoingPaymentFactory)) + def props(nodeParams: NodeParams, outgoingPaymentFactory: MultiPartPaymentFactory, router: ActorRef) = Props(new PaymentInitiator(nodeParams, outgoingPaymentFactory, router)) // @formatter:off sealed trait PendingPayment { @@ -273,6 +278,7 @@ object PaymentInitiator { sealed trait SendRequestedPayment { // @formatter:off + def replyTo: ActorRef def recipientAmount: MilliSatoshi def invoice: Invoice def recipientNodeId: PublicKey = invoice.nodeId @@ -300,7 +306,8 @@ 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(recipientAmount: MilliSatoshi, + case class SendTrampolinePayment(replyTo: ActorRef, + recipientAmount: MilliSatoshi, invoice: Invoice, trampolineNodeId: PublicKey, trampolineAttempts: Seq[(MilliSatoshi, CltvExpiryDelta)], @@ -308,13 +315,14 @@ object PaymentInitiator { /** * @param recipientAmount amount that should be received by the final recipient (usually from a Bolt 11 invoice). - * @param invoice invoice. + * @param invoice invoice to pay that may have been modified from the original invoice. * @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, @@ -324,7 +332,13 @@ object PaymentInitiator { routeParams: RouteParams, payerKey_opt: Option[PrivateKey] = None, userCustomTlvs: Set[GenericTlv] = Set.empty, - blockUntilComplete: Boolean = false) extends SendRequestedPayment + 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) + } + } /** * @param recipientAmount amount that should be received by the final recipient. @@ -370,7 +384,7 @@ 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 Bolt 11 invoice. + * @param invoice invoice to pay that may have been modified from the original invoice. * @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 @@ -378,13 +392,22 @@ 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(recipientAmount: MilliSatoshi, + case class SendPaymentToRoute(replyTo: ActorRef, + recipientAmount: MilliSatoshi, invoice: Invoice, route: PredefinedRoute, externalId: Option[String], parentId: Option[UUID], - trampoline_opt: Option[TrampolineAttempt]) extends SendRequestedPayment + 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) + } + } /** * @param paymentId id of the outgoing payment (mapped to a single outgoing HTLC). @@ -406,7 +429,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 Bolt 11 invoice. + * @param invoice 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 @@ -433,4 +456,85 @@ 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/PaymentLifecycle.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala index 39bb0687fd..381ce075e1 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/send/PaymentLifecycle.scala @@ -300,7 +300,7 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A log.info("received an update for a routing hint (shortChannelId={} nodeId={} enabled={} update={})", failure.update.shortChannelId, nodeId, failure.update.channelFlags.isEnabled, failure.update) if (failure.update.channelFlags.isEnabled) { data.recipient.extraEdges.map { - case edge: ExtraEdge if edge.sourceNodeId == Right(nodeId) && edge.targetNodeId == hop.nextNodeId => edge.update(failure.update) + case edge: ExtraEdge if edge.sourceNodeId == nodeId && edge.targetNodeId == hop.nextNodeId => edge.update(failure.update) case edge: ExtraEdge => edge } } else { @@ -308,10 +308,10 @@ class PaymentLifecycle(nodeParams: NodeParams, cfg: SendPaymentConfig, router: A // contain channel flags to indicate that it's disabled // we want the exclusion to be router-wide so that sister payments in the case of MPP are aware the channel is faulty data.recipient.extraEdges - .find(edge => edge.sourceNodeId == Right(nodeId) && edge.targetNodeId == hop.nextNodeId) - .foreach(edge => router ! ExcludeChannel(ChannelDesc(edge.shortChannelId, nodeId, edge.targetNodeId), Some(nodeParams.routerConf.channelExcludeDuration))) + .find(edge => edge.sourceNodeId == nodeId && edge.targetNodeId == hop.nextNodeId) + .foreach(edge => router ! ExcludeChannel(ChannelDesc(edge.shortChannelId, edge.sourceNodeId, edge.targetNodeId), Some(nodeParams.routerConf.channelExcludeDuration))) // we remove this edge for our next payment attempt - data.recipient.extraEdges.filterNot(edge => edge.sourceNodeId == Right(nodeId) && edge.targetNodeId == hop.nextNodeId) + data.recipient.extraEdges.filterNot(edge => edge.sourceNodeId == nodeId && edge.targetNodeId == hop.nextNodeId) } } case None => 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 63d79dd70e..abda773d99 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,11 +21,11 @@ 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, PaymentBlindedRoute} +import fr.acinq.eclair.payment.{Bolt11Invoice, Bolt12Invoice, OutgoingPaymentPacket} 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.{Alias, CltvExpiry, Features, InvoiceFeature, MilliSatoshi, MilliSatoshiLong, ShortChannelId} +import fr.acinq.eclair.{CltvExpiry, Features, InvoiceFeature, MilliSatoshi, MilliSatoshiLong, ShortChannelId} import scodec.bits.ByteVector /** @@ -121,30 +121,18 @@ case class BlindedRecipient(nodeId: PublicKey, features: Features[InvoiceFeature], totalAmount: MilliSatoshi, expiry: CltvExpiry, - blindedPaths: Map[Alias, PaymentBlindedRoute], + blindedHops: Seq[BlindedHop], customTlvs: Set[GenericTlv] = Set.empty) extends Recipient { - require(blindedPaths.nonEmpty, "blinded routes must be provided") + require(blindedHops.nonEmpty, "blinded routes must be provided") - override val extraEdges = blindedPaths.map { case (scid, path) => - val introductionNodeId = path.route match { - case OfferTypes.BlindedPath(route) => Right(route.introductionNodeId) - case OfferTypes.CompactBlindedPath(introductionNode, _, _) => Left(introductionNode) - } - ExtraEdge( - introductionNodeId, - nodeId, - scid, - path.paymentInfo.feeBase, - path.paymentInfo.feeProportionalMillionths, - path.paymentInfo.cltvExpiryDelta, - path.paymentInfo.minHtlc, - Some(path.paymentInfo.maxHtlc)) - }.toSeq + override val extraEdges = blindedHops.map { h => + ExtraEdge(h.route.introductionNodeId, nodeId, h.dummyId, h.paymentInfo.feeBase, h.paymentInfo.feeProportionalMillionths, h.paymentInfo.cltvExpiryDelta, h.paymentInfo.minHtlc, Some(h.paymentInfo.maxHtlc)) + } private def validateRoute(route: Route): Either[OutgoingPaymentError, BlindedHop] = { route.finalHop_opt match { case Some(blindedHop: BlindedHop) => Right(blindedHop) - case _ => Left(MissingBlindedHop) + case _ => Left(MissingBlindedHop(blindedHops.map(_.route.introductionNodeId).toSet)) } } @@ -178,8 +166,18 @@ case class BlindedRecipient(nodeId: PublicKey, } object BlindedRecipient { - def apply(invoice: Bolt12Invoice, totalAmount: MilliSatoshi, expiry: CltvExpiry, customTlvs: Set[GenericTlv]): BlindedRecipient = { - BlindedRecipient(invoice.nodeId, invoice.features, totalAmount, expiry, invoice.blindedPaths.map((ShortChannelId.generateLocalAlias(), _)).toMap, customTlvs) + def apply(invoice: Bolt12Invoice, totalAmount: MilliSatoshi, expiry: CltvExpiry, customTlvs: Set[GenericTlv]): Option[BlindedRecipient] = { + val blindedHops = invoice.blindedPaths.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 + } + }) + Some(BlindedRecipient(invoice.nodeId, invoice.features, totalAmount, expiry, blindedHops, customTlvs)) } } @@ -206,7 +204,7 @@ case class ClearTrampolineRecipient(invoice: Bolt11Invoice, override val nodeId = invoice.nodeId override val features = invoice.features - override val extraEdges = Seq(ExtraEdge(Right(trampolineNodeId), nodeId, ShortChannelId.generateLocalAlias(), trampolineFee, 0, trampolineHop.cltvExpiryDelta, 1 msat, None)) + override val extraEdges = Seq(ExtraEdge(trampolineNodeId, nodeId, ShortChannelId.generateLocalAlias(), trampolineFee, 0, trampolineHop.cltvExpiryDelta, 1 msat, None)) private def validateRoute(route: Route): Either[OutgoingPaymentError, NodeHop] = { route.finalHop_opt match { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala index 32fbb90869..89af943807 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/Graph.scala @@ -21,12 +21,11 @@ import fr.acinq.bitcoin.scalacompat.{Btc, BtcDouble, MilliBtc, Satoshi} import fr.acinq.eclair._ import fr.acinq.eclair.payment.Invoice import fr.acinq.eclair.payment.relay.Relayer.RelayFees -import fr.acinq.eclair.router.Graph.GraphStructure.{DirectedGraph, GraphEdge} +import fr.acinq.eclair.router.Graph.GraphStructure.{GraphEdge, DirectedGraph} import fr.acinq.eclair.router.Router._ import fr.acinq.eclair.wire.protocol.{ChannelUpdate, NodeAnnouncement} import scala.annotation.tailrec -import scala.collection.immutable.SortedMap import scala.collection.mutable object Graph { @@ -613,23 +612,15 @@ object Graph { balance_opt = pc.getBalanceSameSideAs(u) ) - def fromExtraEdge(e: Invoice.ExtraEdge, publicChannels: SortedMap[RealShortChannelId, PublicChannel]): Option[GraphEdge] = { + def apply(e: Invoice.ExtraEdge): GraphEdge = { val maxBtc = 21e6.btc - val sourceNodeId = e.sourceNodeId match { - case Left(scidDir) => publicChannels.get(scidDir.scid) match { - case Some(pc) if scidDir.isNode1 => pc.nodeId1 - case Some(pc) => pc.nodeId2 - case None => return None - } - case Right(publicKey) => publicKey - } - Some(GraphEdge( - desc = ChannelDesc(e.shortChannelId, sourceNodeId, e.targetNodeId), + GraphEdge( + desc = ChannelDesc(e.shortChannelId, e.sourceNodeId, e.targetNodeId), params = HopRelayParams.FromHint(e), // Routing hints don't include the channel's capacity, so we assume it's big enough. capacity = maxBtc.toSatoshi, balance_opt = None, - )) + ) } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala index be54356e94..b0b1e4c4c9 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/RouteCalculation.scala @@ -22,18 +22,16 @@ import com.softwaremill.quicklens.ModifyPimp import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.eclair.Logs.LogCategory import fr.acinq.eclair._ -import fr.acinq.eclair.crypto.Sphinx.RouteBlinding +import fr.acinq.eclair.message.SendingMessage import fr.acinq.eclair.payment.send._ import fr.acinq.eclair.router.Graph.GraphStructure.DirectedGraph.graphEdgeToHop import fr.acinq.eclair.router.Graph.GraphStructure.{DirectedGraph, GraphEdge} import fr.acinq.eclair.router.Graph.{InfiniteLoop, MessagePath, NegativeProbability, RichWeight} import fr.acinq.eclair.router.Monitoring.{Metrics, Tags} import fr.acinq.eclair.router.Router._ -import fr.acinq.eclair.wire.protocol.OfferTypes import kamon.tag.TagSet import scala.annotation.tailrec -import scala.collection.immutable.SortedMap import scala.collection.mutable import scala.util.{Failure, Random, Success, Try} @@ -68,7 +66,7 @@ object RouteCalculation { paymentHash_opt = fr.paymentContext.map(_.paymentHash))) { implicit val sender: ActorRef = ctx.self // necessary to preserve origin when sending messages to other actors - val extraEdges = fr.extraEdges.flatMap(GraphEdge.fromExtraEdge(_, d.channels)) + val extraEdges = fr.extraEdges.map(GraphEdge(_)) val g = extraEdges.foldLeft(d.graphWithBalances.graph) { case (g: DirectedGraph, e: GraphEdge) => g.addEdge(e) } fr.route match { @@ -131,7 +129,7 @@ object RouteCalculation { * * The routes found must then be post-processed by calling [[addFinalHop]]. */ - private def computeTarget(r: RouteRequest, ignoredEdges: Set[ChannelDesc], publicChannels: SortedMap[RealShortChannelId, PublicChannel]): (PublicKey, MilliSatoshi, MilliSatoshi, Set[GraphEdge]) = { + private def computeTarget(r: RouteRequest, ignoredEdges: Set[ChannelDesc]): (PublicKey, MilliSatoshi, MilliSatoshi, Set[GraphEdge]) = { val pendingAmount = r.pendingPayments.map(_.amount).sum val totalMaxFee = r.routeParams.getMaxFee(r.target.totalAmount) val pendingChannelFee = r.pendingPayments.map(_.channelFee(r.routeParams.includeLocalChannelCost)).sum @@ -141,8 +139,8 @@ object RouteCalculation { val amountToSend = recipient.totalAmount - pendingAmount val maxFee = totalMaxFee - pendingChannelFee val extraEdges = recipient.extraEdges - .flatMap(GraphEdge.fromExtraEdge(_, publicChannels)) - .filterNot(e => (e.desc.a == r.source) || (e.desc.b == r.source)) // we ignore routing hints for our own channels, we have more accurate information + .filter(_.sourceNodeId != r.source) // we ignore routing hints for our own channels, we have more accurate information + .map(GraphEdge(_)) .filterNot(e => ignoredEdges.contains(e.desc)) .toSet (targetNodeId, amountToSend, maxFee, extraEdges) @@ -158,7 +156,7 @@ object RouteCalculation { .map(_.copy(targetNodeId = targetNodeId)) .filterNot(e => ignoredEdges.exists(_.shortChannelId == e.shortChannelId)) // For blinded routes, the maximum htlc field is used to indicate the maximum amount that can be sent through the route. - .flatMap(e => GraphEdge.fromExtraEdge(e, publicChannels).map(_.copy(balance_opt = e.htlcMaximum_opt))) + .map(e => GraphEdge(e).copy(balance_opt = e.htlcMaximum_opt)) .toSet val amountToSend = recipient.totalAmount - pendingAmount // When we are the introduction node and includeLocalChannelCost is false, we cannot easily remove the fee for @@ -184,16 +182,11 @@ object RouteCalculation { case _: SpontaneousRecipient => Some(route) case recipient: ClearTrampolineRecipient => Some(route.copy(finalHop_opt = Some(recipient.trampolineHop))) case recipient: BlindedRecipient => - route.hops.lastOption.flatMap(lastHop =>{ - val alias = Alias(lastHop.shortChannelId.toLong) - recipient.blindedPaths.get(alias).map(path => { - val blindedRoute = path.route match { - case OfferTypes.BlindedPath(blindedRoute) => blindedRoute - case OfferTypes.CompactBlindedPath(_, blindingKey, blindedNodes) => RouteBlinding.BlindedRoute(lastHop.nodeId, blindingKey, blindedNodes) - } - Route(route.amount, route.hops.dropRight(1), Some(BlindedHop(alias, blindedRoute, path.paymentInfo))) - }) - }) + route.hops.lastOption.flatMap { + hop => recipient.blindedHops.find(_.dummyId == hop.shortChannelId) + }.map { + blindedHop => Route(route.amount, route.hops.dropRight(1), Some(blindedHop)) + } } }) } @@ -207,7 +200,7 @@ object RouteCalculation { implicit val sender: ActorRef = ctx.self // necessary to preserve origin when sending messages to other actors val ignoredEdges = r.ignore.channels ++ d.excludedChannels.keySet - val (targetNodeId, amountToSend, maxFee, extraEdges) = computeTarget(r, ignoredEdges, d.channels) + val (targetNodeId, amountToSend, maxFee, extraEdges) = computeTarget(r, ignoredEdges) val routesToFind = if (r.routeParams.randomize) DEFAULT_ROUTES_COUNT else 1 log.info(s"finding routes ${r.source}->$targetNodeId with assistedChannels={} ignoreNodes={} ignoreChannels={} excludedChannels={}", extraEdges.map(_.desc.shortChannelId).mkString(","), r.ignore.nodes.map(_.value).mkString(","), r.ignore.channels.mkString(","), d.excludedChannels.mkString(",")) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala index 41db1ea12a..e25d69f682 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala @@ -344,7 +344,13 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I val secret = randomBytes32() val pr = Bolt11Invoice(Block.LivenetGenesisBlock.hash, Some(1234 msat), ByteVector32.One, randomKey(), Right(randomBytes32()), CltvExpiryDelta(18)) eclair.sendToRoute(Some(1200 msat), Some("42"), Some(parentId), pr, route, Some(secret), Some(100 msat), Some(CltvExpiryDelta(144))) - paymentInitiator.expectMsg(SendPaymentToRoute(1200 msat, pr, route, Some("42"), Some(parentId), Some(TrampolineAttempt(secret, 100 msat, CltvExpiryDelta(144))))) + val sendPaymentToRoute = paymentInitiator.expectMsgType[SendPaymentToRoute] + assert(sendPaymentToRoute.recipientAmount == 1200.msat) + assert(sendPaymentToRoute.invoice == pr) + assert(sendPaymentToRoute.route == route) + assert(sendPaymentToRoute.externalId.contains("42")) + assert(sendPaymentToRoute.parentId.contains(parentId)) + assert(sendPaymentToRoute.trampoline_opt.contains(TrampolineAttempt(secret, 100 msat, CltvExpiryDelta(144)))) } test("find routes") { f => diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala index e5157c2873..142e4a2a45 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala @@ -388,9 +388,8 @@ trait ChannelStateTestsBase extends Assertions with Eventually { } def makeSingleHopRoute(amount: MilliSatoshi, destination: PublicKey): Route = { - val dummyNodeId = randomKey().publicKey - val dummyParams = HopRelayParams.FromHint(Invoice.ExtraEdge(Right(dummyNodeId), destination, ShortChannelId(0), 0 msat, 0, CltvExpiryDelta(0), 0 msat, None)) - Route(amount, Seq(ChannelHop(ShortChannelId(0), dummyNodeId, dummyParams.extraHop.targetNodeId, dummyParams)), None) + val dummyParams = HopRelayParams.FromHint(Invoice.ExtraEdge(randomKey().publicKey, destination, ShortChannelId(0), 0 msat, 0, CltvExpiryDelta(0), 0 msat, None)) + Route(amount, Seq(ChannelHop(ShortChannelId(0), dummyParams.extraHop.sourceNodeId, dummyParams.extraHop.targetNodeId, dummyParams)), None) } def addHtlc(amount: MilliSatoshi, s: TestFSMRef[ChannelState, ChannelData, Channel], r: TestFSMRef[ChannelState, ChannelData, Channel], s2r: TestProbe, r2s: TestProbe): (ByteVector32, UpdateAddHtlc) = { 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 2d432ac40e..ac08bb4ec2 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 @@ -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(amount, invoice, nodes("G").nodeParams.nodeId, attempts, routeParams = integrationTestRouteParams) + val payment = SendTrampolinePayment(sender.ref, 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(amount, invoice, nodes("C").nodeParams.nodeId, Seq((750000 msat, CltvExpiryDelta(288))), routeParams = integrationTestRouteParams) + val payment = SendTrampolinePayment(sender.ref, 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(amount, invoice, nodes("C").nodeParams.nodeId, Seq((1500000 msat, CltvExpiryDelta(432))), routeParams = integrationTestRouteParams) + val payment = SendTrampolinePayment(sender.ref, 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) @@ -621,7 +621,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { assert(invoice.features.hasFeature(Features.BasicMultiPartPayment)) assert(invoice.features.hasFeature(Features.TrampolinePaymentPrototype)) - val payment = SendTrampolinePayment(amount, invoice, nodes("C").nodeParams.nodeId, Seq((250000 msat, CltvExpiryDelta(288))), routeParams = integrationTestRouteParams) + val payment = SendTrampolinePayment(sender.ref, 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(amount, invoice, nodes("B").nodeParams.nodeId, Seq((450000 msat, CltvExpiryDelta(288))), routeParams = integrationTestRouteParams) + val payment = SendTrampolinePayment(sender.ref, 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(amount, invoice, nodes("C").nodeParams.nodeId, Seq((500000 msat, CltvExpiryDelta(288))), routeParams = integrationTestRouteParams) + val payment = SendTrampolinePayment(sender.ref, 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/basic/fixtures/MinimalNodeFixture.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala index 4df68766cc..dadd736db4 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), "payment-initiator") + val paymentInitiator = system.actorOf(PaymentInitiator.props(nodeParams, paymentFactory, router), "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) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala index bce0ec6195..e767e2c5b5 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartHandlerSpec.scala @@ -271,9 +271,9 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike val invoiceReq = InvoiceRequest(offer, 25_000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash) val router = TestProbe() val (a, b, c, d) = (randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, nodeParams.nodeId) - val hop_ab = Router.ChannelHop(ShortChannelId(1), a, b, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(Right(a), b, ShortChannelId(1), 1000 msat, 0, CltvExpiryDelta(100), 1 msat, None))) - val hop_bd = Router.ChannelHop(ShortChannelId(2), b, d, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(Right(b), d, ShortChannelId(2), 800 msat, 0, CltvExpiryDelta(50), 1 msat, None))) - val hop_cd = Router.ChannelHop(ShortChannelId(3), c, d, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(Right(c), d, ShortChannelId(3), 0 msat, 0, CltvExpiryDelta(75), 1 msat, None))) + val hop_ab = Router.ChannelHop(ShortChannelId(1), a, b, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(a, b, ShortChannelId(1), 1000 msat, 0, CltvExpiryDelta(100), 1 msat, None))) + val hop_bd = Router.ChannelHop(ShortChannelId(2), b, d, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(b, d, ShortChannelId(2), 800 msat, 0, CltvExpiryDelta(50), 1 msat, None))) + val hop_cd = Router.ChannelHop(ShortChannelId(3), c, d, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(c, d, ShortChannelId(3), 0 msat, 0, CltvExpiryDelta(75), 1 msat, None))) val receivingRoutes = Seq( ReceivingRoute(Seq(a, b, d), CltvExpiryDelta(100), Seq(DummyBlindedHop(150 msat, 0, CltvExpiryDelta(25)))), ReceivingRoute(Seq(c, d), CltvExpiryDelta(50), Seq(DummyBlindedHop(250 msat, 0, CltvExpiryDelta(10)), DummyBlindedHop(150 msat, 0, CltvExpiryDelta(80)))), @@ -317,7 +317,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike val invoiceReq = InvoiceRequest(offer, 25_000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash) val router = TestProbe() val (a, b, c) = (randomKey().publicKey, randomKey().publicKey, nodeParams.nodeId) - val hop_ac = Router.ChannelHop(ShortChannelId(1), a, c, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(Right(a), c, ShortChannelId(1), 100 msat, 0, CltvExpiryDelta(50), 1 msat, None))) + val hop_ac = Router.ChannelHop(ShortChannelId(1), a, c, Router.HopRelayParams.FromHint(Invoice.ExtraEdge(a, c, ShortChannelId(1), 100 msat, 0, CltvExpiryDelta(50), 1 msat, None))) val receivingRoutes = Seq( ReceivingRoute(Seq(a, c), CltvExpiryDelta(100)), ReceivingRoute(Seq(b, c), CltvExpiryDelta(100)), diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala index d2dbba681e..ee5da5b8ff 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/MultiPartPaymentLifecycleSpec.scala @@ -351,7 +351,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS import f._ // The B -> E channel is private and provided in the invoice routing hints. - val extraEdge = ExtraEdge(Right(b), e, hop_be.shortChannelId, hop_be.params.relayFees.feeBase, hop_be.params.relayFees.feeProportionalMillionths, hop_be.params.cltvExpiryDelta, hop_be.params.htlcMinimum, hop_be.params.htlcMaximum_opt) + val extraEdge = ExtraEdge(b, e, hop_be.shortChannelId, hop_be.params.relayFees.feeBase, hop_be.params.relayFees.feeProportionalMillionths, hop_be.params.cltvExpiryDelta, hop_be.params.htlcMinimum, hop_be.params.htlcMaximum_opt) val recipient = ClearRecipient(e, Features.empty, finalAmount, expiry, randomBytes32(), Seq(extraEdge)) val payment = SendMultiPartPayment(sender.ref, recipient, 3, routeParams) sender.send(payFsm, payment) @@ -374,7 +374,7 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS import f._ // The B -> E channel is private and provided in the invoice routing hints. - val extraEdge = ExtraEdge(Right(b), e, hop_be.shortChannelId, hop_be.params.relayFees.feeBase, hop_be.params.relayFees.feeProportionalMillionths, hop_be.params.cltvExpiryDelta, hop_be.params.htlcMinimum, hop_be.params.htlcMaximum_opt) + val extraEdge = ExtraEdge(b, e, hop_be.shortChannelId, hop_be.params.relayFees.feeBase, hop_be.params.relayFees.feeProportionalMillionths, hop_be.params.cltvExpiryDelta, hop_be.params.htlcMinimum, hop_be.params.htlcMaximum_opt) val recipient = ClearRecipient(e, Features.empty, finalAmount, expiry, randomBytes32(), Seq(extraEdge)) val payment = SendMultiPartPayment(sender.ref, recipient, 3, routeParams) sender.send(payFsm, payment) @@ -417,9 +417,9 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS test("update routing hints") { () => val recipient = ClearRecipient(e, Features.empty, finalAmount, expiry, randomBytes32(), Seq( - ExtraEdge(Right(a), b, ShortChannelId(1), 10 msat, 0, CltvExpiryDelta(12), 1 msat, None), - ExtraEdge(Right(b), c, ShortChannelId(2), 0 msat, 100, CltvExpiryDelta(24), 1 msat, None), - ExtraEdge(Right(a), c, ShortChannelId(3), 1 msat, 10, CltvExpiryDelta(144), 1 msat, None) + ExtraEdge(a, b, ShortChannelId(1), 10 msat, 0, CltvExpiryDelta(12), 1 msat, None), + ExtraEdge(b, c, ShortChannelId(2), 0 msat, 100, CltvExpiryDelta(24), 1 msat, None), + ExtraEdge(a, c, ShortChannelId(3), 1 msat, 10, CltvExpiryDelta(144), 1 msat, None) )) def makeChannelUpdate(shortChannelId: ShortChannelId, feeBase: MilliSatoshi, feeProportional: Long, cltvExpiryDelta: CltvExpiryDelta): ChannelUpdate = { @@ -433,9 +433,9 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS UnreadableRemoteFailure(finalAmount, Nil) ) val extraEdges1 = Seq( - ExtraEdge(Right(a), b, ShortChannelId(1), 10 msat, 0, CltvExpiryDelta(12), 1 msat, None), - ExtraEdge(Right(b), c, ShortChannelId(2), 15 msat, 150, CltvExpiryDelta(48), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)), - ExtraEdge(Right(a), c, ShortChannelId(3), 1 msat, 10, CltvExpiryDelta(144), 1 msat, None) + ExtraEdge(a, b, ShortChannelId(1), 10 msat, 0, CltvExpiryDelta(12), 1 msat, None), + ExtraEdge(b, c, ShortChannelId(2), 15 msat, 150, CltvExpiryDelta(48), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)), + ExtraEdge(a, c, ShortChannelId(3), 1 msat, 10, CltvExpiryDelta(144), 1 msat, None) ) assert(extraEdges1.zip(PaymentFailure.updateExtraEdges(failures, recipient).extraEdges).forall { case (e1, e2) => e1 == e2 }) } @@ -447,9 +447,9 @@ class MultiPartPaymentLifecycleSpec extends TestKitBaseClass with FixtureAnyFunS RemoteFailure(finalAmount, Nil, Sphinx.DecryptedFailurePacket(a, FeeInsufficient(100 msat, makeChannelUpdate(ShortChannelId(1), 23 msat, 23, CltvExpiryDelta(23))))), ) val extraEdges1 = Seq( - ExtraEdge(Right(a), b, ShortChannelId(1), 23 msat, 23, CltvExpiryDelta(23), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)), - ExtraEdge(Right(b), c, ShortChannelId(2), 21 msat, 21, CltvExpiryDelta(21), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)), - ExtraEdge(Right(a), c, ShortChannelId(3), 22 msat, 22, CltvExpiryDelta(22), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)) + ExtraEdge(a, b, ShortChannelId(1), 23 msat, 23, CltvExpiryDelta(23), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)), + ExtraEdge(b, c, ShortChannelId(2), 21 msat, 21, CltvExpiryDelta(21), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)), + ExtraEdge(a, c, ShortChannelId(3), 22 msat, 22, CltvExpiryDelta(22), defaultChannelUpdate.htlcMinimumMsat, Some(defaultChannelUpdate.htlcMaximumMsat)) ) assert(extraEdges1.zip(PaymentFailure.updateExtraEdges(failures, recipient).extraEdges).forall { case (e1, e2) => e1 == e2 }) } 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 d3e4c91188..9dd5a0cfd5 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 @@ -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) + case class FixtureParam(nodeParams: NodeParams, initiator: TestActorRef[PaymentInitiator], payFsm: TestProbe, multiPartPayFsm: TestProbe, sender: TestProbe, eventListener: TestProbe, router: TestProbe) val featuresWithoutMpp: Features[Bolt11Feature] = Features( VariableLengthOnion -> Mandatory, @@ -114,9 +114,10 @@ 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))) - withFixture(test.toNoArgTest(FixtureParam(nodeParams, initiator, payFsm, multiPartPayFsm, sender, eventListener))) + val initiator = TestActorRef(new PaymentInitiator(nodeParams, FakePaymentFactory(payFsm, multiPartPayFsm), router.ref)) + withFixture(test.toNoArgTest(FixtureParam(nodeParams, initiator, payFsm, multiPartPayFsm, sender, eventListener, router))) } test("forward payment with user custom tlv records") { f => @@ -181,7 +182,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(finalAmount, invoice, route, None, None, None) + val request = SendPaymentToRoute(sender.ref, finalAmount, invoice, 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)) @@ -267,7 +268,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(finalAmount, invoice, route, None, None, None) + val req = SendPaymentToRoute(sender.ref, finalAmount, invoice, 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)) @@ -376,7 +377,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike 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(finalAmount, invoice, b, Seq((trampolineFees, CltvExpiryDelta(12))), nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendTrampolinePayment(sender.ref, finalAmount, invoice, b, Seq((trampolineFees, CltvExpiryDelta(12))), nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) val id = sender.expectMsgType[UUID] multiPartPayFsm.expectMsgType[SendPaymentConfig] @@ -403,7 +404,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(finalAmount, invoice, b, Seq((trampolineFees, CltvExpiryDelta(12))), routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendTrampolinePayment(sender.ref, finalAmount, invoice, b, Seq((trampolineFees, CltvExpiryDelta(12))), routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) sender.expectMsgType[UUID] multiPartPayFsm.expectMsgType[SendPaymentConfig] @@ -427,7 +428,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(finalAmount, invoice, b, Seq((trampolineFees, CltvExpiryDelta(12))), routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendTrampolinePayment(sender.ref, 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] @@ -442,7 +443,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(finalAmount, invoice, b, trampolineAttempts, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendTrampolinePayment(sender.ref, finalAmount, invoice, b, trampolineAttempts, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) val id = sender.expectMsgType[UUID] val cfg = multiPartPayFsm.expectMsgType[SendPaymentConfig] @@ -479,7 +480,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(finalAmount, invoice, b, trampolineAttempts, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendTrampolinePayment(sender.ref, finalAmount, invoice, b, trampolineAttempts, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) sender.expectMsgType[UUID] val cfg = multiPartPayFsm.expectMsgType[SendPaymentConfig] @@ -510,7 +511,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(finalAmount, invoice, b, trampolineAttempts, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) + val req = SendTrampolinePayment(sender.ref, finalAmount, invoice, b, trampolineAttempts, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams) sender.send(initiator, req) sender.expectMsgType[UUID] @@ -538,7 +539,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(finalAmount, invoice, route, None, None, Some(trampolineAttempt)) + val req = SendPaymentToRoute(sender.ref, finalAmount, invoice, 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/PaymentLifecycleSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala index 1a5c309f35..a598b0e583 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentLifecycleSpec.scala @@ -197,7 +197,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { val recipientNodeId = randomKey().publicKey val route = PredefinedNodeRoute(defaultAmountMsat, Seq(a, b, c, recipientNodeId)) - val extraEdges = Seq(ExtraEdge(Right(c), recipientNodeId, ShortChannelId(561), 1 msat, 100, CltvExpiryDelta(144), 1 msat, None)) + val extraEdges = Seq(ExtraEdge(c, recipientNodeId, ShortChannelId(561), 1 msat, 100, CltvExpiryDelta(144), 1 msat, None)) val recipient = ClearRecipient(recipientNodeId, Features.empty, defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret, extraEdges) val request = SendPaymentToRoute(sender.ref, Left(route), recipient) @@ -594,8 +594,8 @@ class PaymentLifecycleSpec extends BaseRouterSpec { // we build an assisted route for channel bc and cd val recipient = ClearRecipient(d, Features.empty, defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret, Seq( - ExtraEdge(Right(b), c, scid_bc, update_bc.feeBaseMsat, update_bc.feeProportionalMillionths, update_bc.cltvExpiryDelta, 1 msat, None), - ExtraEdge(Right(c), d, scid_cd, update_cd.feeBaseMsat, update_cd.feeProportionalMillionths, update_cd.cltvExpiryDelta, 1 msat, None) + ExtraEdge(b, c, scid_bc, update_bc.feeBaseMsat, update_bc.feeProportionalMillionths, update_bc.cltvExpiryDelta, 1 msat, None), + ExtraEdge(c, d, scid_cd, update_cd.feeBaseMsat, update_cd.feeProportionalMillionths, update_cd.cltvExpiryDelta, 1 msat, None) )) val request = SendPaymentToNode(sender.ref, recipient, 5, defaultRouteParams) sender.send(paymentFSM, request) @@ -638,7 +638,7 @@ class PaymentLifecycleSpec extends BaseRouterSpec { // we build an assisted route for channel cd val recipient = ClearRecipient(d, Features.empty, defaultAmountMsat, defaultExpiry, defaultInvoice.paymentSecret, Seq( - ExtraEdge(Right(c), d, scid_cd, update_cd.feeBaseMsat, update_cd.feeProportionalMillionths, update_cd.cltvExpiryDelta, 1 msat, None) + ExtraEdge(c, d, scid_cd, update_cd.feeBaseMsat, update_cd.feeProportionalMillionths, update_cd.cltvExpiryDelta, 1 msat, None) )) val request = SendPaymentToNode(sender.ref, recipient, 1, defaultRouteParams) sender.send(paymentFSM, request) 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 a1d90a0974..bdd6edca8c 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 @@ -28,7 +28,7 @@ import fr.acinq.eclair.payment.IncomingPaymentPacket.{ChannelRelayPacket, FinalP import fr.acinq.eclair.payment.OutgoingPaymentPacket._ import fr.acinq.eclair.payment.send.{BlindedRecipient, ClearRecipient, ClearTrampolineRecipient} import fr.acinq.eclair.router.BaseRouterSpec.{blindedRouteFromHops, channelHopFromUpdate} -import fr.acinq.eclair.router.{BlindedRouteCreation, Router} +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 @@ -161,7 +161,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { test("build outgoing blinded payment") { val (invoice, route, recipient) = longBlindedHops(hex"deadbeef") assert(recipient.extraEdges.length == 1) - assert(recipient.extraEdges.head.sourceNodeId == Right(c)) + assert(recipient.extraEdges.head.sourceNodeId == c) assert(recipient.extraEdges.head.targetNodeId == invoice.nodeId) val Right(payment) = buildOutgoingPayment(ActorRef.noSender, priv_a.privateKey, Upstream.Local(UUID.randomUUID()), paymentHash, route, recipient) assert(payment.outgoingChannel == channelUpdate_ab.shortChannelId) @@ -220,11 +220,9 @@ 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 recipient = BlindedRecipient(invoice, amount_bc, expiry_bc, Set.empty) + val Some(recipient) = BlindedRecipient(invoice, amount_bc, expiry_bc, Set.empty) val hops = Seq(channelHopFromUpdate(a, b, channelUpdate_ab), channelHopFromUpdate(b, c, channelUpdate_bc)) - val (alias, blindedPath) = recipient.blindedPaths.head - val finalHop = Router.BlindedHop(alias, blindedPath.route.asInstanceOf[OfferTypes.BlindedPath].route, blindedPath.paymentInfo) - val Right(payment) = buildOutgoingPayment(ActorRef.noSender, priv_a.privateKey, Upstream.Local(UUID.randomUUID()), paymentHash, Route(amount_bc, hops, Some(finalHop)), recipient) + 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) assert(payment.cmd.amount == amount_ab) assert(payment.cmd.cltvExpiry == expiry_ab) @@ -252,7 +250,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { test("build outgoing blinded payment starting at our node") { val (route, recipient) = singleBlindedHop(hex"123456") assert(recipient.extraEdges.length == 1) - assert(recipient.extraEdges.head.sourceNodeId == Right(a)) + assert(recipient.extraEdges.head.sourceNodeId == a) val Right(payment) = buildOutgoingPayment(ActorRef.noSender, priv_a.privateKey, Upstream.Local(UUID.randomUUID()), paymentHash, route, recipient) assert(payment.outgoingChannel == channelUpdate_ab.shortChannelId) assert(payment.cmd.amount == finalAmount) @@ -420,7 +418,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { assert(buildOutgoingPayment(ActorRef.noSender, priv_a.privateKey, Upstream.Local(UUID.randomUUID()), paymentHash, route, recipient).isRight) val routeMissingBlindedHop = route.copy(finalHop_opt = None) val Left(failure) = buildOutgoingPayment(ActorRef.noSender, priv_a.privateKey, Upstream.Local(UUID.randomUUID()), paymentHash, routeMissingBlindedHop, recipient) - assert(failure == MissingBlindedHop) + assert(failure == MissingBlindedHop(Set(c))) } test("fail to decrypt when the onion is invalid") { @@ -473,10 +471,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 recipient = BlindedRecipient(invoice, amount_bc, expiry_bc, Set.empty) - val (alias, blindedPath) = recipient.blindedPaths.head - val finalHop = Router.BlindedHop(alias, blindedPath.route.asInstanceOf[OfferTypes.BlindedPath].route, blindedPath.paymentInfo) - val route = Route(amount_bc, Seq(channelHopFromUpdate(a, b, channelUpdate_ab)), Some(finalHop)) + val Some(recipient) = BlindedRecipient(invoice, amount_bc, expiry_bc, Set.empty) + val route = Route(amount_bc, Seq(channelHopFromUpdate(a, b, channelUpdate_ab)), Some(recipient.blindedHops.head)) (route, recipient) } val Right(payment) = buildOutgoingPayment(ActorRef.noSender, priv_a.privateKey, Upstream.Local(UUID.randomUUID()), paymentHash, route, recipient) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala index 71c7ab83ab..19b88cf48d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala @@ -733,7 +733,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl assert(outgoingPayment.recipient.nodeId == outgoingNodeId) assert(outgoingPayment.recipient.totalAmount == outgoingAmount) assert(outgoingPayment.recipient.expiry == outgoingExpiry) - assert(outgoingPayment.recipient.extraEdges.head == ExtraEdge(Right(hints.head.nodeId), outgoingNodeId, ShortChannelId(42), 10 msat, 1, CltvExpiryDelta(12), 1 msat, None)) + assert(outgoingPayment.recipient.extraEdges.head == ExtraEdge(hints.head.nodeId, outgoingNodeId, ShortChannelId(42), 10 msat, 1, CltvExpiryDelta(12), 1 msat, None)) assert(outgoingPayment.recipient.isInstanceOf[ClearRecipient]) val recipient = outgoingPayment.recipient.asInstanceOf[ClearRecipient] assert(recipient.nextTrampolineOnion_opt.isEmpty) @@ -777,7 +777,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl assert(outgoingPayment.recipient.nodeId == outgoingNodeId) assert(outgoingPayment.amount == outgoingAmount) assert(outgoingPayment.recipient.expiry == outgoingExpiry) - assert(outgoingPayment.recipient.extraEdges.head == ExtraEdge(Right(hints.head.nodeId), outgoingNodeId, ShortChannelId(42), 10 msat, 1, CltvExpiryDelta(12), 1 msat, None)) + assert(outgoingPayment.recipient.extraEdges.head == ExtraEdge(hints.head.nodeId, outgoingNodeId, ShortChannelId(42), 10 msat, 1, CltvExpiryDelta(12), 1 msat, None)) assert(outgoingPayment.recipient.isInstanceOf[ClearRecipient]) val recipient = outgoingPayment.recipient.asInstanceOf[ClearRecipient] assert(recipient.nextTrampolineOnion_opt.isEmpty) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/BalanceEstimateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/BalanceEstimateSpec.scala index 3469afb1e5..9784f93345 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/BalanceEstimateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/BalanceEstimateSpec.scala @@ -38,7 +38,7 @@ class BalanceEstimateSpec extends AnyFunSuite { def makeEdge(nodeId1: PublicKey, nodeId2: PublicKey, channelId: Long, capacity: Satoshi): GraphEdge = GraphEdge( ChannelDesc(ShortChannelId(channelId), nodeId1, nodeId2), - HopRelayParams.FromHint(Invoice.ExtraEdge(Right(nodeId1), nodeId2, ShortChannelId(channelId), 0 msat, 0, CltvExpiryDelta(0), 0 msat, None)), + HopRelayParams.FromHint(Invoice.ExtraEdge(nodeId1, nodeId2, ShortChannelId(channelId), 0 msat, 0, CltvExpiryDelta(0), 0 msat, None)), capacity, None) def makeEdge(channelId: Long, capacity: Satoshi): GraphEdge = 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 35575ab3d9..081630b6e6 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 @@ -254,8 +254,7 @@ object BaseRouterSpec { preimage: ByteVector32 = randomBytes32(), pathId: ByteVector = randomBytes(32)): (Bolt12Invoice, BlindedHop, BlindedRecipient) = { val (invoice, recipient) = blindedRoutesFromPaths(amount, expiry, Seq(hops), routeExpiry, preimage, pathId) - val (alias, blindedPath) = recipient.blindedPaths.head - (invoice, BlindedHop(alias, blindedPath.route.asInstanceOf[OfferTypes.BlindedPath].route, blindedPath.paymentInfo), recipient) + (invoice, recipient.blindedHops.head, recipient) } def blindedRoutesFromPaths(amount: MilliSatoshi, @@ -276,7 +275,7 @@ object BaseRouterSpec { PaymentBlindedRoute(blindedRoute, paymentInfo) }) val invoice = Bolt12Invoice(invoiceRequest, preimage, recipientKey, 300 seconds, features, blindedRoutes) - val recipient = BlindedRecipient(invoice, amount, expiry, Set.empty) + val Some(recipient) = BlindedRecipient(invoice, amount, expiry, Set.empty) (invoice, recipient) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala index 67b106b944..a8c82aa9be 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala @@ -541,8 +541,8 @@ class RouterSpec extends BaseRouterSpec { val sender = TestProbe() val r = randomKey().publicKey val hopsToRecipient = Seq( - ChannelHop(ShortChannelId(10000), b, r, HopRelayParams.FromHint(ExtraEdge(Right(b), r, ShortChannelId(10000), 800 msat, 0, CltvExpiryDelta(36), 1 msat, Some(400_000 msat)))) :: Nil, - ChannelHop(ShortChannelId(10001), c, r, HopRelayParams.FromHint(ExtraEdge(Right(c), r, ShortChannelId(10001), 500 msat, 0, CltvExpiryDelta(36), 1 msat, Some(400_000 msat)))) :: Nil, + ChannelHop(ShortChannelId(10000), b, r, HopRelayParams.FromHint(ExtraEdge(b, r, ShortChannelId(10000), 800 msat, 0, CltvExpiryDelta(36), 1 msat, Some(400_000 msat)))) :: Nil, + ChannelHop(ShortChannelId(10001), c, r, HopRelayParams.FromHint(ExtraEdge(c, r, ShortChannelId(10001), 500 msat, 0, CltvExpiryDelta(36), 1 msat, Some(400_000 msat)))) :: Nil, ) { @@ -551,14 +551,14 @@ class RouterSpec extends BaseRouterSpec { sender.send(router, RouteRequest(a, recipient, DEFAULT_ROUTE_PARAMS, allowMultiPart = true)) val routes = sender.expectMsgType[RouteResponse].routes assert(routes.length == 2) - assert(routes.flatMap(_.finalHop_opt) == recipient.blindedPaths.map{case (alias, blindedPath) => BlindedHop(alias, blindedPath.route.asInstanceOf[OfferTypes.BlindedPath].route, blindedPath.paymentInfo)}) + assert(routes.flatMap(_.finalHop_opt) == recipient.blindedHops) assert(routes.map(route => route2NodeIds(route)).toSet == Set(Seq(a, b), Seq(a, b, c))) assert(routes.map(route => route.blindedFee + route.channelFee(false)).toSet == Set(510 msat, 800 msat)) } { // One blinded route is ignored, we use the other one: val (_, recipient) = blindedRoutesFromPaths(300_000 msat, DEFAULT_EXPIRY, hopsToRecipient, DEFAULT_EXPIRY) - val ignored = Ignore(Set.empty, Set(ChannelDesc(recipient.extraEdges.last.shortChannelId, recipient.extraEdges.last.sourceNodeId.toOption.get, recipient.extraEdges.last.targetNodeId))) + val ignored = Ignore(Set.empty, Set(ChannelDesc(recipient.extraEdges.last.shortChannelId, recipient.extraEdges.last.sourceNodeId, recipient.extraEdges.last.targetNodeId))) sender.send(router, RouteRequest(a, recipient, DEFAULT_ROUTE_PARAMS, ignore = ignored)) val routes = sender.expectMsgType[RouteResponse].routes assert(routes.length == 1) @@ -569,7 +569,7 @@ class RouterSpec extends BaseRouterSpec { { // One blinded route is ignored, the other one doesn't have enough capacity: val (_, recipient) = blindedRoutesFromPaths(500_000 msat, DEFAULT_EXPIRY, hopsToRecipient, DEFAULT_EXPIRY) - val ignored = Ignore(Set.empty, Set(ChannelDesc(recipient.extraEdges.last.shortChannelId, recipient.extraEdges.last.sourceNodeId.toOption.get, recipient.extraEdges.last.targetNodeId))) + val ignored = Ignore(Set.empty, Set(ChannelDesc(recipient.extraEdges.last.shortChannelId, recipient.extraEdges.last.sourceNodeId, recipient.extraEdges.last.targetNodeId))) sender.send(router, RouteRequest(a, recipient, DEFAULT_ROUTE_PARAMS, allowMultiPart = true, ignore = ignored)) sender.expectMsg(Failure(RouteNotFound)) } @@ -834,7 +834,7 @@ class RouterSpec extends BaseRouterSpec { { val amount = 10_000.msat - val invoiceRoutingHint = Invoice.ExtraEdge(Right(b), targetNodeId, RealShortChannelId(BlockHeight(420000), 516, 1105), 10 msat, 150, CltvExpiryDelta(96), 1 msat, None) + val invoiceRoutingHint = Invoice.ExtraEdge(b, targetNodeId, RealShortChannelId(BlockHeight(420000), 516, 1105), 10 msat, 150, CltvExpiryDelta(96), 1 msat, None) val preComputedRoute = PredefinedChannelRoute(amount, targetNodeId, Seq(scid_ab, invoiceRoutingHint.shortChannelId)) // the amount affects the way we estimate the channel capacity of the hinted channel assert(amount < RoutingHeuristics.CAPACITY_CHANNEL_LOW) @@ -850,7 +850,7 @@ class RouterSpec extends BaseRouterSpec { } { val amount = RoutingHeuristics.CAPACITY_CHANNEL_LOW * 2 - val invoiceRoutingHint = Invoice.ExtraEdge(Right(h), targetNodeId, RealShortChannelId(BlockHeight(420000), 516, 1105), 10 msat, 150, CltvExpiryDelta(96), 1 msat, None) + val invoiceRoutingHint = Invoice.ExtraEdge(h, targetNodeId, RealShortChannelId(BlockHeight(420000), 516, 1105), 10 msat, 150, CltvExpiryDelta(96), 1 msat, None) val preComputedRoute = PredefinedChannelRoute(amount, targetNodeId, Seq(scid_ag_private, scid_gh, invoiceRoutingHint.shortChannelId)) // the amount affects the way we estimate the channel capacity of the hinted channel assert(amount > RoutingHeuristics.CAPACITY_CHANNEL_LOW) From dabfb795f891059fcae6e4a351f1aead2e84a1e3 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Thu, 9 Nov 2023 17:11:25 +0100 Subject: [PATCH 07/12] small improvements --- .../scala/fr/acinq/eclair/wire/protocol/OfferCodecs.scala | 8 +------- .../scala/fr/acinq/eclair/wire/protocol/OfferTypes.scala | 7 ++++++- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferCodecs.scala index 420778c72f..f71e2d7f28 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferCodecs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferCodecs.scala @@ -65,13 +65,7 @@ object OfferCodecs { ("blinding" | publicKey) :: ("path" | blindedNodesCodec)).as[CompactBlindedPath] - private val pathCodec: Codec[BlindedContactInfo] = fallback(blindedPathCodec, compactBlindedPathCodec).xmap({ - case Left(path) => path - case Right(compact) => compact - }, { - case path: BlindedPath => Left(path) - case compact: CompactBlindedPath => Right(compact) - }) + private val pathCodec: Codec[BlindedContactInfo] = choice(compactBlindedPathCodec.upcast[BlindedContactInfo], blindedPathCodec.upcast[BlindedContactInfo]) private val offerPaths: Codec[OfferPaths] = tlvField(list(pathCodec).xmap[Seq[BlindedContactInfo]](_.toSeq, _.toList)) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferTypes.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferTypes.scala index 4f6c183bb6..6d49408b96 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferTypes.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferTypes.scala @@ -37,11 +37,16 @@ import scala.util.{Failure, Try} object OfferTypes { case class ShortChannelIdDir(isNode1: Boolean, scid: RealShortChannelId) + // @formatter:off + /** Data provided to reach the issuer of an offer or invoice. */ sealed trait ContactInfo + /** If the offer or invoice issuer doesn't want to hide their identity, they can directly share their public nodeId. */ + case class RecipientNodeId(nodeId: PublicKey) extends ContactInfo + /** If the offer or invoice issuer wants to hide their identity, they instead provide blinded paths. */ sealed trait BlindedContactInfo extends ContactInfo case class BlindedPath(route: BlindedRoute) extends BlindedContactInfo case class CompactBlindedPath(introductionNode: ShortChannelIdDir, blindingKey: PublicKey, blindedNodes: Seq[BlindedNode]) extends BlindedContactInfo - case class RecipientNodeId(nodeId: PublicKey) extends ContactInfo + // @formatter:on sealed trait Bolt12Tlv extends Tlv From ca140110d69afdffa3b3ed3e1a2ea738bf34f400 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Fri, 10 Nov 2023 14:15:34 +0100 Subject: [PATCH 08/12] More tests --- .../fr/acinq/eclair/message/Postman.scala | 2 +- .../payment/send/PaymentInitiator.scala | 2 +- .../eclair/wire/protocol/OfferCodecs.scala | 3 +- .../eclair/payment/PaymentInitiatorSpec.scala | 74 ++++++++++++++++++- .../eclair/wire/protocol/OfferTypesSpec.scala | 25 ++++++- 5 files changed, 98 insertions(+), 8 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/message/Postman.scala b/eclair-core/src/main/scala/fr/acinq/eclair/message/Postman.scala index 2f6842d958..2f805cebd0 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/message/Postman.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/message/Postman.scala @@ -161,7 +161,7 @@ private class SendingMessage(nodeParams: NodeParams, private def waitForNodeId(compactBlindedPath: CompactBlindedPath): Behavior[Command] = { Behaviors.receiveMessagePartial { case WrappedNodeIdResponse(None) => - replyTo ! Postman.MessageFailed("Unknown target") + replyTo ! Postman.MessageFailed(s"Could not resolve introduction node for compact blinded path (scid=${compactBlindedPath.introductionNode.scid.toCoordinatesString})") Behaviors.stopped case WrappedNodeIdResponse(Some(nodeId)) => sendToDestination(OnionMessages.BlindedPath(BlindedRoute(nodeId, compactBlindedPath.blindingKey, compactBlindedPath.blindedNodes))) 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 515c986b08..9204a4489e 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 @@ -47,7 +47,7 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn case r: SendPaymentToNode => val replyTo = if (r.replyTo == ActorRef.noSender) sender() else r.replyTo val paymentCfg = r.paymentConfig() - if (!r.blockUntilComplete) { + if (!r.blockUntilComplete && r.paymentConfig_opt.isEmpty) { // Immediately return the paymentId replyTo ! paymentCfg.id } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferCodecs.scala index f71e2d7f28..913036f8ac 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferCodecs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/OfferCodecs.scala @@ -65,13 +65,12 @@ object OfferCodecs { ("blinding" | publicKey) :: ("path" | blindedNodesCodec)).as[CompactBlindedPath] - private val pathCodec: Codec[BlindedContactInfo] = choice(compactBlindedPathCodec.upcast[BlindedContactInfo], blindedPathCodec.upcast[BlindedContactInfo]) + val pathCodec: Codec[BlindedContactInfo] = choice(compactBlindedPathCodec.upcast[BlindedContactInfo], blindedPathCodec.upcast[BlindedContactInfo]) private val offerPaths: Codec[OfferPaths] = tlvField(list(pathCodec).xmap[Seq[BlindedContactInfo]](_.toSeq, _.toList)) private val offerIssuer: Codec[OfferIssuer] = tlvField(utf8) - private val offerQuantityMax: Codec[OfferQuantityMax] = tlvField(tu64overflow) private val offerNodeId: Codec[OfferNodeId] = tlvField(publicKey) 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 9dd5a0cfd5..27218a53c2 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} +import fr.acinq.eclair.router.{BlindedRouteCreation, RouteNotFound, Router} import fr.acinq.eclair.wire.protocol.OfferTypes.{InvoiceRequest, Offer} import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{Bolt11Feature, Bolt12Feature, CltvExpiry, CltvExpiryDelta, Feature, Features, MilliSatoshiLong, NodeParams, PaymentFinalExpiryConf, TestConstants, TestKitBaseClass, TimestampSecond, UnknownFeature, randomBytes32, randomKey} +import fr.acinq.eclair.{Bolt11Feature, Bolt12Feature, CltvExpiry, CltvExpiryDelta, Feature, Features, MilliSatoshiLong, NodeParams, PaymentFinalExpiryConf, RealShortChannelId, TestConstants, TestKitBaseClass, TimestampSecond, UnknownFeature, randomBytes32, randomKey} import org.scalatest.funsuite.FixtureAnyFunSuiteLike import org.scalatest.{Outcome, Tag} import scodec.bits.{ByteVector, HexStringSyntax} @@ -372,6 +372,76 @@ 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)))) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/OfferTypesSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/OfferTypesSpec.scala index 59079b09d8..2a9cba1c1b 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/OfferTypesSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/OfferTypesSpec.scala @@ -17,13 +17,14 @@ package fr.acinq.eclair.wire.protocol import fr.acinq.bitcoin.Bech32 -import fr.acinq.bitcoin.scalacompat.Crypto.PrivateKey +import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32} import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional} import fr.acinq.eclair.Features.BasicMultiPartPayment +import fr.acinq.eclair.crypto.Sphinx.RouteBlinding.{BlindedNode, BlindedRoute} import fr.acinq.eclair.wire.protocol.OfferCodecs.{invoiceRequestTlvCodec, offerTlvCodec} import fr.acinq.eclair.wire.protocol.OfferTypes._ -import fr.acinq.eclair.{Features, MilliSatoshiLong, randomBytes32, randomKey} +import fr.acinq.eclair.{Features, MilliSatoshiLong, RealShortChannelId, randomBytes32, randomKey} import org.scalatest.funsuite.AnyFunSuite import scodec.bits.{ByteVector, HexStringSyntax} @@ -261,4 +262,24 @@ class OfferTypesSpec extends AnyFunSuite { } } + test("compact blinded route") { + case class TestCase(encoded: ByteVector, decoded: BlindedContactInfo) + + val testCases = Seq( + TestCase(hex"00 00000000000004d2 0379b470d00b78ded936f8972a0f3ecda2bb6e6df40dcd581dbaeb3742b30008ff 01 02fba71b72623187dd24670110eec870e28b848f255ba2edc0486d3a8e89ec44b7 0002 1dea", + CompactBlindedPath(ShortChannelIdDir(isNode1 = true, RealShortChannelId(1234)), PublicKey(hex"0379b470d00b78ded936f8972a0f3ecda2bb6e6df40dcd581dbaeb3742b30008ff"), Seq(BlindedNode(PublicKey(hex"02fba71b72623187dd24670110eec870e28b848f255ba2edc0486d3a8e89ec44b7"), hex"1dea")))), + TestCase(hex"01 000000000000ddd5 0353a081bb02d6e361be3df3e92b41b788ca65667f6ea0c01e2bfa03664460ef86 01 03bce3f0cdb4172caac82ec8a9251eb35df1201bdcb977c5a03f3624ec4156a65f 0003 c0ffee", + CompactBlindedPath(ShortChannelIdDir(isNode1 = false, RealShortChannelId(56789)), PublicKey(hex"0353a081bb02d6e361be3df3e92b41b788ca65667f6ea0c01e2bfa03664460ef86"), Seq(BlindedNode(PublicKey(hex"03bce3f0cdb4172caac82ec8a9251eb35df1201bdcb977c5a03f3624ec4156a65f"), hex"c0ffee")))), + TestCase(hex"022d3b15cea00ee4a8e710b082bef18f0f3409cc4e7aff41c26eb0a4d3ab20dd73 0379a3b6e4bceb7519d09db776994b1f82cf6a9fa4d3ec2e52314c5938f2f9f966 01 02b446aaa523df82a992ab468e5298eabb6168e2c466455c210d8c97dbb8981328 0002 cafe", + BlindedPath(BlindedRoute(PublicKey(hex"022d3b15cea00ee4a8e710b082bef18f0f3409cc4e7aff41c26eb0a4d3ab20dd73"), PublicKey(hex"0379a3b6e4bceb7519d09db776994b1f82cf6a9fa4d3ec2e52314c5938f2f9f966"), Seq(BlindedNode(PublicKey(hex"02b446aaa523df82a992ab468e5298eabb6168e2c466455c210d8c97dbb8981328"), hex"cafe"))))), + TestCase(hex"03ba3c458e3299eb19d2e07ae86453f4290bcdf8689707f0862f35194397c45922 028aa5d1a10463d598a0a0ab7296af21619049f94fe03ef664a87561009e58c3dd 01 02988d7381d0434cfebbe521031505fb9987ae6cefd0bab0e5927852eb96bb6cc2 0003 ec1a13", + BlindedPath(BlindedRoute(PublicKey(hex"03ba3c458e3299eb19d2e07ae86453f4290bcdf8689707f0862f35194397c45922"), PublicKey(hex"028aa5d1a10463d598a0a0ab7296af21619049f94fe03ef664a87561009e58c3dd"), Seq(BlindedNode(PublicKey(hex"02988d7381d0434cfebbe521031505fb9987ae6cefd0bab0e5927852eb96bb6cc2"), hex"ec1a13"))))), + ) + + testCases.foreach { + case TestCase(encoded, decoded) => + assert(OfferCodecs.pathCodec.encode(decoded).require.bytes == encoded) + assert(OfferCodecs.pathCodec.decode(encoded.bits).require.value == decoded) + } + } } From c53b52c4bb3ac3ea182293e43f00b92d7c65d513 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Mon, 13 Nov 2023 13:41:10 +0100 Subject: [PATCH 09/12] 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) } From 0e47682d55929cad7d245681608b9611569ee693 Mon Sep 17 00:00:00 2001 From: t-bast Date: Mon, 13 Nov 2023 16:07:55 +0100 Subject: [PATCH 10/12] Rename `ResolvedPaymentBlindedRoute` This is just an automated renaming: * PaymentBlindedRoute -> PaymentBlindedContactInfo * ResolvedPaymentBlindedRoute -> PaymentBlindedRoute --- .../acinq/eclair/payment/Bolt12Invoice.scala | 8 ++++---- .../payment/receive/MultiPartHandler.scala | 4 ++-- .../eclair/payment/send/OfferPayment.scala | 20 +++++++++---------- .../payment/send/PaymentInitiator.scala | 4 ++-- .../acinq/eclair/payment/send/Recipient.scala | 4 ++-- .../fr/acinq/eclair/db/PaymentsDbSpec.scala | 2 +- .../eclair/payment/Bolt12InvoiceSpec.scala | 4 ++-- .../eclair/payment/PaymentInitiatorSpec.scala | 8 ++++---- .../eclair/payment/PaymentPacketSpec.scala | 8 ++++---- .../payment/send/OfferPaymentSpec.scala | 18 ++++++++--------- .../acinq/eclair/router/BaseRouterSpec.scala | 6 +++--- .../protocol/MessageOnionCodecsSpec.scala | 4 ++-- 12 files changed, 45 insertions(+), 45 deletions(-) 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 03c4ab715d..5963479bc4 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 @@ -53,7 +53,7 @@ case class Bolt12Invoice(records: TlvStream[InvoiceTlv]) extends Invoice { // We add invoice features that are implicitly required for Bolt 12 (the spec doesn't allow explicitly setting them). f.add(Features.VariableLengthOnion, FeatureSupport.Mandatory).add(Features.RouteBlinding, FeatureSupport.Mandatory) } - val blindedPaths: Seq[PaymentBlindedRoute] = records.get[InvoicePaths].get.paths.zip(records.get[InvoiceBlindedPay].get.paymentInfo).map { case (route, info) => PaymentBlindedRoute(route, info) } + val blindedPaths: Seq[PaymentBlindedContactInfo] = records.get[InvoicePaths].get.paths.zip(records.get[InvoiceBlindedPay].get.paymentInfo).map { case (route, info) => PaymentBlindedContactInfo(route, info) } val fallbacks: Option[Seq[FallbackAddress]] = records.get[InvoiceFallbacks].map(_.addresses) val signature: ByteVector64 = records.get[Signature].get.signature @@ -87,9 +87,9 @@ case class Bolt12Invoice(records: TlvStream[InvoiceTlv]) extends Invoice { } -case class PaymentBlindedRoute(route: BlindedContactInfo, paymentInfo: PaymentInfo) +case class PaymentBlindedContactInfo(route: BlindedContactInfo, paymentInfo: PaymentInfo) -case class ResolvedPaymentBlindedRoute(route: BlindedRoute, paymentInfo: PaymentInfo) +case class PaymentBlindedRoute(route: BlindedRoute, paymentInfo: PaymentInfo) object Bolt12Invoice { val hrp = "lni" @@ -110,7 +110,7 @@ object Bolt12Invoice { nodeKey: PrivateKey, invoiceExpiry: FiniteDuration, features: Features[Bolt12Feature], - paths: Seq[PaymentBlindedRoute], + paths: Seq[PaymentBlindedContactInfo], additionalTlvs: Set[InvoiceTlv] = Set.empty, customTlvs: Set[GenericTlv] = Set.empty): Bolt12Invoice = { require(request.amount.nonEmpty || request.offer.amount.nonEmpty) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala index a1c1e262e4..7764f11128 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/receive/MultiPartHandler.scala @@ -373,7 +373,7 @@ object MultiPartHandler { case None => OfferTypes.BlindedPath(blindedRoute.route) } val paymentInfo = aggregatePaymentInfo(r.amount, dummyHops, nodeParams.channelConf.minFinalExpiryDelta) - Future.successful(PaymentBlindedRoute(contactInfo, paymentInfo)) + Future.successful(PaymentBlindedContactInfo(contactInfo, paymentInfo)) } else { implicit val timeout: Timeout = 10.seconds r.router.ask(Router.FinalizeRoute(Router.PredefinedNodeRoute(r.amount, route.nodes))).mapTo[Router.RouteResponse].map(routeResponse => { @@ -384,7 +384,7 @@ object MultiPartHandler { case None => OfferTypes.BlindedPath(blindedRoute.route) } val paymentInfo = aggregatePaymentInfo(r.amount, clearRoute.hops ++ dummyHops, nodeParams.channelConf.minFinalExpiryDelta) - PaymentBlindedRoute(contactInfo, paymentInfo) + PaymentBlindedContactInfo(contactInfo, paymentInfo) }) } })).map(paths => { 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 213300a582..81d492e756 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 @@ -25,7 +25,7 @@ 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.payment.{PaymentBlindedContactInfo, PaymentBlindedRoute} import fr.acinq.eclair.router.Router import fr.acinq.eclair.router.Router.RouteParams import fr.acinq.eclair.wire.protocol.MessageOnion.{FinalPayload, InvoicePayload} @@ -134,7 +134,7 @@ object OfferPayment { Behaviors.receiveMessagePartial { case WrappedMessageResponse(Postman.Response(payload: InvoicePayload)) if payload.invoice.validateFor(request).isRight => 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 } + val scids = payload.invoice.blindedPaths.collect { case PaymentBlindedContactInfo(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. @@ -154,8 +154,8 @@ object OfferPayment { paymentInitiator: ActorRef, router: ActorRef, sendPaymentToNode: SendPaymentToNode, - toResolve: Seq[PaymentBlindedRoute], - resolved: Seq[ResolvedPaymentBlindedRoute], + toResolve: Seq[PaymentBlindedContactInfo], + resolved: Seq[PaymentBlindedRoute], scids: Seq[RealShortChannelId]): Behavior[Command] = { if (toResolve.isEmpty) { if (resolved.isEmpty) { @@ -167,9 +167,9 @@ object OfferPayment { 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) => + case PaymentBlindedContactInfo(BlindedPath(route), paymentInfo) => + resolve(context, paymentInitiator, router, sendPaymentToNode, toResolve.tail, resolved :+ PaymentBlindedRoute(route, paymentInfo), scids) + case PaymentBlindedContactInfo(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) } @@ -182,14 +182,14 @@ object OfferPayment { sendPaymentToNode: SendPaymentToNode, compactRoute: CompactBlindedPath, paymentInfo: PaymentInfo, - toResolve: Seq[PaymentBlindedRoute], - resolved: Seq[ResolvedPaymentBlindedRoute], + toResolve: Seq[PaymentBlindedContactInfo], + resolved: Seq[PaymentBlindedRoute], 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) + val resolvedPaymentBlindedRoute = PaymentBlindedRoute(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 032c5184ea..806ec7fdba 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 @@ -320,7 +320,7 @@ object PaymentInitiator { case class SendPaymentToNode(replyTo: ActorRef, recipientAmount: MilliSatoshi, invoice: Invoice, - resolvedPaths: Seq[ResolvedPaymentBlindedRoute], + resolvedPaths: Seq[PaymentBlindedRoute], maxAttempts: Int, externalId: Option[String] = None, routeParams: RouteParams, @@ -384,7 +384,7 @@ object PaymentInitiator { */ case class SendPaymentToRoute(recipientAmount: MilliSatoshi, invoice: Invoice, - resolvedPaths: Seq[ResolvedPaymentBlindedRoute], + resolvedPaths: Seq[PaymentBlindedRoute], route: PredefinedRoute, externalId: Option[String], parentId: Option[UUID], 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 3eb95ef5df..dce6a65128 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,7 +21,7 @@ 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, ResolvedPaymentBlindedRoute} +import fr.acinq.eclair.payment.{Bolt11Invoice, Bolt12Invoice, OutgoingPaymentPacket, PaymentBlindedRoute} import fr.acinq.eclair.router.Router._ import fr.acinq.eclair.wire.protocol.PaymentOnion.{FinalPayload, IntermediatePayload, OutgoingBlindedPerHopPayload} import fr.acinq.eclair.wire.protocol.{GenericTlv, OnionRoutingPacket, PaymentOnionCodecs} @@ -166,7 +166,7 @@ case class BlindedRecipient(nodeId: PublicKey, } object BlindedRecipient { - def apply(invoice: Bolt12Invoice, paths: Seq[ResolvedPaymentBlindedRoute], totalAmount: MilliSatoshi, expiry: CltvExpiry, customTlvs: Set[GenericTlv]): BlindedRecipient = { + def apply(invoice: Bolt12Invoice, paths: Seq[PaymentBlindedRoute], 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 diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala index 81d731d2a1..1eddf80102 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/PaymentsDbSpec.scala @@ -801,7 +801,7 @@ object PaymentsDbSpec { def createBolt12Invoice(amount: MilliSatoshi, payerKey: PrivateKey, recipientKey: PrivateKey, preimage: ByteVector32): Bolt12Invoice = { val offer = Offer(Some(amount), "some offer", recipientKey.publicKey, Features.empty, Block.TestnetGenesisBlock.hash) val invoiceRequest = InvoiceRequest(offer, 789 msat, 1, Features.empty, payerKey, Block.TestnetGenesisBlock.hash) - val dummyRoute = PaymentBlindedRoute(BlindedPath(RouteBlinding.create(randomKey(), Seq(randomKey().publicKey), Seq(randomBytes(100))).route), PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 0 msat, Features.empty)) + val dummyRoute = PaymentBlindedContactInfo(BlindedPath(RouteBlinding.create(randomKey(), Seq(randomKey().publicKey), Seq(randomBytes(100))).route), PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 0 msat, Features.empty)) Bolt12Invoice(invoiceRequest, preimage, recipientKey, 1 hour, Features.empty, Seq(dummyRoute)) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt12InvoiceSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt12InvoiceSpec.scala index 99d6d9c3c2..b9538c7ab5 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt12InvoiceSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/Bolt12InvoiceSpec.scala @@ -49,9 +49,9 @@ class Bolt12InvoiceSpec extends AnyFunSuite { signedInvoice } - def createPaymentBlindedRoute(nodeId: PublicKey, sessionKey: PrivateKey = randomKey(), pathId: ByteVector = randomBytes32()): PaymentBlindedRoute = { + def createPaymentBlindedRoute(nodeId: PublicKey, sessionKey: PrivateKey = randomKey(), pathId: ByteVector = randomBytes32()): PaymentBlindedContactInfo = { val selfPayload = blindedRouteDataCodec.encode(TlvStream(PathId(pathId), PaymentConstraints(CltvExpiry(1234567), 0 msat), AllowedFeatures(Features.empty))).require.bytes - PaymentBlindedRoute(OfferTypes.BlindedPath(Sphinx.RouteBlinding.create(sessionKey, Seq(nodeId), Seq(selfPayload)).route), PaymentInfo(1 msat, 2, CltvExpiryDelta(3), 4 msat, 5 msat, Features.empty)) + PaymentBlindedContactInfo(OfferTypes.BlindedPath(Sphinx.RouteBlinding.create(sessionKey, Seq(nodeId), Seq(selfPayload)).route), PaymentInfo(1 msat, 2, CltvExpiryDelta(3), 4 msat, 5 msat, Features.empty)) } test("check invoice signature") { 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 6509d5350c..ef8fd8df15 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 @@ -298,14 +298,14 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike val invoiceRequest = InvoiceRequest(offer, finalAmount, 1, features, randomKey(), Block.RegtestGenesisBlock.hash) val blindedRoute = OfferTypes.BlindedPath(BlindedRouteCreation.createBlindedRouteWithoutHops(e, hex"2a2a2a2a", 1 msat, CltvExpiry(500_000)).route) val paymentInfo = OfferTypes.PaymentInfo(1_000 msat, 0, CltvExpiryDelta(24), 0 msat, finalAmount, Features.empty) - Bolt12Invoice(invoiceRequest, paymentPreimage, priv_e.privateKey, 300 seconds, features, Seq(PaymentBlindedRoute(blindedRoute, paymentInfo))) + Bolt12Invoice(invoiceRequest, paymentPreimage, priv_e.privateKey, 300 seconds, features, Seq(PaymentBlindedContactInfo(blindedRoute, paymentInfo))) } test("forward single-part blinded payment") { f => import f._ val payerKey = randomKey() val invoice = createBolt12Invoice(Features.empty, payerKey) - val resolvedPaths = invoice.blindedPaths.map(path => ResolvedPaymentBlindedRoute(path.route.asInstanceOf[BlindedPath].route, path.paymentInfo)) + val resolvedPaths = invoice.blindedPaths.map(path => PaymentBlindedRoute(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] @@ -336,7 +336,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike import f._ val payerKey = randomKey() val invoice = createBolt12Invoice(Features(BasicMultiPartPayment -> Optional), payerKey) - val resolvedPaths = invoice.blindedPaths.map(path => ResolvedPaymentBlindedRoute(path.route.asInstanceOf[BlindedPath].route, path.paymentInfo)) + val resolvedPaths = invoice.blindedPaths.map(path => PaymentBlindedRoute(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] @@ -365,7 +365,7 @@ 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 resolvedPaths = invoice.blindedPaths.map(path => ResolvedPaymentBlindedRoute(path.route.asInstanceOf[BlindedPath].route, path.paymentInfo)) + val resolvedPaths = invoice.blindedPaths.map(path => PaymentBlindedRoute(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] 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 5893a57361..128b360929 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 @@ -219,8 +219,8 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { val invoiceRequest = InvoiceRequest(offer, amount_bc, 1, features, randomKey(), Block.RegtestGenesisBlock.hash) 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 resolvedPaths = invoice.blindedPaths.map(path => ResolvedPaymentBlindedRoute(path.route.asInstanceOf[BlindedPath].route, path.paymentInfo)) + val invoice = Bolt12Invoice(invoiceRequest, paymentPreimage, recipientKey, 300 seconds, features, Seq(PaymentBlindedContactInfo(blindedRoute, paymentInfo))) + val resolvedPaths = invoice.blindedPaths.map(path => PaymentBlindedRoute(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) @@ -471,8 +471,8 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll { val tmpBlindedRoute = BlindedRouteCreation.createBlindedRouteFromHops(Seq(channelHopFromUpdate(b, c, channelUpdate_bc)), hex"deadbeef", 1 msat, CltvExpiry(500_000)).route 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 resolvedPaths = invoice.blindedPaths.map(path => ResolvedPaymentBlindedRoute(path.route.asInstanceOf[BlindedPath].route, path.paymentInfo)) + val invoice = Bolt12Invoice(invoiceRequest, paymentPreimage, priv_c.privateKey, 300 seconds, features, Seq(PaymentBlindedContactInfo(blindedRoute, paymentInfo))) + val resolvedPaths = invoice.blindedPaths.map(path => PaymentBlindedRoute(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 100474c164..b7b0f5a2f3 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 @@ -26,7 +26,7 @@ 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.payment.{Bolt12Invoice, PaymentBlindedContactInfo} import fr.acinq.eclair.router.Router import fr.acinq.eclair.router.Router.RouteParams import fr.acinq.eclair.wire.protocol.MessageOnion.InvoicePayload @@ -74,7 +74,7 @@ class OfferPaymentSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app val Right(invoiceRequest) = InvoiceRequest.validate(message.get[OnionMessagePayloadTlv.InvoiceRequest].get.tlvs) val preimage = randomBytes32() - val paymentRoute = PaymentBlindedRoute(OfferTypes.BlindedPath(RouteBlinding.create(randomKey(), Seq(merchantKey.publicKey), Seq(hex"7777")).route), PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 1_000_000_000 msat, Features.empty)) + val paymentRoute = PaymentBlindedContactInfo(OfferTypes.BlindedPath(RouteBlinding.create(randomKey(), Seq(merchantKey.publicKey), Seq(hex"7777")).route), PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 1_000_000_000 msat, Features.empty)) val invoice = Bolt12Invoice(invoiceRequest, preimage, merchantKey, 1 minute, Features.empty, Seq(paymentRoute)) replyTo ! Postman.Response(InvoicePayload(TlvStream(OnionMessagePayloadTlv.Invoice(invoice.records)), TlvStream.empty)) val send = paymentInitiator.expectMsgType[SendPaymentToNode] @@ -121,7 +121,7 @@ class OfferPaymentSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app val Right(invoiceRequest) = InvoiceRequest.validate(message.get[OnionMessagePayloadTlv.InvoiceRequest].get.tlvs) val preimage = randomBytes32() - val paymentRoute = PaymentBlindedRoute(OfferTypes.BlindedPath(RouteBlinding.create(randomKey(), Seq(merchantKey.publicKey), Seq(hex"7777")).route), PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 1_000_000_000 msat, Features.empty)) + val paymentRoute = PaymentBlindedContactInfo(OfferTypes.BlindedPath(RouteBlinding.create(randomKey(), Seq(merchantKey.publicKey), Seq(hex"7777")).route), PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 1_000_000_000 msat, Features.empty)) val invoice = Bolt12Invoice(invoiceRequest, preimage, randomKey(), 1 minute, Features.empty, Seq(paymentRoute)) replyTo ! Postman.Response(InvoicePayload(TlvStream(OnionMessagePayloadTlv.Invoice(invoice.records)), TlvStream.empty)) @@ -148,12 +148,12 @@ class OfferPaymentSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app 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)), + PaymentBlindedContactInfo(OfferTypes.BlindedPath(blindedRoutes(0)), PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 1_000_000_000 msat, Features.empty)), + PaymentBlindedContactInfo(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)), + PaymentBlindedContactInfo(OfferTypes.BlindedPath(blindedRoutes(2)), PaymentInfo(2 msat, 22, CltvExpiryDelta(222), 0 msat, 1_000_000_000 msat, Features.empty)), + PaymentBlindedContactInfo(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)), + PaymentBlindedContactInfo(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)), + PaymentBlindedContactInfo(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)) 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 d69212665b..633cf0bea9 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,7 +31,7 @@ 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, ResolvedPaymentBlindedRoute} +import fr.acinq.eclair.payment.{Bolt12Invoice, PaymentBlindedContactInfo, PaymentBlindedRoute} import fr.acinq.eclair.router.Announcements._ import fr.acinq.eclair.router.BaseRouterSpec.channelAnnouncement import fr.acinq.eclair.router.Router._ @@ -272,10 +272,10 @@ object BaseRouterSpec { val blindedRoutes = paths.map(hops => { val blindedRoute = OfferTypes.BlindedPath(BlindedRouteCreation.createBlindedRouteFromHops(hops, pathId, 1 msat, routeExpiry).route) val paymentInfo = BlindedRouteCreation.aggregatePaymentInfo(amount, hops, Channel.MIN_CLTV_EXPIRY_DELTA) - PaymentBlindedRoute(blindedRoute, paymentInfo) + PaymentBlindedContactInfo(blindedRoute, paymentInfo) }) val invoice = Bolt12Invoice(invoiceRequest, preimage, recipientKey, 300 seconds, features, blindedRoutes) - val resolvedPaths = invoice.blindedPaths.map(path => ResolvedPaymentBlindedRoute(path.route.asInstanceOf[BlindedPath].route, path.paymentInfo)) + val resolvedPaths = invoice.blindedPaths.map(path => PaymentBlindedRoute(path.route.asInstanceOf[BlindedPath].route, path.paymentInfo)) val recipient = BlindedRecipient(invoice, resolvedPaths, amount, expiry, Set.empty) (invoice, recipient) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/MessageOnionCodecsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/MessageOnionCodecsSpec.scala index 58b5d65e27..84c186298b 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/MessageOnionCodecsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/MessageOnionCodecsSpec.scala @@ -4,7 +4,7 @@ import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32} import fr.acinq.eclair.crypto.Sphinx import fr.acinq.eclair.crypto.Sphinx.RouteBlinding -import fr.acinq.eclair.payment.{Bolt12Invoice, PaymentBlindedRoute} +import fr.acinq.eclair.payment.{Bolt12Invoice, PaymentBlindedContactInfo} import fr.acinq.eclair.wire.protocol.MessageOnion.{FinalPayload, IntermediatePayload, InvalidResponsePayload, InvoiceErrorPayload, InvoicePayload, InvoiceRequestPayload} import fr.acinq.eclair.wire.protocol.MessageOnionCodecs._ import fr.acinq.eclair.wire.protocol.OfferTypes.PaymentInfo @@ -95,7 +95,7 @@ class MessageOnionCodecsSpec extends AnyFunSuiteLike { val payerKey = randomKey() val request = OfferTypes.InvoiceRequest(offer, 100_000 msat, 1, Features.empty, payerKey, Block.LivenetGenesisBlock.hash) val selfPayload = blindedRouteDataCodec.encode(TlvStream(PathId(randomBytes32()), PaymentConstraints(CltvExpiry(1234567), 0 msat), AllowedFeatures(Features.empty))).require.bytes - val route = PaymentBlindedRoute(OfferTypes.BlindedPath(Sphinx.RouteBlinding.create(randomKey(), Seq(nodeKey.publicKey), Seq(selfPayload)).route), PaymentInfo(1 msat, 2, CltvExpiryDelta(3), 4 msat, 5 msat, Features.empty)) + val route = PaymentBlindedContactInfo(OfferTypes.BlindedPath(Sphinx.RouteBlinding.create(randomKey(), Seq(nodeKey.publicKey), Seq(selfPayload)).route), PaymentInfo(1 msat, 2, CltvExpiryDelta(3), 4 msat, 5 msat, Features.empty)) val invoice = Bolt12Invoice(request, randomBytes32(), nodeKey, 300 seconds, Features.empty, Seq(route)) val testCasesInvalid = Seq[TlvStream[OnionMessagePayloadTlv]]( From 91201d8e1daa17bb683f142a4186bb874d63c8fc Mon Sep 17 00:00:00 2001 From: t-bast Date: Tue, 14 Nov 2023 09:11:53 +0100 Subject: [PATCH 11/12] Refactor `OfferPayment` Remove parameter duplication by using a private class. --- .../eclair/payment/send/OfferPayment.scala | 107 ++++++++---------- .../acinq/eclair/payment/send/Recipient.scala | 1 + .../integration/PaymentIntegrationSpec.scala | 23 ++-- .../fr/acinq/eclair/message/PostmanSpec.scala | 16 +-- 4 files changed, 75 insertions(+), 72 deletions(-) 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 81d492e756..7f59b6727c 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 @@ -25,14 +25,16 @@ 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.{PaymentBlindedContactInfo, PaymentBlindedRoute} +import fr.acinq.eclair.payment.{Bolt12Invoice, PaymentBlindedContactInfo, PaymentBlindedRoute} 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.{BlindedPath, CompactBlindedPath, InvoiceRequest, Offer, PaymentInfo} +import fr.acinq.eclair.wire.protocol.OfferTypes._ import fr.acinq.eclair.wire.protocol.{OnionMessagePayloadTlv, TlvStream} import fr.acinq.eclair.{Features, InvoiceFeature, MilliSatoshi, NodeParams, RealShortChannelId, TimestampSecond, randomKey} +import scala.annotation.tailrec + object OfferPayment { sealed trait Failure @@ -53,7 +55,7 @@ object OfferPayment { } case class UnknownShortChannelIds(scids: Seq[RealShortChannelId]) extends Failure { - override def toString: String = s"Unknown short channel ids: $scids" + override def toString: String = s"Unknown short channel ids: ${scids.mkString(",")}" } sealed trait Command @@ -99,50 +101,44 @@ object OfferPayment { } else { val payerKey = randomKey() val request = InvoiceRequest(offer, amount, quantity, nodeParams.features.bolt12Features(), payerKey, nodeParams.chainHash) - sendInvoiceRequest(nodeParams, postman, router, paymentInitiator, context, request, payerKey, replyTo, 0, sendPaymentConfig) + val offerPayment = new OfferPayment(replyTo, nodeParams, postman, router, paymentInitiator, payerKey, request, sendPaymentConfig, context) + offerPayment.sendInvoiceRequest(attemptNumber = 0) } }) } - - def sendInvoiceRequest(nodeParams: NodeParams, - postman: typed.ActorRef[Postman.Command], - router: ActorRef, - paymentInitiator: ActorRef, - context: ActorContext[Command], - request: InvoiceRequest, - payerKey: PrivateKey, - replyTo: ActorRef, - attemptNumber: Int, - sendPaymentConfig: SendPaymentConfig): Behavior[Command] = { - val contactInfo = request.offer.contactInfos(attemptNumber % request.offer.contactInfos.length) - val messageContent = TlvStream[OnionMessagePayloadTlv](OnionMessagePayloadTlv.InvoiceRequest(request.records)) +} + +private class OfferPayment(replyTo: ActorRef, + nodeParams: NodeParams, + postman: typed.ActorRef[Postman.Command], + router: ActorRef, + paymentInitiator: ActorRef, + payerKey: PrivateKey, + invoiceRequest: InvoiceRequest, + sendPaymentConfig: OfferPayment.SendPaymentConfig, + context: ActorContext[OfferPayment.Command]) { + + import OfferPayment._ + + def sendInvoiceRequest(attemptNumber: Int): Behavior[Command] = { + val contactInfo = invoiceRequest.offer.contactInfos(attemptNumber % invoiceRequest.offer.contactInfos.length) + val messageContent = TlvStream[OnionMessagePayloadTlv](OnionMessagePayloadTlv.InvoiceRequest(invoiceRequest.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, router, paymentInitiator, context, request, payerKey, replyTo, attemptNumber + 1, sendPaymentConfig) + waitForInvoice(attemptNumber + 1) } - def waitForInvoice(nodeParams: NodeParams, - postman: typed.ActorRef[Postman.Command], - router: ActorRef, - paymentInitiator: ActorRef, - context: ActorContext[Command], - request: InvoiceRequest, - payerKey: PrivateKey, - replyTo: ActorRef, - attemptNumber: Int, - sendPaymentConfig: SendPaymentConfig): Behavior[Command] = { + private def waitForInvoice(attemptNumber: Int): Behavior[Command] = { Behaviors.receiveMessagePartial { - case WrappedMessageResponse(Postman.Response(payload: InvoicePayload)) if payload.invoice.validateFor(request).isRight => - 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 PaymentBlindedContactInfo(CompactBlindedPath(scdidDir, _, _), _) => scdidDir.scid } - resolve(context, paymentInitiator, router, sendPaymentToNode, payload.invoice.blindedPaths, Nil, scids) + case WrappedMessageResponse(Postman.Response(payload: InvoicePayload)) if payload.invoice.validateFor(invoiceRequest).isRight => + resolveCompactBlindedPaths(payload.invoice, payload.invoice.blindedPaths, Nil) 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) + replyTo ! InvalidInvoiceResponse(invoiceRequest, payload) Behaviors.stopped case WrappedMessageResponse(Postman.NoReply) if attemptNumber < nodeParams.onionMessageConfig.maxAttempts => // We didn't get a response, let's retry. - sendInvoiceRequest(nodeParams, postman, router, paymentInitiator, context, request, payerKey, replyTo, attemptNumber, sendPaymentConfig) + sendInvoiceRequest(attemptNumber) case WrappedMessageResponse(_) => // We can't reach the offer node or the offer node can't reach us. replyTo ! NoInvoiceResponse @@ -150,46 +146,43 @@ object OfferPayment { } } - def resolve(context: ActorContext[Command], - paymentInitiator: ActorRef, - router: ActorRef, - sendPaymentToNode: SendPaymentToNode, - toResolve: Seq[PaymentBlindedContactInfo], - resolved: Seq[PaymentBlindedRoute], - scids: Seq[RealShortChannelId]): Behavior[Command] = { + /** + * Blinded paths in Bolt 12 invoices may encode the introduction node with an scid and a direction: we need to resolve + * that to a nodeId in order to reach that introduction node and use the blinded path. + */ + @tailrec + private def resolveCompactBlindedPaths(invoice: Bolt12Invoice, toResolve: Seq[PaymentBlindedContactInfo], resolved: Seq[PaymentBlindedRoute]): Behavior[Command] = { if (toResolve.isEmpty) { if (resolved.isEmpty) { - // No route could be resolved - sendPaymentToNode.replyTo ! UnknownShortChannelIds(scids) + // We don't know how to reach any of the blinded paths' introduction nodes. + val scids = invoice.blindedPaths.collect { case PaymentBlindedContactInfo(CompactBlindedPath(scdidDir, _, _), _) => scdidDir.scid } + replyTo ! UnknownShortChannelIds(scids) } else { - paymentInitiator ! sendPaymentToNode.copy(resolvedPaths = resolved) + paymentInitiator ! SendPaymentToNode(replyTo, invoice.amount, invoice, resolved, maxAttempts = sendPaymentConfig.maxAttempts, externalId = sendPaymentConfig.externalId_opt, routeParams = sendPaymentConfig.routeParams, payerKey_opt = Some(payerKey), blockUntilComplete = sendPaymentConfig.blocking) } Behaviors.stopped } else { toResolve.head match { case PaymentBlindedContactInfo(BlindedPath(route), paymentInfo) => - resolve(context, paymentInitiator, router, sendPaymentToNode, toResolve.tail, resolved :+ PaymentBlindedRoute(route, paymentInfo), scids) + resolveCompactBlindedPaths(invoice, toResolve.tail, resolved :+ PaymentBlindedRoute(route, paymentInfo)) case PaymentBlindedContactInfo(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) + waitForNodeId(invoice, route, paymentInfo, toResolve.tail, resolved) } } } - def waitForNodeId(context: ActorContext[Command], - paymentInitiator: ActorRef, - router: ActorRef, - sendPaymentToNode: SendPaymentToNode, - compactRoute: CompactBlindedPath, - paymentInfo: PaymentInfo, - toResolve: Seq[PaymentBlindedContactInfo], - resolved: Seq[PaymentBlindedRoute], - scids: Seq[RealShortChannelId]): Behavior[Command] = + private def waitForNodeId(invoice: Bolt12Invoice, + compactRoute: CompactBlindedPath, + paymentInfo: PaymentInfo, + toResolve: Seq[PaymentBlindedContactInfo], + resolved: Seq[PaymentBlindedRoute]): Behavior[Command] = Behaviors.receiveMessagePartial { case WrappedNodeId(None) => - resolve(context, paymentInitiator, router, sendPaymentToNode, toResolve, resolved, scids) + resolveCompactBlindedPaths(invoice, toResolve, resolved) case WrappedNodeId(Some(nodeId)) => val resolvedPaymentBlindedRoute = PaymentBlindedRoute(BlindedRoute(nodeId, compactRoute.blindingKey, compactRoute.blindedNodes), paymentInfo) - resolve(context, paymentInitiator, router, sendPaymentToNode, toResolve, resolved :+ resolvedPaymentBlindedRoute, scids) + resolveCompactBlindedPaths(invoice, toResolve, resolved :+ resolvedPaymentBlindedRoute) } + } \ No newline at end of file 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 dce6a65128..75bf4436f7 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 @@ -166,6 +166,7 @@ case class BlindedRecipient(nodeId: PublicKey, } object BlindedRecipient { + /** @param paths the caller must resolve the scid of compact blinded paths, otherwise they will be ignored. */ def apply(invoice: Bolt12Invoice, paths: Seq[PaymentBlindedRoute], totalAmount: MilliSatoshi, expiry: CltvExpiry, customTlvs: Set[GenericTlv]): BlindedRecipient = { val blindedHops = paths.map( path => { 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 2cff4c6e57..443238a798 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 @@ -828,11 +828,18 @@ class PaymentIntegrationSpec extends IntegrationSpec { } test("send to compact route") { + val probe = TestProbe() val recipientKey = randomKey() val amount = 10_000_000 msat val chain = nodes("C").nodeParams.chainHash val pathId = randomBytes32() - val offerPath = OfferTypes.BlindedPath(buildRoute(randomKey(), Seq(IntermediateNode(nodes("B").nodeParams.nodeId), IntermediateNode(nodes("C").nodeParams.nodeId)), Recipient(nodes("C").nodeParams.nodeId, Some(pathId)))) + val scidDirEB = { + probe.send(nodes("B").router, Router.GetChannels) + val Some(channelBE) = probe.expectMsgType[Iterable[ChannelAnnouncement]].find(ann => Set(ann.nodeId1, ann.nodeId2) == Set(nodes("B").nodeParams.nodeId, nodes("E").nodeParams.nodeId)) + ShortChannelIdDir(channelBE.nodeId1 == nodes("B").nodeParams.nodeId, channelBE.shortChannelId) + } + val offerBlindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(nodes("B").nodeParams.nodeId), IntermediateNode(nodes("C").nodeParams.nodeId)), Recipient(nodes("C").nodeParams.nodeId, Some(pathId))) + val offerPath = OfferTypes.CompactBlindedPath(scidDirEB, offerBlindedRoute.blindingKey, offerBlindedRoute.blindedNodes) val offer = Offer(Some(amount), "test offer", recipientKey.publicKey, nodes("C").nodeParams.features.bolt12Features(), chain, additionalTlvs = Set(OfferPaths(Seq(offerPath)))) val offerHandler = TypedProbe[HandlerCommand]()(nodes("C").system.toTyped) nodes("C").offerManager ! RegisterOffer(offer, recipientKey, Some(pathId), offerHandler.ref) @@ -842,18 +849,18 @@ class PaymentIntegrationSpec extends IntegrationSpec { alice.payOfferBlocking(offer, amount, 1, maxAttempts_opt = Some(3))(30 seconds).pipeTo(sender.ref) val handleInvoiceRequest = offerHandler.expectMessageType[HandleInvoiceRequest] - val probe = TestProbe() - probe.send(nodes("C").router, Router.GetChannels) - val b = nodes("B").nodeParams.nodeId - val channelWithB = probe.expectMsgType[Iterable[ChannelAnnouncement]].find(ann => ann.nodeId1 == b || ann.nodeId2 == b).get + val scidDirCB = { + probe.send(nodes("B").router, Router.GetChannels) + val Some(channelBC) = probe.expectMsgType[Iterable[ChannelAnnouncement]].find(ann => Set(ann.nodeId1, ann.nodeId2) == Set(nodes("B").nodeParams.nodeId, nodes("C").nodeParams.nodeId)) + ShortChannelIdDir(channelBC.nodeId1 == nodes("B").nodeParams.nodeId, channelBC.shortChannelId) + } val receivingRoutes = Seq( - ReceivingRoute(Seq(nodes("B").nodeParams.nodeId, nodes("C").nodeParams.nodeId), CltvExpiryDelta(555), Seq(DummyBlindedHop(55 msat, 55, CltvExpiryDelta(55))), Some(ShortChannelIdDir(channelWithB.nodeId1 == b, channelWithB.shortChannelId))) + ReceivingRoute(Seq(nodes("B").nodeParams.nodeId, nodes("C").nodeParams.nodeId), CltvExpiryDelta(555), Seq(DummyBlindedHop(55 msat, 55, CltvExpiryDelta(55))), Some(scidDirCB)) ) - handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, receivingRoutes, pluginData_opt = Some(hex"eff0")) + handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, receivingRoutes) val handlePayment = offerHandler.expectMessageType[HandlePayment] assert(handlePayment.offerId == offer.offerId) - assert(handlePayment.pluginData_opt.contains(hex"eff0")) handlePayment.replyTo ! PaymentActor.AcceptPayment() val paymentSent = sender.expectMsgType[PaymentSent] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala index 9956d6864d..57c946d124 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/message/PostmanSpec.scala @@ -41,6 +41,8 @@ import org.scalatest.Outcome import org.scalatest.funsuite.FixtureAnyFunSuiteLike import scodec.bits.HexStringSyntax +import scala.concurrent.duration.DurationInt + class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("application")) with FixtureAnyFunSuiteLike { case class FixtureParam(postman: ActorRef[Command], nodeParams: NodeParams, messageSender: TestProbe[OnionMessageResponse], switchboard: TestProbe[Any], offerManager: TestProbe[RequestInvoice], router: TestProbe[Router.PostmanRequest]) @@ -94,7 +96,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat testKit.system.eventStream ! EventStream.Publish(ReceiveMessage(replyPayload)) messageSender.expectMessage(Response(replyPayload)) - messageSender.expectNoMessage() + messageSender.expectNoMessage(10 millis) } test("sending failure") { f => @@ -113,7 +115,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat replyTo ! Disconnected(messageId) messageSender.expectMessage(MessageFailed("Peer is not connected")) - messageSender.expectNoMessage() + messageSender.expectNoMessage(10 millis) } test("timeout") { f => @@ -140,7 +142,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat val ReceiveMessage(replyPayload) = OnionMessages.process(nodeParams.privateKey, reply) testKit.system.eventStream ! EventStream.Publish(ReceiveMessage(replyPayload)) - messageSender.expectNoMessage() + messageSender.expectNoMessage(10 millis) } test("do not expect reply") { f => @@ -162,7 +164,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat assert(finalPayload.records.get[ReplyPath].isEmpty) messageSender.expectMessage(MessageSent) - messageSender.expectNoMessage() + messageSender.expectNoMessage(10 millis) } test("send to route that starts at ourselves") { f => @@ -180,7 +182,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat assert(finalPayload.records.get[ReplyPath].isEmpty) messageSender.expectMessage(MessageSent) - messageSender.expectNoMessage() + messageSender.expectNoMessage(10 millis) } test("forward invoice request to offer manager") { f => @@ -269,7 +271,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat assert(finalPayload.records.get[ReplyPath].isEmpty) messageSender.expectMessage(MessageSent) - messageSender.expectNoMessage() + messageSender.expectNoMessage(10 millis) } test("send to compact route that starts at ourselves") { f => @@ -298,6 +300,6 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat assert(finalPayload.records.get[ReplyPath].isEmpty) messageSender.expectMessage(MessageSent) - messageSender.expectNoMessage() + messageSender.expectNoMessage(10 millis) } } From 05002597b0b30749f158c1a79bb3245a7a57c4e1 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Tue, 14 Nov 2023 10:35:48 +0100 Subject: [PATCH 12/12] comments --- .../scala/fr/acinq/eclair/payment/send/OfferPayment.scala | 4 ++-- .../main/scala/fr/acinq/eclair/payment/send/Recipient.scala | 5 ++++- 2 files changed, 6 insertions(+), 3 deletions(-) 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 7f59b6727c..2fc19f0703 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 @@ -154,7 +154,7 @@ private class OfferPayment(replyTo: ActorRef, private def resolveCompactBlindedPaths(invoice: Bolt12Invoice, toResolve: Seq[PaymentBlindedContactInfo], resolved: Seq[PaymentBlindedRoute]): Behavior[Command] = { if (toResolve.isEmpty) { if (resolved.isEmpty) { - // We don't know how to reach any of the blinded paths' introduction nodes. + // We couldn't identify any of the blinded paths' introduction nodes because the scids are unknown. val scids = invoice.blindedPaths.collect { case PaymentBlindedContactInfo(CompactBlindedPath(scdidDir, _, _), _) => scdidDir.scid } replyTo ! UnknownShortChannelIds(scids) } else { @@ -185,4 +185,4 @@ private class OfferPayment(replyTo: ActorRef, resolveCompactBlindedPaths(invoice, toResolve, resolved :+ resolvedPaymentBlindedRoute) } -} \ No newline at end of file +} 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 75bf4436f7..18605956ce 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 @@ -166,7 +166,10 @@ case class BlindedRecipient(nodeId: PublicKey, } object BlindedRecipient { - /** @param paths the caller must resolve the scid of compact blinded paths, otherwise they will be ignored. */ + /** + * @param invoice Bolt invoice. Paths from the invoice must be passed as `paths` with compact paths expanded to include the node id. + * @param paths Payment paths to use to reach the recipient. + */ def apply(invoice: Bolt12Invoice, paths: Seq[PaymentBlindedRoute], totalAmount: MilliSatoshi, expiry: CltvExpiry, customTlvs: Set[GenericTlv]): BlindedRecipient = { val blindedHops = paths.map( path => {