Skip to content

Commit 372222d

Browse files
authored
Check peer features before attempting wake-up (#2979)
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
1 parent fb58d8c commit 372222d

File tree

3 files changed

+57
-40
lines changed

3 files changed

+57
-40
lines changed

eclair-core/src/main/scala/fr/acinq/eclair/Features.scala

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -305,6 +305,12 @@ object Features {
305305
val mandatory = 54
306306
}
307307

308+
/** This feature bit indicates that the node is a mobile wallet that can be woken up via push notifications. */
309+
case object WakeUpNotificationClient extends Feature with InitFeature {
310+
val rfcName = "wake_up_notification_client"
311+
val mandatory = 132
312+
}
313+
308314
// TODO: @t-bast: update feature bits once spec-ed (currently reserved here: https://github.com/lightningnetwork/lightning-rfc/issues/605)
309315
// We're not advertising these bits yet in our announcements, clients have to assume support.
310316
// This is why we haven't added them yet to `areSupported`.
@@ -369,6 +375,7 @@ object Features {
369375
PaymentMetadata,
370376
ZeroConf,
371377
KeySend,
378+
WakeUpNotificationClient,
372379
TrampolinePaymentPrototype,
373380
AsyncPaymentPrototype,
374381
SplicePrototype,

eclair-core/src/main/scala/fr/acinq/eclair/payment/relay/NodeRelay.scala

Lines changed: 37 additions & 34 deletions
Original file line numberDiff line numberDiff line change
@@ -66,6 +66,7 @@ object NodeRelay {
6666
private case class WrappedPaymentFailed(paymentFailed: PaymentFailed) extends Command
6767
private case class WrappedPeerReadyResult(result: PeerReadyNotifier.Result) extends Command
6868
private case class WrappedResolvedPaths(resolved: Seq[ResolvedPath]) extends Command
69+
private case class WrappedPeerInfo(isPeer: Boolean, remoteFeatures_opt: Option[Features[InitFeature]]) extends Command
6970
private case class WrappedOnTheFlyFundingResponse(result: Peer.ProposeOnTheFlyFundingResponse) extends Command
7071
// @formatter:on
7172

@@ -137,24 +138,6 @@ object NodeRelay {
137138
}
138139
}
139140

140-
/** This function identifies whether the next node is a wallet node directly connected to us, and returns its node_id. */
141-
private def nextWalletNodeId(nodeParams: NodeParams, recipient: Recipient): Option[PublicKey] = {
142-
recipient match {
143-
// This recipient is only used when we're the payment initiator.
144-
case _: SpontaneousRecipient => None
145-
// When relaying to a trampoline node, the next node may be a wallet node directly connected to us, but we don't
146-
// want to have false positives. Feature branches should check an internal DB/cache to confirm.
147-
case r: ClearRecipient if r.nextTrampolineOnion_opt.nonEmpty => None
148-
// If we're relaying to a non-trampoline recipient, it's never a wallet node.
149-
case _: ClearRecipient => None
150-
// When using blinded paths, we may be the introduction node for a wallet node directly connected to us.
151-
case r: BlindedRecipient => r.blindedHops.head.resolved.route match {
152-
case BlindedPathsResolver.PartialBlindedRoute(walletNodeId: EncodedNodeId.WithPublicKey.Wallet, _, _) => Some(walletNodeId.publicKey)
153-
case _ => None
154-
}
155-
}
156-
}
157-
158141
/** Compute route params that honor our fee and cltv requirements. */
159142
private def computeRouteParams(nodeParams: NodeParams, amountIn: MilliSatoshi, expiryIn: CltvExpiry, amountOut: MilliSatoshi, expiryOut: CltvExpiry): RouteParams = {
160143
nodeParams.routerConf.pathFindingExperimentConf.getRandomConf().getDefaultRouteParams
@@ -264,14 +247,14 @@ class NodeRelay private(nodeParams: NodeParams,
264247
val paymentSecret = randomBytes32() // we generate a new secret to protect against probing attacks
265248
val recipient = ClearRecipient(payloadOut.outgoingNodeId, Features.empty, payloadOut.amountToForward, payloadOut.outgoingCltv, paymentSecret, nextTrampolineOnion_opt = nextPacket_opt)
266249
context.log.debug("forwarding payment to the next trampoline node {}", recipient.nodeId)
267-
ensureRecipientReady(upstream, recipient, nextPayload, nextPacket_opt)
250+
attemptWakeUpIfRecipientIsWallet(upstream, recipient, nextPayload, nextPacket_opt)
268251
case payloadOut: IntermediatePayload.NodeRelay.ToNonTrampoline =>
269252
val paymentSecret = payloadOut.paymentSecret
270253
val features = Features(payloadOut.invoiceFeatures).invoiceFeatures()
271254
val extraEdges = payloadOut.invoiceRoutingInfo.flatMap(Bolt11Invoice.toExtraEdges(_, payloadOut.outgoingNodeId))
272255
val recipient = ClearRecipient(payloadOut.outgoingNodeId, features, payloadOut.amountToForward, payloadOut.outgoingCltv, paymentSecret, extraEdges, payloadOut.paymentMetadata)
273256
context.log.debug("forwarding payment to non-trampoline recipient {}", recipient.nodeId)
274-
ensureRecipientReady(upstream, recipient, nextPayload, None)
257+
attemptWakeUpIfRecipientIsWallet(upstream, recipient, nextPayload, None)
275258
case payloadOut: IntermediatePayload.NodeRelay.ToBlindedPaths =>
276259
// Blinded paths in Bolt 12 invoices may encode the introduction node with an scid and a direction: we need to
277260
// resolve that to a nodeId in order to reach that introduction node and use the blinded path.
@@ -287,29 +270,48 @@ class NodeRelay private(nodeParams: NodeParams,
287270
// We don't have access to the invoice: we use the only node_id that somewhat makes sense for the recipient.
288271
val blindedNodeId = resolved.head.route.blindedNodeIds.last
289272
val recipient = BlindedRecipient.fromPaths(blindedNodeId, Features(payloadOut.invoiceFeatures).invoiceFeatures(), payloadOut.amountToForward, payloadOut.outgoingCltv, resolved, Set.empty)
290-
context.log.debug("forwarding payment to blinded recipient {}", recipient.nodeId)
291-
ensureRecipientReady(upstream, recipient, nextPayload, nextPacket_opt)
273+
resolved.head.route match {
274+
case BlindedPathsResolver.PartialBlindedRoute(walletNodeId: EncodedNodeId.WithPublicKey.Wallet, _, _) if nodeParams.peerWakeUpConfig.enabled =>
275+
context.log.debug("forwarding payment to blinded peer {}", walletNodeId.publicKey)
276+
attemptWakeUp(upstream, walletNodeId.publicKey, recipient, nextPayload, nextPacket_opt)
277+
case _ =>
278+
context.log.debug("forwarding payment to blinded recipient {}", recipient.nodeId)
279+
relay(upstream, recipient, None, None, nextPayload, nextPacket_opt)
280+
}
292281
}
293282
}
294283
}
295284
}
296285

297-
/**
298-
* The next node may be a mobile wallet directly connected to us: in that case, we'll need to wake them up before
299-
* relaying the payment.
300-
*/
301-
private def ensureRecipientReady(upstream: Upstream.Hot.Trampoline, recipient: Recipient, nextPayload: IntermediatePayload.NodeRelay, nextPacket_opt: Option[OnionRoutingPacket]): Behavior[Command] = {
302-
nextWalletNodeId(nodeParams, recipient) match {
303-
case Some(walletNodeId) if nodeParams.peerWakeUpConfig.enabled => waitForPeerReady(upstream, walletNodeId, recipient, nextPayload, nextPacket_opt)
304-
case walletNodeId_opt => relay(upstream, recipient, walletNodeId_opt, None, nextPayload, nextPacket_opt)
286+
/** 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. */
287+
private def attemptWakeUpIfRecipientIsWallet(upstream: Upstream.Hot.Trampoline, recipient: Recipient, nextPayload: IntermediatePayload.NodeRelay, nextPacket_opt: Option[OnionRoutingPacket]): Behavior[Command] = {
288+
if (nodeParams.peerWakeUpConfig.enabled) {
289+
val forwardNodeIdFailureAdapter = context.messageAdapter[Register.ForwardNodeIdFailure[Peer.GetPeerInfo]](_ => WrappedPeerInfo(isPeer = false, remoteFeatures_opt = None))
290+
val peerInfoAdapter = context.messageAdapter[Peer.PeerInfoResponse] {
291+
case _: Peer.PeerNotFound => WrappedPeerInfo(isPeer = false, remoteFeatures_opt = None)
292+
case info: Peer.PeerInfo => WrappedPeerInfo(isPeer = true, info.features)
293+
}
294+
register ! Register.ForwardNodeId(forwardNodeIdFailureAdapter, recipient.nodeId, Peer.GetPeerInfo(Some(peerInfoAdapter)))
295+
Behaviors.receiveMessagePartial {
296+
rejectExtraHtlcPartialFunction orElse {
297+
case info: WrappedPeerInfo =>
298+
if (info.isPeer && info.remoteFeatures_opt.exists(_.hasFeature(Features.WakeUpNotificationClient))) {
299+
attemptWakeUp(upstream, recipient.nodeId, recipient, nextPayload, nextPacket_opt)
300+
} else {
301+
relay(upstream, recipient, None, None, nextPayload, nextPacket_opt)
302+
}
303+
}
304+
}
305+
} else {
306+
relay(upstream, recipient, None, None, nextPayload, nextPacket_opt)
305307
}
306308
}
307309

308310
/**
309311
* The next node is the payment recipient. They are directly connected to us and may be offline. We try to wake them
310312
* up and will relay the payment once they're connected and channels are reestablished.
311313
*/
312-
private def waitForPeerReady(upstream: Upstream.Hot.Trampoline, walletNodeId: PublicKey, recipient: Recipient, nextPayload: IntermediatePayload.NodeRelay, nextPacket_opt: Option[OnionRoutingPacket]): Behavior[Command] = {
314+
private def attemptWakeUp(upstream: Upstream.Hot.Trampoline, walletNodeId: PublicKey, recipient: Recipient, nextPayload: IntermediatePayload.NodeRelay, nextPacket_opt: Option[OnionRoutingPacket]): Behavior[Command] = {
313315
context.log.info("trying to wake up next peer (nodeId={})", walletNodeId)
314316
val notifier = context.spawnAnonymous(PeerReadyNotifier(walletNodeId, timeout_opt = Some(Left(nodeParams.peerWakeUpConfig.timeout))))
315317
notifier ! PeerReadyNotifier.NotifyWhenPeerReady(context.messageAdapter(WrappedPeerReadyResult))
@@ -349,7 +351,7 @@ class NodeRelay private(nodeParams: NodeParams,
349351
}
350352
val payFSM = outgoingPaymentFactory.spawnOutgoingPayFSM(context, paymentCfg, useMultiPart)
351353
payFSM ! payment
352-
sending(upstream, recipient, recipientFeatures_opt, payloadOut, TimestampMilli.now(), fulfilledUpstream = false)
354+
sending(upstream, recipient, walletNodeId_opt, recipientFeatures_opt, payloadOut, TimestampMilli.now(), fulfilledUpstream = false)
353355
}
354356

355357
/**
@@ -361,6 +363,7 @@ class NodeRelay private(nodeParams: NodeParams,
361363
*/
362364
private def sending(upstream: Upstream.Hot.Trampoline,
363365
recipient: Recipient,
366+
walletNodeId_opt: Option[PublicKey],
364367
recipientFeatures_opt: Option[Features[InitFeature]],
365368
nextPayload: IntermediatePayload.NodeRelay,
366369
startedAt: TimestampMilli,
@@ -373,7 +376,7 @@ class NodeRelay private(nodeParams: NodeParams,
373376
// We want to fulfill upstream as soon as we receive the preimage (even if not all HTLCs have fulfilled downstream).
374377
context.log.debug("got preimage from downstream")
375378
fulfillPayment(upstream, paymentPreimage)
376-
sending(upstream, recipient, recipientFeatures_opt, nextPayload, startedAt, fulfilledUpstream = true)
379+
sending(upstream, recipient, walletNodeId_opt, recipientFeatures_opt, nextPayload, startedAt, fulfilledUpstream = true)
377380
} else {
378381
// we don't want to fulfill multiple times
379382
Behaviors.same
@@ -388,7 +391,7 @@ class NodeRelay private(nodeParams: NodeParams,
388391
recordRelayDuration(startedAt, isSuccess = true)
389392
stopping()
390393
case WrappedPaymentFailed(PaymentFailed(_, _, failures, _)) =>
391-
nextWalletNodeId(nodeParams, recipient) match {
394+
walletNodeId_opt match {
392395
case Some(walletNodeId) if shouldAttemptOnTheFlyFunding(nodeParams, recipientFeatures_opt, failures)(context) =>
393396
context.log.info("trampoline payment failed, attempting on-the-fly funding")
394397
attemptOnTheFlyFunding(upstream, walletNodeId, recipient, nextPayload, failures, startedAt)

eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/NodeRelayerSpec.scala

Lines changed: 13 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,6 @@
1616

1717
package fr.acinq.eclair.payment.relay
1818

19-
import akka.actor.Status
2019
import akka.actor.testkit.typed.scaladsl.{ScalaTestWithActorTestKit, TestProbe}
2120
import akka.actor.typed.ActorRef
2221
import akka.actor.typed.eventstream.EventStream
@@ -115,6 +114,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
115114
.modify(_.peerWakeUpConfig.enabled).setToIf(test.tags.contains(wakeUpEnabled))(true)
116115
.modify(_.peerWakeUpConfig.timeout).setToIf(test.tags.contains(wakeUpTimeout))(100 millis)
117116
.modify(_.features.activated).usingIf(test.tags.contains(onTheFlyFunding))(_ + (Features.OnTheFlyFunding -> FeatureSupport.Optional))
117+
.modify(_.features.activated).usingIf(test.tags.contains(wakeUpEnabled))(_ + (Features.WakeUpNotificationClient -> FeatureSupport.Optional))
118118
val router = TestProbe[Any]("router")
119119
val register = TestProbe[Any]("register")
120120
val eventListener = TestProbe[PaymentEvent]("event-listener")
@@ -630,10 +630,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
630630
register.expectNoMessage(100 millis)
631631
}
632632

633-
// The two tests below are disabled by default, since there is no default mechanism to flag the next trampoline node
634-
// as being a wallet node. Feature branches that support wallet software should restore those tests and flag the
635-
// outgoing node_id as being a wallet node.
636-
ignore("relay incoming multi-part payment with on-the-fly funding", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
633+
test("relay incoming multi-part payment with on-the-fly funding", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
637634
import f._
638635

639636
val (peerReadyManager, switchboard) = createWakeUpActors()
@@ -642,6 +639,11 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
642639
val (nodeRelayer, parent) = f.createNodeRelay(incomingMultiPart.head)
643640
incomingMultiPart.foreach(p => nodeRelayer ! NodeRelay.Relay(p, randomKey().publicKey))
644641

642+
// We first check if the outgoing node is our peer and supports wake-up notifications.
643+
val peerFeaturesRequest = register.expectMessageType[Register.ForwardNodeId[Peer.GetPeerInfo]]
644+
assert(peerFeaturesRequest.nodeId == outgoingNodeId)
645+
peerFeaturesRequest.message.replyTo.foreach(_ ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.DISCONNECTED, Some(nodeParams.features.initFeatures()), None, Set.empty))
646+
645647
// The remote node is a wallet node: we wake them up before relaying the payment.
646648
val wakeUp = peerReadyManager.expectMessageType[PeerReadyManager.Register]
647649
assert(wakeUp.remoteNodeId == outgoingNodeId)
@@ -676,7 +678,7 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
676678
parent.expectMessageType[NodeRelayer.RelayComplete]
677679
}
678680

679-
ignore("relay incoming multi-part payment with on-the-fly funding (non-liquidity failure)", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
681+
test("relay incoming multi-part payment with on-the-fly funding (non-liquidity failure)", Tag(wakeUpEnabled), Tag(onTheFlyFunding)) { f =>
680682
import f._
681683

682684
val (peerReadyManager, switchboard) = createWakeUpActors()
@@ -685,6 +687,11 @@ class NodeRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("appl
685687
val (nodeRelayer, parent) = f.createNodeRelay(incomingMultiPart.head)
686688
incomingMultiPart.foreach(p => nodeRelayer ! NodeRelay.Relay(p, randomKey().publicKey))
687689

690+
// We first check if the outgoing node is our peer and supports wake-up notifications.
691+
val peerFeaturesRequest = register.expectMessageType[Register.ForwardNodeId[Peer.GetPeerInfo]]
692+
assert(peerFeaturesRequest.nodeId == outgoingNodeId)
693+
peerFeaturesRequest.message.replyTo.foreach(_ ! Peer.PeerInfo(TestProbe[Any]().ref.toClassic, outgoingNodeId, Peer.DISCONNECTED, Some(nodeParams.features.initFeatures()), None, Set.empty))
694+
688695
// The remote node is a wallet node: we wake them up before relaying the payment.
689696
peerReadyManager.expectMessageType[PeerReadyManager.Register].replyTo ! PeerReadyManager.Registered(outgoingNodeId, otherAttempts = 0)
690697
val peerInfo = switchboard.expectMessageType[Switchboard.GetPeerInfo]

0 commit comments

Comments
 (0)