Skip to content

Commit

Permalink
Refactor trampoline-to-legacy payments (#2948)
Browse files Browse the repository at this point in the history
We refactor trampoline-to-legacy payments to use a dedicated class, like
what we do for trampoline-to-blinded-paths payments. This allows us to
supports two encodings for those payments:

- one where the trampoline onion contains a dummy payload for the
  recipient that must be ignored (current Phoenix wallets), which
  wastes space in the onion for legacy reasons
- one where we don't include a dummy payload for the recipient, which
  is more efficient and similar to trampoline-to-blinded-paths
  • Loading branch information
t-bast authored Nov 27, 2024
1 parent a624b82 commit ab94128
Show file tree
Hide file tree
Showing 7 changed files with 168 additions and 96 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import fr.acinq.eclair.channel.{CMD_ADD_HTLC, CMD_FAIL_HTLC, CannotExtractShared
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.payment.send.Recipient
import fr.acinq.eclair.router.Router.Route
import fr.acinq.eclair.wire.protocol.OnionPaymentPayloadTlv.OutgoingBlindedPaths
import fr.acinq.eclair.wire.protocol.OnionPaymentPayloadTlv.{InvoiceRoutingInfo, OutgoingBlindedPaths}
import fr.acinq.eclair.wire.protocol.PaymentOnion.{FinalPayload, IntermediatePayload, PerHopPayload}
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, Feature, Features, MilliSatoshi, ShortChannelId, UInt64, randomKey}
Expand Down Expand Up @@ -59,6 +59,7 @@ object IncomingPaymentPacket {
def innerPayload: IntermediatePayload.NodeRelay
}
case class RelayToTrampolinePacket(add: UpdateAddHtlc, outerPayload: FinalPayload.Standard, innerPayload: IntermediatePayload.NodeRelay.Standard, nextPacket: OnionRoutingPacket) extends NodeRelayPacket
case class RelayToNonTrampolinePacket(add: UpdateAddHtlc, outerPayload: FinalPayload.Standard, innerPayload: IntermediatePayload.NodeRelay.ToNonTrampoline) extends NodeRelayPacket
case class RelayToBlindedPathsPacket(add: UpdateAddHtlc, outerPayload: FinalPayload.Standard, innerPayload: IntermediatePayload.NodeRelay.ToBlindedPaths) extends NodeRelayPacket
// @formatter:on

Expand Down Expand Up @@ -113,16 +114,18 @@ object IncomingPaymentPacket {
* @return whether the payment is to be relayed or if our node is the final recipient (or an error).
*/
def decrypt(add: UpdateAddHtlc, privateKey: PrivateKey, features: Features[Feature]): Either[FailureMessage, IncomingPaymentPacket] = {
// We first derive the decryption key used to peel the onion.
// We first derive the decryption key used to peel the outer onion.
val outerOnionDecryptionKey = add.blinding_opt match {
case Some(blinding) => Sphinx.RouteBlinding.derivePrivateKey(privateKey, blinding)
case None => privateKey
}
decryptOnion(add.paymentHash, outerOnionDecryptionKey, add.onionRoutingPacket).flatMap {
case DecodedOnionPacket(payload, Some(nextPacket)) =>
// We are an intermediate node: we need to relay to one of our peers.
payload.get[OnionPaymentPayloadTlv.EncryptedRecipientData] match {
case Some(_) if !features.hasFeature(Features.RouteBlinding) => Left(InvalidOnionPayload(UInt64(10), 0))
case Some(encrypted) =>
// We are inside a blinded path: channel relay information is encrypted.
decryptEncryptedRecipientData(add, privateKey, payload, encrypted.data).flatMap {
case DecodedEncryptedRecipientData(blindedPayload, nextBlinding) =>
validateBlindedChannelRelayPayload(add, payload, blindedPayload, nextBlinding, nextPacket).flatMap {
Expand All @@ -132,14 +135,18 @@ object IncomingPaymentPacket {
}
}
case None if add.blinding_opt.isDefined => Left(InvalidOnionBlinding(Sphinx.hash(add.onionRoutingPacket)))
case None => IntermediatePayload.ChannelRelay.Standard.validate(payload).left.map(_.failureMessage).map {
payload => ChannelRelayPacket(add, payload, nextPacket)
}
case None =>
// We are not inside a blinded path: channel relay information is directly available.
IntermediatePayload.ChannelRelay.Standard.validate(payload).left.map(_.failureMessage).map(payload => ChannelRelayPacket(add, payload, nextPacket))
}
case DecodedOnionPacket(payload, None) =>
// We are the final node for the outer onion, so we are either:
// - the final recipient of the payment.
// - an intermediate trampoline node.
payload.get[OnionPaymentPayloadTlv.EncryptedRecipientData] match {
case Some(_) if !features.hasFeature(Features.RouteBlinding) => Left(InvalidOnionPayload(UInt64(10), 0))
case Some(encrypted) =>
// We are the final recipient of a blinded payment.
decryptEncryptedRecipientData(add, privateKey, payload, encrypted.data).flatMap {
case DecodedEncryptedRecipientData(blindedPayload, _) => validateBlindedFinalPayload(add, payload, blindedPayload)
}
Expand All @@ -151,15 +158,33 @@ object IncomingPaymentPacket {
// NB: when we enable blinded trampoline routes, we will need to check if the outer onion contains a
// blinding point and use it to derive the decryption key for the blinded trampoline onion.
decryptOnion(add.paymentHash, privateKey, trampolinePacket).flatMap {
case DecodedOnionPacket(innerPayload, Some(next)) => validateNodeRelay(add, payload, innerPayload, next)
case DecodedOnionPacket(innerPayload, Some(next)) =>
// We are an intermediate trampoline node.
if (innerPayload.get[InvoiceRoutingInfo].isDefined) {
// The payment recipient doesn't support trampoline.
// They can be reached with the invoice data provided.
// The payer is a wallet using the legacy trampoline feature.
validateTrampolineToNonTrampoline(add, payload, innerPayload)
} else {
validateNodeRelay(add, payload, innerPayload, next)
}
case DecodedOnionPacket(innerPayload, None) =>
if (innerPayload.get[OutgoingBlindedPaths].isDefined) {
// The payment recipient doesn't support trampoline.
// They can be reached using the blinded paths provided.
validateTrampolineToBlindedPaths(add, payload, innerPayload)
} else if (innerPayload.get[InvoiceRoutingInfo].isDefined) {
// The payment recipient doesn't support trampoline.
// They can be reached with the invoice data provided.
validateTrampolineToNonTrampoline(add, payload, innerPayload)
} else {
// We're the final recipient of this trampoline payment.
validateTrampolineFinalPayload(add, payload, innerPayload)
}
}
case None => validateFinalPayload(add, payload)
case None =>
// We are the final recipient of a standard (non-blinded, non-trampoline) payment.
validateFinalPayload(add, payload)
}
}
}
Expand Down Expand Up @@ -224,6 +249,16 @@ object IncomingPaymentPacket {
}
}

private def validateTrampolineToNonTrampoline(add: UpdateAddHtlc, outerPayload: TlvStream[OnionPaymentPayloadTlv], innerPayload: TlvStream[OnionPaymentPayloadTlv]): Either[FailureMessage, RelayToNonTrampolinePacket] = {
FinalPayload.Standard.validate(outerPayload).left.map(_.failureMessage).flatMap { outerPayload =>
IntermediatePayload.NodeRelay.ToNonTrampoline.validate(innerPayload).left.map(_.failureMessage).flatMap {
case _ if add.amountMsat < outerPayload.amount => Left(FinalIncorrectHtlcAmount(add.amountMsat))
case _ if add.cltvExpiry != outerPayload.expiry => Left(FinalIncorrectCltvExpiry(add.cltvExpiry))
case innerPayload => Right(RelayToNonTrampolinePacket(add, outerPayload, innerPayload))
}
}
}

private def validateTrampolineToBlindedPaths(add: UpdateAddHtlc, outerPayload: TlvStream[OnionPaymentPayloadTlv], innerPayload: TlvStream[OnionPaymentPayloadTlv]): Either[FailureMessage, RelayToBlindedPathsPacket] = {
FinalPayload.Standard.validate(outerPayload).left.map(_.failureMessage).flatMap { outerPayload =>
IntermediatePayload.NodeRelay.ToBlindedPaths.validate(innerPayload).left.map(_.failureMessage).flatMap {
Expand Down Expand Up @@ -255,10 +290,9 @@ object OutgoingPaymentPacket {
case class PaymentPayloads(amount: MilliSatoshi, expiry: CltvExpiry, payloads: Seq[NodePayload], outerBlinding_opt: Option[PublicKey])

sealed trait OutgoingPaymentError extends Throwable
case class CannotCreateOnion(message: String) extends OutgoingPaymentError { override def getMessage: String = message }
private case class CannotCreateOnion(message: String) extends OutgoingPaymentError { override def getMessage: String = message }
case class InvalidRouteRecipient(expected: PublicKey, actual: PublicKey) extends OutgoingPaymentError { override def getMessage: String = s"expected route to $expected, got route to $actual" }
case class IndirectRelayInBlindedRoute(nextNodeId: PublicKey) extends OutgoingPaymentError { override def getMessage: String = s"must relay directly to node $nextNodeId inside blinded route" }
case class MissingTrampolineHop(trampolineNodeId: PublicKey) extends OutgoingPaymentError { override def getMessage: String = s"expected route to trampoline node $trampolineNodeId" }
case class MissingBlindedHop(introductionNodeIds: Set[PublicKey]) extends OutgoingPaymentError { override def getMessage: String = s"expected blinded route using one of the following introduction nodes: ${introductionNodeIds.mkString(", ")}" }
case object EmptyRoute extends OutgoingPaymentError { override def getMessage: String = "route cannot be empty" }
// @formatter:on
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,7 @@ object NodeRelay {
val incomingPaymentHandler = context.actorOf(MultiPartPaymentFSM.props(nodeParams, paymentHash, totalAmountIn, mppFsmAdapters))
val nextPacket_opt = nodeRelayPacket match {
case IncomingPaymentPacket.RelayToTrampolinePacket(_, _, _, nextPacket) => Some(nextPacket)
case _: IncomingPaymentPacket.RelayToNonTrampolinePacket => None
case _: IncomingPaymentPacket.RelayToBlindedPathsPacket => None
}
new NodeRelay(nodeParams, parent, register, relayId, paymentHash, nodeRelayPacket.outerPayload.paymentSecret, context, outgoingPaymentFactory, router)
Expand All @@ -126,12 +127,7 @@ object NodeRelay {
} else if (payloadOut.amountToForward <= MilliSatoshi(0)) {
Some(InvalidOnionPayload(UInt64(2), 0))
} else {
payloadOut match {
// If we're relaying a standard payment to a non-trampoline recipient, we need the payment secret.
case payloadOut: IntermediatePayload.NodeRelay.Standard if payloadOut.invoiceFeatures.isDefined && payloadOut.paymentSecret.isEmpty => Some(InvalidOnionPayload(UInt64(8), 0))
case _: IntermediatePayload.NodeRelay.Standard => None
case _: IntermediatePayload.NodeRelay.ToBlindedPaths => None
}
None
}
}

Expand Down Expand Up @@ -193,6 +189,7 @@ object NodeRelay {
// Otherwise, we try to find a downstream error that we could decrypt.
val outgoingNodeFailure = nextPayload match {
case nextPayload: IntermediatePayload.NodeRelay.Standard => failures.collectFirst { case RemoteFailure(_, _, e) if e.originNode == nextPayload.outgoingNodeId => e.failureMessage }
case nextPayload: IntermediatePayload.NodeRelay.ToNonTrampoline => failures.collectFirst { case RemoteFailure(_, _, e) if e.originNode == nextPayload.outgoingNodeId => e.failureMessage }
// When using blinded paths, we will never get a failure from the final node (for privacy reasons).
case _: IntermediatePayload.NodeRelay.ToBlindedPaths => None
}
Expand Down Expand Up @@ -257,20 +254,17 @@ class NodeRelay private(nodeParams: NodeParams,
private def resolveNextNode(upstream: Upstream.Hot.Trampoline, nextPayload: IntermediatePayload.NodeRelay, nextPacket_opt: Option[OnionRoutingPacket]): Behavior[Command] = {
nextPayload match {
case payloadOut: IntermediatePayload.NodeRelay.Standard =>
// If invoice features are provided in the onion, the sender is asking us to relay to a non-trampoline recipient.
payloadOut.invoiceFeatures match {
case Some(features) =>
val extraEdges = payloadOut.invoiceRoutingInfo.getOrElse(Nil).flatMap(Bolt11Invoice.toExtraEdges(_, payloadOut.outgoingNodeId))
val paymentSecret = payloadOut.paymentSecret.get // NB: we've verified that there was a payment secret in validateRelay
val recipient = ClearRecipient(payloadOut.outgoingNodeId, Features(features).invoiceFeatures(), payloadOut.amountToForward, payloadOut.outgoingCltv, paymentSecret, extraEdges, payloadOut.paymentMetadata)
context.log.debug("forwarding payment to non-trampoline recipient {}", recipient.nodeId)
ensureRecipientReady(upstream, recipient, nextPayload, None)
case None =>
val paymentSecret = randomBytes32() // we generate a new secret to protect against probing attacks
val recipient = ClearRecipient(payloadOut.outgoingNodeId, Features.empty, payloadOut.amountToForward, payloadOut.outgoingCltv, paymentSecret, nextTrampolineOnion_opt = nextPacket_opt)
context.log.debug("forwarding payment to the next trampoline node {}", recipient.nodeId)
ensureRecipientReady(upstream, recipient, nextPayload, nextPacket_opt)
}
val paymentSecret = randomBytes32() // we generate a new secret to protect against probing attacks
val recipient = ClearRecipient(payloadOut.outgoingNodeId, Features.empty, payloadOut.amountToForward, payloadOut.outgoingCltv, paymentSecret, nextTrampolineOnion_opt = nextPacket_opt)
context.log.debug("forwarding payment to the next trampoline node {}", recipient.nodeId)
ensureRecipientReady(upstream, recipient, nextPayload, nextPacket_opt)
case payloadOut: IntermediatePayload.NodeRelay.ToNonTrampoline =>
val paymentSecret = payloadOut.paymentSecret
val features = Features(payloadOut.invoiceFeatures).invoiceFeatures()
val extraEdges = payloadOut.invoiceRoutingInfo.flatMap(Bolt11Invoice.toExtraEdges(_, payloadOut.outgoingNodeId))
val recipient = ClearRecipient(payloadOut.outgoingNodeId, features, payloadOut.amountToForward, payloadOut.outgoingCltv, paymentSecret, extraEdges, payloadOut.paymentMetadata)
context.log.debug("forwarding payment to non-trampoline recipient {}", recipient.nodeId)
ensureRecipientReady(upstream, recipient, nextPayload, None)
case payloadOut: IntermediatePayload.NodeRelay.ToBlindedPaths =>
// Blinded paths in Bolt 12 invoices may encode the introduction node with an scid and a direction: we need to
// resolve that to a nodeId in order to reach that introduction node and use the blinded path.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ import fr.acinq.eclair.payment.PaymentSent.PartialPayment
import fr.acinq.eclair.payment._
import fr.acinq.eclair.router.Router.RouteParams
import fr.acinq.eclair.wire.protocol.{PaymentOnion, PaymentOnionCodecs}
import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, Features, Logs, MilliSatoshi, MilliSatoshiLong, NodeParams, ShortChannelId, randomBytes32}
import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, Features, Logs, MilliSatoshi, NodeParams, randomBytes32}

import java.util.UUID

Expand Down Expand Up @@ -216,11 +216,8 @@ object TrampolinePayment {
val trampolinePayload = PaymentOnion.IntermediatePayload.NodeRelay.Standard(totalAmount, expiry, invoice.nodeId)
buildOnion(NodePayload(trampolineNodeId, trampolinePayload) :: NodePayload(invoice.nodeId, finalPayload) :: Nil, invoice.paymentHash, None).toOption.get
case invoice: Bolt11Invoice =>
// The recipient doesn't support trampoline: the trampoline node will convert the payment to a non-trampoline payment.
// The final payload will thus never reach the recipient, so we create the smallest payload possible to avoid overflowing the trampoline onion size.
val dummyPayload = PaymentOnion.IntermediatePayload.ChannelRelay.Standard(ShortChannelId(0), 0 msat, CltvExpiry(0))
val trampolinePayload = PaymentOnion.IntermediatePayload.NodeRelay.Standard.createNodeRelayToNonTrampolinePayload(totalAmount, totalAmount, expiry, invoice.nodeId, invoice)
buildOnion(NodePayload(trampolineNodeId, trampolinePayload) :: NodePayload(invoice.nodeId, dummyPayload) :: Nil, invoice.paymentHash, None).toOption.get
val trampolinePayload = PaymentOnion.IntermediatePayload.NodeRelay.ToNonTrampoline(totalAmount, totalAmount, expiry, invoice.nodeId, invoice)
buildOnion(NodePayload(trampolineNodeId, trampolinePayload) :: Nil, invoice.paymentHash, None).toOption.get
case invoice: Bolt12Invoice =>
val trampolinePayload = PaymentOnion.IntermediatePayload.NodeRelay.ToBlindedPaths(totalAmount, expiry, invoice)
buildOnion(NodePayload(trampolineNodeId, trampolinePayload) :: Nil, invoice.paymentHash, None).toOption.get
Expand Down
Loading

0 comments on commit ab94128

Please sign in to comment.