Skip to content

Commit

Permalink
Decrypt on-the-fly funding trampoline failures (#2960)
Browse files Browse the repository at this point in the history
While we ignore the actual failure and always return a temporary failure
upstream, it's useful to decrypt the recipient failure to log it. When
we add support for trampoline errors, we will need the decrypted failure
to be able to re-wrap it with trampoline onion secrets.
  • Loading branch information
t-bast authored Dec 5, 2024
1 parent feef44b commit 8381fc4
Show file tree
Hide file tree
Showing 8 changed files with 59 additions and 35 deletions.
27 changes: 14 additions & 13 deletions eclair-core/src/main/scala/fr/acinq/eclair/io/Peer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.blockchain.{CurrentBlockHeight, CurrentFeerates, OnChainChannelFunder, OnchainPubkeyCache}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.db.PendingCommandsDb
import fr.acinq.eclair.io.MessageRelay.Status
import fr.acinq.eclair.io.Monitoring.{Metrics, Tags}
Expand All @@ -44,7 +45,7 @@ import fr.acinq.eclair.remote.EclairInternalsSerializer.RemoteTypes
import fr.acinq.eclair.router.Router
import fr.acinq.eclair.wire.protocol
import fr.acinq.eclair.wire.protocol.FailureMessageCodecs.createBadOnionFailure
import fr.acinq.eclair.wire.protocol.{AddFeeCredit, ChannelTlv, CurrentFeeCredit, Error, FailureReason, HasChannelId, HasTemporaryChannelId, LightningMessage, LiquidityAds, NodeAddress, OnTheFlyFundingFailureMessage, OnionMessage, OnionRoutingPacket, RecommendedFeerates, RoutingMessage, SpliceInit, TemporaryChannelFailure, TlvStream, TxAbort, UnknownMessage, Warning, WillAddHtlc, WillFailHtlc, WillFailMalformedHtlc}
import fr.acinq.eclair.wire.protocol.{AddFeeCredit, ChannelTlv, CurrentFeeCredit, Error, FailureReason, HasChannelId, HasTemporaryChannelId, LightningMessage, LiquidityAds, NodeAddress, OnTheFlyFundingFailureMessage, OnionMessage, OnionRoutingPacket, RecommendedFeerates, RoutingMessage, SpliceInit, TlvStream, TxAbort, UnknownMessage, Warning, WillAddHtlc, WillFailHtlc, WillFailMalformedHtlc}

/**
* This actor represents a logical peer. There is one [[Peer]] per unique remote node id at all time.
Expand Down Expand Up @@ -267,27 +268,27 @@ class Peer(val nodeParams: NodeParams,
status.timer.cancel()
val timer = context.system.scheduler.scheduleOnce(nodeParams.onTheFlyFundingConfig.proposalTimeout, self, OnTheFlyFundingTimeout(cmd.paymentHash))(context.dispatcher)
pending.copy(
proposed = pending.proposed :+ OnTheFlyFunding.Proposal(htlc, cmd.upstream),
proposed = pending.proposed :+ OnTheFlyFunding.Proposal(htlc, cmd.upstream, cmd.onionSharedSecrets),
status = OnTheFlyFunding.Status.Proposed(timer)
)
case status: OnTheFlyFunding.Status.AddedToFeeCredit =>
log.info("received extra payment for on-the-fly funding that was added to fee credit (payment_hash={}, amount={})", cmd.paymentHash, cmd.amount)
val proposal = OnTheFlyFunding.Proposal(htlc, cmd.upstream)
val proposal = OnTheFlyFunding.Proposal(htlc, cmd.upstream, cmd.onionSharedSecrets)
proposal.createFulfillCommands(status.preimage).foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
pending.copy(proposed = pending.proposed :+ proposal)
case status: OnTheFlyFunding.Status.Funded =>
log.info("rejecting extra payment for on-the-fly funding that has already been funded with txId={} (payment_hash={}, amount={})", status.txId, cmd.paymentHash, cmd.amount)
// The payer is buggy and is paying the same payment_hash multiple times. We could simply claim that
// extra payment for ourselves, but we're nice and instead immediately fail it.
val proposal = OnTheFlyFunding.Proposal(htlc, cmd.upstream)
proposal.createFailureCommands(None).foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
val proposal = OnTheFlyFunding.Proposal(htlc, cmd.upstream, cmd.onionSharedSecrets)
proposal.createFailureCommands(None)(log).foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
pending
}
case None =>
self ! Peer.OutgoingMessage(htlc, d.peerConnection)
Metrics.OnTheFlyFunding.withTag(Tags.OnTheFlyFundingState, Tags.OnTheFlyFundingStates.Proposed).increment()
val timer = context.system.scheduler.scheduleOnce(nodeParams.onTheFlyFundingConfig.proposalTimeout, self, OnTheFlyFundingTimeout(cmd.paymentHash))(context.dispatcher)
OnTheFlyFunding.Pending(Seq(OnTheFlyFunding.Proposal(htlc, cmd.upstream)), OnTheFlyFunding.Status.Proposed(timer))
OnTheFlyFunding.Pending(Seq(OnTheFlyFunding.Proposal(htlc, cmd.upstream, cmd.onionSharedSecrets)), OnTheFlyFunding.Status.Proposed(timer))
}
pendingOnTheFlyFunding += (htlc.paymentHash -> pending)
stay()
Expand All @@ -303,7 +304,7 @@ class Peer(val nodeParams: NodeParams,
case msg: WillFailHtlc => FailureReason.EncryptedDownstreamFailure(msg.reason)
case msg: WillFailMalformedHtlc => FailureReason.LocalFailure(createBadOnionFailure(msg.onionHash, msg.failureCode))
}
htlc.createFailureCommands(Some(failure)).foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
htlc.createFailureCommands(Some(failure))(log).foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
val proposed1 = pending.proposed.filterNot(_.htlc.id == msg.id)
if (proposed1.isEmpty) {
Metrics.OnTheFlyFunding.withTag(Tags.OnTheFlyFundingState, Tags.OnTheFlyFundingStates.Rejected).increment()
Expand Down Expand Up @@ -335,7 +336,7 @@ class Peer(val nodeParams: NodeParams,
pending.status match {
case _: OnTheFlyFunding.Status.Proposed =>
log.warning("on-the-fly funding proposal timed out for payment_hash={}", timeout.paymentHash)
pending.createFailureCommands().foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
pending.createFailureCommands(log).foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
Metrics.OnTheFlyFunding.withTag(Tags.OnTheFlyFundingState, Tags.OnTheFlyFundingStates.Expired).increment()
pendingOnTheFlyFunding -= timeout.paymentHash
self ! Peer.OutgoingMessage(Warning(s"on-the-fly funding proposal timed out for payment_hash=${timeout.paymentHash}"), d.peerConnection)
Expand Down Expand Up @@ -584,14 +585,14 @@ class Peer(val nodeParams: NodeParams,
case _: OnTheFlyFunding.Status.Proposed =>
log.warning("proposed will_add_htlc expired for payment_hash={}", paymentHash)
Metrics.OnTheFlyFunding.withTag(Tags.OnTheFlyFundingState, Tags.OnTheFlyFundingStates.Timeout).increment()
pending.createFailureCommands().foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
pending.createFailureCommands(log).foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
case _: OnTheFlyFunding.Status.AddedToFeeCredit =>
// Nothing to do, we already fulfilled the upstream HTLCs.
log.debug("forgetting will_add_htlc added to fee credit for payment_hash={}", paymentHash)
case _: OnTheFlyFunding.Status.Funded =>
log.warning("funded will_add_htlc expired for payment_hash={}, our peer may be malicious", paymentHash)
Metrics.OnTheFlyFunding.withTag(Tags.OnTheFlyFundingState, Tags.OnTheFlyFundingStates.Timeout).increment()
pending.createFailureCommands().foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
pending.createFailureCommands(log).foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
nodeParams.db.liquidity.removePendingOnTheFlyFunding(remoteNodeId, paymentHash)
}
}
Expand Down Expand Up @@ -675,7 +676,7 @@ class Peer(val nodeParams: NodeParams,
// We emit a relay event: since we waited for on-chain funding before relaying the payment, the timestamps
// won't be accurate, but everything else is.
pending.proposed.foreach {
case OnTheFlyFunding.Proposal(htlc, upstream) => upstream match {
case OnTheFlyFunding.Proposal(htlc, upstream, _) => upstream match {
case _: Upstream.Local => ()
case u: Upstream.Hot.Channel =>
val incoming = PaymentRelayed.IncomingPart(u.add.amountMsat, u.add.channelId, u.receivedAt)
Expand Down Expand Up @@ -810,7 +811,7 @@ class Peer(val nodeParams: NodeParams,
case status: OnTheFlyFunding.Status.Proposed =>
log.info("cancelling on-the-fly funding for payment_hash={}", paymentHash)
status.timer.cancel()
pending.createFailureCommands().foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
pending.createFailureCommands(log).foreach { case (channelId, cmd) => PendingCommandsDb.safeSend(register, nodeParams.db.pendingCommands, channelId, cmd) }
true
// We keep proposals that have been added to fee credit until we reach the HTLC expiry or we restart. This
// guarantees that our peer cannot concurrently add to their fee credit a payment for which we've signed a
Expand Down Expand Up @@ -983,7 +984,7 @@ object Peer {
case class SpawnChannelNonInitiator(open: Either[protocol.OpenChannel, protocol.OpenDualFundedChannel], channelConfig: ChannelConfig, channelType: SupportedChannelType, addFunding_opt: Option[LiquidityAds.AddFunding], localParams: LocalParams, peerConnection: ActorRef)

/** If [[Features.OnTheFlyFunding]] is supported and we're connected, relay a funding proposal to our peer. */
case class ProposeOnTheFlyFunding(replyTo: typed.ActorRef[ProposeOnTheFlyFundingResponse], amount: MilliSatoshi, paymentHash: ByteVector32, expiry: CltvExpiry, onion: OnionRoutingPacket, nextPathKey_opt: Option[PublicKey], upstream: Upstream.Hot)
case class ProposeOnTheFlyFunding(replyTo: typed.ActorRef[ProposeOnTheFlyFundingResponse], amount: MilliSatoshi, paymentHash: ByteVector32, expiry: CltvExpiry, onion: OnionRoutingPacket, onionSharedSecrets: Seq[Sphinx.SharedSecret], nextPathKey_opt: Option[PublicKey], upstream: Upstream.Hot)

sealed trait ProposeOnTheFlyFundingResponse
object ProposeOnTheFlyFundingResponse {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -186,7 +186,8 @@ class ChannelRelay private(nodeParams: NodeParams,
context.log.info("rejecting htlc reason={}", cmdFail.reason)
safeSendAndStop(r.add.channelId, cmdFail)
case RelayNeedsFunding(nextNodeId, cmdFail) =>
val cmd = Peer.ProposeOnTheFlyFunding(onTheFlyFundingResponseAdapter, r.amountToForward, r.add.paymentHash, r.outgoingCltv, r.nextPacket, nextPathKey_opt, upstream)
// Note that in the channel relay case, we don't have any outgoing onion shared secrets.
val cmd = Peer.ProposeOnTheFlyFunding(onTheFlyFundingResponseAdapter, r.amountToForward, r.add.paymentHash, r.outgoingCltv, r.nextPacket, Nil, nextPathKey_opt, upstream)
register ! Register.ForwardNodeId(forwardNodeIdFailureAdapter, nextNodeId, cmd)
waitForOnTheFlyFundingResponse(cmdFail)
case RelaySuccess(selectedChannelId, cmdAdd) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -422,7 +422,7 @@ class NodeRelay private(nodeParams: NodeParams,
case Right(nextPacket) =>
val forwardNodeIdFailureAdapter = context.messageAdapter[Register.ForwardNodeIdFailure[Peer.ProposeOnTheFlyFunding]](_ => WrappedOnTheFlyFundingResponse(Peer.ProposeOnTheFlyFundingResponse.NotAvailable("peer not found")))
val onTheFlyFundingResponseAdapter = context.messageAdapter[Peer.ProposeOnTheFlyFundingResponse](WrappedOnTheFlyFundingResponse)
val cmd = Peer.ProposeOnTheFlyFunding(onTheFlyFundingResponseAdapter, amountOut, paymentHash, expiryOut, nextPacket.cmd.onion, nextPacket.cmd.nextPathKey_opt, upstream)
val cmd = Peer.ProposeOnTheFlyFunding(onTheFlyFundingResponseAdapter, amountOut, paymentHash, expiryOut, nextPacket.cmd.onion, nextPacket.sharedSecrets, nextPacket.cmd.nextPathKey_opt, upstream)
register ! Register.ForwardNodeId(forwardNodeIdFailureAdapter, walletNodeId, cmd)
Behaviors.receiveMessagePartial {
rejectExtraHtlcPartialFunction orElse {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import akka.actor.Cancellable
import akka.actor.typed.scaladsl.adapter.TypedActorRefOps
import akka.actor.typed.scaladsl.{ActorContext, Behaviors, StashBuffer}
import akka.actor.typed.{ActorRef, Behavior}
import akka.event.LoggingAdapter
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto, TxId}
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
Expand Down Expand Up @@ -87,12 +88,12 @@ object OnTheFlyFunding {
// @formatter:on

/** An on-the-fly funding proposal sent to our peer. */
case class Proposal(htlc: WillAddHtlc, upstream: Upstream.Hot) {
case class Proposal(htlc: WillAddHtlc, upstream: Upstream.Hot, onionSharedSecrets: Seq[Sphinx.SharedSecret]) {
/** Maximum fees that can be collected from this HTLC. */
def maxFees(htlcMinimum: MilliSatoshi): MilliSatoshi = htlc.amount - htlcMinimum

/** Create commands to fail all upstream HTLCs. */
def createFailureCommands(failure_opt: Option[FailureReason]): Seq[(ByteVector32, CMD_FAIL_HTLC)] = upstream match {
def createFailureCommands(failure_opt: Option[FailureReason])(implicit log: LoggingAdapter): Seq[(ByteVector32, CMD_FAIL_HTLC)] = upstream match {
case _: Upstream.Local => Nil
case u: Upstream.Hot.Channel =>
val failure = htlc.pathKey_opt match {
Expand All @@ -101,11 +102,18 @@ object OnTheFlyFunding {
}
Seq(u.add.channelId -> CMD_FAIL_HTLC(u.add.id, failure, commit = true))
case u: Upstream.Hot.Trampoline =>
// In the trampoline case, we currently ignore downstream failures: we should add dedicated failures to the
// BOLTs to better handle those cases.
val failure = failure_opt match {
case Some(f) => f match {
case _: FailureReason.EncryptedDownstreamFailure => FailureReason.LocalFailure(TemporaryNodeFailure())
case f: FailureReason.EncryptedDownstreamFailure =>
// In the trampoline case, we currently ignore downstream failures: we should add dedicated failures to
// the BOLTs to better handle those cases.
Sphinx.FailurePacket.decrypt(f.packet, onionSharedSecrets) match {
case Left(Sphinx.CannotDecryptFailurePacket(_)) =>
log.warning("couldn't decrypt downstream on-the-fly funding failure")
case Right(f) =>
log.warning("downstream on-the-fly funding failure: {}", f.failureMessage.message)
}
FailureReason.LocalFailure(TemporaryNodeFailure())
case _: FailureReason.LocalFailure => f
}
case None => FailureReason.LocalFailure(UnknownNextPeer())
Expand All @@ -131,7 +139,7 @@ object OnTheFlyFunding {
def maxFees(htlcMinimum: MilliSatoshi): MilliSatoshi = proposed.map(_.maxFees(htlcMinimum)).sum

/** Create commands to fail all upstream HTLCs. */
def createFailureCommands(): Seq[(ByteVector32, CMD_FAIL_HTLC)] = proposed.flatMap(_.createFailureCommands(None))
def createFailureCommands(implicit log: LoggingAdapter): Seq[(ByteVector32, CMD_FAIL_HTLC)] = proposed.flatMap(_.createFailureCommands(None))

/** Create commands to fulfill all upstream HTLCs. */
def createFulfillCommands(preimage: ByteVector32): Seq[(ByteVector32, CMD_FULFILL_HTLC)] = proposed.flatMap(_.createFulfillCommands(preimage))
Expand Down Expand Up @@ -355,7 +363,13 @@ object OnTheFlyFunding {
.typecase(0x01, upstreamChannel)
.typecase(0x02, upstreamTrampoline)

val proposal: Codec[Proposal] = (("willAddHtlc" | lengthDelimited(willAddHtlcCodec)) :: ("upstream" | upstream)).as[Proposal]
val proposal: Codec[Proposal] = (
("willAddHtlc" | lengthDelimited(willAddHtlcCodec)) ::
("upstream" | upstream) ::
// We don't need to persist the onion shared secrets: we only persist on-the-fly funding proposals once they
// have been funded, at which point we will ignore downstream failures.
("onionSharedSecrets" | provide(Seq.empty[Sphinx.SharedSecret]))
).as[Proposal]

val proposals: Codec[Seq[Proposal]] = listOfN(uint16, proposal).xmap(_.toSeq, _.toList)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,28 +78,28 @@ class LiquidityDbSpec extends AnyFunSuite {
val pendingAlice = Seq(
OnTheFlyFunding.Pending(
proposed = Seq(
OnTheFlyFunding.Proposal(createWillAdd(20_000 msat, paymentHash1, CltvExpiry(500)), upstream(0)),
OnTheFlyFunding.Proposal(createWillAdd(1 msat, paymentHash1, CltvExpiry(750), Some(randomKey().publicKey)), upstream(1)),
OnTheFlyFunding.Proposal(createWillAdd(20_000 msat, paymentHash1, CltvExpiry(500)), upstream(0), Nil),
OnTheFlyFunding.Proposal(createWillAdd(1 msat, paymentHash1, CltvExpiry(750), Some(randomKey().publicKey)), upstream(1), Nil),
),
status = OnTheFlyFunding.Status.Funded(randomBytes32(), TxId(randomBytes32()), 7, 500 msat)
),
OnTheFlyFunding.Pending(
proposed = Seq(
OnTheFlyFunding.Proposal(createWillAdd(195_000_000 msat, paymentHash2, CltvExpiry(1000)), Upstream.Hot.Trampoline(upstream(2) :: upstream(3) :: Nil)),
OnTheFlyFunding.Proposal(createWillAdd(195_000_000 msat, paymentHash2, CltvExpiry(1000)), Upstream.Hot.Trampoline(upstream(2) :: upstream(3) :: Nil), Nil),
),
status = OnTheFlyFunding.Status.Funded(randomBytes32(), TxId(randomBytes32()), 3, 0 msat)
)
)
val pendingBob = Seq(
OnTheFlyFunding.Pending(
proposed = Seq(
OnTheFlyFunding.Proposal(createWillAdd(20_000 msat, paymentHash1, CltvExpiry(42)), upstream(0)),
OnTheFlyFunding.Proposal(createWillAdd(20_000 msat, paymentHash1, CltvExpiry(42)), upstream(0), Nil),
),
status = OnTheFlyFunding.Status.Funded(randomBytes32(), TxId(randomBytes32()), 11, 3_500 msat)
),
OnTheFlyFunding.Pending(
proposed = Seq(
OnTheFlyFunding.Proposal(createWillAdd(24_000_000 msat, paymentHash2, CltvExpiry(800_000), Some(randomKey().publicKey)), Upstream.Local(UUID.randomUUID())),
OnTheFlyFunding.Proposal(createWillAdd(24_000_000 msat, paymentHash2, CltvExpiry(800_000), Some(randomKey().publicKey)), Upstream.Local(UUID.randomUUID()), Nil),
),
status = OnTheFlyFunding.Status.Funded(randomBytes32(), TxId(randomBytes32()), 0, 10_000 msat)
)
Expand Down
Loading

0 comments on commit 8381fc4

Please sign in to comment.