mirror of
https://github.com/ACINQ/eclair.git
synced 2025-03-13 11:35:47 +01:00
Check peer features before attempting wake-up
Before relaying a trampoline payment, we check if the next node is one of our direct peers. If that's the case, we check their features to see if they support waking up through notifications, in which case we go through the wake-up flow. Note that we don't need this for channel relay, because: - if it is a Bolt 12 payment, the recipient will use a `wallet_node_id` to let us know that they support wake-up notifications - if it is a Bolt 11 payment, we use a custom `short_channel_id` for phoenix that also lets us know that they support wake-up notifications
This commit is contained in:
parent
0c9b78cb9a
commit
9e98aa7a6b
3 changed files with 56 additions and 39 deletions
|
@ -305,6 +305,12 @@ object Features {
|
|||
val mandatory = 54
|
||||
}
|
||||
|
||||
/** This feature bit indicates that the node is a mobile wallet that can be woken up via push notifications. */
|
||||
case object WakeUpNotificationClient extends Feature with InitFeature {
|
||||
val rfcName = "wake_up_notification_client"
|
||||
val mandatory = 132
|
||||
}
|
||||
|
||||
// TODO: @t-bast: update feature bits once spec-ed (currently reserved here: https://github.com/lightningnetwork/lightning-rfc/issues/605)
|
||||
// We're not advertising these bits yet in our announcements, clients have to assume support.
|
||||
// This is why we haven't added them yet to `areSupported`.
|
||||
|
@ -369,6 +375,7 @@ object Features {
|
|||
PaymentMetadata,
|
||||
ZeroConf,
|
||||
KeySend,
|
||||
WakeUpNotificationClient,
|
||||
TrampolinePaymentPrototype,
|
||||
AsyncPaymentPrototype,
|
||||
SplicePrototype,
|
||||
|
|
|
@ -66,6 +66,7 @@ object NodeRelay {
|
|||
private case class WrappedPaymentFailed(paymentFailed: PaymentFailed) extends Command
|
||||
private case class WrappedPeerReadyResult(result: PeerReadyNotifier.Result) extends Command
|
||||
private case class WrappedResolvedPaths(resolved: Seq[ResolvedPath]) extends Command
|
||||
private case class WrappedPeerInfo(isPeer: Boolean, remoteFeatures_opt: Option[Features[InitFeature]]) extends Command
|
||||
private case class WrappedOnTheFlyFundingResponse(result: Peer.ProposeOnTheFlyFundingResponse) extends Command
|
||||
// @formatter:on
|
||||
|
||||
|
@ -137,24 +138,6 @@ object NodeRelay {
|
|||
}
|
||||
}
|
||||
|
||||
/** This function identifies whether the next node is a wallet node directly connected to us, and returns its node_id. */
|
||||
private def nextWalletNodeId(nodeParams: NodeParams, recipient: Recipient): Option[PublicKey] = {
|
||||
recipient match {
|
||||
// This recipient is only used when we're the payment initiator.
|
||||
case _: SpontaneousRecipient => None
|
||||
// When relaying to a trampoline node, the next node may be a wallet node directly connected to us, but we don't
|
||||
// want to have false positives. Feature branches should check an internal DB/cache to confirm.
|
||||
case r: ClearRecipient if r.nextTrampolineOnion_opt.nonEmpty => None
|
||||
// If we're relaying to a non-trampoline recipient, it's never a wallet node.
|
||||
case _: ClearRecipient => None
|
||||
// When using blinded paths, we may be the introduction node for a wallet node directly connected to us.
|
||||
case r: BlindedRecipient => r.blindedHops.head.resolved.route match {
|
||||
case BlindedPathsResolver.PartialBlindedRoute(walletNodeId: EncodedNodeId.WithPublicKey.Wallet, _, _) => Some(walletNodeId.publicKey)
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Compute route params that honor our fee and cltv requirements. */
|
||||
private def computeRouteParams(nodeParams: NodeParams, amountIn: MilliSatoshi, expiryIn: CltvExpiry, amountOut: MilliSatoshi, expiryOut: CltvExpiry): RouteParams = {
|
||||
nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams
|
||||
|
@ -264,14 +247,14 @@ class NodeRelay private(nodeParams: NodeParams,
|
|||
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)
|
||||
checkIfRecipientIsWallet(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)
|
||||
checkIfRecipientIsWallet(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.
|
||||
|
@ -287,21 +270,40 @@ class NodeRelay private(nodeParams: NodeParams,
|
|||
// We don't have access to the invoice: we use the only node_id that somewhat makes sense for the recipient.
|
||||
val blindedNodeId = resolved.head.route.blindedNodeIds.last
|
||||
val recipient = BlindedRecipient.fromPaths(blindedNodeId, Features(payloadOut.invoiceFeatures).invoiceFeatures(), payloadOut.amountToForward, payloadOut.outgoingCltv, resolved, Set.empty)
|
||||
context.log.debug("forwarding payment to blinded recipient {}", recipient.nodeId)
|
||||
ensureRecipientReady(upstream, recipient, nextPayload, nextPacket_opt)
|
||||
resolved.head.route match {
|
||||
case BlindedPathsResolver.PartialBlindedRoute(walletNodeId: EncodedNodeId.WithPublicKey.Wallet, _, _) if nodeParams.peerWakeUpConfig.enabled =>
|
||||
context.log.debug("forwarding payment to blinded peer {}", walletNodeId.publicKey)
|
||||
waitForPeerReady(upstream, walletNodeId.publicKey, recipient, nextPayload, nextPacket_opt)
|
||||
case _ =>
|
||||
context.log.debug("forwarding payment to blinded recipient {}", recipient.nodeId)
|
||||
relay(upstream, recipient, None, None, nextPayload, nextPacket_opt)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The next node may be a mobile wallet directly connected to us: in that case, we'll need to wake them up before
|
||||
* relaying the payment.
|
||||
*/
|
||||
private def ensureRecipientReady(upstream: Upstream.Hot.Trampoline, recipient: Recipient, nextPayload: IntermediatePayload.NodeRelay, nextPacket_opt: Option[OnionRoutingPacket]): Behavior[Command] = {
|
||||
nextWalletNodeId(nodeParams, recipient) match {
|
||||
case Some(walletNodeId) if nodeParams.peerWakeUpConfig.enabled => waitForPeerReady(upstream, walletNodeId, recipient, nextPayload, nextPacket_opt)
|
||||
case walletNodeId_opt => relay(upstream, recipient, walletNodeId_opt, None, nextPayload, nextPacket_opt)
|
||||
/** The next node may be a mobile wallet directly connected to us: in that case, we'll need to wake them up before relaying the payment. */
|
||||
private def checkIfRecipientIsWallet(upstream: Upstream.Hot.Trampoline, recipient: Recipient, nextPayload: IntermediatePayload.NodeRelay, nextPacket_opt: Option[OnionRoutingPacket]): Behavior[Command] = {
|
||||
if (nodeParams.peerWakeUpConfig.enabled) {
|
||||
val forwardNodeIdFailureAdapter = context.messageAdapter[Register.ForwardNodeIdFailure[Peer.GetPeerInfo]](_ => WrappedPeerInfo(isPeer = false, remoteFeatures_opt = None))
|
||||
val peerInfoAdapter = context.messageAdapter[Peer.PeerInfoResponse] {
|
||||
case _: Peer.PeerNotFound => WrappedPeerInfo(isPeer = false, remoteFeatures_opt = None)
|
||||
case info: Peer.PeerInfo => WrappedPeerInfo(isPeer = true, info.features)
|
||||
}
|
||||
register ! Register.ForwardNodeId(forwardNodeIdFailureAdapter, recipient.nodeId, Peer.GetPeerInfo(Some(peerInfoAdapter)))
|
||||
Behaviors.receiveMessagePartial {
|
||||
rejectExtraHtlcPartialFunction orElse {
|
||||
case info: WrappedPeerInfo =>
|
||||
if (info.isPeer && info.remoteFeatures_opt.exists(_.hasFeature(Features.WakeUpNotificationClient))) {
|
||||
waitForPeerReady(upstream, recipient.nodeId, recipient, nextPayload, nextPacket_opt)
|
||||
} else {
|
||||
relay(upstream, recipient, None, None, nextPayload, nextPacket_opt)
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
relay(upstream, recipient, None, None, nextPayload, nextPacket_opt)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -349,7 +351,7 @@ class NodeRelay private(nodeParams: NodeParams,
|
|||
}
|
||||
val payFSM = outgoingPaymentFactory.spawnOutgoingPayFSM(context, paymentCfg, useMultiPart)
|
||||
payFSM ! payment
|
||||
sending(upstream, recipient, recipientFeatures_opt, payloadOut, TimestampMilli.now(), fulfilledUpstream = false)
|
||||
sending(upstream, recipient, walletNodeId_opt, recipientFeatures_opt, payloadOut, TimestampMilli.now(), fulfilledUpstream = false)
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -361,6 +363,7 @@ class NodeRelay private(nodeParams: NodeParams,
|
|||
*/
|
||||
private def sending(upstream: Upstream.Hot.Trampoline,
|
||||
recipient: Recipient,
|
||||
walletNodeId_opt: Option[PublicKey],
|
||||
recipientFeatures_opt: Option[Features[InitFeature]],
|
||||
nextPayload: IntermediatePayload.NodeRelay,
|
||||
startedAt: TimestampMilli,
|
||||
|
@ -373,7 +376,7 @@ class NodeRelay private(nodeParams: NodeParams,
|
|||
// We want to fulfill upstream as soon as we receive the preimage (even if not all HTLCs have fulfilled downstream).
|
||||
context.log.debug("got preimage from downstream")
|
||||
fulfillPayment(upstream, paymentPreimage)
|
||||
sending(upstream, recipient, recipientFeatures_opt, nextPayload, startedAt, fulfilledUpstream = true)
|
||||
sending(upstream, recipient, walletNodeId_opt, recipientFeatures_opt, nextPayload, startedAt, fulfilledUpstream = true)
|
||||
} else {
|
||||
// we don't want to fulfill multiple times
|
||||
Behaviors.same
|
||||
|
@ -388,7 +391,7 @@ class NodeRelay private(nodeParams: NodeParams,
|
|||
recordRelayDuration(startedAt, isSuccess = true)
|
||||
stopping()
|
||||
case WrappedPaymentFailed(PaymentFailed(_, _, failures, _)) =>
|
||||
nextWalletNodeId(nodeParams, recipient) match {
|
||||
walletNodeId_opt match {
|
||||
case Some(walletNodeId) if shouldAttemptOnTheFlyFunding(nodeParams, recipientFeatures_opt, failures)(context) =>
|
||||
context.log.info("trampoline payment failed, attempting on-the-fly funding")
|
||||
attemptOnTheFlyFunding(upstream, walletNodeId, recipient, nextPayload, failures, startedAt)
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
|
||||
package fr.acinq.eclair.payment.relay
|
||||
|
||||
import akka.actor.Status
|
||||
import akka.actor.testkit.typed.scaladsl.{ScalaTestWithActorTestKit, TestProbe}
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.eventstream.EventStream
|
||||
|
@ -115,6 +114,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
|||
.modify(_.peerWakeUpConfig.enabled).setToIf(test.tags.contains(wakeUpEnabled))(true)
|
||||
.modify(_.peerWakeUpConfig.timeout).setToIf(test.tags.contains(wakeUpTimeout))(100 millis)
|
||||
.modify(_.features.activated).usingIf(test.tags.contains(onTheFlyFunding))(_ + (Features.OnTheFlyFunding -> FeatureSupport.Optional))
|
||||
.modify(_.features.activated).usingIf(test.tags.contains(wakeUpEnabled))(_ + (Features.WakeUpNotificationClient -> FeatureSupport.Optional))
|
||||
val router = TestProbe[Any]("router")
|
||||
val register = TestProbe[Any]("register")
|
||||
val eventListener = TestProbe[PaymentEvent]("event-listener")
|
||||
|
@ -630,10 +630,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
|||
register.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
// The two tests below are disabled by default, since there is no default mechanism to flag the next trampoline node
|
||||
// as being a wallet node. Feature branches that support wallet software should restore those tests and flag the
|
||||
// outgoing node_id as being a wallet node.
|
||||
ignore("relay incoming multi-part payment with on-the-fly funding", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
|
||||
test("relay incoming multi-part payment with on-the-fly funding", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val (peerReadyManager, switchboard) = createWakeUpActors()
|
||||
|
@ -642,6 +639,11 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
|||
val (nodeRelayer, parent) = f.createNodeRelay(incomingMultiPart.head)
|
||||
incomingMultiPart.foreach(p => nodeRelayer ! NodeRelay.Relay(p, randomKey().publicKey))
|
||||
|
||||
// We first check if the outgoing node is our peer and supports wake-up notifications.
|
||||
val peerFeaturesRequest = register.expectMessageType[Register.ForwardNodeId[Peer.GetPeerInfo]]
|
||||
assert(peerFeaturesRequest.nodeId == outgoingNodeId)
|
||||
peerFeaturesRequest.message.replyTo.foreach(_ ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.DISCONNECTED, Some(nodeParams.features.initFeatures()), None, Set.empty))
|
||||
|
||||
// The remote node is a wallet node: we wake them up before relaying the payment.
|
||||
val wakeUp = peerReadyManager.expectMessageType[PeerReadyManager.Register]
|
||||
assert(wakeUp.remoteNodeId == outgoingNodeId)
|
||||
|
@ -676,7 +678,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
|||
parent.expectMessageType[NodeRelayer.RelayComplete]
|
||||
}
|
||||
|
||||
ignore("relay incoming multi-part payment with on-the-fly funding (non-liquidity failure)", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
|
||||
test("relay incoming multi-part payment with on-the-fly funding (non-liquidity failure)", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val (peerReadyManager, switchboard) = createWakeUpActors()
|
||||
|
@ -685,6 +687,11 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
|
|||
val (nodeRelayer, parent) = f.createNodeRelay(incomingMultiPart.head)
|
||||
incomingMultiPart.foreach(p => nodeRelayer ! NodeRelay.Relay(p, randomKey().publicKey))
|
||||
|
||||
// We first check if the outgoing node is our peer and supports wake-up notifications.
|
||||
val peerFeaturesRequest = register.expectMessageType[Register.ForwardNodeId[Peer.GetPeerInfo]]
|
||||
assert(peerFeaturesRequest.nodeId == outgoingNodeId)
|
||||
peerFeaturesRequest.message.replyTo.foreach(_ ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.DISCONNECTED, Some(nodeParams.features.initFeatures()), None, Set.empty))
|
||||
|
||||
// The remote node is a wallet node: we wake them up before relaying the payment.
|
||||
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 0)
|
||||
val peerInfo = switchboard.expectMessageType[Switchboard.GetPeerInfo]
|
||||
|
|
Loading…
Add table
Reference in a new issue