mirror of
https://github.com/ACINQ/eclair.git
synced 2025-02-22 06:21:42 +01:00
Implicit node id in offers with blinded paths (#2852)
To save space in the offer, we can skip the node id for offers that use blinded paths. The node id used to sign the invoice will be the last blinded node id of the path used to request the invoice. We also make the description optional for offers without amount.
This commit is contained in:
parent
35295af73c
commit
c4934930aa
27 changed files with 329 additions and 250 deletions
|
@ -457,27 +457,27 @@ object InvoiceSerializer extends MinimalSerializer({
|
||||||
JObject(fieldList)
|
JObject(fieldList)
|
||||||
case p: Bolt12Invoice =>
|
case p: Bolt12Invoice =>
|
||||||
val fieldList = List(
|
val fieldList = List(
|
||||||
JField("amount", JLong(p.amount.toLong)),
|
Some(JField("amount", JLong(p.amount.toLong))),
|
||||||
JField("nodeId", JString(p.nodeId.toString())),
|
Some(JField("nodeId", JString(p.nodeId.toString()))),
|
||||||
JField("paymentHash", JString(p.paymentHash.toString())),
|
Some(JField("paymentHash", JString(p.paymentHash.toString()))),
|
||||||
p.description.fold(string => JField("description", JString(string)), hash => JField("descriptionHash", JString(hash.toHex))),
|
p.description.map(string => JField("description", JString(string))),
|
||||||
JField("features", Extraction.decompose(p.features)(
|
Some(JField("features", Extraction.decompose(p.features)(
|
||||||
DefaultFormats +
|
DefaultFormats +
|
||||||
FeatureKeySerializer +
|
FeatureKeySerializer +
|
||||||
FeatureSupportSerializer +
|
FeatureSupportSerializer +
|
||||||
UnknownFeatureSerializer
|
UnknownFeatureSerializer
|
||||||
)),
|
))),
|
||||||
JField("blindedPaths", JArray(p.blindedPaths.map(path => {
|
Some(JField("blindedPaths", JArray(p.blindedPaths.map(path => {
|
||||||
val introductionNode = path.route.introductionNodeId.toString
|
val introductionNode = path.route.introductionNodeId.toString
|
||||||
val blindedNodes = path.route.blindedNodes
|
val blindedNodes = path.route.blindedNodes
|
||||||
JObject(List(
|
JObject(List(
|
||||||
JField("introductionNodeId", JString(introductionNode)),
|
JField("introductionNodeId", JString(introductionNode)),
|
||||||
JField("blindedNodeIds", JArray(blindedNodes.map(n => JString(n.blindedPublicKey.toString)).toList))
|
JField("blindedNodeIds", JArray(blindedNodes.map(n => JString(n.blindedPublicKey.toString)).toList))
|
||||||
))
|
))
|
||||||
}).toList)),
|
}).toList))),
|
||||||
JField("createdAt", JLong(p.createdAt.toLong)),
|
Some(JField("createdAt", JLong(p.createdAt.toLong))),
|
||||||
JField("expiresAt", JLong((p.createdAt + p.relativeExpiry).toLong)),
|
Some(JField("expiresAt", JLong((p.createdAt + p.relativeExpiry).toLong))),
|
||||||
JField("serialized", JString(p.toString)))
|
Some(JField("serialized", JString(p.toString)))).flatten
|
||||||
JObject(fieldList)
|
JObject(fieldList)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
|
|
@ -92,18 +92,18 @@ object OnionMessages {
|
||||||
|
|
||||||
def buildRoute(blindingSecret: PrivateKey,
|
def buildRoute(blindingSecret: PrivateKey,
|
||||||
intermediateNodes: Seq[IntermediateNode],
|
intermediateNodes: Seq[IntermediateNode],
|
||||||
recipient: Recipient): Sphinx.RouteBlinding.BlindedRoute = {
|
recipient: Recipient): Sphinx.RouteBlinding.BlindedRouteDetails = {
|
||||||
val intermediatePayloads = buildIntermediatePayloads(intermediateNodes, EncodedNodeId(recipient.nodeId))
|
val intermediatePayloads = buildIntermediatePayloads(intermediateNodes, EncodedNodeId(recipient.nodeId))
|
||||||
val tlvs: Set[RouteBlindingEncryptedDataTlv] = Set(recipient.padding.map(Padding), recipient.pathId.map(PathId)).flatten
|
val tlvs: Set[RouteBlindingEncryptedDataTlv] = Set(recipient.padding.map(Padding), recipient.pathId.map(PathId)).flatten
|
||||||
val lastPayload = RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec.encode(TlvStream(tlvs, recipient.customTlvs)).require.bytes
|
val lastPayload = RouteBlindingEncryptedDataCodecs.blindedRouteDataCodec.encode(TlvStream(tlvs, recipient.customTlvs)).require.bytes
|
||||||
Sphinx.RouteBlinding.create(blindingSecret, intermediateNodes.map(_.publicKey) :+ recipient.nodeId, intermediatePayloads :+ lastPayload).route
|
Sphinx.RouteBlinding.create(blindingSecret, intermediateNodes.map(_.publicKey) :+ recipient.nodeId, intermediatePayloads :+ lastPayload)
|
||||||
}
|
}
|
||||||
|
|
||||||
private[message] def buildRouteFrom(blindingSecret: PrivateKey,
|
private[message] def buildRouteFrom(blindingSecret: PrivateKey,
|
||||||
intermediateNodes: Seq[IntermediateNode],
|
intermediateNodes: Seq[IntermediateNode],
|
||||||
destination: Destination): Sphinx.RouteBlinding.BlindedRoute = {
|
destination: Destination): Sphinx.RouteBlinding.BlindedRoute = {
|
||||||
destination match {
|
destination match {
|
||||||
case recipient: Recipient => buildRoute(blindingSecret, intermediateNodes, recipient)
|
case recipient: Recipient => buildRoute(blindingSecret, intermediateNodes, recipient).route
|
||||||
case BlindedPath(route) if intermediateNodes.isEmpty => route
|
case BlindedPath(route) if intermediateNodes.isEmpty => route
|
||||||
case BlindedPath(route) =>
|
case BlindedPath(route) =>
|
||||||
val intermediatePayloads = buildIntermediatePayloads(intermediateNodes, route.introductionNodeId, Some(route.blindingKey))
|
val intermediatePayloads = buildIntermediatePayloads(intermediateNodes, route.introductionNodeId, Some(route.blindingKey))
|
||||||
|
@ -112,11 +112,7 @@ object OnionMessages {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// @formatter:off
|
case class MessageTooLarge(payloadSize: Long)
|
||||||
sealed trait BuildMessageError
|
|
||||||
case class MessageTooLarge(payloadSize: Long) extends BuildMessageError
|
|
||||||
case class InvalidDestination(destination: Destination) extends BuildMessageError
|
|
||||||
// @formatter:on
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Builds an encrypted onion containing a message that should be relayed to the destination.
|
* Builds an encrypted onion containing a message that should be relayed to the destination.
|
||||||
|
@ -132,7 +128,7 @@ object OnionMessages {
|
||||||
blindingSecret: PrivateKey,
|
blindingSecret: PrivateKey,
|
||||||
intermediateNodes: Seq[IntermediateNode],
|
intermediateNodes: Seq[IntermediateNode],
|
||||||
destination: Destination,
|
destination: Destination,
|
||||||
content: TlvStream[OnionMessagePayloadTlv]): Either[BuildMessageError, OnionMessage] = {
|
content: TlvStream[OnionMessagePayloadTlv]): Either[MessageTooLarge, OnionMessage] = {
|
||||||
val route = buildRouteFrom(blindingSecret, intermediateNodes, destination)
|
val route = buildRouteFrom(blindingSecret, intermediateNodes, destination)
|
||||||
val lastPayload = MessageOnionCodecs.perHopPayloadCodec.encode(TlvStream(content.records + EncryptedData(route.encryptedPayloads.last), content.unknown)).require.bytes
|
val lastPayload = MessageOnionCodecs.perHopPayloadCodec.encode(TlvStream(content.records + EncryptedData(route.encryptedPayloads.last), content.unknown)).require.bytes
|
||||||
val payloads = route.encryptedPayloads.dropRight(1).map(encTlv => MessageOnionCodecs.perHopPayloadCodec.encode(TlvStream(EncryptedData(encTlv))).require.bytes) :+ lastPayload
|
val payloads = route.encryptedPayloads.dropRight(1).map(encTlv => MessageOnionCodecs.perHopPayloadCodec.encode(TlvStream(EncryptedData(encTlv))).require.bytes) :+ lastPayload
|
||||||
|
@ -156,7 +152,7 @@ object OnionMessages {
|
||||||
sealed trait Action
|
sealed trait Action
|
||||||
case class DropMessage(reason: DropReason) extends Action
|
case class DropMessage(reason: DropReason) extends Action
|
||||||
case class SendMessage(nextNode: Either[ShortChannelId, EncodedNodeId], message: OnionMessage) extends Action
|
case class SendMessage(nextNode: Either[ShortChannelId, EncodedNodeId], message: OnionMessage) extends Action
|
||||||
case class ReceiveMessage(finalPayload: FinalPayload) extends Action
|
case class ReceiveMessage(finalPayload: FinalPayload, blindedKey: PrivateKey) extends Action
|
||||||
|
|
||||||
sealed trait DropReason
|
sealed trait DropReason
|
||||||
case class CannotDecryptOnion(message: String) extends DropReason { override def toString = s"can't decrypt onion: $message" }
|
case class CannotDecryptOnion(message: String) extends DropReason { override def toString = s"can't decrypt onion: $message" }
|
||||||
|
@ -200,12 +196,12 @@ object OnionMessages {
|
||||||
case Left(f) => DropMessage(f)
|
case Left(f) => DropMessage(f)
|
||||||
case Right(DecodedEncryptedData(blindedPayload, nextBlinding)) => nextPacket_opt match {
|
case Right(DecodedEncryptedData(blindedPayload, nextBlinding)) => nextPacket_opt match {
|
||||||
case Some(nextPacket) => validateRelayPayload(payload, blindedPayload, nextBlinding, nextPacket)
|
case Some(nextPacket) => validateRelayPayload(payload, blindedPayload, nextBlinding, nextPacket)
|
||||||
case None => validateFinalPayload(payload, blindedPayload)
|
case None => validateFinalPayload(payload, blindedPayload, blindedPrivateKey)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
case None => nextPacket_opt match {
|
case None => nextPacket_opt match {
|
||||||
case Some(_) => DropMessage(CannotDecryptBlindedPayload("encrypted_data is missing"))
|
case Some(_) => DropMessage(CannotDecryptBlindedPayload("encrypted_data is missing"))
|
||||||
case None => validateFinalPayload(payload, TlvStream.empty)
|
case None => validateFinalPayload(payload, TlvStream.empty, blindedPrivateKey)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -218,10 +214,10 @@ object OnionMessages {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private def validateFinalPayload(payload: TlvStream[OnionMessagePayloadTlv], blindedPayload: TlvStream[RouteBlindingEncryptedDataTlv]): Action = {
|
private def validateFinalPayload(payload: TlvStream[OnionMessagePayloadTlv], blindedPayload: TlvStream[RouteBlindingEncryptedDataTlv], blindedKey: PrivateKey): Action = {
|
||||||
FinalPayload.validate(payload, blindedPayload) match {
|
FinalPayload.validate(payload, blindedPayload) match {
|
||||||
case Left(f) => DropMessage(CannotDecodeBlindedPayload(f.failureMessage.message))
|
case Left(f) => DropMessage(CannotDecodeBlindedPayload(f.failureMessage.message))
|
||||||
case Right(finalPayload) => ReceiveMessage(finalPayload)
|
case Right(finalPayload) => ReceiveMessage(finalPayload, blindedKey)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,7 @@ import akka.actor.typed.eventstream.EventStream
|
||||||
import akka.actor.typed.scaladsl.{ActorContext, Behaviors}
|
import akka.actor.typed.scaladsl.{ActorContext, Behaviors}
|
||||||
import akka.actor.typed.{ActorRef, Behavior}
|
import akka.actor.typed.{ActorRef, Behavior}
|
||||||
import fr.acinq.bitcoin.scalacompat.ByteVector32
|
import fr.acinq.bitcoin.scalacompat.ByteVector32
|
||||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
|
||||||
import fr.acinq.eclair.crypto.Sphinx.RouteBlinding.BlindedRoute
|
import fr.acinq.eclair.crypto.Sphinx.RouteBlinding.BlindedRoute
|
||||||
import fr.acinq.eclair.io.MessageRelay
|
import fr.acinq.eclair.io.MessageRelay
|
||||||
import fr.acinq.eclair.message.OnionMessages.{Destination, RoutingStrategy}
|
import fr.acinq.eclair.message.OnionMessages.{Destination, RoutingStrategy}
|
||||||
|
@ -54,7 +54,7 @@ object Postman {
|
||||||
replyTo: ActorRef[OnionMessageResponse]) extends Command
|
replyTo: ActorRef[OnionMessageResponse]) extends Command
|
||||||
case class Subscribe(pathId: ByteVector32, replyTo: ActorRef[OnionMessageResponse]) extends Command
|
case class Subscribe(pathId: ByteVector32, replyTo: ActorRef[OnionMessageResponse]) extends Command
|
||||||
private case class Unsubscribe(pathId: ByteVector32) extends Command
|
private case class Unsubscribe(pathId: ByteVector32) extends Command
|
||||||
case class WrappedMessage(finalPayload: FinalPayload) extends Command
|
case class WrappedMessage(finalPayload: FinalPayload, blindedKey: PrivateKey) extends Command
|
||||||
|
|
||||||
sealed trait OnionMessageResponse
|
sealed trait OnionMessageResponse
|
||||||
case object NoReply extends OnionMessageResponse
|
case object NoReply extends OnionMessageResponse
|
||||||
|
@ -66,16 +66,16 @@ object Postman {
|
||||||
|
|
||||||
def apply(nodeParams: NodeParams, switchboard: akka.actor.ActorRef, router: ActorRef[Router.PostmanRequest], 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 => {
|
Behaviors.setup(context => {
|
||||||
context.system.eventStream ! EventStream.Subscribe(context.messageAdapter[OnionMessages.ReceiveMessage](r => WrappedMessage(r.finalPayload)))
|
context.system.eventStream ! EventStream.Subscribe(context.messageAdapter[OnionMessages.ReceiveMessage](r => WrappedMessage(r.finalPayload, r.blindedKey)))
|
||||||
|
|
||||||
// For messages expecting a reply, send reply or failure to send
|
// For messages expecting a reply, send reply or failure to send
|
||||||
val subscribed = new mutable.HashMap[ByteVector32, ActorRef[OnionMessageResponse]]()
|
val subscribed = new mutable.HashMap[ByteVector32, ActorRef[OnionMessageResponse]]()
|
||||||
|
|
||||||
Behaviors.receiveMessage {
|
Behaviors.receiveMessage {
|
||||||
case WrappedMessage(invoiceRequestPayload: InvoiceRequestPayload) =>
|
case WrappedMessage(invoiceRequestPayload: InvoiceRequestPayload, blindedKey) =>
|
||||||
offerManager ! OfferManager.RequestInvoice(invoiceRequestPayload, context.self)
|
offerManager ! OfferManager.RequestInvoice(invoiceRequestPayload, blindedKey, context.self)
|
||||||
Behaviors.same
|
Behaviors.same
|
||||||
case WrappedMessage(finalPayload) =>
|
case WrappedMessage(finalPayload, _) =>
|
||||||
finalPayload.pathId_opt match {
|
finalPayload.pathId_opt match {
|
||||||
case Some(pathId) if pathId.length == 32 =>
|
case Some(pathId) if pathId.length == 32 =>
|
||||||
val id = ByteVector32(pathId)
|
val id = ByteVector32(pathId)
|
||||||
|
@ -200,7 +200,7 @@ private class SendingMessage(nodeParams: NodeParams,
|
||||||
val numHopsToAdd = 0.max(nodeParams.onionMessageConfig.minIntermediateHops - intermediateNodes.length - 1)
|
val numHopsToAdd = 0.max(nodeParams.onionMessageConfig.minIntermediateHops - intermediateNodes.length - 1)
|
||||||
val intermediateHops = OnionMessages.IntermediateNode(plainNodeId, destination.introductionNodeId) +: (intermediateNodes.reverse ++ Seq.fill(numHopsToAdd)(nodeParams.nodeId)).map(OnionMessages.IntermediateNode(_))
|
val intermediateHops = OnionMessages.IntermediateNode(plainNodeId, destination.introductionNodeId) +: (intermediateNodes.reverse ++ Seq.fill(numHopsToAdd)(nodeParams.nodeId)).map(OnionMessages.IntermediateNode(_))
|
||||||
val lastHop = OnionMessages.Recipient(nodeParams.nodeId, Some(messageId))
|
val lastHop = OnionMessages.Recipient(nodeParams.nodeId, Some(messageId))
|
||||||
Some(OnionMessages.buildRoute(randomKey(), intermediateHops, lastHop))
|
Some(OnionMessages.buildRoute(randomKey(), intermediateHops, lastHop).route)
|
||||||
} else {
|
} else {
|
||||||
None
|
None
|
||||||
}
|
}
|
||||||
|
|
|
@ -44,7 +44,7 @@ case class Bolt12Invoice(records: TlvStream[InvoiceTlv]) extends Invoice {
|
||||||
override val amount_opt: Option[MilliSatoshi] = Some(amount)
|
override val amount_opt: Option[MilliSatoshi] = Some(amount)
|
||||||
override val nodeId: Crypto.PublicKey = records.get[InvoiceNodeId].get.nodeId
|
override val nodeId: Crypto.PublicKey = records.get[InvoiceNodeId].get.nodeId
|
||||||
override val paymentHash: ByteVector32 = records.get[InvoicePaymentHash].get.hash
|
override val paymentHash: ByteVector32 = records.get[InvoicePaymentHash].get.hash
|
||||||
override val description: Either[String, ByteVector32] = Left(invoiceRequest.offer.description)
|
val description: Option[String] = invoiceRequest.offer.description
|
||||||
override val createdAt: TimestampSecond = records.get[InvoiceCreatedAt].get.timestamp
|
override val createdAt: TimestampSecond = records.get[InvoiceCreatedAt].get.timestamp
|
||||||
override val relativeExpiry: FiniteDuration = FiniteDuration(records.get[InvoiceRelativeExpiry].map(_.seconds).getOrElse(DEFAULT_EXPIRY_SECONDS), TimeUnit.SECONDS)
|
override val relativeExpiry: FiniteDuration = FiniteDuration(records.get[InvoiceRelativeExpiry].map(_.seconds).getOrElse(DEFAULT_EXPIRY_SECONDS), TimeUnit.SECONDS)
|
||||||
override val features: Features[InvoiceFeature] = {
|
override val features: Features[InvoiceFeature] = {
|
||||||
|
@ -57,10 +57,10 @@ case class Bolt12Invoice(records: TlvStream[InvoiceTlv]) extends Invoice {
|
||||||
val signature: ByteVector64 = records.get[Signature].get.signature
|
val signature: ByteVector64 = records.get[Signature].get.signature
|
||||||
|
|
||||||
// It is assumed that the request is valid for this offer.
|
// It is assumed that the request is valid for this offer.
|
||||||
def validateFor(request: InvoiceRequest): Either[String, Unit] = {
|
def validateFor(request: InvoiceRequest, pathNodeId: PublicKey): Either[String, Unit] = {
|
||||||
if (invoiceRequest.unsigned != request.unsigned) {
|
if (invoiceRequest.unsigned != request.unsigned) {
|
||||||
Left("Invoice does not match request")
|
Left("Invoice does not match request")
|
||||||
} else if (nodeId != invoiceRequest.offer.nodeId) {
|
} else if (nodeId != invoiceRequest.offer.nodeId.getOrElse(pathNodeId)) {
|
||||||
Left("Wrong node id")
|
Left("Wrong node id")
|
||||||
} else if (isExpired()) {
|
} else if (isExpired()) {
|
||||||
Left("Invoice expired")
|
Left("Invoice expired")
|
||||||
|
@ -169,7 +169,7 @@ case class MinimalBolt12Invoice(records: TlvStream[InvoiceTlv]) extends Invoice
|
||||||
override val amount_opt: Option[MilliSatoshi] = records.get[InvoiceAmount].map(_.amount)
|
override val amount_opt: Option[MilliSatoshi] = records.get[InvoiceAmount].map(_.amount)
|
||||||
override val nodeId: Crypto.PublicKey = records.get[InvoiceNodeId].get.nodeId
|
override val nodeId: Crypto.PublicKey = records.get[InvoiceNodeId].get.nodeId
|
||||||
override val paymentHash: ByteVector32 = records.get[InvoicePaymentHash].get.hash
|
override val paymentHash: ByteVector32 = records.get[InvoicePaymentHash].get.hash
|
||||||
override val description: Either[String, ByteVector32] = Left(records.get[OfferDescription].get.description)
|
val description: Option[String] = records.get[OfferDescription].map(_.description)
|
||||||
override val createdAt: TimestampSecond = records.get[InvoiceCreatedAt].get.timestamp
|
override val createdAt: TimestampSecond = records.get[InvoiceCreatedAt].get.timestamp
|
||||||
override val relativeExpiry: FiniteDuration = FiniteDuration(records.get[InvoiceRelativeExpiry].map(_.seconds).getOrElse(Bolt12Invoice.DEFAULT_EXPIRY_SECONDS), TimeUnit.SECONDS)
|
override val relativeExpiry: FiniteDuration = FiniteDuration(records.get[InvoiceRelativeExpiry].map(_.seconds).getOrElse(Bolt12Invoice.DEFAULT_EXPIRY_SECONDS), TimeUnit.SECONDS)
|
||||||
override val features: Features[InvoiceFeature] = {
|
override val features: Features[InvoiceFeature] = {
|
||||||
|
@ -203,7 +203,7 @@ object MinimalBolt12Invoice {
|
||||||
OfferTypes.InvoiceCreatedAt(createdAt),
|
OfferTypes.InvoiceCreatedAt(createdAt),
|
||||||
OfferTypes.InvoicePaymentHash(paymentHash),
|
OfferTypes.InvoicePaymentHash(paymentHash),
|
||||||
OfferTypes.InvoiceAmount(amount),
|
OfferTypes.InvoiceAmount(amount),
|
||||||
OfferTypes.InvoiceNodeId(offer.nodeId),
|
OfferTypes.InvoiceNodeId(offer.contactInfos.head.nodeId),
|
||||||
) ++ additionalTlvs, offer.records.unknown ++ customTlvs))
|
) ++ additionalTlvs, offer.records.unknown ++ customTlvs))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -31,7 +31,6 @@ trait Invoice {
|
||||||
def amount_opt: Option[MilliSatoshi]
|
def amount_opt: Option[MilliSatoshi]
|
||||||
def createdAt: TimestampSecond
|
def createdAt: TimestampSecond
|
||||||
def paymentHash: ByteVector32
|
def paymentHash: ByteVector32
|
||||||
def description: Either[String, ByteVector32]
|
|
||||||
def relativeExpiry: FiniteDuration
|
def relativeExpiry: FiniteDuration
|
||||||
def features: Features[InvoiceFeature]
|
def features: Features[InvoiceFeature]
|
||||||
def isExpired(now: TimestampSecond = TimestampSecond.now()): Boolean = createdAt + relativeExpiry.toSeconds <= now
|
def isExpired(now: TimestampSecond = TimestampSecond.now()): Boolean = createdAt + relativeExpiry.toSeconds <= now
|
||||||
|
|
|
@ -50,7 +50,7 @@ object OfferManager {
|
||||||
* @param pathId_opt If the offer uses a blinded path, the path id of this blinded path.
|
* @param pathId_opt If the offer uses a blinded path, the path id of this blinded path.
|
||||||
* @param handler An actor that will be in charge of accepting or rejecting invoice requests and payments for this offer.
|
* @param handler An actor that will be in charge of accepting or rejecting invoice requests and payments for this offer.
|
||||||
*/
|
*/
|
||||||
case class RegisterOffer(offer: Offer, nodeKey: PrivateKey, pathId_opt: Option[ByteVector32], handler: ActorRef[HandlerCommand]) extends Command
|
case class RegisterOffer(offer: Offer, nodeKey: Option[PrivateKey], pathId_opt: Option[ByteVector32], handler: ActorRef[HandlerCommand]) extends Command
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Forget about an offer. Invoice requests and payment attempts for this offer will be ignored.
|
* Forget about an offer. Invoice requests and payment attempts for this offer will be ignored.
|
||||||
|
@ -59,7 +59,7 @@ object OfferManager {
|
||||||
*/
|
*/
|
||||||
case class DisableOffer(offer: Offer) extends Command
|
case class DisableOffer(offer: Offer) extends Command
|
||||||
|
|
||||||
case class RequestInvoice(messagePayload: MessageOnion.InvoiceRequestPayload, postman: ActorRef[Postman.SendMessage]) extends Command
|
case class RequestInvoice(messagePayload: MessageOnion.InvoiceRequestPayload, blindedKey: PrivateKey, postman: ActorRef[Postman.SendMessage]) extends Command
|
||||||
|
|
||||||
case class ReceivePayment(replyTo: ActorRef[MultiPartHandler.GetIncomingPaymentActor.Command], paymentHash: ByteVector32, payload: FinalPayload.Blinded) extends Command
|
case class ReceivePayment(replyTo: ActorRef[MultiPartHandler.GetIncomingPaymentActor.Command], paymentHash: ByteVector32, payload: FinalPayload.Blinded) extends Command
|
||||||
|
|
||||||
|
@ -87,7 +87,7 @@ object OfferManager {
|
||||||
*/
|
*/
|
||||||
case class HandlePayment(replyTo: ActorRef[PaymentActor.Command], offerId: ByteVector32, pluginData_opt: Option[ByteVector] = None) extends HandlerCommand
|
case class HandlePayment(replyTo: ActorRef[PaymentActor.Command], offerId: ByteVector32, pluginData_opt: Option[ByteVector] = None) extends HandlerCommand
|
||||||
|
|
||||||
private case class RegisteredOffer(offer: Offer, nodeKey: PrivateKey, pathId_opt: Option[ByteVector32], handler: ActorRef[HandlerCommand])
|
private case class RegisteredOffer(offer: Offer, nodeKey: Option[PrivateKey], pathId_opt: Option[ByteVector32], handler: ActorRef[HandlerCommand])
|
||||||
|
|
||||||
def apply(nodeParams: NodeParams, router: akka.actor.ActorRef, paymentTimeout: FiniteDuration): Behavior[Command] = {
|
def apply(nodeParams: NodeParams, router: akka.actor.ActorRef, paymentTimeout: FiniteDuration): Behavior[Command] = {
|
||||||
Behaviors.setup { context =>
|
Behaviors.setup { context =>
|
||||||
|
@ -104,10 +104,10 @@ object OfferManager {
|
||||||
normal(registeredOffers + (offer.offerId -> RegisteredOffer(offer, nodeKey, pathId_opt, handler)))
|
normal(registeredOffers + (offer.offerId -> RegisteredOffer(offer, nodeKey, pathId_opt, handler)))
|
||||||
case DisableOffer(offer) =>
|
case DisableOffer(offer) =>
|
||||||
normal(registeredOffers - offer.offerId)
|
normal(registeredOffers - offer.offerId)
|
||||||
case RequestInvoice(messagePayload, postman) =>
|
case RequestInvoice(messagePayload, blindedKey, postman) =>
|
||||||
registeredOffers.get(messagePayload.invoiceRequest.offer.offerId) match {
|
registeredOffers.get(messagePayload.invoiceRequest.offer.offerId) match {
|
||||||
case Some(registered) if registered.pathId_opt.map(_.bytes) == messagePayload.pathId_opt && messagePayload.invoiceRequest.isValid =>
|
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, messagePayload.replyPath, postman))
|
val child = context.spawnAnonymous(InvoiceRequestActor(nodeParams, messagePayload.invoiceRequest, registered.handler, registered.nodeKey.getOrElse(blindedKey), router, messagePayload.replyPath, postman))
|
||||||
child ! InvoiceRequestActor.RequestInvoice
|
child ! InvoiceRequestActor.RequestInvoice
|
||||||
case _ => context.log.debug("offer {} is not registered or invoice request is invalid", messagePayload.invoiceRequest.offer.offerId)
|
case _ => context.log.debug("offer {} is not registered or invoice request is invalid", messagePayload.invoiceRequest.offer.offerId)
|
||||||
}
|
}
|
||||||
|
@ -116,8 +116,8 @@ object OfferManager {
|
||||||
MinimalInvoiceData.decode(payload.pathId) match {
|
MinimalInvoiceData.decode(payload.pathId) match {
|
||||||
case Some(signed) =>
|
case Some(signed) =>
|
||||||
registeredOffers.get(signed.offerId) match {
|
registeredOffers.get(signed.offerId) match {
|
||||||
case Some(RegisteredOffer(offer, nodeKey, _, handler)) =>
|
case Some(RegisteredOffer(offer, _, _, handler)) =>
|
||||||
MinimalInvoiceData.verify(nodeKey.publicKey, signed) match {
|
MinimalInvoiceData.verify(nodeParams.nodeId, signed) match {
|
||||||
case Some(metadata) if Crypto.sha256(metadata.preimage) == paymentHash =>
|
case Some(metadata) if Crypto.sha256(metadata.preimage) == paymentHash =>
|
||||||
val child = context.spawnAnonymous(PaymentActor(nodeParams, replyTo, offer, metadata, paymentTimeout))
|
val child = context.spawnAnonymous(PaymentActor(nodeParams, replyTo, offer, metadata, paymentTimeout))
|
||||||
handler ! HandlePayment(child, signed.offerId, metadata.pluginData_opt)
|
handler ! HandlePayment(child, signed.offerId, metadata.pluginData_opt)
|
||||||
|
@ -196,7 +196,7 @@ object OfferManager {
|
||||||
case ApproveRequest(amount, routes, pluginData_opt, additionalTlvs, customTlvs) =>
|
case ApproveRequest(amount, routes, pluginData_opt, additionalTlvs, customTlvs) =>
|
||||||
val preimage = randomBytes32()
|
val preimage = randomBytes32()
|
||||||
val metadata = MinimalInvoiceData(preimage, invoiceRequest.payerId, TimestampSecond.now(), invoiceRequest.quantity, amount, pluginData_opt)
|
val metadata = MinimalInvoiceData(preimage, invoiceRequest.payerId, TimestampSecond.now(), invoiceRequest.quantity, amount, pluginData_opt)
|
||||||
val pathId = MinimalInvoiceData.encode(nodeKey, invoiceRequest.offer.offerId, metadata)
|
val pathId = MinimalInvoiceData.encode(nodeParams.privateKey, invoiceRequest.offer.offerId, metadata)
|
||||||
val receivePayment = MultiPartHandler.ReceiveOfferPayment(context.messageAdapter[CreateInvoiceActor.Bolt12InvoiceResponse](WrappedInvoiceResponse), nodeKey, invoiceRequest, routes, router, preimage, pathId, additionalTlvs, customTlvs)
|
val receivePayment = MultiPartHandler.ReceiveOfferPayment(context.messageAdapter[CreateInvoiceActor.Bolt12InvoiceResponse](WrappedInvoiceResponse), nodeKey, invoiceRequest, routes, router, preimage, pathId, additionalTlvs, customTlvs)
|
||||||
val child = context.spawnAnonymous(CreateInvoiceActor(nodeParams))
|
val child = context.spawnAnonymous(CreateInvoiceActor(nodeParams))
|
||||||
child ! CreateInvoiceActor.CreateBolt12Invoice(receivePayment)
|
child ! CreateInvoiceActor.CreateBolt12Invoice(receivePayment)
|
||||||
|
|
|
@ -300,7 +300,6 @@ object MultiPartHandler {
|
||||||
pathId: ByteVector,
|
pathId: ByteVector,
|
||||||
additionalTlvs: Set[InvoiceTlv] = Set.empty,
|
additionalTlvs: Set[InvoiceTlv] = Set.empty,
|
||||||
customTlvs: Set[GenericTlv] = Set.empty) extends ReceivePayment {
|
customTlvs: Set[GenericTlv] = Set.empty) extends ReceivePayment {
|
||||||
require(nodeKey.publicKey == invoiceRequest.offer.nodeId, "the node id of the invoice must be the same as the one from the offer")
|
|
||||||
require(routes.forall(_.nodes.nonEmpty), "each route must have at least one node")
|
require(routes.forall(_.nodes.nonEmpty), "each route must have at least one node")
|
||||||
require(invoiceRequest.offer.amount.nonEmpty || invoiceRequest.amount.nonEmpty, "an amount must be specified in the offer or in the invoice request")
|
require(invoiceRequest.offer.amount.nonEmpty || invoiceRequest.amount.nonEmpty, "an amount must be specified in the offer or in the invoice request")
|
||||||
|
|
||||||
|
|
|
@ -126,12 +126,12 @@ private class OfferPayment(replyTo: ActorRef,
|
||||||
val messageContent = TlvStream[OnionMessagePayloadTlv](OnionMessagePayloadTlv.InvoiceRequest(invoiceRequest.records))
|
val messageContent = TlvStream[OnionMessagePayloadTlv](OnionMessagePayloadTlv.InvoiceRequest(invoiceRequest.records))
|
||||||
val routingStrategy = if (sendPaymentConfig.connectDirectly) OnionMessages.RoutingStrategy.connectDirectly else OnionMessages.RoutingStrategy.FindRoute
|
val routingStrategy = if (sendPaymentConfig.connectDirectly) OnionMessages.RoutingStrategy.connectDirectly else OnionMessages.RoutingStrategy.FindRoute
|
||||||
postman ! SendMessage(contactInfo, routingStrategy, messageContent, expectsReply = true, context.messageAdapter(WrappedMessageResponse))
|
postman ! SendMessage(contactInfo, routingStrategy, messageContent, expectsReply = true, context.messageAdapter(WrappedMessageResponse))
|
||||||
waitForInvoice(attemptNumber + 1)
|
waitForInvoice(attemptNumber + 1, contactInfo.nodeId)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def waitForInvoice(attemptNumber: Int): Behavior[Command] = {
|
private def waitForInvoice(attemptNumber: Int, pathNodeId: PublicKey): Behavior[Command] = {
|
||||||
Behaviors.receiveMessagePartial {
|
Behaviors.receiveMessagePartial {
|
||||||
case WrappedMessageResponse(Postman.Response(payload: InvoicePayload)) if payload.invoice.validateFor(invoiceRequest).isRight =>
|
case WrappedMessageResponse(Postman.Response(payload: InvoicePayload)) if payload.invoice.validateFor(invoiceRequest, pathNodeId).isRight =>
|
||||||
sendPaymentConfig.trampoline match {
|
sendPaymentConfig.trampoline match {
|
||||||
case Some(trampoline) =>
|
case Some(trampoline) =>
|
||||||
paymentInitiator ! SendTrampolinePayment(replyTo, payload.invoice.amount, payload.invoice, trampoline.nodeId, trampoline.attempts, sendPaymentConfig.routeParams)
|
paymentInitiator ! SendTrampolinePayment(replyTo, payload.invoice.amount, payload.invoice, trampoline.nodeId, trampoline.attempts, sendPaymentConfig.routeParams)
|
||||||
|
|
|
@ -37,11 +37,15 @@ import scala.util.{Failure, Try}
|
||||||
object OfferTypes {
|
object OfferTypes {
|
||||||
// @formatter:off
|
// @formatter:off
|
||||||
/** Data provided to reach the issuer of an offer or invoice. */
|
/** Data provided to reach the issuer of an offer or invoice. */
|
||||||
sealed trait ContactInfo
|
sealed trait ContactInfo {
|
||||||
|
val nodeId: PublicKey
|
||||||
|
}
|
||||||
/** If the offer or invoice issuer doesn't want to hide their identity, they can directly share their public nodeId. */
|
/** 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
|
case class RecipientNodeId(nodeId: PublicKey) extends ContactInfo
|
||||||
/** If the offer or invoice issuer wants to hide their identity, they instead provide blinded paths. */
|
/** If the offer or invoice issuer wants to hide their identity, they instead provide blinded paths. */
|
||||||
case class BlindedPath(route: BlindedRoute) extends ContactInfo
|
case class BlindedPath(route: BlindedRoute) extends ContactInfo {
|
||||||
|
override val nodeId: PublicKey = route.blindedNodeIds.last
|
||||||
|
}
|
||||||
// @formatter:on
|
// @formatter:on
|
||||||
|
|
||||||
sealed trait Bolt12Tlv extends Tlv
|
sealed trait Bolt12Tlv extends Tlv
|
||||||
|
@ -233,15 +237,15 @@ object OfferTypes {
|
||||||
case Some(_) => None // TODO: add exchange rates
|
case Some(_) => None // TODO: add exchange rates
|
||||||
case None => records.get[OfferAmount].map(_.amount)
|
case None => records.get[OfferAmount].map(_.amount)
|
||||||
}
|
}
|
||||||
val description: String = records.get[OfferDescription].get.description
|
val description: Option[String] = records.get[OfferDescription].map(_.description)
|
||||||
val features: Features[Bolt12Feature] = records.get[OfferFeatures].map(_.features.bolt12Features()).getOrElse(Features.empty)
|
val features: Features[Bolt12Feature] = records.get[OfferFeatures].map(_.features.bolt12Features()).getOrElse(Features.empty)
|
||||||
val expiry: Option[TimestampSecond] = records.get[OfferAbsoluteExpiry].map(_.absoluteExpiry)
|
val expiry: Option[TimestampSecond] = records.get[OfferAbsoluteExpiry].map(_.absoluteExpiry)
|
||||||
private val paths: Option[Seq[BlindedPath]] = records.get[OfferPaths].map(_.paths.map(BlindedPath))
|
private val paths: Option[Seq[BlindedPath]] = records.get[OfferPaths].map(_.paths.map(BlindedPath))
|
||||||
val issuer: Option[String] = records.get[OfferIssuer].map(_.issuer)
|
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 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 nodeId: Option[PublicKey] = records.get[OfferNodeId].map(_.publicKey)
|
||||||
|
|
||||||
val contactInfos: Seq[ContactInfo] = paths.getOrElse(Seq(RecipientNodeId(nodeId)))
|
val contactInfos: Seq[ContactInfo] = paths.getOrElse(Seq(RecipientNodeId(nodeId.get)))
|
||||||
|
|
||||||
def encode(): String = {
|
def encode(): String = {
|
||||||
val data = OfferCodecs.offerTlvCodec.encode(records).require.bytes
|
val data = OfferCodecs.offerTlvCodec.encode(records).require.bytes
|
||||||
|
@ -264,25 +268,44 @@ object OfferTypes {
|
||||||
* @param chain chain on which the offer is valid.
|
* @param chain chain on which the offer is valid.
|
||||||
*/
|
*/
|
||||||
def apply(amount_opt: Option[MilliSatoshi],
|
def apply(amount_opt: Option[MilliSatoshi],
|
||||||
description: String,
|
description_opt: Option[String],
|
||||||
nodeId: PublicKey,
|
nodeId: PublicKey,
|
||||||
features: Features[Bolt12Feature],
|
features: Features[Bolt12Feature],
|
||||||
chain: BlockHash,
|
chain: BlockHash,
|
||||||
additionalTlvs: Set[OfferTlv] = Set.empty,
|
additionalTlvs: Set[OfferTlv] = Set.empty,
|
||||||
customTlvs: Set[GenericTlv] = Set.empty): Offer = {
|
customTlvs: Set[GenericTlv] = Set.empty): Offer = {
|
||||||
|
require(amount_opt.isEmpty || description_opt.nonEmpty)
|
||||||
val tlvs: Set[OfferTlv] = Set(
|
val tlvs: Set[OfferTlv] = Set(
|
||||||
if (chain != Block.LivenetGenesisBlock.hash) Some(OfferChains(Seq(chain))) else None,
|
if (chain != Block.LivenetGenesisBlock.hash) Some(OfferChains(Seq(chain))) else None,
|
||||||
amount_opt.map(OfferAmount),
|
amount_opt.map(OfferAmount),
|
||||||
Some(OfferDescription(description)),
|
description_opt.map(OfferDescription),
|
||||||
if (!features.isEmpty) Some(OfferFeatures(features.unscoped())) else None,
|
if (!features.isEmpty) Some(OfferFeatures(features.unscoped())) else None,
|
||||||
Some(OfferNodeId(nodeId)),
|
Some(OfferNodeId(nodeId)),
|
||||||
).flatten ++ additionalTlvs
|
).flatten ++ additionalTlvs
|
||||||
Offer(TlvStream(tlvs, customTlvs))
|
Offer(TlvStream(tlvs, customTlvs))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
def withPaths(amount_opt: Option[MilliSatoshi],
|
||||||
|
description_opt: Option[String],
|
||||||
|
paths: Seq[BlindedRoute],
|
||||||
|
features: Features[Bolt12Feature],
|
||||||
|
chain: BlockHash,
|
||||||
|
additionalTlvs: Set[OfferTlv] = Set.empty,
|
||||||
|
customTlvs: Set[GenericTlv] = Set.empty): Offer = {
|
||||||
|
require(amount_opt.isEmpty || description_opt.nonEmpty)
|
||||||
|
val tlvs: Set[OfferTlv] = Set(
|
||||||
|
if (chain != Block.LivenetGenesisBlock.hash) Some(OfferChains(Seq(chain))) else None,
|
||||||
|
amount_opt.map(OfferAmount),
|
||||||
|
description_opt.map(OfferDescription),
|
||||||
|
if (!features.isEmpty) Some(OfferFeatures(features.unscoped())) else None,
|
||||||
|
Some(OfferPaths(paths))
|
||||||
|
).flatten ++ additionalTlvs
|
||||||
|
Offer(TlvStream(tlvs, customTlvs))
|
||||||
|
}
|
||||||
|
|
||||||
def validate(records: TlvStream[OfferTlv]): Either[InvalidTlvPayload, Offer] = {
|
def validate(records: TlvStream[OfferTlv]): Either[InvalidTlvPayload, Offer] = {
|
||||||
if (records.get[OfferDescription].isEmpty) return Left(MissingRequiredTlv(UInt64(10)))
|
if (records.get[OfferDescription].isEmpty && records.get[OfferAmount].nonEmpty) return Left(MissingRequiredTlv(UInt64(10)))
|
||||||
if (records.get[OfferNodeId].isEmpty) return Left(MissingRequiredTlv(UInt64(22)))
|
if (records.get[OfferNodeId].isEmpty && records.get[OfferPaths].forall(_.paths.isEmpty)) return Left(MissingRequiredTlv(UInt64(22)))
|
||||||
if (records.unknown.exists(_.tag >= UInt64(80))) return Left(ForbiddenTlv(records.unknown.find(_.tag >= UInt64(80)).get.tag))
|
if (records.unknown.exists(_.tag >= UInt64(80))) return Left(ForbiddenTlv(records.unknown.find(_.tag >= UInt64(80)).get.tag))
|
||||||
Right(Offer(records))
|
Right(Offer(records))
|
||||||
}
|
}
|
||||||
|
|
|
@ -665,7 +665,8 @@ class PaymentsDbSpec extends AnyFunSuite {
|
||||||
|
|
||||||
val paidInvoice1 = Bolt11Invoice(Block.TestnetGenesisBlock.hash, Some(561 msat), randomBytes32(), alicePriv, Left("invoice #7"), CltvExpiryDelta(18), timestamp = TimestampSecond.now() - 5.seconds)
|
val paidInvoice1 = Bolt11Invoice(Block.TestnetGenesisBlock.hash, Some(561 msat), randomBytes32(), alicePriv, Left("invoice #7"), CltvExpiryDelta(18), timestamp = TimestampSecond.now() - 5.seconds)
|
||||||
val paidInvoice2 = Bolt11Invoice(Block.TestnetGenesisBlock.hash, Some(1105 msat), randomBytes32(), bobPriv, Left("invoice #8"), CltvExpiryDelta(18), expirySeconds = Some(60), timestamp = TimestampSecond.now() - 4.seconds)
|
val paidInvoice2 = Bolt11Invoice(Block.TestnetGenesisBlock.hash, Some(1105 msat), randomBytes32(), bobPriv, Left("invoice #8"), CltvExpiryDelta(18), expirySeconds = Some(60), timestamp = TimestampSecond.now() - 4.seconds)
|
||||||
val offer = Offer(None, "offer", randomKey().publicKey, Features.empty, Block.TestnetGenesisBlock.hash)
|
val nodeId = randomKey().publicKey
|
||||||
|
val offer = Offer(None, Some("offer"), nodeId, Features.empty, Block.TestnetGenesisBlock.hash)
|
||||||
val paidInvoice3 = MinimalBolt12Invoice(offer, Block.TestnetGenesisBlock.hash, 1729 msat, 1, randomBytes32(), randomKey().publicKey, TimestampSecond.now() - 3.seconds)
|
val paidInvoice3 = MinimalBolt12Invoice(offer, Block.TestnetGenesisBlock.hash, 1729 msat, 1, randomBytes32(), randomKey().publicKey, TimestampSecond.now() - 3.seconds)
|
||||||
val receivedAt1 = TimestampMilli.now() + 1.milli
|
val receivedAt1 = TimestampMilli.now() + 1.milli
|
||||||
val receivedAt2 = TimestampMilli.now() + 2.milli
|
val receivedAt2 = TimestampMilli.now() + 2.milli
|
||||||
|
@ -799,7 +800,7 @@ object PaymentsDbSpec {
|
||||||
val (paymentHash1, paymentHash2, paymentHash3, paymentHash4) = (Crypto.sha256(preimage1), Crypto.sha256(preimage2), Crypto.sha256(preimage3), Crypto.sha256(preimage4))
|
val (paymentHash1, paymentHash2, paymentHash3, paymentHash4) = (Crypto.sha256(preimage1), Crypto.sha256(preimage2), Crypto.sha256(preimage3), Crypto.sha256(preimage4))
|
||||||
|
|
||||||
def createBolt12Invoice(amount: MilliSatoshi, payerKey: PrivateKey, recipientKey: PrivateKey, preimage: ByteVector32): Bolt12Invoice = {
|
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 offer = Offer(Some(amount), Some("some offer"), recipientKey.publicKey, Features.empty, Block.TestnetGenesisBlock.hash)
|
||||||
val invoiceRequest = InvoiceRequest(offer, 789 msat, 1, Features.empty, payerKey, Block.TestnetGenesisBlock.hash)
|
val invoiceRequest = InvoiceRequest(offer, 789 msat, 1, Features.empty, payerKey, Block.TestnetGenesisBlock.hash)
|
||||||
val dummyRoute = PaymentBlindedRoute(RouteBlinding.create(randomKey(), Seq(randomKey().publicKey), Seq(randomBytes(100))).route, PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 0 msat, 0 msat, Features.empty))
|
val dummyRoute = PaymentBlindedRoute(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))
|
Bolt12Invoice(invoiceRequest, preimage, recipientKey, 1 hour, Features.empty, Seq(dummyRoute))
|
||||||
|
|
|
@ -82,7 +82,7 @@ class MessageIntegrationSpec extends IntegrationSpec {
|
||||||
val eventListener = TestProbe()
|
val eventListener = TestProbe()
|
||||||
nodes("B").system.eventStream.subscribe(eventListener.ref, classOf[OnionMessages.ReceiveMessage])
|
nodes("B").system.eventStream.subscribe(eventListener.ref, classOf[OnionMessages.ReceiveMessage])
|
||||||
|
|
||||||
val blindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(nodes("A").nodeParams.nodeId), IntermediateNode(nodes("B").nodeParams.nodeId), IntermediateNode(nodes("B").nodeParams.nodeId)), Recipient(nodes("B").nodeParams.nodeId, None))
|
val blindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(nodes("A").nodeParams.nodeId), IntermediateNode(nodes("B").nodeParams.nodeId), IntermediateNode(nodes("B").nodeParams.nodeId)), Recipient(nodes("B").nodeParams.nodeId, None)).route
|
||||||
assert(blindedRoute.introductionNodeId == EncodedNodeId(nodes("A").nodeParams.nodeId))
|
assert(blindedRoute.introductionNodeId == EncodedNodeId(nodes("A").nodeParams.nodeId))
|
||||||
|
|
||||||
alice.sendOnionMessage(None, Right(blindedRoute), expectsReply = false, ByteVector.empty).pipeTo(probe.ref)
|
alice.sendOnionMessage(None, Right(blindedRoute), expectsReply = false, ByteVector.empty).pipeTo(probe.ref)
|
||||||
|
|
|
@ -694,13 +694,13 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
||||||
val chain = nodes("D").nodeParams.chainHash
|
val chain = nodes("D").nodeParams.chainHash
|
||||||
val pathId = randomBytes32()
|
val pathId = randomBytes32()
|
||||||
val offerPaths = Seq(
|
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("G").nodeParams.nodeId), IntermediateNode(nodes("C").nodeParams.nodeId)), Recipient(nodes("D").nodeParams.nodeId, Some(pathId))).route,
|
||||||
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("B").nodeParams.nodeId), IntermediateNode(nodes("C").nodeParams.nodeId)), Recipient(nodes("D").nodeParams.nodeId, Some(pathId))).route,
|
||||||
buildRoute(randomKey(), Seq(IntermediateNode(nodes("E").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))).route
|
||||||
)
|
)
|
||||||
val offer = Offer(Some(amount), "test offer", recipientKey.publicKey, nodes("D").nodeParams.features.bolt12Features(), chain, additionalTlvs = Set(OfferPaths(offerPaths)))
|
val offer = Offer(Some(amount), Some("test offer"), recipientKey.publicKey, nodes("D").nodeParams.features.bolt12Features(), chain, additionalTlvs = Set(OfferPaths(offerPaths)))
|
||||||
val offerHandler = TypedProbe[HandlerCommand]()(nodes("D").system.toTyped)
|
val offerHandler = TypedProbe[HandlerCommand]()(nodes("D").system.toTyped)
|
||||||
nodes("D").offerManager ! RegisterOffer(offer, recipientKey, Some(pathId), offerHandler.ref)
|
nodes("D").offerManager ! RegisterOffer(offer, Some(recipientKey), Some(pathId), offerHandler.ref)
|
||||||
|
|
||||||
val sender = TestProbe()
|
val sender = TestProbe()
|
||||||
val bob = new EclairImpl(nodes("B"))
|
val bob = new EclairImpl(nodes("B"))
|
||||||
|
@ -731,9 +731,9 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
||||||
test("send a blinded payment D->C with empty blinded routes") {
|
test("send a blinded payment D->C with empty blinded routes") {
|
||||||
val amount = 25_000_000 msat
|
val amount = 25_000_000 msat
|
||||||
val chain = nodes("C").nodeParams.chainHash
|
val chain = nodes("C").nodeParams.chainHash
|
||||||
val offer = Offer(Some(amount), "test offer", nodes("C").nodeParams.nodeId, nodes("C").nodeParams.features.bolt12Features(), chain)
|
val offer = Offer(Some(amount), Some("test offer"), nodes("C").nodeParams.nodeId, nodes("C").nodeParams.features.bolt12Features(), chain)
|
||||||
val offerHandler = TypedProbe[HandlerCommand]()(nodes("C").system.toTyped)
|
val offerHandler = TypedProbe[HandlerCommand]()(nodes("C").system.toTyped)
|
||||||
nodes("C").offerManager ! RegisterOffer(offer, nodes("C").nodeParams.privateKey, None, offerHandler.ref)
|
nodes("C").offerManager ! RegisterOffer(offer, Some(nodes("C").nodeParams.privateKey), None, offerHandler.ref)
|
||||||
|
|
||||||
val sender = TestProbe()
|
val sender = TestProbe()
|
||||||
val dave = new EclairImpl(nodes("D"))
|
val dave = new EclairImpl(nodes("D"))
|
||||||
|
@ -766,10 +766,10 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
||||||
val amount = 50_000_000 msat
|
val amount = 50_000_000 msat
|
||||||
val chain = nodes("A").nodeParams.chainHash
|
val chain = nodes("A").nodeParams.chainHash
|
||||||
val pathId = randomBytes32()
|
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 = buildRoute(randomKey(), Seq(IntermediateNode(nodes("A").nodeParams.nodeId), IntermediateNode(nodes("A").nodeParams.nodeId)), Recipient(nodes("A").nodeParams.nodeId, Some(pathId))).route
|
||||||
val offer = Offer(Some(amount), "test offer", recipientKey.publicKey, nodes("A").nodeParams.features.bolt12Features(), chain, additionalTlvs = Set(OfferPaths(Seq(offerPath))))
|
val offer = Offer(Some(amount), Some("test offer"), recipientKey.publicKey, nodes("A").nodeParams.features.bolt12Features(), chain, additionalTlvs = Set(OfferPaths(Seq(offerPath))))
|
||||||
val offerHandler = TypedProbe[HandlerCommand]()(nodes("A").system.toTyped)
|
val offerHandler = TypedProbe[HandlerCommand]()(nodes("A").system.toTyped)
|
||||||
nodes("A").offerManager ! RegisterOffer(offer, recipientKey, Some(pathId), offerHandler.ref)
|
nodes("A").offerManager ! RegisterOffer(offer, Some(recipientKey), Some(pathId), offerHandler.ref)
|
||||||
|
|
||||||
val sender = TestProbe()
|
val sender = TestProbe()
|
||||||
val bob = new EclairImpl(nodes("B"))
|
val bob = new EclairImpl(nodes("B"))
|
||||||
|
@ -800,10 +800,10 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
||||||
val amount = 10_000_000 msat
|
val amount = 10_000_000 msat
|
||||||
val chain = nodes("C").nodeParams.chainHash
|
val chain = nodes("C").nodeParams.chainHash
|
||||||
val pathId = randomBytes32()
|
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 = buildRoute(randomKey(), Seq(IntermediateNode(nodes("B").nodeParams.nodeId), IntermediateNode(nodes("C").nodeParams.nodeId)), Recipient(nodes("C").nodeParams.nodeId, Some(pathId))).route
|
||||||
val offer = Offer(Some(amount), "tricky test offer", recipientKey.publicKey, nodes("C").nodeParams.features.bolt12Features(), chain, additionalTlvs = Set(OfferPaths(Seq(offerPath))))
|
val offer = Offer(Some(amount), Some("tricky test offer"), recipientKey.publicKey, nodes("C").nodeParams.features.bolt12Features(), chain, additionalTlvs = Set(OfferPaths(Seq(offerPath))))
|
||||||
val offerHandler = TypedProbe[HandlerCommand]()(nodes("C").system.toTyped)
|
val offerHandler = TypedProbe[HandlerCommand]()(nodes("C").system.toTyped)
|
||||||
nodes("C").offerManager ! RegisterOffer(offer, recipientKey, Some(pathId), offerHandler.ref)
|
nodes("C").offerManager ! RegisterOffer(offer, Some(recipientKey), Some(pathId), offerHandler.ref)
|
||||||
|
|
||||||
val sender = TestProbe()
|
val sender = TestProbe()
|
||||||
val bob = new EclairImpl(nodes("B"))
|
val bob = new EclairImpl(nodes("B"))
|
||||||
|
@ -832,9 +832,9 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
||||||
test("send a blinded payment A->D with trampoline") {
|
test("send a blinded payment A->D with trampoline") {
|
||||||
val amount = 10_000_000 msat
|
val amount = 10_000_000 msat
|
||||||
val chain = nodes("D").nodeParams.chainHash
|
val chain = nodes("D").nodeParams.chainHash
|
||||||
val offer = Offer(Some(amount), "test offer", nodes("D").nodeParams.nodeId, nodes("D").nodeParams.features.bolt12Features(), chain)
|
val offer = Offer(Some(amount), Some("test offer"), nodes("D").nodeParams.nodeId, nodes("D").nodeParams.features.bolt12Features(), chain)
|
||||||
val offerHandler = TypedProbe[HandlerCommand]()(nodes("D").system.toTyped)
|
val offerHandler = TypedProbe[HandlerCommand]()(nodes("D").system.toTyped)
|
||||||
nodes("D").offerManager ! RegisterOffer(offer, nodes("D").nodeParams.privateKey, None, offerHandler.ref)
|
nodes("D").offerManager ! RegisterOffer(offer, Some(nodes("D").nodeParams.privateKey), None, offerHandler.ref)
|
||||||
|
|
||||||
val sender = TestProbe()
|
val sender = TestProbe()
|
||||||
val alice = new EclairImpl(nodes("A"))
|
val alice = new EclairImpl(nodes("A"))
|
||||||
|
@ -875,11 +875,11 @@ class PaymentIntegrationSpec extends IntegrationSpec {
|
||||||
val Some(channelBE) = probe.expectMsgType[Iterable[ChannelAnnouncement]].find(ann => Set(ann.nodeId1, ann.nodeId2) == Set(nodes("B").nodeParams.nodeId, nodes("E").nodeParams.nodeId))
|
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)
|
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 offerBlindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(nodes("B").nodeParams.nodeId), IntermediateNode(nodes("C").nodeParams.nodeId)), Recipient(nodes("C").nodeParams.nodeId, Some(pathId))).route
|
||||||
val offerPath = BlindedRoute(scidDirEB, offerBlindedRoute.blindingKey, offerBlindedRoute.blindedNodes)
|
val offerPath = BlindedRoute(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 offer = Offer(Some(amount), Some("test offer"), recipientKey.publicKey, nodes("C").nodeParams.features.bolt12Features(), chain, additionalTlvs = Set(OfferPaths(Seq(offerPath))))
|
||||||
val offerHandler = TypedProbe[HandlerCommand]()(nodes("C").system.toTyped)
|
val offerHandler = TypedProbe[HandlerCommand]()(nodes("C").system.toTyped)
|
||||||
nodes("C").offerManager ! RegisterOffer(offer, recipientKey, Some(pathId), offerHandler.ref)
|
nodes("C").offerManager ! RegisterOffer(offer, Some(recipientKey), Some(pathId), offerHandler.ref)
|
||||||
|
|
||||||
val sender = TestProbe()
|
val sender = TestProbe()
|
||||||
val alice = new EclairImpl(nodes("A"))
|
val alice = new EclairImpl(nodes("A"))
|
||||||
|
|
|
@ -29,6 +29,7 @@ import fr.acinq.eclair.channel.{DATA_NORMAL, RealScidStatus}
|
||||||
import fr.acinq.eclair.integration.basic.fixtures.MinimalNodeFixture
|
import fr.acinq.eclair.integration.basic.fixtures.MinimalNodeFixture
|
||||||
import fr.acinq.eclair.integration.basic.fixtures.MinimalNodeFixture.{connect, getChannelData, getPeerChannels, getRouterData, knownFundingTxs, nodeParamsFor, openChannel, watcherAutopilot}
|
import fr.acinq.eclair.integration.basic.fixtures.MinimalNodeFixture.{connect, getChannelData, getPeerChannels, getRouterData, knownFundingTxs, nodeParamsFor, openChannel, watcherAutopilot}
|
||||||
import fr.acinq.eclair.integration.basic.fixtures.composite.ThreeNodesFixture
|
import fr.acinq.eclair.integration.basic.fixtures.composite.ThreeNodesFixture
|
||||||
|
import fr.acinq.eclair.message.OnionMessages
|
||||||
import fr.acinq.eclair.message.OnionMessages.{IntermediateNode, Recipient, buildRoute}
|
import fr.acinq.eclair.message.OnionMessages.{IntermediateNode, Recipient, buildRoute}
|
||||||
import fr.acinq.eclair.payment._
|
import fr.acinq.eclair.payment._
|
||||||
import fr.acinq.eclair.payment.offer.OfferManager
|
import fr.acinq.eclair.payment.offer.OfferManager
|
||||||
|
@ -122,9 +123,9 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience {
|
||||||
import f._
|
import f._
|
||||||
|
|
||||||
val sender = TestProbe("sender")
|
val sender = TestProbe("sender")
|
||||||
val offer = Offer(None, "test", recipient.nodeId, Features.empty, recipient.nodeParams.chainHash)
|
val offer = Offer(None, Some("test"), recipient.nodeId, Features.empty, recipient.nodeParams.chainHash)
|
||||||
val handler = recipient.system.spawnAnonymous(offerHandler(amount, routes))
|
val handler = recipient.system.spawnAnonymous(offerHandler(amount, routes))
|
||||||
recipient.offerManager ! OfferManager.RegisterOffer(offer, recipient.nodeParams.privateKey, None, handler)
|
recipient.offerManager ! OfferManager.RegisterOffer(offer, Some(recipient.nodeParams.privateKey), None, handler)
|
||||||
val offerPayment = payer.system.spawnAnonymous(OfferPayment(payer.nodeParams, payer.postman, payer.router, 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)
|
val sendPaymentConfig = OfferPayment.SendPaymentConfig(None, connectDirectly = false, maxAttempts = 1, payer.routeParams, blocking = true)
|
||||||
offerPayment ! OfferPayment.PayOffer(sender.ref, offer, amount, 1, sendPaymentConfig)
|
offerPayment ! OfferPayment.PayOffer(sender.ref, offer, amount, 1, sendPaymentConfig)
|
||||||
|
@ -140,11 +141,11 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience {
|
||||||
val offerPaths = routes.map(route => {
|
val offerPaths = routes.map(route => {
|
||||||
val ourNodeId = route.nodes.last
|
val ourNodeId = route.nodes.last
|
||||||
val intermediateNodes = route.nodes.dropRight(1).map(IntermediateNode(_)) ++ route.dummyHops.map(_ => IntermediateNode(ourNodeId))
|
val intermediateNodes = route.nodes.dropRight(1).map(IntermediateNode(_)) ++ route.dummyHops.map(_ => IntermediateNode(ourNodeId))
|
||||||
buildRoute(randomKey(), intermediateNodes, Recipient(ourNodeId, Some(pathId)))
|
buildRoute(randomKey(), intermediateNodes, Recipient(ourNodeId, Some(pathId))).route
|
||||||
})
|
})
|
||||||
val offer = Offer(None, "test", recipientKey.publicKey, Features.empty, recipient.nodeParams.chainHash, additionalTlvs = Set(OfferPaths(offerPaths)))
|
val offer = Offer(None, Some("test"), recipientKey.publicKey, Features.empty, recipient.nodeParams.chainHash, additionalTlvs = Set(OfferPaths(offerPaths)))
|
||||||
val handler = recipient.system.spawnAnonymous(offerHandler(amount, routes))
|
val handler = recipient.system.spawnAnonymous(offerHandler(amount, routes))
|
||||||
recipient.offerManager ! OfferManager.RegisterOffer(offer, recipientKey, Some(pathId), handler)
|
recipient.offerManager ! OfferManager.RegisterOffer(offer, Some(recipientKey), Some(pathId), handler)
|
||||||
val offerPayment = payer.system.spawnAnonymous(OfferPayment(payer.nodeParams, payer.postman, payer.router, 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)
|
val sendPaymentConfig = OfferPayment.SendPaymentConfig(None, connectDirectly = false, maxAttempts = 1, payer.routeParams, blocking = true)
|
||||||
offerPayment ! OfferPayment.PayOffer(sender.ref, offer, amount, 1, sendPaymentConfig)
|
offerPayment ! OfferPayment.PayOffer(sender.ref, offer, amount, 1, sendPaymentConfig)
|
||||||
|
@ -156,9 +157,9 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience {
|
||||||
|
|
||||||
val sender = TestProbe("sender")
|
val sender = TestProbe("sender")
|
||||||
val paymentInterceptor = TestProbe("payment-interceptor")
|
val paymentInterceptor = TestProbe("payment-interceptor")
|
||||||
val offer = Offer(None, "test", recipient.nodeId, Features.empty, recipient.nodeParams.chainHash)
|
val offer = Offer(None, Some("test"), recipient.nodeId, Features.empty, recipient.nodeParams.chainHash)
|
||||||
val handler = recipient.system.spawnAnonymous(offerHandler(recipientAmount, routes))
|
val handler = recipient.system.spawnAnonymous(offerHandler(recipientAmount, routes))
|
||||||
recipient.offerManager ! OfferManager.RegisterOffer(offer, recipient.nodeParams.privateKey, None, handler)
|
recipient.offerManager ! OfferManager.RegisterOffer(offer, Some(recipient.nodeParams.privateKey), None, handler)
|
||||||
val offerPayment = payer.system.spawnAnonymous(OfferPayment(payer.nodeParams, payer.postman, payer.router, 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)
|
val sendPaymentConfig = OfferPayment.SendPaymentConfig(None, connectDirectly = false, maxAttempts = 1, payer.routeParams, blocking = true)
|
||||||
offerPayment ! OfferPayment.PayOffer(sender.ref, offer, recipientAmount, 1, sendPaymentConfig)
|
offerPayment ! OfferPayment.PayOffer(sender.ref, offer, recipientAmount, 1, sendPaymentConfig)
|
||||||
|
@ -172,7 +173,7 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience {
|
||||||
assert(result.isInstanceOf[PaymentSent])
|
assert(result.isInstanceOf[PaymentSent])
|
||||||
val payment = result.asInstanceOf[PaymentSent]
|
val payment = result.asInstanceOf[PaymentSent]
|
||||||
assert(payment.recipientAmount == amount)
|
assert(payment.recipientAmount == amount)
|
||||||
assert(payment.recipientNodeId == offer.nodeId)
|
assert(offer.nodeId.forall(_ == payment.recipientNodeId))
|
||||||
assert(payment.parts.map(_.amount).sum == amount)
|
assert(payment.parts.map(_.amount).sum == amount)
|
||||||
payment
|
payment
|
||||||
}
|
}
|
||||||
|
@ -359,20 +360,39 @@ class OfferPaymentSpec extends FixtureSpec with IntegrationPatience {
|
||||||
val recipientKey = randomKey()
|
val recipientKey = randomKey()
|
||||||
val pathId = randomBytes32()
|
val pathId = randomBytes32()
|
||||||
|
|
||||||
val blindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(bob.nodeId), IntermediateNode(carol.nodeId)), Recipient(carol.nodeId, Some(pathId)))
|
val blindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(bob.nodeId), IntermediateNode(carol.nodeId)), Recipient(carol.nodeId, Some(pathId))).route
|
||||||
val offer = Offer(None, "test", recipientKey.publicKey, Features.empty, carol.nodeParams.chainHash, additionalTlvs = Set(OfferPaths(Seq(blindedRoute))))
|
val offer = Offer(None, Some("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 scid_bc = getPeerChannels(bob, carol.nodeId).head.data.asInstanceOf[DATA_NORMAL].shortIds.real.toOption.get
|
||||||
val compactBlindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(bob.nodeId, EncodedNodeId(bob.nodeId), Some(scid_bc)), IntermediateNode(carol.nodeId, EncodedNodeId(carol.nodeId), Some(ShortChannelId.toSelf))), Recipient(carol.nodeId, Some(pathId)))
|
val compactBlindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(bob.nodeId, EncodedNodeId(bob.nodeId), Some(scid_bc)), IntermediateNode(carol.nodeId, EncodedNodeId(carol.nodeId), Some(ShortChannelId.toSelf))), Recipient(carol.nodeId, Some(pathId))).route
|
||||||
val compactOffer = Offer(None, "test", recipientKey.publicKey, Features.empty, carol.nodeParams.chainHash, additionalTlvs = Set(OfferPaths(Seq(compactBlindedRoute))))
|
val compactOffer = Offer(None, Some("test"), recipientKey.publicKey, Features.empty, carol.nodeParams.chainHash, additionalTlvs = Set(OfferPaths(Seq(compactBlindedRoute))))
|
||||||
assert(compactOffer.toString.length < offer.toString.length)
|
assert(compactOffer.toString.length < offer.toString.length)
|
||||||
|
|
||||||
val receivingRoute = ReceivingRoute(Seq(bob.nodeId, carol.nodeId), maxFinalExpiryDelta)
|
val receivingRoute = ReceivingRoute(Seq(bob.nodeId, carol.nodeId), maxFinalExpiryDelta)
|
||||||
val handler = carol.system.spawnAnonymous(offerHandler(amount, Seq(receivingRoute)))
|
val handler = carol.system.spawnAnonymous(offerHandler(amount, Seq(receivingRoute)))
|
||||||
carol.offerManager ! OfferManager.RegisterOffer(compactOffer, recipientKey, Some(pathId), handler)
|
carol.offerManager ! OfferManager.RegisterOffer(compactOffer, Some(recipientKey), Some(pathId), handler)
|
||||||
val offerPayment = alice.system.spawnAnonymous(OfferPayment(alice.nodeParams, alice.postman, alice.router, 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)
|
val sendPaymentConfig = OfferPayment.SendPaymentConfig(None, connectDirectly = false, maxAttempts = 1, alice.routeParams, blocking = true)
|
||||||
offerPayment ! OfferPayment.PayOffer(probe.ref, compactOffer, amount, 1, sendPaymentConfig)
|
offerPayment ! OfferPayment.PayOffer(probe.ref, compactOffer, amount, 1, sendPaymentConfig)
|
||||||
val payment = verifyPaymentSuccess(compactOffer, amount, probe.expectMsgType[PaymentEvent])
|
val payment = verifyPaymentSuccess(compactOffer, amount, probe.expectMsgType[PaymentEvent])
|
||||||
assert(payment.parts.length == 1)
|
assert(payment.parts.length == 1)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
test("send payment a->b->c offer with implicit node id") { f =>
|
||||||
|
import f._
|
||||||
|
|
||||||
|
val sender = TestProbe("sender")
|
||||||
|
val pathId = randomBytes32()
|
||||||
|
val amount = 25_000_000 msat
|
||||||
|
|
||||||
|
val offerPaths = Seq(OnionMessages.buildRoute(randomKey(), Seq(IntermediateNode(bob.nodeId)), Recipient(carol.nodeId, Some(pathId))).route)
|
||||||
|
val offer = Offer.withPaths(None, Some("implicit node id"), offerPaths, Features.empty, carol.nodeParams.chainHash)
|
||||||
|
val handler = carol.system.spawnAnonymous(offerHandler(amount, Seq(ReceivingRoute(Seq(bob.nodeId, carol.nodeId), maxFinalExpiryDelta))))
|
||||||
|
carol.offerManager ! OfferManager.RegisterOffer(offer, None, Some(pathId), handler)
|
||||||
|
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(sender.ref, offer, amount, 1, sendPaymentConfig)
|
||||||
|
val result = sender.expectMsgType[PaymentEvent]
|
||||||
|
val payment = verifyPaymentSuccess(offer, amount, result)
|
||||||
|
assert(payment.parts.length == 1)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -208,7 +208,7 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
assert(connectToNextPeer.nodeId == bobId)
|
assert(connectToNextPeer.nodeId == bobId)
|
||||||
connectToNextPeer.replyTo ! PeerConnection.ConnectionResult.AlreadyConnected(peerConnection.ref.toClassic, peer.ref.toClassic)
|
connectToNextPeer.replyTo ! PeerConnection.ConnectionResult.AlreadyConnected(peerConnection.ref.toClassic, peer.ref.toClassic)
|
||||||
val messageToBob = peer.expectMessageType[Peer.RelayOnionMessage].msg
|
val messageToBob = peer.expectMessageType[Peer.RelayOnionMessage].msg
|
||||||
val OnionMessages.ReceiveMessage(payload) = OnionMessages.process(Bob.nodeParams.privateKey, messageToBob)
|
val OnionMessages.ReceiveMessage(payload, _) = OnionMessages.process(Bob.nodeParams.privateKey, messageToBob)
|
||||||
assert(payload.records.unknown == Set(GenericTlv(UInt64(31), hex"f3ed")))
|
assert(payload.records.unknown == Set(GenericTlv(UInt64(31), hex"f3ed")))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -231,7 +231,7 @@ class MessageRelaySpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
assert(getNodeId.shortChannelId == RealShortChannelId(123L))
|
assert(getNodeId.shortChannelId == RealShortChannelId(123L))
|
||||||
getNodeId.replyTo ! Some(aliceId)
|
getNodeId.replyTo ! Some(aliceId)
|
||||||
|
|
||||||
val OnionMessages.ReceiveMessage(finalPayload) = probe.expectMessageType[OnionMessages.ReceiveMessage]
|
val OnionMessages.ReceiveMessage(finalPayload, _) = probe.expectMessageType[OnionMessages.ReceiveMessage]
|
||||||
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -46,7 +46,7 @@ class OnionMessagesSpec extends AnyFunSuite {
|
||||||
val Right(message) = buildMessage(sessionKey, blindingSecret, Nil, Recipient(destination.publicKey, None), TlvStream.empty)
|
val Right(message) = buildMessage(sessionKey, blindingSecret, Nil, Recipient(destination.publicKey, None), TlvStream.empty)
|
||||||
|
|
||||||
process(destination, message) match {
|
process(destination, message) match {
|
||||||
case ReceiveMessage(finalPayload) => assert(finalPayload.pathId_opt.isEmpty)
|
case ReceiveMessage(finalPayload, _) => assert(finalPayload.pathId_opt.isEmpty)
|
||||||
case x => fail(x.toString)
|
case x => fail(x.toString)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -106,7 +106,7 @@ class OnionMessagesSpec extends AnyFunSuite {
|
||||||
val onionForAlice = OnionMessage(blindingSecret.publicKey, packet)
|
val onionForAlice = OnionMessage(blindingSecret.publicKey, packet)
|
||||||
|
|
||||||
// Building the onion with functions from `OnionMessages`
|
// Building the onion with functions from `OnionMessages`
|
||||||
val replyPath = buildRoute(blindingOverride, IntermediateNode(carol.publicKey, EncodedNodeId(carol.publicKey), padding = Some(hex"0000000000000000000000000000000000000000000000000000000000000000000000")) :: Nil, Recipient(dave.publicKey, pathId = Some(hex"01234567")))
|
val replyPath = buildRoute(blindingOverride, IntermediateNode(carol.publicKey, EncodedNodeId(carol.publicKey), padding = Some(hex"0000000000000000000000000000000000000000000000000000000000000000000000")) :: Nil, Recipient(dave.publicKey, pathId = Some(hex"01234567"))).route
|
||||||
assert(replyPath == routeFromCarol)
|
assert(replyPath == routeFromCarol)
|
||||||
val Right(message) = buildMessage(sessionKey, blindingSecret, IntermediateNode(alice.publicKey) :: IntermediateNode(bob.publicKey) :: Nil, BlindedPath(replyPath), TlvStream.empty)
|
val Right(message) = buildMessage(sessionKey, blindingSecret, IntermediateNode(alice.publicKey) :: IntermediateNode(bob.publicKey) :: Nil, BlindedPath(replyPath), TlvStream.empty)
|
||||||
assert(message == onionForAlice)
|
assert(message == onionForAlice)
|
||||||
|
@ -122,7 +122,7 @@ class OnionMessagesSpec extends AnyFunSuite {
|
||||||
case SendMessage(Right(nextNodeId), onionForDave) =>
|
case SendMessage(Right(nextNodeId), onionForDave) =>
|
||||||
assert(nextNodeId == EncodedNodeId(dave.publicKey))
|
assert(nextNodeId == EncodedNodeId(dave.publicKey))
|
||||||
process(dave, onionForDave) match {
|
process(dave, onionForDave) match {
|
||||||
case ReceiveMessage(finalPayload) => assert(finalPayload.pathId_opt.contains(hex"01234567"))
|
case ReceiveMessage(finalPayload, _) => assert(finalPayload.pathId_opt.contains(hex"01234567"))
|
||||||
case x => fail(x.toString)
|
case x => fail(x.toString)
|
||||||
}
|
}
|
||||||
case x => fail(x.toString)
|
case x => fail(x.toString)
|
||||||
|
@ -206,7 +206,7 @@ class OnionMessagesSpec extends AnyFunSuite {
|
||||||
val blindingSecret = randomKey()
|
val blindingSecret = randomKey()
|
||||||
val blindingOverride = randomKey()
|
val blindingOverride = randomKey()
|
||||||
val destination = randomKey()
|
val destination = randomKey()
|
||||||
val replyPath = buildRoute(blindingOverride, IntermediateNode(destination.publicKey) :: Nil, Recipient(destination.publicKey, pathId = Some(hex"01234567")))
|
val replyPath = buildRoute(blindingOverride, IntermediateNode(destination.publicKey) :: Nil, Recipient(destination.publicKey, pathId = Some(hex"01234567"))).route
|
||||||
assert(replyPath.blindingKey == blindingOverride.publicKey)
|
assert(replyPath.blindingKey == blindingOverride.publicKey)
|
||||||
assert(replyPath.introductionNodeId == EncodedNodeId(destination.publicKey))
|
assert(replyPath.introductionNodeId == EncodedNodeId(destination.publicKey))
|
||||||
val Right(message) = buildMessage(sessionKey, blindingSecret, Nil, BlindedPath(replyPath), TlvStream.empty)
|
val Right(message) = buildMessage(sessionKey, blindingSecret, Nil, BlindedPath(replyPath), TlvStream.empty)
|
||||||
|
@ -216,7 +216,7 @@ class OnionMessagesSpec extends AnyFunSuite {
|
||||||
case SendMessage(Right(EncodedNodeId.Plain(nextNodeId2)), message2) =>
|
case SendMessage(Right(EncodedNodeId.Plain(nextNodeId2)), message2) =>
|
||||||
assert(nextNodeId2 == destination.publicKey)
|
assert(nextNodeId2 == destination.publicKey)
|
||||||
process(destination, message2) match {
|
process(destination, message2) match {
|
||||||
case ReceiveMessage(finalPayload) => assert(finalPayload.pathId_opt.contains(hex"01234567"))
|
case ReceiveMessage(finalPayload, _) => assert(finalPayload.pathId_opt.contains(hex"01234567"))
|
||||||
case x => fail(x.toString)
|
case x => fail(x.toString)
|
||||||
}
|
}
|
||||||
case x => fail(x.toString)
|
case x => fail(x.toString)
|
||||||
|
@ -240,7 +240,7 @@ class OnionMessagesSpec extends AnyFunSuite {
|
||||||
case SendMessage(Right(nextNodeId), onionForCarol) =>
|
case SendMessage(Right(nextNodeId), onionForCarol) =>
|
||||||
assert(nextNodeId == EncodedNodeId(carol.publicKey))
|
assert(nextNodeId == EncodedNodeId(carol.publicKey))
|
||||||
process(carol, onionForCarol) match {
|
process(carol, onionForCarol) match {
|
||||||
case ReceiveMessage(finalPayload) => assert(finalPayload.pathId_opt.contains(pathId))
|
case ReceiveMessage(finalPayload, _) => assert(finalPayload.pathId_opt.contains(pathId))
|
||||||
case x => fail(x.toString)
|
case x => fail(x.toString)
|
||||||
}
|
}
|
||||||
case x => fail(x.toString)
|
case x => fail(x.toString)
|
||||||
|
@ -292,7 +292,7 @@ class OnionMessagesSpec extends AnyFunSuite {
|
||||||
val pathBobToDave =
|
val pathBobToDave =
|
||||||
buildRoute(blindingSecretBob,
|
buildRoute(blindingSecretBob,
|
||||||
Seq(makeIntermediateNode(bob, (testVector \ "generate" \ "hops")(1) \ "tlvs"), makeIntermediateNode(carol, (testVector \ "generate" \ "hops")(2) \ "tlvs")),
|
Seq(makeIntermediateNode(bob, (testVector \ "generate" \ "hops")(1) \ "tlvs"), makeIntermediateNode(carol, (testVector \ "generate" \ "hops")(2) \ "tlvs")),
|
||||||
makeRecipient(dave, (testVector \ "generate" \ "hops")(3) \ "tlvs"))
|
makeRecipient(dave, (testVector \ "generate" \ "hops")(3) \ "tlvs")).route
|
||||||
val blindingSecretAlice = PrivateKey(ByteVector32.fromValidHex(((testVector \ "generate" \ "hops")(0) \ "blinding_secret").extract[String]))
|
val blindingSecretAlice = PrivateKey(ByteVector32.fromValidHex(((testVector \ "generate" \ "hops")(0) \ "blinding_secret").extract[String]))
|
||||||
val intermediateAlice = Seq(makeIntermediateNode(alice, (testVector \ "generate" \ "hops")(0) \ "tlvs"))
|
val intermediateAlice = Seq(makeIntermediateNode(alice, (testVector \ "generate" \ "hops")(0) \ "tlvs"))
|
||||||
val pathAliceToDave = buildRouteFrom(blindingSecretAlice, intermediateAlice, BlindedPath(pathBobToDave))
|
val pathAliceToDave = buildRouteFrom(blindingSecretAlice, intermediateAlice, BlindedPath(pathBobToDave))
|
||||||
|
@ -336,7 +336,7 @@ class OnionMessagesSpec extends AnyFunSuite {
|
||||||
case SendMessage(Right(nextNodeId), onionForDave) =>
|
case SendMessage(Right(nextNodeId), onionForDave) =>
|
||||||
assert(nextNodeId == EncodedNodeId(dave.publicKey))
|
assert(nextNodeId == EncodedNodeId(dave.publicKey))
|
||||||
process(dave, onionForDave) match {
|
process(dave, onionForDave) match {
|
||||||
case ReceiveMessage(finalPayload) => assert(finalPayload.pathId_opt.contains(pathId))
|
case ReceiveMessage(finalPayload, _) => assert(finalPayload.pathId_opt.contains(pathId))
|
||||||
case x => fail(x.toString)
|
case x => fail(x.toString)
|
||||||
}
|
}
|
||||||
case x => fail(x.toString)
|
case x => fail(x.toString)
|
||||||
|
@ -366,7 +366,7 @@ class OnionMessagesSpec extends AnyFunSuite {
|
||||||
case SendMessage(Left(outgoingChannelId), onionForCarol) =>
|
case SendMessage(Left(outgoingChannelId), onionForCarol) =>
|
||||||
assert(outgoingChannelId == bob2carol)
|
assert(outgoingChannelId == bob2carol)
|
||||||
process(carol, onionForCarol) match {
|
process(carol, onionForCarol) match {
|
||||||
case ReceiveMessage(finalPayload) => assert(finalPayload.pathId_opt.contains(pathId))
|
case ReceiveMessage(finalPayload, _) => assert(finalPayload.pathId_opt.contains(pathId))
|
||||||
case x => fail(x.toString)
|
case x => fail(x.toString)
|
||||||
}
|
}
|
||||||
case x => fail(x.toString)
|
case x => fail(x.toString)
|
||||||
|
|
|
@ -73,13 +73,13 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||||
}
|
}
|
||||||
|
|
||||||
@tailrec
|
@tailrec
|
||||||
private def receive(privateKeys: Seq[PrivateKey], message: OnionMessage): MessageOnion.FinalPayload = {
|
private def receive(privateKeys: Seq[PrivateKey], message: OnionMessage): ReceiveMessage = {
|
||||||
OnionMessages.process(privateKeys.head, message) match {
|
OnionMessages.process(privateKeys.head, message) match {
|
||||||
case OnionMessages.SendMessage(nextNode, nextMessage) if nextNode == Left(ShortChannelId.toSelf) || nextNode == Right(EncodedNodeId(privateKeys.head.publicKey)) =>
|
case OnionMessages.SendMessage(nextNode, nextMessage) if nextNode == Left(ShortChannelId.toSelf) || nextNode == Right(EncodedNodeId(privateKeys.head.publicKey)) =>
|
||||||
receive(privateKeys, nextMessage)
|
receive(privateKeys, nextMessage)
|
||||||
case OnionMessages.SendMessage(nextNode, nextMessage) if nextNode == Right(EncodedNodeId(privateKeys(1).publicKey)) =>
|
case OnionMessages.SendMessage(nextNode, nextMessage) if nextNode == Right(EncodedNodeId(privateKeys(1).publicKey)) =>
|
||||||
receive(privateKeys.tail, nextMessage)
|
receive(privateKeys.tail, nextMessage)
|
||||||
case ReceiveMessage(finalPayload) => finalPayload
|
case r: ReceiveMessage => r
|
||||||
case _ => fail()
|
case _ => fail()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -98,15 +98,15 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||||
|
|
||||||
val Peer.RelayOnionMessage(messageId, message, Some(replyTo)) = expectRelayToConnected(switchboard, recipientKey.publicKey)
|
val Peer.RelayOnionMessage(messageId, message, Some(replyTo)) = expectRelayToConnected(switchboard, recipientKey.publicKey)
|
||||||
replyTo ! Sent(messageId)
|
replyTo ! Sent(messageId)
|
||||||
val ReceiveMessage(finalPayload) = OnionMessages.process(recipientKey, message)
|
val ReceiveMessage(finalPayload, _) = OnionMessages.process(recipientKey, message)
|
||||||
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
||||||
|
|
||||||
val replyPath = finalPayload.records.get[ReplyPath].get.blindedRoute
|
val replyPath = finalPayload.records.get[ReplyPath].get.blindedRoute
|
||||||
val Right(reply) = buildMessage(randomKey(), randomKey(), Nil, BlindedPath(replyPath), TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(55), hex"1234"))))
|
val Right(reply) = buildMessage(randomKey(), randomKey(), Nil, BlindedPath(replyPath), TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(55), hex"1234"))))
|
||||||
val replyPayload = receive(Seq(recipientKey, nodeParams.privateKey), reply)
|
val ReceiveMessage(replyPayload, blindedKey) = receive(Seq(recipientKey, nodeParams.privateKey), reply)
|
||||||
|
|
||||||
testKit.system.eventStream ! EventStream.Publish(ReceiveMessage(replyPayload))
|
testKit.system.eventStream ! EventStream.Publish(ReceiveMessage(replyPayload, blindedKey))
|
||||||
testKit.system.eventStream ! EventStream.Publish(ReceiveMessage(replyPayload))
|
testKit.system.eventStream ! EventStream.Publish(ReceiveMessage(replyPayload, blindedKey))
|
||||||
|
|
||||||
messageSender.expectMessage(Response(replyPayload))
|
messageSender.expectMessage(Response(replyPayload))
|
||||||
messageSender.expectNoMessage(10 millis)
|
messageSender.expectNoMessage(10 millis)
|
||||||
|
@ -145,15 +145,15 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||||
|
|
||||||
val Peer.RelayOnionMessage(messageId, message, Some(replyTo)) = expectRelayToConnected(switchboard, recipientKey.publicKey)
|
val Peer.RelayOnionMessage(messageId, message, Some(replyTo)) = expectRelayToConnected(switchboard, recipientKey.publicKey)
|
||||||
replyTo ! Sent(messageId)
|
replyTo ! Sent(messageId)
|
||||||
val ReceiveMessage(finalPayload) = OnionMessages.process(recipientKey, message)
|
val ReceiveMessage(finalPayload, _) = OnionMessages.process(recipientKey, message)
|
||||||
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
||||||
|
|
||||||
messageSender.expectMessage(NoReply)
|
messageSender.expectMessage(NoReply)
|
||||||
|
|
||||||
val replyPath = finalPayload.records.get[ReplyPath].get.blindedRoute
|
val replyPath = finalPayload.records.get[ReplyPath].get.blindedRoute
|
||||||
val Right(reply) = buildMessage(randomKey(), randomKey(), Nil, BlindedPath(replyPath), TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(55), hex"1234"))))
|
val Right(reply) = buildMessage(randomKey(), randomKey(), Nil, BlindedPath(replyPath), TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(55), hex"1234"))))
|
||||||
val replyPayload = receive(Seq(recipientKey, nodeParams.privateKey), reply)
|
val receiveReply = receive(Seq(recipientKey, nodeParams.privateKey), reply)
|
||||||
testKit.system.eventStream ! EventStream.Publish(ReceiveMessage(replyPayload))
|
testKit.system.eventStream ! EventStream.Publish(receiveReply)
|
||||||
|
|
||||||
messageSender.expectNoMessage(10 millis)
|
messageSender.expectNoMessage(10 millis)
|
||||||
}
|
}
|
||||||
|
@ -172,7 +172,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||||
|
|
||||||
val Peer.RelayOnionMessage(messageId, message, Some(replyTo)) = expectRelayToConnected(switchboard, recipientKey.publicKey)
|
val Peer.RelayOnionMessage(messageId, message, Some(replyTo)) = expectRelayToConnected(switchboard, recipientKey.publicKey)
|
||||||
replyTo ! Sent(messageId)
|
replyTo ! Sent(messageId)
|
||||||
val ReceiveMessage(finalPayload) = OnionMessages.process(recipientKey, message)
|
val ReceiveMessage(finalPayload, _) = OnionMessages.process(recipientKey, message)
|
||||||
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
||||||
assert(finalPayload.records.get[ReplyPath].isEmpty)
|
assert(finalPayload.records.get[ReplyPath].isEmpty)
|
||||||
|
|
||||||
|
@ -185,12 +185,12 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||||
|
|
||||||
val recipientKey = randomKey()
|
val recipientKey = randomKey()
|
||||||
|
|
||||||
val blindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(nodeParams.nodeId)), Recipient(recipientKey.publicKey, None))
|
val blindedRoute = buildRoute(randomKey(), Seq(IntermediateNode(nodeParams.nodeId)), Recipient(recipientKey.publicKey, None)).route
|
||||||
postman ! SendMessage(OfferTypes.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)
|
val Peer.RelayOnionMessage(messageId, message, Some(replyTo)) = expectRelayToConnected(switchboard, recipientKey.publicKey)
|
||||||
replyTo ! Sent(messageId)
|
replyTo ! Sent(messageId)
|
||||||
val ReceiveMessage(finalPayload) = OnionMessages.process(recipientKey, message)
|
val ReceiveMessage(finalPayload, _) = OnionMessages.process(recipientKey, message)
|
||||||
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
||||||
assert(finalPayload.records.get[ReplyPath].isEmpty)
|
assert(finalPayload.records.get[ReplyPath].isEmpty)
|
||||||
|
|
||||||
|
@ -201,11 +201,11 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||||
test("forward invoice request to offer manager") { f =>
|
test("forward invoice request to offer manager") { f =>
|
||||||
import f._
|
import f._
|
||||||
|
|
||||||
val offer = OfferTypes.Offer(None, "", randomKey().publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
val offer = OfferTypes.Offer(None, None, randomKey().publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
||||||
val invoiceRequest = OfferTypes.InvoiceRequest(offer, 1000 msat, 1, Features.empty, randomKey(), Block.LivenetGenesisBlock.hash)
|
val invoiceRequest = OfferTypes.InvoiceRequest(offer, 1000 msat, 1, Features.empty, randomKey(), Block.LivenetGenesisBlock.hash)
|
||||||
val replyPath = BlindedRoute(EncodedNodeId(randomKey().publicKey), randomKey().publicKey, Seq(BlindedNode(randomKey().publicKey, hex"")))
|
val replyPath = BlindedRoute(EncodedNodeId(randomKey().publicKey), randomKey().publicKey, Seq(BlindedNode(randomKey().publicKey, hex"")))
|
||||||
val invoiceRequestPayload = MessageOnion.InvoiceRequestPayload(TlvStream(InvoiceRequest(invoiceRequest.records), ReplyPath(replyPath)), TlvStream(PathId(hex"abcd")))
|
val invoiceRequestPayload = MessageOnion.InvoiceRequestPayload(TlvStream(InvoiceRequest(invoiceRequest.records), ReplyPath(replyPath)), TlvStream(PathId(hex"abcd")))
|
||||||
postman ! WrappedMessage(invoiceRequestPayload)
|
postman ! WrappedMessage(invoiceRequestPayload, randomKey())
|
||||||
|
|
||||||
val request = offerManager.expectMessageType[RequestInvoice]
|
val request = offerManager.expectMessageType[RequestInvoice]
|
||||||
assert(request.messagePayload.pathId_opt.contains(hex"abcd"))
|
assert(request.messagePayload.pathId_opt.contains(hex"abcd"))
|
||||||
|
@ -231,7 +231,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||||
assert(next3 == EncodedNodeId(c.publicKey))
|
assert(next3 == EncodedNodeId(c.publicKey))
|
||||||
val OnionMessages.SendMessage(Right(next4), message4) = OnionMessages.process(c, message3)
|
val OnionMessages.SendMessage(Right(next4), message4) = OnionMessages.process(c, message3)
|
||||||
assert(next4 == EncodedNodeId(d.publicKey))
|
assert(next4 == EncodedNodeId(d.publicKey))
|
||||||
val OnionMessages.ReceiveMessage(payload) = OnionMessages.process(d, message4)
|
val OnionMessages.ReceiveMessage(payload, _) = OnionMessages.process(d, message4)
|
||||||
assert(payload.records.unknown == Set(GenericTlv(UInt64(11), hex"012345")))
|
assert(payload.records.unknown == Set(GenericTlv(UInt64(11), hex"012345")))
|
||||||
assert(payload.records.get[ReplyPath].nonEmpty)
|
assert(payload.records.get[ReplyPath].nonEmpty)
|
||||||
val replyPath = payload.records.get[ReplyPath].get.blindedRoute
|
val replyPath = payload.records.get[ReplyPath].get.blindedRoute
|
||||||
|
@ -240,10 +240,11 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||||
assert(nodeParams.onionMessageConfig.minIntermediateHops > 5)
|
assert(nodeParams.onionMessageConfig.minIntermediateHops > 5)
|
||||||
|
|
||||||
val Right(reply) = OnionMessages.buildMessage(randomKey(), randomKey(), Nil, OnionMessages.BlindedPath(replyPath), TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(13), hex"6789"))))
|
val Right(reply) = OnionMessages.buildMessage(randomKey(), randomKey(), Nil, OnionMessages.BlindedPath(replyPath), TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(13), hex"6789"))))
|
||||||
val replyPayload = receive(Seq(d, c, b, a, nodeParams.privateKey), reply)
|
val receiveReply = receive(Seq(d, c, b, a, nodeParams.privateKey), reply)
|
||||||
|
assert(receiveReply.finalPayload.records.unknown == Set(GenericTlv(UInt64(13), hex"6789")))
|
||||||
|
|
||||||
postman ! WrappedMessage(replyPayload)
|
postman ! WrappedMessage(receiveReply.finalPayload, receiveReply.blindedKey)
|
||||||
assert(replyPayload.records.unknown == Set(GenericTlv(UInt64(13), hex"6789")))
|
messageSender.expectMessage(Response(receiveReply.finalPayload))
|
||||||
}
|
}
|
||||||
|
|
||||||
test("send to compact route") { f =>
|
test("send to compact route") { f =>
|
||||||
|
@ -251,7 +252,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||||
|
|
||||||
val recipientKey = randomKey()
|
val recipientKey = randomKey()
|
||||||
|
|
||||||
val route = buildRoute(randomKey(), Seq(), Recipient(recipientKey.publicKey, None))
|
val route = buildRoute(randomKey(), Seq(), Recipient(recipientKey.publicKey, None)).route
|
||||||
val compactRoute = OfferTypes.BlindedPath(route.copy(introductionNodeId = EncodedNodeId.ShortChannelIdDir(isNode1 = false, RealShortChannelId(1234))))
|
val compactRoute = OfferTypes.BlindedPath(route.copy(introductionNodeId = EncodedNodeId.ShortChannelIdDir(isNode1 = false, RealShortChannelId(1234))))
|
||||||
postman ! SendMessage(compactRoute, FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = false, messageSender.ref)
|
postman ! SendMessage(compactRoute, FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = false, messageSender.ref)
|
||||||
|
|
||||||
|
@ -272,7 +273,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||||
replyConnectedTo ! ConnectionResult.Connected(peerConnection.ref.toClassic, peer.ref.toClassic)
|
replyConnectedTo ! ConnectionResult.Connected(peerConnection.ref.toClassic, peer.ref.toClassic)
|
||||||
val Peer.RelayOnionMessage(messageId, message, Some(replySentTo)) = peer.expectMessageType[Peer.RelayOnionMessage]
|
val Peer.RelayOnionMessage(messageId, message, Some(replySentTo)) = peer.expectMessageType[Peer.RelayOnionMessage]
|
||||||
replySentTo ! Sent(messageId)
|
replySentTo ! Sent(messageId)
|
||||||
val ReceiveMessage(finalPayload) = OnionMessages.process(recipientKey, message)
|
val ReceiveMessage(finalPayload, _) = OnionMessages.process(recipientKey, message)
|
||||||
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
||||||
assert(finalPayload.records.get[ReplyPath].isEmpty)
|
assert(finalPayload.records.get[ReplyPath].isEmpty)
|
||||||
|
|
||||||
|
@ -285,7 +286,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||||
|
|
||||||
val recipientKey = randomKey()
|
val recipientKey = randomKey()
|
||||||
|
|
||||||
val route = buildRoute(randomKey(), Seq(IntermediateNode(nodeParams.nodeId)), Recipient(recipientKey.publicKey, None))
|
val route = buildRoute(randomKey(), Seq(IntermediateNode(nodeParams.nodeId)), Recipient(recipientKey.publicKey, None)).route
|
||||||
val compactRoute = OfferTypes.BlindedPath(route.copy(introductionNodeId = EncodedNodeId.ShortChannelIdDir(isNode1 = true, RealShortChannelId(1234))))
|
val compactRoute = OfferTypes.BlindedPath(route.copy(introductionNodeId = EncodedNodeId.ShortChannelIdDir(isNode1 = true, RealShortChannelId(1234))))
|
||||||
postman ! SendMessage(compactRoute, FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = false, messageSender.ref)
|
postman ! SendMessage(compactRoute, FindRoute, TlvStream(Set.empty[OnionMessagePayloadTlv], Set(GenericTlv(UInt64(33), hex"abcd"))), expectsReply = false, messageSender.ref)
|
||||||
|
|
||||||
|
@ -301,7 +302,7 @@ class PostmanSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat
|
||||||
replyConnectedTo ! ConnectionResult.Connected(peerConnection.ref.toClassic, peer.ref.toClassic)
|
replyConnectedTo ! ConnectionResult.Connected(peerConnection.ref.toClassic, peer.ref.toClassic)
|
||||||
val Peer.RelayOnionMessage(messageId, message, Some(replySentTo)) = peer.expectMessageType[Peer.RelayOnionMessage]
|
val Peer.RelayOnionMessage(messageId, message, Some(replySentTo)) = peer.expectMessageType[Peer.RelayOnionMessage]
|
||||||
replySentTo ! Sent(messageId)
|
replySentTo ! Sent(messageId)
|
||||||
val ReceiveMessage(finalPayload) = OnionMessages.process(recipientKey, message)
|
val ReceiveMessage(finalPayload, _) = OnionMessages.process(recipientKey, message)
|
||||||
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
assert(finalPayload.records.unknown == Set(GenericTlv(UInt64(33), hex"abcd")))
|
||||||
assert(finalPayload.records.get[ReplyPath].isEmpty)
|
assert(finalPayload.records.get[ReplyPath].isEmpty)
|
||||||
|
|
||||||
|
|
|
@ -22,6 +22,9 @@ import fr.acinq.bitcoin.scalacompat.{Block, BlockHash, ByteVector32}
|
||||||
import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional}
|
import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional}
|
||||||
import fr.acinq.eclair.Features.{BasicMultiPartPayment, VariableLengthOnion}
|
import fr.acinq.eclair.Features.{BasicMultiPartPayment, VariableLengthOnion}
|
||||||
import fr.acinq.eclair.crypto.Sphinx
|
import fr.acinq.eclair.crypto.Sphinx
|
||||||
|
import fr.acinq.eclair.crypto.Sphinx.RouteBlinding
|
||||||
|
import fr.acinq.eclair.message.OnionMessages
|
||||||
|
import fr.acinq.eclair.message.OnionMessages.{IntermediateNode, Recipient}
|
||||||
import fr.acinq.eclair.payment.Bolt12Invoice.hrp
|
import fr.acinq.eclair.payment.Bolt12Invoice.hrp
|
||||||
import fr.acinq.eclair.wire.protocol.OfferCodecs.{invoiceRequestTlvCodec, invoiceTlvCodec}
|
import fr.acinq.eclair.wire.protocol.OfferCodecs.{invoiceRequestTlvCodec, invoiceTlvCodec}
|
||||||
import fr.acinq.eclair.wire.protocol.OfferTypes._
|
import fr.acinq.eclair.wire.protocol.OfferTypes._
|
||||||
|
@ -56,7 +59,7 @@ class Bolt12InvoiceSpec extends AnyFunSuite {
|
||||||
|
|
||||||
test("check invoice signature") {
|
test("check invoice signature") {
|
||||||
val (nodeKey, payerKey, chain) = (randomKey(), randomKey(), BlockHash(randomBytes32()))
|
val (nodeKey, payerKey, chain) = (randomKey(), randomKey(), BlockHash(randomBytes32()))
|
||||||
val offer = Offer(Some(10000 msat), "test offer", nodeKey.publicKey, Features.empty, chain)
|
val offer = Offer(Some(10000 msat), Some("test offer"), nodeKey.publicKey, Features.empty, chain)
|
||||||
val request = InvoiceRequest(offer, 11000 msat, 1, Features.empty, payerKey, chain)
|
val request = InvoiceRequest(offer, 11000 msat, 1, Features.empty, payerKey, chain)
|
||||||
val invoice = Bolt12Invoice(request, randomBytes32(), nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
val invoice = Bolt12Invoice(request, randomBytes32(), nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
||||||
assert(invoice.checkSignature())
|
assert(invoice.checkSignature())
|
||||||
|
@ -73,55 +76,69 @@ class Bolt12InvoiceSpec extends AnyFunSuite {
|
||||||
|
|
||||||
test("check invoice signature with unknown field from invoice request") {
|
test("check invoice signature with unknown field from invoice request") {
|
||||||
val (nodeKey, payerKey, chain) = (randomKey(), randomKey(), BlockHash(randomBytes32()))
|
val (nodeKey, payerKey, chain) = (randomKey(), randomKey(), BlockHash(randomBytes32()))
|
||||||
val offer = Offer(Some(10000 msat), "test offer", nodeKey.publicKey, Features.empty, chain)
|
val offer = Offer(Some(10000 msat), Some("test offer"), nodeKey.publicKey, Features.empty, chain)
|
||||||
val basicRequest = InvoiceRequest(offer, 11000 msat, 1, Features.empty, payerKey, chain)
|
val basicRequest = InvoiceRequest(offer, 11000 msat, 1, Features.empty, payerKey, chain)
|
||||||
val requestWithUnknownTlv = basicRequest.copy(records = TlvStream(basicRequest.records.records, Set(GenericTlv(UInt64(87), hex"0404"))))
|
val requestWithUnknownTlv = basicRequest.copy(records = TlvStream(basicRequest.records.records, Set(GenericTlv(UInt64(87), hex"0404"))))
|
||||||
val invoice = Bolt12Invoice(requestWithUnknownTlv, randomBytes32(), nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
val invoice = Bolt12Invoice(requestWithUnknownTlv, randomBytes32(), nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
||||||
assert(invoice.records.unknown == Set(GenericTlv(UInt64(87), hex"0404")))
|
assert(invoice.records.unknown == Set(GenericTlv(UInt64(87), hex"0404")))
|
||||||
assert(invoice.validateFor(requestWithUnknownTlv).isRight)
|
assert(invoice.validateFor(requestWithUnknownTlv, nodeKey.publicKey).isRight)
|
||||||
assert(Bolt12Invoice.fromString(invoice.toString).get.toString == invoice.toString)
|
assert(Bolt12Invoice.fromString(invoice.toString).get.toString == invoice.toString)
|
||||||
}
|
}
|
||||||
|
|
||||||
test("check that invoice matches offer") {
|
test("check that invoice matches offer") {
|
||||||
val (nodeKey, payerKey, chain) = (randomKey(), randomKey(), BlockHash(randomBytes32()))
|
val (nodeKey, payerKey, chain) = (randomKey(), randomKey(), BlockHash(randomBytes32()))
|
||||||
val offer = Offer(Some(10000 msat), "test offer", nodeKey.publicKey, Features.empty, chain)
|
val offer = Offer(Some(10000 msat), Some("test offer"), nodeKey.publicKey, Features.empty, chain)
|
||||||
val request = InvoiceRequest(offer, 11000 msat, 1, Features.empty, payerKey, chain)
|
val request = InvoiceRequest(offer, 11000 msat, 1, Features.empty, payerKey, chain)
|
||||||
val invoice = Bolt12Invoice(request, randomBytes32(), nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
val invoice = Bolt12Invoice(request, randomBytes32(), nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
||||||
assert(invoice.validateFor(request).isRight)
|
assert(invoice.validateFor(request, nodeKey.publicKey).isRight)
|
||||||
// amount must match the request
|
// amount must match the request
|
||||||
val withOtherAmount = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case OfferAmount(_) => OfferAmount(9000 msat) case x => x })), nodeKey)
|
val withOtherAmount = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case OfferAmount(_) => OfferAmount(9000 msat) case x => x })), nodeKey)
|
||||||
assert(withOtherAmount.validateFor(request).isLeft)
|
assert(withOtherAmount.validateFor(request, nodeKey.publicKey).isLeft)
|
||||||
// description must match the offer
|
// description must match the offer
|
||||||
val withOtherDescription = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case OfferDescription(_) => OfferDescription("other description") case x => x })), nodeKey)
|
val withOtherDescription = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case OfferDescription(_) => OfferDescription("other description") case x => x })), nodeKey)
|
||||||
assert(withOtherDescription.validateFor(request).isLeft)
|
assert(withOtherDescription.validateFor(request, nodeKey.publicKey).isLeft)
|
||||||
// nodeId must match the offer
|
// nodeId must match the offer
|
||||||
val otherNodeKey = randomKey()
|
val otherNodeKey = randomKey()
|
||||||
val withOtherNodeId = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case OfferNodeId(_) => OfferNodeId(otherNodeKey.publicKey) case x => x })), nodeKey)
|
val withOtherNodeId = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case OfferNodeId(_) => OfferNodeId(otherNodeKey.publicKey) case x => x })), nodeKey)
|
||||||
assert(withOtherNodeId.validateFor(request).isLeft)
|
assert(withOtherNodeId.validateFor(request, nodeKey.publicKey).isLeft)
|
||||||
// issuer must match the offer
|
// issuer must match the offer
|
||||||
val withOtherIssuer = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records ++ Seq(OfferIssuer("spongebob")))), nodeKey)
|
val withOtherIssuer = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records ++ Seq(OfferIssuer("spongebob")))), nodeKey)
|
||||||
assert(withOtherIssuer.validateFor(request).isLeft)
|
assert(withOtherIssuer.validateFor(request, nodeKey.publicKey).isLeft)
|
||||||
|
}
|
||||||
|
|
||||||
|
test("check that invoice matches offer with implicit node id") {
|
||||||
|
val (nodeKey, payerKey, chain) = (randomKey(), randomKey(), BlockHash(randomBytes32()))
|
||||||
|
val path1 = OnionMessages.buildRoute(randomKey(), Seq(IntermediateNode(randomKey().publicKey)), Recipient(nodeKey.publicKey, None))
|
||||||
|
val path2 = OnionMessages.buildRoute(randomKey(), Seq(IntermediateNode(randomKey().publicKey)), Recipient(nodeKey.publicKey, None))
|
||||||
|
val offer = Offer.withPaths(None, None, Seq(path1.route, path2.route), Features.empty, chain)
|
||||||
|
val request = InvoiceRequest(offer, 11000 msat, 1, Features.empty, payerKey, chain)
|
||||||
|
// Invoice is requested using path1.
|
||||||
|
assert(RouteBlinding.derivePrivateKey(nodeKey, path1.lastBlinding).publicKey == path1.route.blindedNodeIds.last)
|
||||||
|
val invoice = Bolt12Invoice(request, randomBytes32(), RouteBlinding.derivePrivateKey(nodeKey, path1.lastBlinding), 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
||||||
|
assert(invoice.validateFor(request, nodeKey.publicKey).isLeft)
|
||||||
|
assert(invoice.validateFor(request, path1.route.blindedNodeIds.last).isRight)
|
||||||
|
assert(invoice.validateFor(request, path2.route.blindedNodeIds.last).isLeft)
|
||||||
}
|
}
|
||||||
|
|
||||||
test("check that invoice matches invoice request") {
|
test("check that invoice matches invoice request") {
|
||||||
val (nodeKey, payerKey, chain) = (randomKey(), randomKey(), BlockHash(randomBytes32()))
|
val (nodeKey, payerKey, chain) = (randomKey(), randomKey(), BlockHash(randomBytes32()))
|
||||||
val offer = Offer(Some(15000 msat), "test offer", nodeKey.publicKey, Features.empty, chain)
|
val offer = Offer(Some(15000 msat), Some("test offer"), nodeKey.publicKey, Features.empty, chain)
|
||||||
val request = InvoiceRequest(offer, 15000 msat, 1, Features.empty, payerKey, chain)
|
val request = InvoiceRequest(offer, 15000 msat, 1, Features.empty, payerKey, chain)
|
||||||
assert(request.quantity_opt.isEmpty) // when paying for a single item, the quantity field must not be present
|
assert(request.quantity_opt.isEmpty) // when paying for a single item, the quantity field must not be present
|
||||||
val invoice = Bolt12Invoice(request, randomBytes32(), nodeKey, 300 seconds, Features(BasicMultiPartPayment -> Optional), Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
val invoice = Bolt12Invoice(request, randomBytes32(), nodeKey, 300 seconds, Features(BasicMultiPartPayment -> Optional), Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
||||||
assert(invoice.validateFor(request).isRight)
|
assert(invoice.validateFor(request, nodeKey.publicKey).isRight)
|
||||||
val withInvalidFeatures = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case InvoiceFeatures(_) => InvoiceFeatures(Features(BasicMultiPartPayment -> Mandatory)) case x => x })), nodeKey)
|
val withInvalidFeatures = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case InvoiceFeatures(_) => InvoiceFeatures(Features(BasicMultiPartPayment -> Mandatory)) case x => x })), nodeKey)
|
||||||
assert(withInvalidFeatures.validateFor(request).isLeft)
|
assert(withInvalidFeatures.validateFor(request, nodeKey.publicKey).isLeft)
|
||||||
val withAmountTooBig = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case InvoiceRequestAmount(_) => InvoiceRequestAmount(20000 msat) case x => x })), nodeKey)
|
val withAmountTooBig = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case InvoiceRequestAmount(_) => InvoiceRequestAmount(20000 msat) case x => x })), nodeKey)
|
||||||
assert(withAmountTooBig.validateFor(request).isLeft)
|
assert(withAmountTooBig.validateFor(request, nodeKey.publicKey).isLeft)
|
||||||
val withQuantity = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records + InvoiceRequestQuantity(2))), nodeKey)
|
val withQuantity = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records + InvoiceRequestQuantity(2))), nodeKey)
|
||||||
assert(withQuantity.validateFor(request).isLeft)
|
assert(withQuantity.validateFor(request, nodeKey.publicKey).isLeft)
|
||||||
val withOtherPayerKey = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case InvoiceRequestPayerId(_) => InvoiceRequestPayerId(randomKey().publicKey) case x => x })), nodeKey)
|
val withOtherPayerKey = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case InvoiceRequestPayerId(_) => InvoiceRequestPayerId(randomKey().publicKey) case x => x })), nodeKey)
|
||||||
assert(withOtherPayerKey.validateFor(request).isLeft)
|
assert(withOtherPayerKey.validateFor(request, nodeKey.publicKey).isLeft)
|
||||||
val withPayerNote = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records + InvoiceRequestPayerNote("I am Batman"))), nodeKey)
|
val withPayerNote = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records + InvoiceRequestPayerNote("I am Batman"))), nodeKey)
|
||||||
assert(withPayerNote.validateFor(request).isLeft)
|
assert(withPayerNote.validateFor(request, nodeKey.publicKey).isLeft)
|
||||||
val withOtherMetadata = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case InvoiceRequestMetadata(_) => InvoiceRequestMetadata(hex"ae46c46b86") case x => x })), nodeKey)
|
val withOtherMetadata = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case InvoiceRequestMetadata(_) => InvoiceRequestMetadata(hex"ae46c46b86") case x => x })), nodeKey)
|
||||||
assert(withOtherMetadata.validateFor(request).isLeft)
|
assert(withOtherMetadata.validateFor(request, nodeKey.publicKey).isLeft)
|
||||||
// Invoice request with more details about the payer.
|
// Invoice request with more details about the payer.
|
||||||
val requestWithPayerDetails = {
|
val requestWithPayerDetails = {
|
||||||
val tlvs: Set[InvoiceRequestTlv] = Set(
|
val tlvs: Set[InvoiceRequestTlv] = Set(
|
||||||
|
@ -137,37 +154,36 @@ class Bolt12InvoiceSpec extends AnyFunSuite {
|
||||||
InvoiceRequest(TlvStream(tlvs + Signature(signature)))
|
InvoiceRequest(TlvStream(tlvs + Signature(signature)))
|
||||||
}
|
}
|
||||||
val withPayerDetails = Bolt12Invoice(requestWithPayerDetails, randomBytes32(), nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
val withPayerDetails = Bolt12Invoice(requestWithPayerDetails, randomBytes32(), nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
||||||
assert(withPayerDetails.validateFor(requestWithPayerDetails).isRight)
|
assert(withPayerDetails.validateFor(requestWithPayerDetails, nodeKey.publicKey).isRight)
|
||||||
assert(withPayerDetails.validateFor(request).isLeft)
|
assert(withPayerDetails.validateFor(request, nodeKey.publicKey).isLeft)
|
||||||
val withOtherPayerNote = signInvoice(Bolt12Invoice(TlvStream(withPayerDetails.records.records.map { case InvoiceRequestPayerNote(_) => InvoiceRequestPayerNote("Or am I Bruce Wayne?") case x => x })), nodeKey)
|
val withOtherPayerNote = signInvoice(Bolt12Invoice(TlvStream(withPayerDetails.records.records.map { case InvoiceRequestPayerNote(_) => InvoiceRequestPayerNote("Or am I Bruce Wayne?") case x => x })), nodeKey)
|
||||||
assert(withOtherPayerNote.validateFor(requestWithPayerDetails).isLeft)
|
assert(withOtherPayerNote.validateFor(requestWithPayerDetails, nodeKey.publicKey).isLeft)
|
||||||
assert(withOtherPayerNote.validateFor(request).isLeft)
|
assert(withOtherPayerNote.validateFor(request, nodeKey.publicKey).isLeft)
|
||||||
}
|
}
|
||||||
|
|
||||||
test("check invoice expiry") {
|
test("check invoice expiry") {
|
||||||
val (nodeKey, payerKey, chain) = (randomKey(), randomKey(), BlockHash(randomBytes32()))
|
val (nodeKey, payerKey, chain) = (randomKey(), randomKey(), BlockHash(randomBytes32()))
|
||||||
val offer = Offer(Some(5000 msat), "test offer", nodeKey.publicKey, Features.empty, chain)
|
val offer = Offer(Some(5000 msat), Some("test offer"), nodeKey.publicKey, Features.empty, chain)
|
||||||
val request = InvoiceRequest(offer, 5000 msat, 1, Features.empty, payerKey, chain)
|
val request = InvoiceRequest(offer, 5000 msat, 1, Features.empty, payerKey, chain)
|
||||||
val invoice = Bolt12Invoice(request, randomBytes32(), nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
val invoice = Bolt12Invoice(request, randomBytes32(), nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
||||||
assert(!invoice.isExpired())
|
assert(!invoice.isExpired())
|
||||||
assert(invoice.validateFor(request).isRight)
|
assert(invoice.validateFor(request, nodeKey.publicKey).isRight)
|
||||||
val expiredInvoice1 = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case InvoiceCreatedAt(_) => InvoiceCreatedAt(0 unixsec) case x => x })), nodeKey)
|
val expiredInvoice1 = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map { case InvoiceCreatedAt(_) => InvoiceCreatedAt(0 unixsec) case x => x })), nodeKey)
|
||||||
assert(expiredInvoice1.isExpired())
|
assert(expiredInvoice1.isExpired())
|
||||||
assert(expiredInvoice1.validateFor(request).isLeft) // when an invoice is expired, we mark it as invalid as well
|
assert(expiredInvoice1.validateFor(request, nodeKey.publicKey).isLeft) // when an invoice is expired, we mark it as invalid as well
|
||||||
val expiredInvoice2 = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map {
|
val expiredInvoice2 = signInvoice(Bolt12Invoice(TlvStream(invoice.records.records.map {
|
||||||
case InvoiceCreatedAt(_) => InvoiceCreatedAt(TimestampSecond.now() - 2000)
|
case InvoiceCreatedAt(_) => InvoiceCreatedAt(TimestampSecond.now() - 2000)
|
||||||
case InvoiceRelativeExpiry(_) => InvoiceRelativeExpiry(1800)
|
case InvoiceRelativeExpiry(_) => InvoiceRelativeExpiry(1800)
|
||||||
case x => x
|
case x => x
|
||||||
})), nodeKey)
|
})), nodeKey)
|
||||||
assert(expiredInvoice2.isExpired())
|
assert(expiredInvoice2.isExpired())
|
||||||
assert(expiredInvoice2.validateFor(request).isLeft) // when an invoice is expired, we mark it as invalid as well
|
assert(expiredInvoice2.validateFor(request, nodeKey.publicKey).isLeft) // when an invoice is expired, we mark it as invalid as well
|
||||||
}
|
}
|
||||||
|
|
||||||
test("decode invalid invoice") {
|
test("decode invalid invoice") {
|
||||||
val nodeKey = randomKey()
|
val nodeKey = randomKey()
|
||||||
val tlvs = Set[InvoiceTlv](
|
val tlvs = Set[InvoiceTlv](
|
||||||
InvoiceRequestMetadata(hex"012345"),
|
InvoiceRequestMetadata(hex"012345"),
|
||||||
OfferDescription("minimal invoice"),
|
|
||||||
OfferNodeId(nodeKey.publicKey),
|
OfferNodeId(nodeKey.publicKey),
|
||||||
InvoiceRequestPayerId(randomKey().publicKey),
|
InvoiceRequestPayerId(randomKey().publicKey),
|
||||||
InvoicePaths(Seq(createPaymentBlindedRoute(randomKey().publicKey).route)),
|
InvoicePaths(Seq(createPaymentBlindedRoute(randomKey().publicKey).route)),
|
||||||
|
@ -238,7 +254,7 @@ class Bolt12InvoiceSpec extends AnyFunSuite {
|
||||||
val Success(codedDecoded) = Bolt12Invoice.fromString(invoice.toString)
|
val Success(codedDecoded) = Bolt12Invoice.fromString(invoice.toString)
|
||||||
assert(codedDecoded.invoiceRequest.chain == chain)
|
assert(codedDecoded.invoiceRequest.chain == chain)
|
||||||
assert(codedDecoded.amount == amount)
|
assert(codedDecoded.amount == amount)
|
||||||
assert(codedDecoded.description == Left(description))
|
assert(codedDecoded.description.contains(description))
|
||||||
assert(codedDecoded.features == features)
|
assert(codedDecoded.features == features)
|
||||||
assert(codedDecoded.invoiceRequest.offer.issuer.contains(issuer))
|
assert(codedDecoded.invoiceRequest.offer.issuer.contains(issuer))
|
||||||
assert(codedDecoded.nodeId.value.drop(1) == nodeKey.publicKey.value.drop(1))
|
assert(codedDecoded.nodeId.value.drop(1) == nodeKey.publicKey.value.drop(1))
|
||||||
|
@ -260,22 +276,22 @@ class Bolt12InvoiceSpec extends AnyFunSuite {
|
||||||
val payerKey = PrivateKey(hex"d817e8896c67d0bcabfdb93da7eb7fc698c829a181f994dd0ad866a8eda745e8")
|
val payerKey = PrivateKey(hex"d817e8896c67d0bcabfdb93da7eb7fc698c829a181f994dd0ad866a8eda745e8")
|
||||||
assert(payerKey.publicKey == PublicKey(hex"031ef4439f638914de79220483dda32dfb7a431e799a5ce5a7643fbd70b2118e4e"))
|
assert(payerKey.publicKey == PublicKey(hex"031ef4439f638914de79220483dda32dfb7a431e799a5ce5a7643fbd70b2118e4e"))
|
||||||
val preimage = ByteVector32(hex"317d1fd8fec5f3ea23044983c2ba2a8043395b2a0790a815c9b12719aa5f1516")
|
val preimage = ByteVector32(hex"317d1fd8fec5f3ea23044983c2ba2a8043395b2a0790a815c9b12719aa5f1516")
|
||||||
val offer = Offer(None, "minimal tip", nodeKey.publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
val offer = Offer(None, None, nodeKey.publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
||||||
val encodedOffer = "lno1pg9k66twd9kkzmpqw35hq93pqf8l2vtlq5w87m4vqfnvtn82adk9wadfgratnp2wg7l7ha4u0gzqw"
|
val encodedOffer = "lno1zcssynl4x9ls28rld6kqyek9en4wkmzhwk55p74es48y00lt76785pq8"
|
||||||
assert(offer.toString == encodedOffer)
|
assert(offer.toString == encodedOffer)
|
||||||
assert(Offer.decode(encodedOffer).get == offer)
|
assert(Offer.decode(encodedOffer).get == offer)
|
||||||
val request = InvoiceRequest(offer, 12000000 msat, 1, Features.empty, payerKey, Block.LivenetGenesisBlock.hash)
|
val request = InvoiceRequest(offer, 12000000 msat, 1, Features.empty, payerKey, Block.LivenetGenesisBlock.hash)
|
||||||
// Invoice request generation is not reproducible because we add randomness in the first TLV.
|
// Invoice request generation is not reproducible because we add randomness in the first TLV.
|
||||||
val encodedRequest = "lnr1qqs289chx8swkpmwf3uzexfxr0kk9syavsjcmkuur5qgjqt60ayjdec2pdkkjmnfd4skcgr5d9cpvggzfl6nzlc9r3lkatqzvmzue6htd3tht22ql2uc2nj8hl4ld0r6qsr4qgr0u2xq4dh3kdevrf4zg6hx8a60jv0gxe0ptgyfc6xkryqqqqqqqpfq8dcmqpvzzqc773pe7cufzn08jgsys0w6xt0m0fp3u7v6tnj6weplh4ctyyvwfmcypemfjk6kryqxycnnmu2vp9tuw00eslf0grp6rf3hk6v76aynyn4lclra0fyyk2gxyf9hx73rnm775204tn8cltacw4s0fzd5c0lxm58s"
|
val encodedRequest = "lnr1qqswluyyp7j9aamd8l2ma23jyvvuvujqu5wq73jp38t02yr72s23evskyypylaf30uz3clmw4spxd3wvat4kc4m449q04wv9ferml6lkh3aqgp6syph79rq2kmcmxukp563ydtnr7a8ex85rvhs45zyudrtpjqqqqqqqq5srkudsqkppqv00gsulvwy3fhneygzg8hdr9hah5sc70xd9eed8vslm6u9jzx8yauzqylul6xp50xd4hn9shs7nhe02yasj9yfxsgkxych4q52hmny95kgtxj73n74m3dkt988r2xppa5xpwxespv8hukqf8mh3m6t277plwmc"
|
||||||
val decodedRequest = InvoiceRequest.decode(encodedRequest).get
|
val decodedRequest = InvoiceRequest.decode(encodedRequest).get
|
||||||
assert(decodedRequest.unsigned.records.filterNot(_.isInstanceOf[InvoiceRequestMetadata]) == request.unsigned.records.filterNot(_.isInstanceOf[InvoiceRequestMetadata]))
|
assert(decodedRequest.unsigned.records.filterNot(_.isInstanceOf[InvoiceRequestMetadata]) == request.unsigned.records.filterNot(_.isInstanceOf[InvoiceRequestMetadata]))
|
||||||
assert(request.isValid)
|
assert(request.isValid)
|
||||||
assert(request.offer == offer)
|
assert(request.offer == offer)
|
||||||
val invoice = Bolt12Invoice(decodedRequest, preimage, nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
val invoice = Bolt12Invoice(decodedRequest, preimage, nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
||||||
assert(Bolt12Invoice.fromString(invoice.toString).get.records == invoice.records)
|
assert(Bolt12Invoice.fromString(invoice.toString).get.records == invoice.records)
|
||||||
assert(invoice.validateFor(decodedRequest).isRight)
|
assert(invoice.validateFor(decodedRequest, nodeKey.publicKey).isRight)
|
||||||
// Invoice generation is not reproducible as the timestamp and blinding point will change but all other fields should be the same.
|
// Invoice generation is not reproducible as the timestamp and blinding point will change but all other fields should be the same.
|
||||||
val encodedInvoice = "lni1qqs289chx8swkpmwf3uzexfxr0kk9syavsjcmkuur5qgjqt60ayjdec2pdkkjmnfd4skcgr5d9cpvggzfl6nzlc9r3lkatqzvmzue6htd3tht22ql2uc2nj8hl4ld0r6qsr4qgr0u2xq4dh3kdevrf4zg6hx8a60jv0gxe0ptgyfc6xkryqqqqqqqpfq8dcmqpvzzqc773pe7cufzn08jgsys0w6xt0m0fp3u7v6tnj6weplh4ctyyvwf6s2qqj075ch7pgu0ah2cqnxchxw46mv2a66js86hxz5u3ala0mtc7syqup2a4g7lywy0zytzjzdhlar5uegx8qj8el2a2hpl7z30cv56fxkhwqpqgpnv93lzfep3m5ppkt3jry0kanpk3uxku733nr03snlzqjls3pejqp65tnf8nf8te9h67ge0lgzum5kypuvqrdz50t238n6g0wrdtv49nrgjk7k26rw7a24arfx9z4dup8379etdpw0tfkg3mwtngsuqqqqqqgqqqqqyqqrqqqqqqqqqqqqgqqqqqqqqqqqq5qqpfqyvwv9m2dxqgqje2pqshlyweee7p4m365legtkdgvy6s02rdqsv38mwnmk8p88cz03dt725qahrvqtqggzfl6nzlc9r3lkatqzvmzue6htd3tht22ql2uc2nj8hl4ld0r6qsrlqsxuf5rcjutppkh79vr6q7vma5yccxhf79ghfg5zkc6z4u3zqzyh0nf50g7w7q4gk32hqg97pn7p9kaz0ddm5fza65ztdqj2sry3gw6l2"
|
val encodedInvoice = "lni1qqswluyyp7j9aamd8l2ma23jyvvuvujqu5wq73jp38t02yr72s23evskyypylaf30uz3clmw4spxd3wvat4kc4m449q04wv9ferml6lkh3aqgp6syph79rq2kmcmxukp563ydtnr7a8ex85rvhs45zyudrtpjqqqqqqqq5srkudsqkppqv00gsulvwy3fhneygzg8hdr9hah5sc70xd9eed8vslm6u9jzx8yag9qqf8l2vtlq5w87m4vqfnvtn82adk9wadfgratnp2wg7l7ha4u0gzqwqahgwxsycqwtqlvu32j8mqxln456sxzh50k6avmgsndugtcp6wqcvqsxft50dexrade3n9us6tegq60tjjuc5f50jg8h43jr02r263wjfnwqqapd2vrfrwj2es7ne0wla08xnndgg655spddpn0zlru8fvqk6776fff60jphldzuw6wxgtxlne7ttvlp4tpmsghfh54atau5gwqqqqqqyqqqqqzqqpsqqqqqqqqqqqyqqqqqqqqqqqq2qqq5szxvtagn6nqyqfv4qsgtlj8vuulq6aca20u59mx5xzdg84pksgxgnahfamrsnnup8ck4l92qwm3kq9syypylaf30uz3clmw4spxd3wvat4kc4m449q04wv9ferml6lkh3aqgplsgrznfv8aysjyphv0usapr06mc4svfj9hlg4k9s263xd50dp0qdttrffypamzdxz84ftcvd52afx0je8adu4ppxq9z7yse0zh9qjmdwgz"
|
||||||
val decodedInvoice = Bolt12Invoice.fromString(encodedInvoice).get
|
val decodedInvoice = Bolt12Invoice.fromString(encodedInvoice).get
|
||||||
assert(decodedInvoice.amount == invoice.amount)
|
assert(decodedInvoice.amount == invoice.amount)
|
||||||
assert(decodedInvoice.nodeId == invoice.nodeId)
|
assert(decodedInvoice.nodeId == invoice.nodeId)
|
||||||
|
@ -284,28 +300,28 @@ class Bolt12InvoiceSpec extends AnyFunSuite {
|
||||||
assert(decodedInvoice.invoiceRequest.unsigned == invoice.invoiceRequest.unsigned)
|
assert(decodedInvoice.invoiceRequest.unsigned == invoice.invoiceRequest.unsigned)
|
||||||
}
|
}
|
||||||
|
|
||||||
test("minimal offer") {
|
test("minimal offer with amount") {
|
||||||
val nodeKey = PrivateKey(hex"3b7a19e8320bb86431cf92cd7c69cc1dc0181c37d5a09875e4603c4e37d3705d")
|
val nodeKey = PrivateKey(hex"3b7a19e8320bb86431cf92cd7c69cc1dc0181c37d5a09875e4603c4e37d3705d")
|
||||||
assert(nodeKey.publicKey == PublicKey(hex"03c48ac97e09f3cbbaeb35b02aaa6d072b57726841a34d25952157caca60a1caf5"))
|
assert(nodeKey.publicKey == PublicKey(hex"03c48ac97e09f3cbbaeb35b02aaa6d072b57726841a34d25952157caca60a1caf5"))
|
||||||
val payerKey = PrivateKey(hex"0e00a9ef505292f90a0e8a7aa99d31750e885c42a3ef8866dd2bf97919aa3891")
|
val payerKey = PrivateKey(hex"0e00a9ef505292f90a0e8a7aa99d31750e885c42a3ef8866dd2bf97919aa3891")
|
||||||
assert(payerKey.publicKey == PublicKey(hex"033e94f2afd568d128f02ece844ad4a0a1ddf2a4e3a08beb2dba11b3f1134b0517"))
|
assert(payerKey.publicKey == PublicKey(hex"033e94f2afd568d128f02ece844ad4a0a1ddf2a4e3a08beb2dba11b3f1134b0517"))
|
||||||
val preimage = ByteVector32(hex"09ad5e952ec39d45461ebdeceac206fb45574ae9054b5a454dd02c65f5ba1b7c")
|
val preimage = ByteVector32(hex"09ad5e952ec39d45461ebdeceac206fb45574ae9054b5a454dd02c65f5ba1b7c")
|
||||||
val offer = Offer(Some(456000000 msat), "minimal offer", nodeKey.publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
val offer = Offer(Some(456000000 msat), Some("minimal offer with amount"), nodeKey.publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
||||||
val encodedOffer = "lno1pqzpktszqq9q6mtfde5k6ctvyphkven9wgtzzq7y3tyhuz0newawkdds924x6pet2aexssdrf5je2g2het9xpgw275"
|
val encodedOffer = "lno1pqzpktszqq9pjmtfde5k6ctvyphkven9wgs8w6t5dqsxzmt0w4h8g93pq0zg4jt7p8euhwhtxkcz42ndqu44wunggx356fv4y9tu4jnq58902"
|
||||||
assert(offer.toString == encodedOffer)
|
assert(offer.toString == encodedOffer)
|
||||||
assert(Offer.decode(encodedOffer).get == offer)
|
assert(Offer.decode(encodedOffer).get == offer)
|
||||||
val request = InvoiceRequest(offer, 456001234 msat, 1, Features.empty, payerKey, Block.LivenetGenesisBlock.hash)
|
val request = InvoiceRequest(offer, 456001234 msat, 1, Features.empty, payerKey, Block.LivenetGenesisBlock.hash)
|
||||||
// Invoice request generation is not reproducible because we add randomness in the first TLV.
|
// Invoice request generation is not reproducible because we add randomness in the first TLV.
|
||||||
val encodedRequest = "lnr1qqsf4h8fsnpjkj057gjg9c3eqhv889440xh0z6f5kng9vsaad8pgq7sgqsdjuqsqpgxk66twd9kkzmpqdanxvetjzcss83y2e9lqnu7tht4ntvp24fksw26hwf5yrg6dyk2jz472efs2rjh42qsxlc5vp2m0rvmjcxn2y34wv0m5lyc7sdj7zksgn35dvxgqqqqqqqzjqsdjupkjtqssx05572ha26x39rczan5yft22pgwa72jw8gytavkm5ydn7yf5kpgh7pq2hlvh7twke5830a44wc0zlrs2kph4ghndm60ahwcznhcd0pcpl332qv5xuemksazy3zx5s63kqmqkphrn9jg4ln55pc6syrwqukejeq"
|
val encodedRequest = "lnr1qqswg5pzt6anzaxaypy8y46zknl8zn2a2jqyzrp74gtfm4lp6utpkzcgqsdjuqsqpgvk66twd9kkzmpqdanxvetjypmkjargypsk6mm4de6pvggrcj9vjlsf709m46e4kq425mg89dthy6zp5dxjt9fp2l9v5c9pet64qgr0u2xq4dh3kdevrf4zg6hx8a60jv0gxe0ptgyfc6xkryqqqqqqqpfqgxewqmf9sggr86209t74drgj3upwe6zy449q58wl9f8r5z97ktd6zxelzy6tq5tlqspdwx0zfhzu3mua0q2r7lgstw09p4qwtpgpewyuwytkpy2jm3hyupk52vc9tgx9dwvngdlgtgg335j029h0whqfxy28gkwewyu860g5x"
|
||||||
val decodedRequest = InvoiceRequest.decode(encodedRequest).get
|
val decodedRequest = InvoiceRequest.decode(encodedRequest).get
|
||||||
assert(decodedRequest.unsigned.records.filterNot(_.isInstanceOf[InvoiceRequestMetadata]) == request.unsigned.records.filterNot(_.isInstanceOf[InvoiceRequestMetadata]))
|
assert(decodedRequest.unsigned.records.filterNot(_.isInstanceOf[InvoiceRequestMetadata]) == request.unsigned.records.filterNot(_.isInstanceOf[InvoiceRequestMetadata]))
|
||||||
assert(request.isValid)
|
assert(request.isValid)
|
||||||
assert(request.offer == offer)
|
assert(request.offer == offer)
|
||||||
val invoice = Bolt12Invoice(decodedRequest, preimage, nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
val invoice = Bolt12Invoice(decodedRequest, preimage, nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
||||||
assert(Bolt12Invoice.fromString(invoice.toString).get.records == invoice.records)
|
assert(Bolt12Invoice.fromString(invoice.toString).get.records == invoice.records)
|
||||||
assert(invoice.validateFor(decodedRequest).isRight)
|
assert(invoice.validateFor(decodedRequest, nodeKey.publicKey).isRight)
|
||||||
// Invoice generation is not reproducible as the timestamp and blinding point will change but all other fields should be the same.
|
// Invoice generation is not reproducible as the timestamp and blinding point will change but all other fields should be the same.
|
||||||
val encodedInvoice = "lni1qqsf4h8fsnpjkj057gjg9c3eqhv889440xh0z6f5kng9vsaad8pgq7sgqsdjuqsqpgxk66twd9kkzmpqdanxvetjzcss83y2e9lqnu7tht4ntvp24fksw26hwf5yrg6dyk2jz472efs2rjh42qsxlc5vp2m0rvmjcxn2y34wv0m5lyc7sdj7zksgn35dvxgqqqqqqqzjqsdjupkjtqssx05572ha26x39rczan5yft22pgwa72jw8gytavkm5ydn7yf5kpgh5zsq83y2e9lqnu7tht4ntvp24fksw26hwf5yrg6dyk2jz472efs2rjh4qfjynufc627cuspz9lqzyk387xgzs4txcw0q97ugxfqm8x5zgj02gqgz4mnucmtxr620e5ttewtsg0s5n88euljnf7puagqje9j6gvaxk3pqqwsmahw79nhuq05zh8k29jk5qngpuny5l2vhjdrexg8hejukaee8fr7963dfag9q3lpcq9tt23f8s4h89cmjqa43u4fhk6l2y8qqqqqqzqqqqqpqqqcqqqqqqqqqqqzqqqqqqqqqqqq9qqq2gprrnp0zefszqyk2sgpvkrnmq53kv7r52rpnmtmd9ukredsnygsnymsurdy6e9la6l4hyz4qgxewqmftqggrcj9vjlsf709m46e4kq425mg89dthy6zp5dxjt9fp2l9v5c9pet6lqsy3s64amqgnlel7hn6fjrnk32xrn0ugr2xzct22ew28zftgmj70q9x2akqm34que8u2qe643cm38jpka6nfca4lfhuq6hgpnpwkpexrc"
|
val encodedInvoice = "lni1qqswg5pzt6anzaxaypy8y46zknl8zn2a2jqyzrp74gtfm4lp6utpkzcgqsdjuqsqpgvk66twd9kkzmpqdanxvetjypmkjargypsk6mm4de6pvggrcj9vjlsf709m46e4kq425mg89dthy6zp5dxjt9fp2l9v5c9pet64qgr0u2xq4dh3kdevrf4zg6hx8a60jv0gxe0ptgyfc6xkryqqqqqqqpfqgxewqmf9sggr86209t74drgj3upwe6zy449q58wl9f8r5z97ktd6zxelzy6tq5t6pgqrcj9vjlsf709m46e4kq425mg89dthy6zp5dxjt9fp2l9v5c9pet6s9x0wj2xtjxkql2urqn70fsyyhy8pcervfcaxdygsu74qe9jcss8uqypwa9rd3q0jh7tpruvr7xq7e4uzrk8z3mn68n5vzhxu4ds6d83qr4cq8f0mp833xq58twvuwlpm4gqkv5uwv07gl665ye2a33mk0tdkkzls04h25z3943cv5nq6e64dharmudq37remmgdvdv2vpt4zrsqqqqqpqqqqqqsqqvqqqqqqqqqqqpqqqqqqqqqqqqzsqq9yq3nzl2gl5cpqzt9gyqktpeas2gmx0p69psea4akj7tpukcfjygfjdcwpkjdvjl7a06mjp2syrvhqd54syypufzkf0cyl8ja6av6mq242d5rjk4mjdpq6xnf9j5s40jk2vzsu4a0sgq5pde5afeshaze029mqk5r48v07ph0uykc3ks034czmw58khfcw9gpv6d9l3nea06ajl4dqjr7ryrv9alx0eff9rklp7gnrkra0vuj3"
|
||||||
val decodedInvoice = Bolt12Invoice.fromString(encodedInvoice).get
|
val decodedInvoice = Bolt12Invoice.fromString(encodedInvoice).get
|
||||||
assert(decodedInvoice.amount == invoice.amount)
|
assert(decodedInvoice.amount == invoice.amount)
|
||||||
assert(decodedInvoice.nodeId == invoice.nodeId)
|
assert(decodedInvoice.nodeId == invoice.nodeId)
|
||||||
|
@ -339,7 +355,7 @@ class Bolt12InvoiceSpec extends AnyFunSuite {
|
||||||
assert(request.offer == offer)
|
assert(request.offer == offer)
|
||||||
val invoice = Bolt12Invoice(decodedRequest, preimage, nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
val invoice = Bolt12Invoice(decodedRequest, preimage, nodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(nodeKey.publicKey)))
|
||||||
assert(Bolt12Invoice.fromString(invoice.toString).get.records == invoice.records)
|
assert(Bolt12Invoice.fromString(invoice.toString).get.records == invoice.records)
|
||||||
assert(invoice.validateFor(decodedRequest).isRight)
|
assert(invoice.validateFor(decodedRequest, nodeKey.publicKey).isRight)
|
||||||
// Invoice generation is not reproducible as the timestamp and blinding point will change but all other fields should be the same.
|
// Invoice generation is not reproducible as the timestamp and blinding point will change but all other fields should be the same.
|
||||||
val encodedInvoice = "lni1qqs8lqvnh3kg9uj003lxlxyj8hthymgq4p9ms0ag0ryx5uw8gsuus4gzypp5jl7hlqnf2ugg7j3slkwwcwht57vhyzzwjr4dq84rxzgqqqqqqzqrqxr2qzsndanxvetjypmkjargypch2ctww35hg7gjz9skc6trv4qxy6t8wd5x7upwvdhk69qzq05pvggry7hatxw6xgn0gcytj64sgtl9tzl4tqs360z7vlkv305evv3qgd84qgzrf9la07pxj4cs3a9rplvuasawhfuewgyyay826q02xvysqqqqqpfqxmwaqptqzjzcyyp8cmgrl28nvm3wlqqheha0t570rgaszg7mzvvzvwmx9s92nmyujkdq5qpj0t74n8dryfh5vz9ed2cy9lj43064sgga830x0mxgh6vkxgsyxnczgew6pkkhja3cl3dfxthumcmp6gkp446ha4tcj884eqch6g57newqzquqmar5nynwtg9lknq98yzslwla3vdxefulhq2jkwnqnsf7umpl5cqr58qkj63hkpl7ffyd6f3qgn3m5kuegehhakvxw7fuw29tf3r5wgj37uecjdw2th4t5fp7f99xvk4f3gwl0wyf2a558wqa9w3pcqqqqqqsqqqqqgqqxqqqqqqqqqqqqsqqqqqqqqqqqpgqqzjqgcuctck2vqsp9j5zqlsxsv7uy23npygenelt4q5sdh8ftc3x7rpd0hqlachjnj9z834s4gpkmhgqkqssxfa06kva5v3x73sgh94tqsh72k9l2kppr579uelvezlfjcezqs607pqxa3afljxyf2ua9dlqs33wrfzakt5tpraklpzfpn63uxa7el475x4sc0w4hs75e3nhe689slfz4ldqlwja3zaq0w3mnz79f4ne0c3r3c"
|
val encodedInvoice = "lni1qqs8lqvnh3kg9uj003lxlxyj8hthymgq4p9ms0ag0ryx5uw8gsuus4gzypp5jl7hlqnf2ugg7j3slkwwcwht57vhyzzwjr4dq84rxzgqqqqqqzqrqxr2qzsndanxvetjypmkjargypch2ctww35hg7gjz9skc6trv4qxy6t8wd5x7upwvdhk69qzq05pvggry7hatxw6xgn0gcytj64sgtl9tzl4tqs360z7vlkv305evv3qgd84qgzrf9la07pxj4cs3a9rplvuasawhfuewgyyay826q02xvysqqqqqpfqxmwaqptqzjzcyyp8cmgrl28nvm3wlqqheha0t570rgaszg7mzvvzvwmx9s92nmyujkdq5qpj0t74n8dryfh5vz9ed2cy9lj43064sgga830x0mxgh6vkxgsyxnczgew6pkkhja3cl3dfxthumcmp6gkp446ha4tcj884eqch6g57newqzquqmar5nynwtg9lknq98yzslwla3vdxefulhq2jkwnqnsf7umpl5cqr58qkj63hkpl7ffyd6f3qgn3m5kuegehhakvxw7fuw29tf3r5wgj37uecjdw2th4t5fp7f99xvk4f3gwl0wyf2a558wqa9w3pcqqqqqqsqqqqqgqqxqqqqqqqqqqqqsqqqqqqqqqqqpgqqzjqgcuctck2vqsp9j5zqlsxsv7uy23npygenelt4q5sdh8ftc3x7rpd0hqlachjnj9z834s4gpkmhgqkqssxfa06kva5v3x73sgh94tqsh72k9l2kppr579uelvezlfjcezqs607pqxa3afljxyf2ua9dlqs33wrfzakt5tpraklpzfpn63uxa7el475x4sc0w4hs75e3nhe689slfz4ldqlwja3zaq0w3mnz79f4ne0c3r3c"
|
||||||
val decodedInvoice = Bolt12Invoice.fromString(encodedInvoice).get
|
val decodedInvoice = Bolt12Invoice.fromString(encodedInvoice).get
|
||||||
|
@ -357,6 +373,6 @@ class Bolt12InvoiceSpec extends AnyFunSuite {
|
||||||
assert(invoice.amount == 10000000.msat)
|
assert(invoice.amount == 10000000.msat)
|
||||||
assert(invoice.nodeId == PublicKey(hex"024cc50d604f657094f8a2ff031ad6d888f9ac220a86b5949cdaaa5a5c03055d69"))
|
assert(invoice.nodeId == PublicKey(hex"024cc50d604f657094f8a2ff031ad6d888f9ac220a86b5949cdaaa5a5c03055d69"))
|
||||||
assert(invoice.paymentHash == ByteVector32(hex"14805a7006b96286e7b0a3f618c1cd7f1059f76da766044c5bfc3fa31d5e9442"))
|
assert(invoice.paymentHash == ByteVector32(hex"14805a7006b96286e7b0a3f618c1cd7f1059f76da766044c5bfc3fa31d5e9442"))
|
||||||
assert(invoice.description == Left("yolo"))
|
assert(invoice.description.contains("yolo"))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -163,7 +163,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||||
}
|
}
|
||||||
{
|
{
|
||||||
val privKey = randomKey()
|
val privKey = randomKey()
|
||||||
val offer = Offer(Some(amountMsat), "a blinded coffee please", privKey.publicKey, Features.empty, Block.RegtestGenesisBlock.hash)
|
val offer = Offer(Some(amountMsat), Some("a blinded coffee please"), privKey.publicKey, Features.empty, Block.RegtestGenesisBlock.hash)
|
||||||
val invoiceReq = InvoiceRequest(offer, amountMsat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash)
|
val invoiceReq = InvoiceRequest(offer, amountMsat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash)
|
||||||
val preimage = randomBytes32()
|
val preimage = randomBytes32()
|
||||||
val pathId = randomBytes32()
|
val pathId = randomBytes32()
|
||||||
|
@ -267,7 +267,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||||
import f._
|
import f._
|
||||||
|
|
||||||
val privKey = randomKey()
|
val privKey = randomKey()
|
||||||
val offer = Offer(Some(25_000 msat), "a blinded coffee please", privKey.publicKey, Features.empty, Block.RegtestGenesisBlock.hash)
|
val offer = Offer(Some(25_000 msat), Some("a blinded coffee please"), privKey.publicKey, Features.empty, Block.RegtestGenesisBlock.hash)
|
||||||
val invoiceReq = InvoiceRequest(offer, 25_000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash)
|
val invoiceReq = InvoiceRequest(offer, 25_000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash)
|
||||||
val router = TestProbe()
|
val router = TestProbe()
|
||||||
val (a, b, c, d) = (randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, nodeParams.nodeId)
|
val (a, b, c, d) = (randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, nodeParams.nodeId)
|
||||||
|
@ -291,7 +291,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||||
assert(invoice.nodeId == privKey.publicKey)
|
assert(invoice.nodeId == privKey.publicKey)
|
||||||
assert(invoice.blindedPaths.nonEmpty)
|
assert(invoice.blindedPaths.nonEmpty)
|
||||||
assert(invoice.features.hasFeature(RouteBlinding, Some(Mandatory)))
|
assert(invoice.features.hasFeature(RouteBlinding, Some(Mandatory)))
|
||||||
assert(invoice.description == Left("a blinded coffee please"))
|
assert(invoice.description.contains("a blinded coffee please"))
|
||||||
assert(invoice.invoiceRequest.offer == offer)
|
assert(invoice.invoiceRequest.offer == offer)
|
||||||
assert(invoice.blindedPaths.length == 3)
|
assert(invoice.blindedPaths.length == 3)
|
||||||
assert(invoice.blindedPaths(0).route.blindedNodeIds.length == 4)
|
assert(invoice.blindedPaths(0).route.blindedNodeIds.length == 4)
|
||||||
|
@ -313,7 +313,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||||
import f._
|
import f._
|
||||||
|
|
||||||
val privKey = randomKey()
|
val privKey = randomKey()
|
||||||
val offer = Offer(Some(25_000 msat), "a blinded coffee please", privKey.publicKey, Features.empty, Block.RegtestGenesisBlock.hash)
|
val offer = Offer(Some(25_000 msat), Some("a blinded coffee please"), privKey.publicKey, Features.empty, Block.RegtestGenesisBlock.hash)
|
||||||
val invoiceReq = InvoiceRequest(offer, 25_000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash)
|
val invoiceReq = InvoiceRequest(offer, 25_000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash)
|
||||||
val router = TestProbe()
|
val router = TestProbe()
|
||||||
val (a, b, c) = (randomKey().publicKey, randomKey().publicKey, nodeParams.nodeId)
|
val (a, b, c) = (randomKey().publicKey, randomKey().publicKey, nodeParams.nodeId)
|
||||||
|
@ -488,7 +488,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||||
import f._
|
import f._
|
||||||
|
|
||||||
val nodeKey = randomKey()
|
val nodeKey = randomKey()
|
||||||
val offer = Offer(None, "a blinded coffee please", nodeKey.publicKey, Features.empty, Block.RegtestGenesisBlock.hash)
|
val offer = Offer(None, Some("a blinded coffee please"), nodeKey.publicKey, Features.empty, Block.RegtestGenesisBlock.hash)
|
||||||
val invoiceReq = InvoiceRequest(offer, 5000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash)
|
val invoiceReq = InvoiceRequest(offer, 5000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash)
|
||||||
sender.send(handlerWithRouteBlinding, ReceiveOfferPayment(sender.ref, nodeKey, invoiceReq, createEmptyReceivingRoute(), TestProbe().ref, randomBytes32(), randomBytes32()))
|
sender.send(handlerWithRouteBlinding, ReceiveOfferPayment(sender.ref, nodeKey, invoiceReq, createEmptyReceivingRoute(), TestProbe().ref, randomBytes32(), randomBytes32()))
|
||||||
val invoice = sender.expectMsgType[CreateInvoiceActor.InvoiceCreated].invoice
|
val invoice = sender.expectMsgType[CreateInvoiceActor.InvoiceCreated].invoice
|
||||||
|
@ -505,7 +505,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||||
import f._
|
import f._
|
||||||
|
|
||||||
val nodeKey = randomKey()
|
val nodeKey = randomKey()
|
||||||
val offer = Offer(None, "a blinded coffee please", nodeKey.publicKey, Features.empty, Block.RegtestGenesisBlock.hash)
|
val offer = Offer(None, Some("a blinded coffee please"), nodeKey.publicKey, Features.empty, Block.RegtestGenesisBlock.hash)
|
||||||
val invoiceReq = InvoiceRequest(offer, 5000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash)
|
val invoiceReq = InvoiceRequest(offer, 5000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash)
|
||||||
val preimage = randomBytes32()
|
val preimage = randomBytes32()
|
||||||
val pathId = randomBytes32()
|
val pathId = randomBytes32()
|
||||||
|
@ -531,7 +531,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||||
val nodeKey = randomKey()
|
val nodeKey = randomKey()
|
||||||
val preimage = randomBytes32()
|
val preimage = randomBytes32()
|
||||||
val pathId = randomBytes(128)
|
val pathId = randomBytes(128)
|
||||||
val offer = Offer(None, "a blinded coffee please", nodeKey.publicKey, Features.empty, Block.RegtestGenesisBlock.hash)
|
val offer = Offer(None, Some("a blinded coffee please"), nodeKey.publicKey, Features.empty, Block.RegtestGenesisBlock.hash)
|
||||||
val invoiceReq = InvoiceRequest(offer, 5000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash)
|
val invoiceReq = InvoiceRequest(offer, 5000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash)
|
||||||
sender.send(handlerWithRouteBlinding, ReceiveOfferPayment(sender.ref, nodeKey, invoiceReq, createEmptyReceivingRoute(), TestProbe().ref, preimage, pathId))
|
sender.send(handlerWithRouteBlinding, ReceiveOfferPayment(sender.ref, nodeKey, invoiceReq, createEmptyReceivingRoute(), TestProbe().ref, preimage, pathId))
|
||||||
val invoice = sender.expectMsgType[CreateInvoiceActor.InvoiceCreated].invoice
|
val invoice = sender.expectMsgType[CreateInvoiceActor.InvoiceCreated].invoice
|
||||||
|
@ -550,7 +550,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||||
import f._
|
import f._
|
||||||
|
|
||||||
val nodeKey = randomKey()
|
val nodeKey = randomKey()
|
||||||
val offer = Offer(None, "a blinded coffee please", nodeKey.publicKey, Features.empty, Block.RegtestGenesisBlock.hash)
|
val offer = Offer(None, Some("a blinded coffee please"), nodeKey.publicKey, Features.empty, Block.RegtestGenesisBlock.hash)
|
||||||
val invoiceReq = InvoiceRequest(offer, 5000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash)
|
val invoiceReq = InvoiceRequest(offer, 5000 msat, 1, featuresWithRouteBlinding.bolt12Features(), randomKey(), Block.RegtestGenesisBlock.hash)
|
||||||
val preimage = randomBytes32()
|
val preimage = randomBytes32()
|
||||||
val pathId = randomBytes32()
|
val pathId = randomBytes32()
|
||||||
|
|
|
@ -295,7 +295,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
||||||
}
|
}
|
||||||
|
|
||||||
def createBolt12Invoice(features: Features[Bolt12Feature], payerKey: PrivateKey): Bolt12Invoice = {
|
def createBolt12Invoice(features: Features[Bolt12Feature], payerKey: PrivateKey): Bolt12Invoice = {
|
||||||
val offer = Offer(None, "Bolt12 r0cks", e, features, Block.RegtestGenesisBlock.hash)
|
val offer = Offer(None, Some("Bolt12 r0cks"), e, features, Block.RegtestGenesisBlock.hash)
|
||||||
val invoiceRequest = InvoiceRequest(offer, finalAmount, 1, features, randomKey(), 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 = 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)
|
val paymentInfo = OfferTypes.PaymentInfo(1_000 msat, 0, CltvExpiryDelta(24), 0 msat, finalAmount, Features.empty)
|
||||||
|
|
|
@ -217,7 +217,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||||
// a -> b -> c where c uses a 0-hop blinded route.
|
// a -> b -> c where c uses a 0-hop blinded route.
|
||||||
val recipientKey = randomKey()
|
val recipientKey = randomKey()
|
||||||
val features = Features[Bolt12Feature](BasicMultiPartPayment -> Optional)
|
val features = Features[Bolt12Feature](BasicMultiPartPayment -> Optional)
|
||||||
val offer = Offer(None, "Bolt12 r0cks", recipientKey.publicKey, features, Block.RegtestGenesisBlock.hash)
|
val offer = Offer(None, Some("Bolt12 r0cks"), recipientKey.publicKey, features, Block.RegtestGenesisBlock.hash)
|
||||||
val invoiceRequest = InvoiceRequest(offer, amount_bc, 1, features, randomKey(), 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 = 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 paymentInfo = PaymentInfo(0 msat, 0, CltvExpiryDelta(0), 1 msat, amount_bc, Features.empty)
|
||||||
|
@ -490,7 +490,7 @@ class PaymentPacketSpec extends AnyFunSuite with BeforeAndAfterAll {
|
||||||
test("fail to decrypt when blinded route data is invalid") {
|
test("fail to decrypt when blinded route data is invalid") {
|
||||||
val (route, recipient) = {
|
val (route, recipient) = {
|
||||||
val features = Features[Bolt12Feature](BasicMultiPartPayment -> Optional)
|
val features = Features[Bolt12Feature](BasicMultiPartPayment -> Optional)
|
||||||
val offer = Offer(None, "Bolt12 r0cks", c, features, Block.RegtestGenesisBlock.hash)
|
val offer = Offer(None, Some("Bolt12 r0cks"), c, features, Block.RegtestGenesisBlock.hash)
|
||||||
val invoiceRequest = InvoiceRequest(offer, amount_bc, 1, features, randomKey(), Block.RegtestGenesisBlock.hash)
|
val invoiceRequest = InvoiceRequest(offer, amount_bc, 1, features, randomKey(), Block.RegtestGenesisBlock.hash)
|
||||||
// We send the wrong blinded payload to the introduction node.
|
// 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 tmpBlindedRoute = BlindedRouteCreation.createBlindedRouteFromHops(Seq(channelHopFromUpdate(b, c, channelUpdate_bc)), hex"deadbeef", 1 msat, CltvExpiry(500_000)).route
|
||||||
|
|
|
@ -20,7 +20,7 @@ import akka.actor.testkit.typed.scaladsl.{ScalaTestWithActorTestKit, TestProbe}
|
||||||
import akka.actor.typed.ActorRef
|
import akka.actor.typed.ActorRef
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import fr.acinq.bitcoin.scalacompat.Crypto.PrivateKey
|
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
|
||||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto}
|
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto}
|
||||||
import fr.acinq.eclair.message.OnionMessages.Recipient
|
import fr.acinq.eclair.message.OnionMessages.Recipient
|
||||||
import fr.acinq.eclair.message.{OnionMessages, Postman}
|
import fr.acinq.eclair.message.{OnionMessages, Postman}
|
||||||
|
@ -59,17 +59,17 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def requestInvoice(payerKey: PrivateKey, offer: Offer, amount: MilliSatoshi, offerManager: ActorRef[Command], postman: ActorRef[Postman.Command], pathId_opt: Option[ByteVector32] = None): Unit = {
|
def requestInvoice(payerKey: PrivateKey, offer: Offer, offerKey: PrivateKey, amount: MilliSatoshi, offerManager: ActorRef[Command], postman: ActorRef[Postman.Command], pathId_opt: Option[ByteVector32] = None): Unit = {
|
||||||
val invoiceRequest = InvoiceRequest(offer, amount, 1, Features.empty, payerKey, offer.chains.head)
|
val invoiceRequest = InvoiceRequest(offer, amount, 1, Features.empty, payerKey, offer.chains.head)
|
||||||
val replyPath = OnionMessages.buildRoute(randomKey(), Nil, Recipient(payerKey.publicKey, None))
|
val replyPath = OnionMessages.buildRoute(randomKey(), Nil, Recipient(payerKey.publicKey, None)).route
|
||||||
val Right(messagePayload: MessageOnion.InvoiceRequestPayload) = MessageOnion.FinalPayload.validate(
|
val Right(messagePayload: MessageOnion.InvoiceRequestPayload) = MessageOnion.FinalPayload.validate(
|
||||||
TlvStream(OnionMessagePayloadTlv.InvoiceRequest(invoiceRequest.records), OnionMessagePayloadTlv.ReplyPath(replyPath)),
|
TlvStream(OnionMessagePayloadTlv.InvoiceRequest(invoiceRequest.records), OnionMessagePayloadTlv.ReplyPath(replyPath)),
|
||||||
pathId_opt.map(pathId => TlvStream[RouteBlindingEncryptedDataTlv](RouteBlindingEncryptedDataTlv.PathId(pathId))).getOrElse(TlvStream.empty),
|
pathId_opt.map(pathId => TlvStream[RouteBlindingEncryptedDataTlv](RouteBlindingEncryptedDataTlv.PathId(pathId))).getOrElse(TlvStream.empty),
|
||||||
)
|
)
|
||||||
offerManager ! RequestInvoice(messagePayload, postman)
|
offerManager ! RequestInvoice(messagePayload, offerKey, postman)
|
||||||
}
|
}
|
||||||
|
|
||||||
def receiveInvoice(f: FixtureParam, amount: MilliSatoshi, payerKey: PrivateKey, handler: TestProbe[HandlerCommand], pluginData_opt: Option[ByteVector] = None): Bolt12Invoice = {
|
def receiveInvoice(f: FixtureParam, amount: MilliSatoshi, payerKey: PrivateKey, pathNodeId: PublicKey, handler: TestProbe[HandlerCommand], pluginData_opt: Option[ByteVector] = None): Bolt12Invoice = {
|
||||||
import f._
|
import f._
|
||||||
|
|
||||||
val handleInvoiceRequest = handler.expectMessageType[HandleInvoiceRequest]
|
val handleInvoiceRequest = handler.expectMessageType[HandleInvoiceRequest]
|
||||||
|
@ -78,7 +78,7 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, Seq(ReceivingRoute(Seq(nodeParams.nodeId), CltvExpiryDelta(1000), Nil)), pluginData_opt)
|
handleInvoiceRequest.replyTo ! InvoiceRequestActor.ApproveRequest(amount, Seq(ReceivingRoute(Seq(nodeParams.nodeId), CltvExpiryDelta(1000), Nil)), pluginData_opt)
|
||||||
val invoiceMessage = postman.expectMessageType[Postman.SendMessage]
|
val invoiceMessage = postman.expectMessageType[Postman.SendMessage]
|
||||||
val Right(invoice) = Bolt12Invoice.validate(invoiceMessage.message.get[OnionMessagePayloadTlv.Invoice].get.tlvs)
|
val Right(invoice) = Bolt12Invoice.validate(invoiceMessage.message.get[OnionMessagePayloadTlv.Invoice].get.tlvs)
|
||||||
assert(invoice.validateFor(handleInvoiceRequest.invoiceRequest).isRight)
|
assert(invoice.validateFor(handleInvoiceRequest.invoiceRequest, pathNodeId).isRight)
|
||||||
assert(invoice.invoiceRequest.payerId == payerKey.publicKey)
|
assert(invoice.invoiceRequest.payerId == payerKey.publicKey)
|
||||||
assert(invoice.amount == amount)
|
assert(invoice.amount == amount)
|
||||||
invoice
|
invoice
|
||||||
|
@ -103,12 +103,12 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
|
|
||||||
val handler = TestProbe[HandlerCommand]()
|
val handler = TestProbe[HandlerCommand]()
|
||||||
val amount = 10_000_000 msat
|
val amount = 10_000_000 msat
|
||||||
val offer = Offer(Some(amount), "offer", nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
val offer = Offer(Some(amount), Some("offer"), nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
||||||
offerManager ! RegisterOffer(offer, nodeParams.privateKey, pathId_opt, handler.ref)
|
offerManager ! RegisterOffer(offer, Some(nodeParams.privateKey), pathId_opt, handler.ref)
|
||||||
// Request invoice.
|
// Request invoice.
|
||||||
val payerKey = randomKey()
|
val payerKey = randomKey()
|
||||||
requestInvoice(payerKey, offer, amount, offerManager, postman.ref, pathId_opt)
|
requestInvoice(payerKey, offer, nodeParams.privateKey, amount, offerManager, postman.ref, pathId_opt)
|
||||||
val invoice = receiveInvoice(f, amount, payerKey, handler, pluginData_opt = Some(hex"deadbeef"))
|
val invoice = receiveInvoice(f, amount, payerKey, nodeParams.nodeId, handler, pluginData_opt = Some(hex"deadbeef"))
|
||||||
// Pay invoice.
|
// Pay invoice.
|
||||||
val paymentPayload = createPaymentPayload(f, invoice)
|
val paymentPayload = createPaymentPayload(f, invoice)
|
||||||
offerManager ! ReceivePayment(paymentHandler.ref, invoice.paymentHash, paymentPayload)
|
offerManager ! ReceivePayment(paymentHandler.ref, invoice.paymentHash, paymentPayload)
|
||||||
|
@ -134,9 +134,9 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
import f._
|
import f._
|
||||||
|
|
||||||
val handler = TestProbe[HandlerCommand]()
|
val handler = TestProbe[HandlerCommand]()
|
||||||
val offer = Offer(Some(10_000_000 msat), "offer", nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
val offer = Offer(Some(10_000_000 msat), Some("offer"), nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
||||||
offerManager ! RegisterOffer(offer, nodeParams.privateKey, None, handler.ref)
|
offerManager ! RegisterOffer(offer, Some(nodeParams.privateKey), None, handler.ref)
|
||||||
requestInvoice(randomKey(), offer, 9_000_000 msat, offerManager, postman.ref)
|
requestInvoice(randomKey(), offer, nodeParams.privateKey, 9_000_000 msat, offerManager, postman.ref)
|
||||||
handler.expectNoMessage(50 millis)
|
handler.expectNoMessage(50 millis)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -145,9 +145,9 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
|
|
||||||
val handler = TestProbe[HandlerCommand]()
|
val handler = TestProbe[HandlerCommand]()
|
||||||
val pathId = randomBytes32()
|
val pathId = randomBytes32()
|
||||||
val offer = Offer(Some(10_000_000 msat), "offer with path_id", nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
val offer = Offer(Some(10_000_000 msat), Some("offer with path_id"), nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
||||||
offerManager ! RegisterOffer(offer, nodeParams.privateKey, Some(pathId), handler.ref)
|
offerManager ! RegisterOffer(offer, Some(nodeParams.privateKey), Some(pathId), handler.ref)
|
||||||
requestInvoice(randomKey(), offer, 10_000_000 msat, offerManager, postman.ref)
|
requestInvoice(randomKey(), offer, nodeParams.privateKey, 10_000_000 msat, offerManager, postman.ref)
|
||||||
handler.expectNoMessage(50 millis)
|
handler.expectNoMessage(50 millis)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -156,9 +156,9 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
|
|
||||||
val handler = TestProbe[HandlerCommand]()
|
val handler = TestProbe[HandlerCommand]()
|
||||||
val pathId = randomBytes32()
|
val pathId = randomBytes32()
|
||||||
val offer = Offer(Some(10_000_000 msat), "offer with path_id", nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
val offer = Offer(Some(10_000_000 msat), Some("offer with path_id"), nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
||||||
offerManager ! RegisterOffer(offer, nodeParams.privateKey, Some(pathId), handler.ref)
|
offerManager ! RegisterOffer(offer, Some(nodeParams.privateKey), Some(pathId), handler.ref)
|
||||||
requestInvoice(randomKey(), offer, 10_000_000 msat, offerManager, postman.ref, pathId_opt = Some(pathId.reverse))
|
requestInvoice(randomKey(), offer, nodeParams.privateKey, 10_000_000 msat, offerManager, postman.ref, pathId_opt = Some(pathId.reverse))
|
||||||
handler.expectNoMessage(50 millis)
|
handler.expectNoMessage(50 millis)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -166,10 +166,10 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
import f._
|
import f._
|
||||||
|
|
||||||
val handler = TestProbe[HandlerCommand]()
|
val handler = TestProbe[HandlerCommand]()
|
||||||
val offer = Offer(Some(10_000_000 msat), "offer", nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
val offer = Offer(Some(10_000_000 msat), Some("offer"), nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
||||||
offerManager ! RegisterOffer(offer, nodeParams.privateKey, None, handler.ref)
|
offerManager ! RegisterOffer(offer, Some(nodeParams.privateKey), None, handler.ref)
|
||||||
offerManager ! DisableOffer(offer)
|
offerManager ! DisableOffer(offer)
|
||||||
requestInvoice(randomKey(), offer, 10_000_000 msat, offerManager, postman.ref)
|
requestInvoice(randomKey(), offer, nodeParams.privateKey, 10_000_000 msat, offerManager, postman.ref)
|
||||||
handler.expectNoMessage(50 millis)
|
handler.expectNoMessage(50 millis)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -177,9 +177,9 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
import f._
|
import f._
|
||||||
|
|
||||||
val handler = TestProbe[HandlerCommand]()
|
val handler = TestProbe[HandlerCommand]()
|
||||||
val offer = Offer(Some(10_000_000 msat), "offer", nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
val offer = Offer(Some(10_000_000 msat), Some("offer"), nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
||||||
offerManager ! RegisterOffer(offer, nodeParams.privateKey, None, handler.ref)
|
offerManager ! RegisterOffer(offer, Some(nodeParams.privateKey), None, handler.ref)
|
||||||
requestInvoice(randomKey(), offer, 10_000_000 msat, offerManager, postman.ref)
|
requestInvoice(randomKey(), offer, nodeParams.privateKey, 10_000_000 msat, offerManager, postman.ref)
|
||||||
val handleInvoiceRequest = handler.expectMessageType[HandleInvoiceRequest]
|
val handleInvoiceRequest = handler.expectMessageType[HandleInvoiceRequest]
|
||||||
handleInvoiceRequest.replyTo ! InvoiceRequestActor.RejectRequest("internal error")
|
handleInvoiceRequest.replyTo ! InvoiceRequestActor.RejectRequest("internal error")
|
||||||
val invoiceMessage = postman.expectMessageType[Postman.SendMessage]
|
val invoiceMessage = postman.expectMessageType[Postman.SendMessage]
|
||||||
|
@ -193,16 +193,16 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
|
|
||||||
val handler = TestProbe[HandlerCommand]()
|
val handler = TestProbe[HandlerCommand]()
|
||||||
val amount = 10_000_000 msat
|
val amount = 10_000_000 msat
|
||||||
val offer1 = Offer(Some(amount), "offer #1", nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
val offer1 = Offer(Some(amount), Some("offer #1"), nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
||||||
val offer2 = Offer(Some(amount), "offer #2", nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
val offer2 = Offer(Some(amount), Some("offer #2"), nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
||||||
offerManager ! RegisterOffer(offer1, nodeParams.privateKey, None, handler.ref)
|
offerManager ! RegisterOffer(offer1, Some(nodeParams.privateKey), None, handler.ref)
|
||||||
offerManager ! RegisterOffer(offer2, nodeParams.privateKey, None, handler.ref)
|
offerManager ! RegisterOffer(offer2, Some(nodeParams.privateKey), None, handler.ref)
|
||||||
// Request invoices for offers #1 and #2.
|
// Request invoices for offers #1 and #2.
|
||||||
val payerKey = randomKey()
|
val payerKey = randomKey()
|
||||||
requestInvoice(payerKey, offer1, amount, offerManager, postman.ref)
|
requestInvoice(payerKey, offer1, nodeParams.privateKey, amount, offerManager, postman.ref)
|
||||||
val invoice1 = receiveInvoice(f, amount, payerKey, handler)
|
val invoice1 = receiveInvoice(f, amount, payerKey, nodeParams.nodeId, handler)
|
||||||
requestInvoice(payerKey, offer2, amount, offerManager, postman.ref)
|
requestInvoice(payerKey, offer2, nodeParams.privateKey, amount, offerManager, postman.ref)
|
||||||
val invoice2 = receiveInvoice(f, amount, payerKey, handler)
|
val invoice2 = receiveInvoice(f, amount, payerKey, nodeParams.nodeId, handler)
|
||||||
// Try paying invoice #1 with data from invoice #2.
|
// Try paying invoice #1 with data from invoice #2.
|
||||||
val paymentPayload = createPaymentPayload(f, invoice2)
|
val paymentPayload = createPaymentPayload(f, invoice2)
|
||||||
offerManager ! ReceivePayment(paymentHandler.ref, invoice1.paymentHash, paymentPayload)
|
offerManager ! ReceivePayment(paymentHandler.ref, invoice1.paymentHash, paymentPayload)
|
||||||
|
@ -215,12 +215,12 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
|
|
||||||
val handler = TestProbe[HandlerCommand]()
|
val handler = TestProbe[HandlerCommand]()
|
||||||
val amount = 10_000_000 msat
|
val amount = 10_000_000 msat
|
||||||
val offer = Offer(Some(amount), "offer", nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
val offer = Offer(Some(amount), Some("offer"), nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
||||||
offerManager ! RegisterOffer(offer, nodeParams.privateKey, None, handler.ref)
|
offerManager ! RegisterOffer(offer, Some(nodeParams.privateKey), None, handler.ref)
|
||||||
// Request invoice.
|
// Request invoice.
|
||||||
val payerKey = randomKey()
|
val payerKey = randomKey()
|
||||||
requestInvoice(payerKey, offer, amount, offerManager, postman.ref)
|
requestInvoice(payerKey, offer, nodeParams.privateKey, amount, offerManager, postman.ref)
|
||||||
val invoice = receiveInvoice(f, amount, payerKey, handler)
|
val invoice = receiveInvoice(f, amount, payerKey, nodeParams.nodeId, handler)
|
||||||
// Try paying the invoice with a modified path_id.
|
// Try paying the invoice with a modified path_id.
|
||||||
val paymentPayload = createPaymentPayload(f, invoice)
|
val paymentPayload = createPaymentPayload(f, invoice)
|
||||||
val Some(pathId) = paymentPayload.blindedRecords.get[RouteBlindingEncryptedDataTlv.PathId].map(_.data)
|
val Some(pathId) = paymentPayload.blindedRecords.get[RouteBlindingEncryptedDataTlv.PathId].map(_.data)
|
||||||
|
@ -238,12 +238,12 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
|
|
||||||
val handler = TestProbe[HandlerCommand]()
|
val handler = TestProbe[HandlerCommand]()
|
||||||
val amount = 10_000_000 msat
|
val amount = 10_000_000 msat
|
||||||
val offer = Offer(Some(amount), "offer", nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
val offer = Offer(Some(amount), Some("offer"), nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
||||||
offerManager ! RegisterOffer(offer, nodeParams.privateKey, None, handler.ref)
|
offerManager ! RegisterOffer(offer, Some(nodeParams.privateKey), None, handler.ref)
|
||||||
// Request invoice.
|
// Request invoice.
|
||||||
val payerKey = randomKey()
|
val payerKey = randomKey()
|
||||||
requestInvoice(payerKey, offer, amount, offerManager, postman.ref)
|
requestInvoice(payerKey, offer, nodeParams.privateKey, amount, offerManager, postman.ref)
|
||||||
val invoice = receiveInvoice(f, amount, payerKey, handler)
|
val invoice = receiveInvoice(f, amount, payerKey, nodeParams.nodeId, handler)
|
||||||
// Try paying the invoice, but the plugin handler doesn't respond.
|
// Try paying the invoice, but the plugin handler doesn't respond.
|
||||||
val paymentPayload = createPaymentPayload(f, invoice)
|
val paymentPayload = createPaymentPayload(f, invoice)
|
||||||
offerManager ! ReceivePayment(paymentHandler.ref, invoice.paymentHash, paymentPayload)
|
offerManager ! ReceivePayment(paymentHandler.ref, invoice.paymentHash, paymentPayload)
|
||||||
|
@ -256,12 +256,12 @@ class OfferManagerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
|
|
||||||
val handler = TestProbe[HandlerCommand]()
|
val handler = TestProbe[HandlerCommand]()
|
||||||
val amount = 10_000_000 msat
|
val amount = 10_000_000 msat
|
||||||
val offer = Offer(Some(amount), "offer", nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
val offer = Offer(Some(amount), Some("offer"), nodeParams.nodeId, Features.empty, nodeParams.chainHash)
|
||||||
offerManager ! RegisterOffer(offer, nodeParams.privateKey, None, handler.ref)
|
offerManager ! RegisterOffer(offer, Some(nodeParams.privateKey), None, handler.ref)
|
||||||
// Request invoice.
|
// Request invoice.
|
||||||
val payerKey = randomKey()
|
val payerKey = randomKey()
|
||||||
requestInvoice(payerKey, offer, amount, offerManager, postman.ref)
|
requestInvoice(payerKey, offer, nodeParams.privateKey, amount, offerManager, postman.ref)
|
||||||
val invoice = receiveInvoice(f, amount, payerKey, handler)
|
val invoice = receiveInvoice(f, amount, payerKey, nodeParams.nodeId, handler)
|
||||||
// Try paying the invoice, but the plugin handler rejects the payment.
|
// Try paying the invoice, but the plugin handler rejects the payment.
|
||||||
val paymentPayload = createPaymentPayload(f, invoice)
|
val paymentPayload = createPaymentPayload(f, invoice)
|
||||||
offerManager ! ReceivePayment(paymentHandler.ref, invoice.paymentHash, paymentPayload)
|
offerManager ! ReceivePayment(paymentHandler.ref, invoice.paymentHash, paymentPayload)
|
||||||
|
|
|
@ -837,7 +837,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||||
import f._
|
import f._
|
||||||
|
|
||||||
val (payerKey, chain) = (randomKey(), BlockHash(randomBytes32()))
|
val (payerKey, chain) = (randomKey(), BlockHash(randomBytes32()))
|
||||||
val offer = Offer(None, "test offer", outgoingNodeId, Features.empty, chain)
|
val offer = Offer(None, Some("test offer"), outgoingNodeId, Features.empty, chain)
|
||||||
val request = InvoiceRequest(offer, outgoingAmount, 1, Features.empty, payerKey, chain)
|
val request = InvoiceRequest(offer, outgoingAmount, 1, Features.empty, payerKey, chain)
|
||||||
val invoice = Bolt12Invoice(request, randomBytes32(), outgoingNodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(outgoingNodeId)))
|
val invoice = Bolt12Invoice(request, randomBytes32(), outgoingNodeKey, 300 seconds, Features.empty, Seq(createPaymentBlindedRoute(outgoingNodeId)))
|
||||||
val incomingPayments = incomingMultiPart.map(incoming => RelayToBlindedPathsPacket(incoming.add, incoming.outerPayload, IntermediatePayload.NodeRelay.ToBlindedPaths(
|
val incomingPayments = incomingMultiPart.map(incoming => RelayToBlindedPathsPacket(incoming.add, incoming.outerPayload, IntermediatePayload.NodeRelay.ToBlindedPaths(
|
||||||
|
@ -876,7 +876,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||||
import f._
|
import f._
|
||||||
|
|
||||||
val (payerKey, chain) = (randomKey(), BlockHash(randomBytes32()))
|
val (payerKey, chain) = (randomKey(), BlockHash(randomBytes32()))
|
||||||
val offer = Offer(None, "test offer", outgoingNodeId, Features(Features.BasicMultiPartPayment -> FeatureSupport.Optional), chain)
|
val offer = Offer(None, Some("test offer"), outgoingNodeId, Features(Features.BasicMultiPartPayment -> FeatureSupport.Optional), chain)
|
||||||
val request = InvoiceRequest(offer, outgoingAmount, 1, Features(Features.BasicMultiPartPayment -> FeatureSupport.Optional), payerKey, chain)
|
val request = InvoiceRequest(offer, outgoingAmount, 1, Features(Features.BasicMultiPartPayment -> FeatureSupport.Optional), payerKey, chain)
|
||||||
val invoice = Bolt12Invoice(request, randomBytes32(), outgoingNodeKey, 300 seconds, Features(Features.BasicMultiPartPayment -> FeatureSupport.Optional), Seq(createPaymentBlindedRoute(outgoingNodeId)))
|
val invoice = Bolt12Invoice(request, randomBytes32(), outgoingNodeKey, 300 seconds, Features(Features.BasicMultiPartPayment -> FeatureSupport.Optional), Seq(createPaymentBlindedRoute(outgoingNodeId)))
|
||||||
val incomingPayments = incomingMultiPart.map(incoming => RelayToBlindedPathsPacket(incoming.add, incoming.outerPayload, IntermediatePayload.NodeRelay.ToBlindedPaths(
|
val incomingPayments = incomingMultiPart.map(incoming => RelayToBlindedPathsPacket(incoming.add, incoming.outerPayload, IntermediatePayload.NodeRelay.ToBlindedPaths(
|
||||||
|
@ -915,7 +915,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||||
import f._
|
import f._
|
||||||
|
|
||||||
val (payerKey, chain) = (randomKey(), BlockHash(randomBytes32()))
|
val (payerKey, chain) = (randomKey(), BlockHash(randomBytes32()))
|
||||||
val offer = Offer(None, "test offer", outgoingNodeId, Features.empty, chain)
|
val offer = Offer(None, Some("test offer"), outgoingNodeId, Features.empty, chain)
|
||||||
val request = InvoiceRequest(offer, outgoingAmount, 1, Features.empty, payerKey, chain)
|
val request = InvoiceRequest(offer, outgoingAmount, 1, Features.empty, payerKey, chain)
|
||||||
val paymentBlindedRoute = createPaymentBlindedRoute(outgoingNodeId)
|
val paymentBlindedRoute = createPaymentBlindedRoute(outgoingNodeId)
|
||||||
val scidDir = ShortChannelIdDir(isNode1 = true, RealShortChannelId(123456L))
|
val scidDir = ShortChannelIdDir(isNode1 = true, RealShortChannelId(123456L))
|
||||||
|
@ -962,7 +962,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
||||||
import f._
|
import f._
|
||||||
|
|
||||||
val (payerKey, chain) = (randomKey(), BlockHash(randomBytes32()))
|
val (payerKey, chain) = (randomKey(), BlockHash(randomBytes32()))
|
||||||
val offer = Offer(None, "test offer", outgoingNodeId, Features.empty, chain)
|
val offer = Offer(None, Some("test offer"), outgoingNodeId, Features.empty, chain)
|
||||||
val request = InvoiceRequest(offer, outgoingAmount, 1, Features.empty, payerKey, chain)
|
val request = InvoiceRequest(offer, outgoingAmount, 1, Features.empty, payerKey, chain)
|
||||||
val paymentBlindedRoute = createPaymentBlindedRoute(outgoingNodeId)
|
val paymentBlindedRoute = createPaymentBlindedRoute(outgoingNodeId)
|
||||||
val scidDir = ShortChannelIdDir(isNode1 = true, RealShortChannelId(123456L))
|
val scidDir = ShortChannelIdDir(isNode1 = true, RealShortChannelId(123456L))
|
||||||
|
|
|
@ -66,7 +66,7 @@ class OfferPaymentSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
val probe = TestProbe()
|
val probe = TestProbe()
|
||||||
val merchantKey = randomKey()
|
val merchantKey = randomKey()
|
||||||
|
|
||||||
val offer = Offer(None, "amountless offer", merchantKey.publicKey, Features.empty, nodeParams.chainHash)
|
val offer = Offer(None, Some("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))
|
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]
|
val Postman.SendMessage(OfferTypes.RecipientNodeId(recipientId), FindRoute, message, expectsReply, replyTo) = postman.expectMessageType[Postman.SendMessage]
|
||||||
assert(recipientId == merchantKey.publicKey)
|
assert(recipientId == merchantKey.publicKey)
|
||||||
|
@ -90,7 +90,7 @@ class OfferPaymentSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
val probe = TestProbe()
|
val probe = TestProbe()
|
||||||
val merchantKey = randomKey()
|
val merchantKey = randomKey()
|
||||||
|
|
||||||
val offer = Offer(None, "amountless offer", merchantKey.publicKey, Features.empty, nodeParams.chainHash)
|
val offer = Offer(None, Some("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))
|
offerPayment ! PayOffer(probe.ref, offer, 40_000_000 msat, 1, SendPaymentConfig(None, connectDirectly = false, 1, routeParams, blocking = false))
|
||||||
for (_ <- 1 to nodeParams.onionMessageConfig.maxAttempts) {
|
for (_ <- 1 to nodeParams.onionMessageConfig.maxAttempts) {
|
||||||
val Postman.SendMessage(OfferTypes.RecipientNodeId(recipientId), FindRoute, message, expectsReply, replyTo) = postman.expectMessageType[Postman.SendMessage]
|
val Postman.SendMessage(OfferTypes.RecipientNodeId(recipientId), FindRoute, message, expectsReply, replyTo) = postman.expectMessageType[Postman.SendMessage]
|
||||||
|
@ -113,7 +113,7 @@ class OfferPaymentSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
val probe = TestProbe()
|
val probe = TestProbe()
|
||||||
val merchantKey = randomKey()
|
val merchantKey = randomKey()
|
||||||
|
|
||||||
val offer = Offer(None, "amountless offer", merchantKey.publicKey, Features.empty, nodeParams.chainHash)
|
val offer = Offer(None, Some("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))
|
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]
|
val Postman.SendMessage(OfferTypes.RecipientNodeId(recipientId), FindRoute, message, expectsReply, replyTo) = postman.expectMessageType[Postman.SendMessage]
|
||||||
assert(recipientId == merchantKey.publicKey)
|
assert(recipientId == merchantKey.publicKey)
|
||||||
|
@ -138,7 +138,7 @@ class OfferPaymentSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
val probe = TestProbe()
|
val probe = TestProbe()
|
||||||
val merchantKey = randomKey()
|
val merchantKey = randomKey()
|
||||||
|
|
||||||
val offer = Offer(None, "offer", merchantKey.publicKey, Features.empty, nodeParams.chainHash)
|
val offer = Offer(None, Some("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))
|
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]
|
val Postman.SendMessage(OfferTypes.RecipientNodeId(recipientId), FindRoute, message, expectsReply, replyTo) = postman.expectMessageType[Postman.SendMessage]
|
||||||
assert(recipientId == merchantKey.publicKey)
|
assert(recipientId == merchantKey.publicKey)
|
||||||
|
@ -181,4 +181,30 @@ class OfferPaymentSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("app
|
||||||
|
|
||||||
TypedProbe().expectTerminated(offerPayment)
|
TypedProbe().expectTerminated(offerPayment)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
test("offer with implicit node id") { f =>
|
||||||
|
import f._
|
||||||
|
|
||||||
|
val probe = TestProbe()
|
||||||
|
|
||||||
|
val merchantKey = randomKey()
|
||||||
|
val route = RouteBlinding.create(randomKey(), Seq.fill(2)(randomKey().publicKey) :+ merchantKey.publicKey, Seq.fill(3)(randomBytes(10)))
|
||||||
|
val offer = Offer.withPaths(None, Some("implicit node id"), Seq(route.route), Features.empty, nodeParams.chainHash)
|
||||||
|
|
||||||
|
offerPayment ! PayOffer(probe.ref, offer, 10_000_000 msat, 1, SendPaymentConfig(None, connectDirectly = false, 1, routeParams, blocking = false))
|
||||||
|
val Postman.SendMessage(OfferTypes.BlindedPath(blindedRoute), FindRoute, message, expectsReply, replyTo) = postman.expectMessageType[Postman.SendMessage]
|
||||||
|
assert(blindedRoute == route.route)
|
||||||
|
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 blindedMerchantKey = RouteBlinding.derivePrivateKey(merchantKey, route.lastBlinding)
|
||||||
|
val invoice = Bolt12Invoice(invoiceRequest, preimage, blindedMerchantKey, 1 minute, Features.empty, Seq(paymentRoute))
|
||||||
|
replyTo ! Postman.Response(InvoicePayload(TlvStream(OnionMessagePayloadTlv.Invoice(invoice.records)), TlvStream.empty))
|
||||||
|
val send = paymentInitiator.expectMsgType[SendPaymentToNode]
|
||||||
|
assert(send.invoice == invoice)
|
||||||
|
TypedProbe().expectTerminated(offerPayment)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -268,7 +268,7 @@ object BaseRouterSpec {
|
||||||
val features = Features[Bolt12Feature](
|
val features = Features[Bolt12Feature](
|
||||||
Features.BasicMultiPartPayment -> FeatureSupport.Optional,
|
Features.BasicMultiPartPayment -> FeatureSupport.Optional,
|
||||||
)
|
)
|
||||||
val offer = Offer(None, "Bolt12 r0cks", recipientKey.publicKey, features, Block.RegtestGenesisBlock.hash)
|
val offer = Offer(None, Some("Bolt12 r0cks"), recipientKey.publicKey, features, Block.RegtestGenesisBlock.hash)
|
||||||
val invoiceRequest = InvoiceRequest(offer, amount, 1, features, randomKey(), Block.RegtestGenesisBlock.hash)
|
val invoiceRequest = InvoiceRequest(offer, amount, 1, features, randomKey(), Block.RegtestGenesisBlock.hash)
|
||||||
val blindedRoutes = paths.map(hops => {
|
val blindedRoutes = paths.map(hops => {
|
||||||
val blindedRoute = BlindedRouteCreation.createBlindedRouteFromHops(hops, pathId, 1 msat, routeExpiry).route
|
val blindedRoute = BlindedRouteCreation.createBlindedRouteFromHops(hops, pathId, 1 msat, routeExpiry).route
|
||||||
|
|
|
@ -91,7 +91,7 @@ class MessageOnionCodecsSpec extends AnyFunSuiteLike {
|
||||||
|
|
||||||
test("validate final payloads") {
|
test("validate final payloads") {
|
||||||
val nodeKey = randomKey()
|
val nodeKey = randomKey()
|
||||||
val offer = OfferTypes.Offer(Some(100_000 msat), "test offer", nodeKey.publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
val offer = OfferTypes.Offer(Some(100_000 msat), Some("test offer"), nodeKey.publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
||||||
val payerKey = randomKey()
|
val payerKey = randomKey()
|
||||||
val request = OfferTypes.InvoiceRequest(offer, 100_000 msat, 1, Features.empty, payerKey, Block.LivenetGenesisBlock.hash)
|
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 selfPayload = blindedRouteDataCodec.encode(TlvStream(PathId(randomBytes32()), PaymentConstraints(CltvExpiry(1234567), 0 msat), AllowedFeatures(Features.empty))).require.bytes
|
||||||
|
|
|
@ -34,22 +34,20 @@ class OfferTypesSpec extends AnyFunSuite {
|
||||||
|
|
||||||
test("invoice request is signed") {
|
test("invoice request is signed") {
|
||||||
val sellerKey = randomKey()
|
val sellerKey = randomKey()
|
||||||
val offer = Offer(Some(100_000 msat), "test offer", sellerKey.publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
val offer = Offer(Some(100_000 msat), Some("test offer"), sellerKey.publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
||||||
val payerKey = randomKey()
|
val payerKey = randomKey()
|
||||||
val request = InvoiceRequest(offer, 100_000 msat, 1, Features.empty, payerKey, Block.LivenetGenesisBlock.hash)
|
val request = InvoiceRequest(offer, 100_000 msat, 1, Features.empty, payerKey, Block.LivenetGenesisBlock.hash)
|
||||||
assert(request.checkSignature())
|
assert(request.checkSignature())
|
||||||
}
|
}
|
||||||
|
|
||||||
test("minimal offer") {
|
test("minimal offer") {
|
||||||
val tlvs = Set[OfferTlv](
|
val tlvs = Set[OfferTlv](OfferNodeId(nodeId))
|
||||||
OfferDescription("basic offer"),
|
|
||||||
OfferNodeId(nodeId))
|
|
||||||
val offer = Offer(TlvStream(tlvs))
|
val offer = Offer(TlvStream(tlvs))
|
||||||
val encoded = "lno1pg9kyctnd93jqmmxvejhy93pqvxl9c6mjgkeaxa6a0vtxqteql688v0ywa8qqwx4j05cyskn8ncrj"
|
val encoded = "lno1zcssxr0juddeytv7nwawhk9nq9us0arnk8j8wnsq8r2e86vzgtfneupe"
|
||||||
assert(Offer.decode(encoded).get == offer)
|
assert(Offer.decode(encoded).get == offer)
|
||||||
assert(offer.amount.isEmpty)
|
assert(offer.amount.isEmpty)
|
||||||
assert(offer.description == "basic offer")
|
assert(offer.description.isEmpty)
|
||||||
assert(offer.nodeId == nodeId)
|
assert(offer.nodeId.contains(nodeId))
|
||||||
// Removing any TLV from the minimal offer makes it invalid.
|
// Removing any TLV from the minimal offer makes it invalid.
|
||||||
for (tlv <- tlvs) {
|
for (tlv <- tlvs) {
|
||||||
val incomplete = TlvStream[OfferTlv](tlvs.filterNot(_ == tlv))
|
val incomplete = TlvStream[OfferTlv](tlvs.filterNot(_ == tlv))
|
||||||
|
@ -70,8 +68,8 @@ class OfferTypesSpec extends AnyFunSuite {
|
||||||
val encoded = "lno1qgsyxjtl6luzd9t3pr62xr7eemp6awnejusgf6gw45q75vcfqqqqqqqgqyeq5ym0venx2u3qwa5hg6pqw96kzmn5d968jys3v9kxjcm9gp3xjemndphhqtnrdak3gqqkyypsmuhrtwfzm85mht4a3vcp0yrlgua3u3m5uqpc6kf7nqjz6v70qwg"
|
val encoded = "lno1qgsyxjtl6luzd9t3pr62xr7eemp6awnejusgf6gw45q75vcfqqqqqqqgqyeq5ym0venx2u3qwa5hg6pqw96kzmn5d968jys3v9kxjcm9gp3xjemndphhqtnrdak3gqqkyypsmuhrtwfzm85mht4a3vcp0yrlgua3u3m5uqpc6kf7nqjz6v70qwg"
|
||||||
assert(Offer.decode(encoded).get == offer)
|
assert(Offer.decode(encoded).get == offer)
|
||||||
assert(offer.amount.contains(50 msat))
|
assert(offer.amount.contains(50 msat))
|
||||||
assert(offer.description == "offer with quantity")
|
assert(offer.description.contains("offer with quantity"))
|
||||||
assert(offer.nodeId == nodeId)
|
assert(offer.nodeId.contains(nodeId))
|
||||||
assert(offer.issuer.contains("alice@bigshop.com"))
|
assert(offer.issuer.contains("alice@bigshop.com"))
|
||||||
assert(offer.quantityMax.contains(Long.MaxValue))
|
assert(offer.quantityMax.contains(Long.MaxValue))
|
||||||
}
|
}
|
||||||
|
@ -85,7 +83,7 @@ class OfferTypesSpec extends AnyFunSuite {
|
||||||
}
|
}
|
||||||
|
|
||||||
test("check that invoice request matches offer") {
|
test("check that invoice request matches offer") {
|
||||||
val offer = Offer(Some(2500 msat), "basic offer", randomKey().publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
val offer = Offer(Some(2500 msat), Some("basic offer"), randomKey().publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
||||||
val payerKey = randomKey()
|
val payerKey = randomKey()
|
||||||
val request = InvoiceRequest(offer, 2500 msat, 1, Features.empty, payerKey, Block.LivenetGenesisBlock.hash)
|
val request = InvoiceRequest(offer, 2500 msat, 1, Features.empty, payerKey, Block.LivenetGenesisBlock.hash)
|
||||||
assert(request.isValid)
|
assert(request.isValid)
|
||||||
|
@ -100,7 +98,7 @@ class OfferTypesSpec extends AnyFunSuite {
|
||||||
}
|
}
|
||||||
|
|
||||||
test("check that invoice request matches offer (with features)") {
|
test("check that invoice request matches offer (with features)") {
|
||||||
val offer = Offer(Some(2500 msat), "offer with features", randomKey().publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
val offer = Offer(Some(2500 msat), Some("offer with features"), randomKey().publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
||||||
val payerKey = randomKey()
|
val payerKey = randomKey()
|
||||||
val request = InvoiceRequest(offer, 2500 msat, 1, Features(BasicMultiPartPayment -> Optional), payerKey, Block.LivenetGenesisBlock.hash)
|
val request = InvoiceRequest(offer, 2500 msat, 1, Features(BasicMultiPartPayment -> Optional), payerKey, Block.LivenetGenesisBlock.hash)
|
||||||
assert(request.isValid)
|
assert(request.isValid)
|
||||||
|
@ -114,7 +112,7 @@ class OfferTypesSpec extends AnyFunSuite {
|
||||||
}
|
}
|
||||||
|
|
||||||
test("check that invoice request matches offer (without amount)") {
|
test("check that invoice request matches offer (without amount)") {
|
||||||
val offer = Offer(None, "offer without amount", randomKey().publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
val offer = Offer(None, Some("offer without amount"), randomKey().publicKey, Features.empty, Block.LivenetGenesisBlock.hash)
|
||||||
val payerKey = randomKey()
|
val payerKey = randomKey()
|
||||||
val request = InvoiceRequest(offer, 500 msat, 1, Features.empty, payerKey, Block.LivenetGenesisBlock.hash)
|
val request = InvoiceRequest(offer, 500 msat, 1, Features.empty, payerKey, Block.LivenetGenesisBlock.hash)
|
||||||
assert(request.isValid)
|
assert(request.isValid)
|
||||||
|
@ -183,18 +181,17 @@ class OfferTypesSpec extends AnyFunSuite {
|
||||||
val payerKey = PrivateKey(hex"527d410ec920b626ece685e8af9abc976a48dbf2fe698c1b35d90a1c5fa2fbca")
|
val payerKey = PrivateKey(hex"527d410ec920b626ece685e8af9abc976a48dbf2fe698c1b35d90a1c5fa2fbca")
|
||||||
val tlvsWithoutSignature = Set[InvoiceRequestTlv](
|
val tlvsWithoutSignature = Set[InvoiceRequestTlv](
|
||||||
InvoiceRequestMetadata(hex"abcdef"),
|
InvoiceRequestMetadata(hex"abcdef"),
|
||||||
OfferDescription("basic offer"),
|
|
||||||
OfferNodeId(nodeId),
|
OfferNodeId(nodeId),
|
||||||
InvoiceRequestPayerId(payerKey.publicKey),
|
InvoiceRequestPayerId(payerKey.publicKey),
|
||||||
)
|
)
|
||||||
val signature = signSchnorr(InvoiceRequest.signatureTag, rootHash(TlvStream[InvoiceRequestTlv](tlvsWithoutSignature), OfferCodecs.invoiceRequestTlvCodec), payerKey)
|
val signature = signSchnorr(InvoiceRequest.signatureTag, rootHash(TlvStream[InvoiceRequestTlv](tlvsWithoutSignature), OfferCodecs.invoiceRequestTlvCodec), payerKey)
|
||||||
val tlvs = tlvsWithoutSignature + Signature(signature)
|
val tlvs = tlvsWithoutSignature + Signature(signature)
|
||||||
val invoiceRequest = InvoiceRequest(TlvStream(tlvs))
|
val invoiceRequest = InvoiceRequest(TlvStream(tlvs))
|
||||||
val encoded = "lnr1qqp6hn00pg9kyctnd93jqmmxvejhy93pqvxl9c6mjgkeaxa6a0vtxqteql688v0ywa8qqwx4j05cyskn8ncrjkppqfxajawru7sa7rt300hfzs2lyk2jrxduxrkx9lmzy6lxcvfhk0j7ruzqc4mtjj5fwukrqp7faqrxn664nmwykad76pu997terewcklsx47apag59wf8exly4tky7y63prr7450n28stqssmzuf48w7e6rjad2eq"
|
val encoded = "lnr1qqp6hn00zcssxr0juddeytv7nwawhk9nq9us0arnk8j8wnsq8r2e86vzgtfneupetqssynwewhp70gwlp4chhm53g90jt9fpnx7rpmrzla3zd0nvxymm8e0p7pq06rwacy8756zgl3hdnsyfepq573astyz94rgn9uhxlyqj4gdyk6q8q0yrv6al909v3435amuvjqvkuq6k8fyld78r8srdyx7wnmwsdu"
|
||||||
assert(InvoiceRequest.decode(encoded).get == invoiceRequest)
|
assert(InvoiceRequest.decode(encoded).get == invoiceRequest)
|
||||||
assert(invoiceRequest.offer.amount.isEmpty)
|
assert(invoiceRequest.offer.amount.isEmpty)
|
||||||
assert(invoiceRequest.offer.description == "basic offer")
|
assert(invoiceRequest.offer.description.isEmpty)
|
||||||
assert(invoiceRequest.offer.nodeId == nodeId)
|
assert(invoiceRequest.offer.nodeId.contains(nodeId))
|
||||||
assert(invoiceRequest.metadata == hex"abcdef")
|
assert(invoiceRequest.metadata == hex"abcdef")
|
||||||
assert(invoiceRequest.payerId == payerKey.publicKey)
|
assert(invoiceRequest.payerId == payerKey.publicKey)
|
||||||
// Removing any TLV from the minimal invoice request makes it invalid.
|
// Removing any TLV from the minimal invoice request makes it invalid.
|
||||||
|
|
|
@ -1232,14 +1232,15 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM
|
||||||
system.eventStream.publish(chcl)
|
system.eventStream.publish(chcl)
|
||||||
wsClient.expectMessage(expectedSerializedChcl)
|
wsClient.expectMessage(expectedSerializedChcl)
|
||||||
|
|
||||||
|
val replyPath = Sphinx.RouteBlinding.create(PrivateKey(hex"414141414141414141414141414141414141414141414141414141414141414101"), Seq(bobNodeId), Seq(hex"000000"))
|
||||||
val Right(payload) = MessageOnion.FinalPayload.validate(TlvStream[OnionMessagePayloadTlv](
|
val Right(payload) = MessageOnion.FinalPayload.validate(TlvStream[OnionMessagePayloadTlv](
|
||||||
Set[OnionMessagePayloadTlv](
|
Set[OnionMessagePayloadTlv](
|
||||||
OnionMessagePayloadTlv.EncryptedData(ByteVector.empty),
|
OnionMessagePayloadTlv.EncryptedData(ByteVector.empty),
|
||||||
OnionMessagePayloadTlv.ReplyPath(Sphinx.RouteBlinding.create(PrivateKey(hex"414141414141414141414141414141414141414141414141414141414141414101"), Seq(bobNodeId), Seq(hex"000000")).route)
|
OnionMessagePayloadTlv.ReplyPath(replyPath.route)
|
||||||
), Set(
|
), Set(
|
||||||
GenericTlv(UInt64(5), hex"1111")
|
GenericTlv(UInt64(5), hex"1111")
|
||||||
)), TlvStream(RouteBlindingEncryptedDataTlv.PathId(hex"2222")))
|
)), TlvStream(RouteBlindingEncryptedDataTlv.PathId(hex"2222")))
|
||||||
val msgrcv = OnionMessages.ReceiveMessage(payload)
|
val msgrcv = OnionMessages.ReceiveMessage(payload, PrivateKey(hex"515151515151515151515151515151515151515151515151515151515151515101"))
|
||||||
val expectedSerializedMsgrcv = """{"type":"onion-message-received","pathId":"2222","tlvs":{"EncryptedData":{"data":""},"ReplyPath":{"blindedRoute":{"introductionNodeId":{"publicKey":"039dc0e0b1d25905e44fdf6f8e89755a5e219685840d0bc1d28d3308f9628a3585"},"blindingKey":"02eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619","blindedNodes":[{"blindedPublicKey":"020303f91e620504cde242df38d04599d8b4d4c555149cc742a5f12de452cbdd40","encryptedPayload":"126a26221759247584d704b382a5789f1d8c5a"}]}},"Unknown5":"1111"}}"""
|
val expectedSerializedMsgrcv = """{"type":"onion-message-received","pathId":"2222","tlvs":{"EncryptedData":{"data":""},"ReplyPath":{"blindedRoute":{"introductionNodeId":{"publicKey":"039dc0e0b1d25905e44fdf6f8e89755a5e219685840d0bc1d28d3308f9628a3585"},"blindingKey":"02eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619","blindedNodes":[{"blindedPublicKey":"020303f91e620504cde242df38d04599d8b4d4c555149cc742a5f12de452cbdd40","encryptedPayload":"126a26221759247584d704b382a5789f1d8c5a"}]}},"Unknown5":"1111"}}"""
|
||||||
assert(serialization.write(msgrcv) == expectedSerializedMsgrcv)
|
assert(serialization.write(msgrcv) == expectedSerializedMsgrcv)
|
||||||
system.eventStream.publish(msgrcv)
|
system.eventStream.publish(msgrcv)
|
||||||
|
|
Loading…
Add table
Reference in a new issue