1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-02-23 06:35:11 +01:00

Add support for sciddir_or_pubkey (#2752)

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.
This commit is contained in:
Thomas HUET 2023-11-14 11:08:33 +01:00 committed by GitHub
parent 7be7d5d524
commit 772e2b20f8
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
31 changed files with 546 additions and 218 deletions

View file

@ -427,7 +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)
val sendPayment = SendPaymentToRoute(recipientAmount, invoice, Nil, route, externalId_opt, parentId_opt, trampoline_opt)
(appKit.paymentInitiator ? sendPayment).mapTo[SendPaymentToRouteResponse]
}
}
@ -442,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)
}
@ -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)
@ -702,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)

View file

@ -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)),

View file

@ -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

View file

@ -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(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)))
}
}
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 {

View file

@ -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}
@ -52,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
@ -86,7 +87,9 @@ case class Bolt12Invoice(records: TlvStream[InvoiceTlv]) extends Invoice {
}
case class PaymentBlindedRoute(route: Sphinx.RouteBlinding.BlindedRoute, paymentInfo: PaymentInfo)
case class PaymentBlindedContactInfo(route: BlindedContactInfo, paymentInfo: PaymentInfo)
case class PaymentBlindedRoute(route: BlindedRoute, paymentInfo: PaymentInfo)
object Bolt12Invoice {
val hrp = "lni"
@ -107,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)

View file

@ -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)

View file

