Skip to content

Commit

Permalink
Abort interactive-tx during funding (#2769)
Browse files Browse the repository at this point in the history
If we abort the interactive-tx protocol while we're waiting for bitcoind
to fund the transaction, we must avoid sending an obsolete `tx_add_input`
for the funded transaction. There is a check in `Channel.scala` to only
send an outgoing interactive-tx message if we have an active interactive-tx
session, but this isn't sufficient in the following scenario:

- we disconnect and thus abort the interactive-tx session while bitcoind
  is funding
- we reconnect and restart another interactive-tx session
- bitcoind responds for the first interactive-tx session

In that scenario we would mix `tx_add_input` from the old and the new
interactive-tx session, which leads to a duplicate `serial_id`.
  • Loading branch information
t-bast authored Nov 3, 2023
1 parent 5a37059 commit 12adf87
Show file tree
Hide file tree
Showing 5 changed files with 41 additions and 18 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -480,7 +480,7 @@ object SpliceStatus {
/** We told our peer we want to splice funds in the channel. */
case class SpliceRequested(cmd: CMD_SPLICE, init: SpliceInit) extends QuiescentSpliceStatus
/** We both agreed to splice and are building the splice transaction. */
case class SpliceInProgress(cmd_opt: Option[CMD_SPLICE], splice: typed.ActorRef[InteractiveTxBuilder.Command], remoteCommitSig: Option[CommitSig]) extends QuiescentSpliceStatus
case class SpliceInProgress(cmd_opt: Option[CMD_SPLICE], sessionId: ByteVector32, splice: typed.ActorRef[InteractiveTxBuilder.Command], remoteCommitSig: Option[CommitSig]) extends QuiescentSpliceStatus
/** The splice transaction has been negotiated, we're exchanging signatures. */
case class SpliceWaitingForSigs(signingSession: InteractiveTxSigningSession.WaitingForSigs) extends QuiescentSpliceStatus
/** The splice attempt was aborted by us, we're waiting for our peer to ack. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -959,15 +959,17 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
targetFeerate = msg.feerate,
requireConfirmedInputs = RequireConfirmedInputs(forLocal = msg.requireConfirmedInputs, forRemote = spliceAck.requireConfirmedInputs)
)
val sessionId = randomBytes32()
val txBuilder = context.spawnAnonymous(InteractiveTxBuilder(
sessionId,
nodeParams, fundingParams,
channelParams = d.commitments.params,
purpose = InteractiveTxBuilder.SpliceTx(parentCommitment),
localPushAmount = spliceAck.pushAmount, remotePushAmount = msg.pushAmount,
wallet
))
txBuilder ! InteractiveTxBuilder.Start(self)
stay() using d.copy(spliceStatus = SpliceStatus.SpliceInProgress(cmd_opt = None, splice = txBuilder, remoteCommitSig = None)) sending spliceAck
stay() using d.copy(spliceStatus = SpliceStatus.SpliceInProgress(cmd_opt = None, sessionId, txBuilder, remoteCommitSig = None)) sending spliceAck
}
case SpliceStatus.SpliceAborted =>
log.info("rejecting splice attempt: our previous tx_abort was not acked")
Expand Down Expand Up @@ -995,23 +997,25 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
targetFeerate = spliceInit.feerate,
requireConfirmedInputs = RequireConfirmedInputs(forLocal = msg.requireConfirmedInputs, forRemote = spliceInit.requireConfirmedInputs)
)
val sessionId = randomBytes32()
val txBuilder = context.spawnAnonymous(InteractiveTxBuilder(
sessionId,
nodeParams, fundingParams,
channelParams = d.commitments.params,
purpose = InteractiveTxBuilder.SpliceTx(parentCommitment),
localPushAmount = cmd.pushAmount, remotePushAmount = msg.pushAmount,
wallet
))
txBuilder ! InteractiveTxBuilder.Start(self)
stay() using d.copy(spliceStatus = SpliceStatus.SpliceInProgress(cmd_opt = Some(cmd), splice = txBuilder, remoteCommitSig = None))
stay() using d.copy(spliceStatus = SpliceStatus.SpliceInProgress(cmd_opt = Some(cmd), sessionId, txBuilder, remoteCommitSig = None))
case _ =>
log.info(s"ignoring unexpected splice_ack=$msg")
stay()
}

case Event(msg: InteractiveTxConstructionMessage, d: DATA_NORMAL) =>
d.spliceStatus match {
case SpliceStatus.SpliceInProgress(_, txBuilder, _) =>
case SpliceStatus.SpliceInProgress(_, _, txBuilder, _) =>
txBuilder ! InteractiveTxBuilder.ReceiveMessage(msg)
stay()
case _ =>
Expand All @@ -1021,7 +1025,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with

case Event(msg: TxAbort, d: DATA_NORMAL) =>
d.spliceStatus match {
case SpliceStatus.SpliceInProgress(cmd_opt, txBuilder, _) =>
case SpliceStatus.SpliceInProgress(cmd_opt, _, txBuilder, _) =>
log.info("our peer aborted the splice attempt: ascii='{}' bin={}", msg.toAscii, msg.data)
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, SpliceAttemptAborted(d.channelId)))
txBuilder ! InteractiveTxBuilder.Abort
Expand Down Expand Up @@ -1053,9 +1057,15 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with

case Event(msg: InteractiveTxBuilder.Response, d: DATA_NORMAL) =>
d.spliceStatus match {
case SpliceStatus.SpliceInProgress(cmd_opt, _, remoteCommitSig_opt) =>
case SpliceStatus.SpliceInProgress(cmd_opt, currentSessionId, _, remoteCommitSig_opt) =>
msg match {
case InteractiveTxBuilder.SendMessage(msg) => stay() sending msg
case InteractiveTxBuilder.SendMessage(sessionId, msg) =>
if (sessionId == currentSessionId) {
stay() sending msg
} else {
log.info("ignoring outgoing interactive-tx message {} from previous session", msg.getClass.getSimpleName)
stay()
}
case InteractiveTxBuilder.Succeeded(signingSession, commitSig) =>
log.info(s"splice tx created with fundingTxIndex=${signingSession.fundingTxIndex} fundingTxId=${signingSession.fundingTx.txId}")
cmd_opt.foreach(cmd => cmd.replyTo ! RES_SPLICE(fundingTxIndex = signingSession.fundingTxIndex, signingSession.fundingTx.txId, signingSession.fundingParams.fundingAmount, signingSession.localCommit.fold(_.spec, _.spec).toLocal))
Expand Down Expand Up @@ -2729,7 +2739,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case SpliceStatus.QuiescenceRequested(cmd) => Some(cmd)
case SpliceStatus.InitiatorQuiescent(cmd) => Some(cmd)
case SpliceStatus.SpliceRequested(cmd, _) => Some(cmd)
case SpliceStatus.SpliceInProgress(cmd_opt, txBuilder, _) =>
case SpliceStatus.SpliceInProgress(cmd_opt, _, txBuilder, _) =>
txBuilder ! InteractiveTxBuilder.Abort
cmd_opt
case _ => None
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import fr.acinq.eclair.channel.publish.TxPublisher.SetChannelId
import fr.acinq.eclair.crypto.ShaChain
import fr.acinq.eclair.io.Peer.OpenChannelResponse
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{MilliSatoshiLong, RealShortChannelId, ToMilliSatoshiConversion, UInt64}
import fr.acinq.eclair.{MilliSatoshiLong, RealShortChannelId, ToMilliSatoshiConversion, UInt64, randomBytes32}

/**
* Created by t-bast on 19/04/2022.
Expand Down Expand Up @@ -212,9 +212,9 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
targetFeerate = open.fundingFeerate,
requireConfirmedInputs = RequireConfirmedInputs(forLocal = open.requireConfirmedInputs, forRemote = accept.requireConfirmedInputs)
)

val purpose = InteractiveTxBuilder.FundingTx(open.commitmentFeerate, open.firstPerCommitmentPoint)
val txBuilder = context.spawnAnonymous(InteractiveTxBuilder(
randomBytes32(),
nodeParams, fundingParams,
channelParams, purpose,
localPushAmount = accept.pushAmount, remotePushAmount = open.pushAmount,
Expand Down Expand Up @@ -277,6 +277,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
)
val purpose = InteractiveTxBuilder.FundingTx(d.lastSent.commitmentFeerate, accept.firstPerCommitmentPoint)
val txBuilder = context.spawnAnonymous(InteractiveTxBuilder(
randomBytes32(),
nodeParams, fundingParams,
channelParams, purpose,
localPushAmount = d.lastSent.pushAmount, remotePushAmount = accept.pushAmount,
Expand Down Expand Up @@ -331,7 +332,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
stay() using d.copy(deferred = Some(commitSig))

case Event(msg: InteractiveTxBuilder.Response, d: DATA_WAIT_FOR_DUAL_FUNDING_CREATED) => msg match {
case InteractiveTxBuilder.SendMessage(msg) => stay() sending msg
case InteractiveTxBuilder.SendMessage(_, msg) => stay() sending msg
case InteractiveTxBuilder.Succeeded(status, commitSig) =>
d.deferred.foreach(self ! _)
d.replyTo_opt.foreach(_ ! OpenChannelResponse.Created(d.channelId, status.fundingTx.txId, status.fundingTx.tx.localFees.truncateToSatoshi))
Expand Down Expand Up @@ -543,6 +544,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
targetFeerate = msg.feerate
)
val txBuilder = context.spawnAnonymous(InteractiveTxBuilder(
randomBytes32(),
nodeParams, fundingParams,
channelParams = d.commitments.params,
purpose = InteractiveTxBuilder.PreviousTxRbf(d.commitments.active.head, 0 msat, 0 msat, previousTransactions = d.allFundingTxs.map(_.sharedTx)),
Expand Down Expand Up @@ -580,6 +582,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
targetFeerate = cmd.targetFeerate,
)
val txBuilder = context.spawnAnonymous(InteractiveTxBuilder(
randomBytes32(),
nodeParams, fundingParams,
channelParams = d.commitments.params,
purpose = InteractiveTxBuilder.PreviousTxRbf(d.commitments.active.head, 0 msat, 0 msat, previousTransactions = d.allFundingTxs.map(_.sharedTx)),
Expand Down Expand Up @@ -657,7 +660,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
d.rbfStatus match {
case RbfStatus.RbfInProgress(cmd_opt, _, remoteCommitSig_opt) =>
msg match {
case InteractiveTxBuilder.SendMessage(msg) => stay() sending msg
case InteractiveTxBuilder.SendMessage(_, msg) => stay() sending msg
case InteractiveTxBuilder.Succeeded(signingSession, commitSig) =>
cmd_opt.foreach(cmd => cmd.replyTo ! RES_BUMP_FUNDING_FEE(rbfIndex = d.previousFundingTxs.length, signingSession.fundingTx.txId, signingSession.fundingTx.tx.localFees.truncateToSatoshi))
remoteCommitSig_opt.foreach(self ! _)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ object InteractiveTxBuilder {
private case object UtxosUnlocked extends Command

sealed trait Response
case class SendMessage(msg: LightningMessage) extends Response
case class SendMessage(sessionId: ByteVector32, msg: LightningMessage) extends Response
case class Succeeded(signingSession: InteractiveTxSigningSession.WaitingForSigs, commitSig: CommitSig) extends Response
sealed trait Failed extends Response { def cause: ChannelException }
case class LocalFailure(cause: ChannelException) extends Failed
Expand Down Expand Up @@ -336,7 +336,8 @@ object InteractiveTxBuilder {
}
// @formatter:on

def apply(nodeParams: NodeParams,
def apply(sessionId: ByteVector32,
nodeParams: NodeParams,
fundingParams: InteractiveTxParams,
channelParams: ChannelParams,
purpose: Purpose,
Expand All @@ -361,7 +362,7 @@ object InteractiveTxBuilder {
replyTo ! LocalFailure(InvalidFundingBalances(channelParams.channelId, fundingParams.fundingAmount, nextLocalBalance, nextRemoteBalance))
Behaviors.stopped
} else {
val actor = new InteractiveTxBuilder(replyTo, nodeParams, channelParams, fundingParams, purpose, localPushAmount, remotePushAmount, wallet, stash, context)
val actor = new InteractiveTxBuilder(replyTo, sessionId, nodeParams, channelParams, fundingParams, purpose, localPushAmount, remotePushAmount, wallet, stash, context)
actor.start()
}
case Abort => Behaviors.stopped
Expand All @@ -377,6 +378,7 @@ object InteractiveTxBuilder {
}

private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Response],
sessionId: ByteVector32,
nodeParams: NodeParams,
channelParams: ChannelParams,
fundingParams: InteractiveTxBuilder.InteractiveTxParams,
Expand Down Expand Up @@ -448,16 +450,16 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
case i: Input.Local => TxAddInput(fundingParams.channelId, i.serialId, Some(i.previousTx), i.previousTxOutput, i.sequence)
case i: Input.Shared => TxAddInput(fundingParams.channelId, i.serialId, i.outPoint, i.sequence)
}
replyTo ! SendMessage(message)
replyTo ! SendMessage(sessionId, message)
val next = session.copy(toSend = tail, localInputs = session.localInputs :+ addInput, txCompleteSent = false)
receive(next)
case (addOutput: Output) +: tail =>
val message = TxAddOutput(fundingParams.channelId, addOutput.serialId, addOutput.amount, addOutput.pubkeyScript)
replyTo ! SendMessage(message)
replyTo ! SendMessage(sessionId, message)
val next = session.copy(toSend = tail, localOutputs = session.localOutputs :+ addOutput, txCompleteSent = false)
receive(next)
case Nil =>
replyTo ! SendMessage(TxComplete(fundingParams.channelId))
replyTo ! SendMessage(sessionId, TxComplete(fundingParams.channelId))
val next = session.copy(txCompleteSent = true)
if (next.isComplete) {
validateAndSign(next)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -122,48 +122,56 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit
}

def spawnTxBuilderAlice(wallet: OnChainWallet, fundingParams: InteractiveTxParams = fundingParamsA): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder(
ByteVector32.Zeroes,
nodeParamsA, fundingParams, channelParamsA,
FundingTx(commitFeerate, firstPerCommitmentPointB),
0 msat, 0 msat,
wallet))

def spawnTxBuilderRbfAlice(fundingParams: InteractiveTxParams, commitment: Commitment, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder(
ByteVector32.Zeroes,
nodeParamsA, fundingParams, channelParamsA,
PreviousTxRbf(commitment, 0 msat, 0 msat, previousTransactions),
0 msat, 0 msat,
wallet))

def spawnTxBuilderSpliceAlice(fundingParams: InteractiveTxParams, commitment: Commitment, wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder(
ByteVector32.Zeroes,
nodeParamsA, fundingParams, channelParamsA,
SpliceTx(commitment),
0 msat, 0 msat,
wallet))

def spawnTxBuilderSpliceRbfAlice(fundingParams: InteractiveTxParams, parentCommitment: Commitment, replacedCommitment: Commitment, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder(
ByteVector32.Zeroes,
nodeParamsA, fundingParams, channelParamsA,
PreviousTxRbf(replacedCommitment, parentCommitment.localCommit.spec.toLocal, parentCommitment.remoteCommit.spec.toLocal, previousTransactions),
0 msat, 0 msat,
wallet))

def spawnTxBuilderBob(wallet: OnChainWallet, fundingParams: InteractiveTxParams = fundingParamsB): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder(
ByteVector32.Zeroes,
nodeParamsB, fundingParams, channelParamsB,
FundingTx(commitFeerate, firstPerCommitmentPointA),
0 msat, 0 msat,
wallet))

def spawnTxBuilderRbfBob(fundingParams: InteractiveTxParams, commitment: Commitment, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder(
ByteVector32.Zeroes,
nodeParamsB, fundingParams, channelParamsB,
PreviousTxRbf(commitment, 0 msat, 0 msat, previousTransactions),
0 msat, 0 msat,
wallet))

def spawnTxBuilderSpliceBob(fundingParams: InteractiveTxParams, commitment: Commitment, wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder(
ByteVector32.Zeroes,
nodeParamsB, fundingParams, channelParamsB,
SpliceTx(commitment),
0 msat, 0 msat,
wallet))

def spawnTxBuilderSpliceRbfBob(fundingParams: InteractiveTxParams, parentCommitment: Commitment, replacedCommitment: Commitment, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder(
ByteVector32.Zeroes,
nodeParamsB, fundingParams, channelParamsB,
PreviousTxRbf(replacedCommitment, parentCommitment.localCommit.spec.toLocal, parentCommitment.remoteCommit.spec.toLocal, previousTransactions),
0 msat, 0 msat,
Expand Down

0 comments on commit 12adf87

Please sign in to comment.