@ -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(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 => {
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)
PaymentBlindedContactInfo(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(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)

View file

@ -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()

View file

@ -20,15 +20,20 @@ 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.{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.{InvoiceRequest, Offer}
import fr.acinq.eclair.wire.protocol.OfferTypes._
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}
import scala.annotation.tailrec
object OfferPayment {
sealed trait Failure
@ -49,6 +54,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.mkString(",")}"
}
sealed trait Command
case class PayOffer(replyTo: ActorRef,
@ -59,6 +68,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 +78,7 @@ object OfferPayment {
def apply(nodeParams: NodeParams,
postman: typed.ActorRef[Postman.Command],
router: ActorRef,
paymentInitiator: ActorRef): Behavior[Command] = {
Behaviors.setup(context =>
Behaviors.receiveMessagePartial {
@ -89,58 +101,88 @@ 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)
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],
paymentInitiator: ActorRef,
context: ActorContext[Command],
request: InvoiceRequest,
payerKey: PrivateKey,
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 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(destination, routingStrategy, messageContent, expectsReply = true, context.messageAdapter(WrappedMessageResponse))
waitForInvoice(nodeParams, postman, paymentInitiator, context, request, payerKey, replyTo, attemptNumber + 1, sendPaymentConfig)
postman ! SendMessage(contactInfo, routingStrategy, messageContent, expectsReply = true, context.messageAdapter(WrappedMessageResponse))
waitForInvoice(attemptNumber + 1)
}
def waitForInvoice(nodeParams: NodeParams,
postman: typed.ActorRef[Postman.Command],
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 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
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, 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
Behaviors.stopped
}
}
/**
* 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) {
// 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 {
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) =>
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(invoice, route, paymentInfo, toResolve.tail, resolved)
}
}
}
private def waitForNodeId(invoice: Bolt12Invoice,
compactRoute: CompactBlindedPath,
paymentInfo: PaymentInfo,
toResolve: Seq[PaymentBlindedContactInfo],
resolved: Seq[PaymentBlindedRoute]): Behavior[Command] =
Behaviors.receiveMessagePartial {
case WrappedNodeId(None) =>
resolveCompactBlindedPaths(invoice, toResolve, resolved)
case WrappedNodeId(Some(nodeId)) =>
val resolvedPaymentBlindedRoute = PaymentBlindedRoute(BlindedRoute(nodeId, compactRoute.blindingKey, compactRoute.blindedNodes), paymentInfo)
resolveCompactBlindedPaths(invoice, toResolve, resolved :+ resolvedPaymentBlindedRoute)
}
}

View file

@ -54,7 +54,7 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn
val finalExpiry = r.finalExpiry(nodeParams)
val recipient = r.invoice match {
case invoice: Bolt11Invoice => ClearRecipient(invoice, r.recipientAmount, finalExpiry, r.userCustomTlvs)
case invoice: Bolt12Invoice => BlindedRecipient(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)
@ -124,7 +124,7 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn
val finalExpiry = r.finalExpiry(nodeParams)
val recipient = r.invoice match {
case invoice: Bolt11Invoice => ClearRecipient(invoice, r.recipientAmount, finalExpiry, Set.empty)
case invoice: Bolt12Invoice => BlindedRecipient(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)
@ -187,7 +187,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)
}
}
@ -309,6 +309,7 @@ object PaymentInitiator {
/**
* @param recipientAmount amount that should be received by the final recipient (usually from a Bolt 11 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.
@ -319,6 +320,7 @@ object PaymentInitiator {
case class SendPaymentToNode(replyTo: ActorRef,
recipientAmount: MilliSatoshi,
invoice: Invoice,
resolvedPaths: Seq[PaymentBlindedRoute],
maxAttempts: Int,
externalId: Option[String] = None,
routeParams: RouteParams,
@ -371,6 +373,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 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
@ -381,6 +384,7 @@ object PaymentInitiator {
*/
case class SendPaymentToRoute(recipientAmount: MilliSatoshi,
invoice: Invoice,
resolvedPaths: Seq[PaymentBlindedRoute],
route: PredefinedRoute,
externalId: Option[String],
parentId: Option[UUID],

View file

@ -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}
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}
@ -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,8 +166,12 @@ case class BlindedRecipient(nodeId: PublicKey,
}
object BlindedRecipient {
def apply(invoice: Bolt12Invoice, totalAmount: MilliSatoshi, expiry: CltvExpiry, customTlvs: Set[GenericTlv]): BlindedRecipient = {
val blindedHops = invoice.blindedPaths.map(
/**
* @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 => {
// 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.

View file

@ -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 }

View file

@ -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,16 +46,31 @@ 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]
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)
@ -114,7 +129,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) ::

View file

@ -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,18 @@ 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)
// @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
// @formatter:on
sealed trait Bolt12Tlv extends Tlv
@ -84,7 +96,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 +156,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 +240,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

View file

@ -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 =>

View file

@ -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 = 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))
}
}

View file

@ -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)

View file

@ -43,8 +43,8 @@ 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.{ChannelAnnouncement, ChannelUpdate, IncorrectOrUnknownPaymentDetails}
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}
import scodec.bits.{ByteVector, HexStringSyntax}
@ -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)
@ -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)
@ -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)
@ -827,6 +827,53 @@ class PaymentIntegrationSpec extends IntegrationSpec {
assert(receivedAmount >= amount)
}
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 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)
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 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(scidDirCB))
)
handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, receivingRoutes)
val handlePayment = offerHandler.expectMessageType[HandlePayment]
assert(handlePayment.offerId == offer.offerId)
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

View file

@ -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]

View file

@ -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)

View file

@ -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
@ -126,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])
@ -141,12 +143,12 @@ 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))
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])
@ -160,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.
@ -360,17 +362,17 @@ 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)
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])

View file

@ -25,30 +25,34 @@ 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}
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
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
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[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 +76,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)
@ -92,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 =>
@ -100,7 +104,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)
@ -111,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 =>
@ -119,7 +123,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)
@ -138,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 =>
@ -146,7 +150,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)
@ -160,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 =>
@ -169,7 +173,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)
@ -178,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 =>
@ -199,7 +203,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)
@ -235,4 +239,67 @@ 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 Peer.Connect(nextNodeId, _, replyConnectedTo, _) = switchboard.expectMessageType[Peer.Connect]
assert(nextNodeId == recipientKey.publicKey)
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)
messageSender.expectMessage(MessageSent)
messageSender.expectNoMessage(10 millis)
}
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 Peer.Connect(nextNodeId, _, replyConnectedTo, _) = switchboard.expectMessageType[Peer.Connect]
assert(nextNodeId == recipientKey.publicKey)
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)
messageSender.expectMessage(MessageSent)
messageSender.expectNoMessage(10 millis)
}
}

View file

@ -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(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") {

View file

@ -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 =>

View file

@ -35,7 +35,7 @@ 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.wire.protocol.OfferTypes.{InvoiceRequest, Offer}
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, TestConstants, TestKitBaseClass, TimestampSecond, UnknownFeature, randomBytes32, randomKey}
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
@ -123,7 +123,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
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]
@ -166,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]
@ -181,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(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))
@ -205,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]
@ -229,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))
@ -253,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))
@ -267,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(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))
@ -296,16 +296,17 @@ 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)))
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 req = SendPaymentToNode(sender.ref, finalAmount, invoice, 1, routeParams = nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams, payerKey_opt = Some(payerKey))
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]
payFsm.expectMsg(SendPaymentConfig(id, id, None, paymentHash, invoice.nodeId, Upstream.Local(id), Some(invoice), Some(payerKey), storeInDb = true, publishEvent = true, recordPathFindingMetrics = true))
@ -335,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 => 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]
multiPartPayFsm.expectMsg(SendPaymentConfig(id, id, None, paymentHash, invoice.nodeId, Upstream.Local(id), Some(invoice), Some(payerKey), storeInDb = true, publishEvent = true, recordPathFindingMetrics = true))
@ -363,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 => 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]
val fail = sender.expectMsgType[PaymentFailed]
@ -538,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(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))

View file

@ -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._
@ -217,10 +217,11 @@ 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 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)
assert(payment.outgoingChannel == channelUpdate_ab.shortChannelId)
@ -468,10 +469,11 @@ 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 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)
}

View file

@ -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),

View file

@ -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.payment.{Bolt12Invoice, PaymentBlindedContactInfo}
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.{OnionMessagePayloadTlv, TlvStream}
import fr.acinq.eclair.{CltvExpiryDelta, Features, MilliSatoshiLong, NodeParams, TestConstants, randomBytes32, randomKey}
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, 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)
}
@ -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 = 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]
@ -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 = 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))
@ -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(
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))
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)
}
}

View file

@ -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, PaymentBlindedContactInfo, PaymentBlindedRoute}
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
@ -270,12 +270,13 @@ 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)
PaymentBlindedContactInfo(blindedRoute, paymentInfo)
})
val invoice = Bolt12Invoice(invoiceRequest, preimage, recipientKey, 300 seconds, features, blindedRoutes)
val recipient = BlindedRecipient(invoice, amount, expiry, Set.empty)
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)
}

View file

@ -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(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]](
@ -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)

View file

@ -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)
}
}
}