From 7d1c23efadb53eb1cd642de11b45a177b6f9dcb3 Mon Sep 17 00:00:00 2001 From: t-bast Date: Wed, 31 Jul 2024 16:45:30 +0200 Subject: [PATCH] Add support for RBF-ing splice transactions If the latest splice transaction doesn't confirm, we allow exchanging `tx_init_rbf` and `tx_ack_rbf` to create another splice transaction to replace it. We use the same funding contribution as the previous splice. We disallow creating another splice transaction using `splice_init` if we have several RBF attempts for the latest splice: we cannot know which one of them will confirm and should be spent by the new splice. TODO: needs tests --- .../fr/acinq/eclair/channel/ChannelData.scala | 97 ++++--- .../eclair/channel/ChannelExceptions.scala | 1 + .../fr/acinq/eclair/channel/fsm/Channel.scala | 265 +++++++++++++++--- .../channel/fsm/ChannelOpenDualFunded.scala | 124 ++++---- .../channel/fsm/DualFundingHandlers.scala | 8 +- .../channel/fund/InteractiveTxBuilder.scala | 26 +- .../channel/fund/InteractiveTxFunder.scala | 6 +- .../channel/version4/ChannelCodecs4.scala | 10 +- .../wire/protocol/LightningMessageTypes.scala | 6 +- .../channel/InteractiveTxBuilderSpec.scala | 26 +- ...WaitForDualFundingConfirmedStateSpec.scala | 40 +-- .../io/PendingChannelsRateLimiterSpec.scala | 4 +- .../channel/version4/ChannelCodecs4Spec.scala | 14 +- 13 files changed, 435 insertions(+), 192 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala index 05aeb6f985..2495e161f0 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala @@ -26,7 +26,7 @@ import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningS import fr.acinq.eclair.io.Peer import fr.acinq.eclair.transactions.CommitmentSpec import fr.acinq.eclair.transactions.Transactions._ -import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelReady, ChannelReestablish, ChannelUpdate, ClosingSigned, CommitSig, FailureMessage, FundingCreated, FundingSigned, Init, OnionRoutingPacket, OpenChannel, OpenDualFundedChannel, Shutdown, SpliceInit, Stfu, TxSignatures, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc} +import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelReady, ChannelReestablish, ChannelUpdate, ClosingSigned, CommitSig, FailureMessage, FundingCreated, FundingSigned, Init, OnionRoutingPacket, OpenChannel, OpenDualFundedChannel, Shutdown, SpliceInit, Stfu, TxInitRbf, TxSignatures, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc} import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Features, InitFeature, MilliSatoshi, MilliSatoshiLong, RealShortChannelId, TimestampMilli, UInt64} import scodec.bits.ByteVector @@ -193,16 +193,16 @@ sealed trait Command extends PossiblyHarmful sealed trait HasReplyToCommand extends Command { def replyTo: ActorRef } sealed trait HasOptionalReplyToCommand extends Command { def replyTo_opt: Option[ActorRef] } -sealed trait ForbiddenCommandDuringSplice extends Command -sealed trait ForbiddenCommandDuringQuiescence extends Command +sealed trait ForbiddenCommandDuringQuiescenceNegotiation extends Command +sealed trait ForbiddenCommandWhenQuiescent extends Command -final case class CMD_ADD_HTLC(replyTo: ActorRef, amount: MilliSatoshi, paymentHash: ByteVector32, cltvExpiry: CltvExpiry, onion: OnionRoutingPacket, nextBlindingKey_opt: Option[PublicKey], confidence: Double, origin: Origin.Hot, commit: Boolean = false) extends HasReplyToCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence -sealed trait HtlcSettlementCommand extends HasOptionalReplyToCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence { def id: Long } +final case class CMD_ADD_HTLC(replyTo: ActorRef, amount: MilliSatoshi, paymentHash: ByteVector32, cltvExpiry: CltvExpiry, onion: OnionRoutingPacket, nextBlindingKey_opt: Option[PublicKey], confidence: Double, origin: Origin.Hot, commit: Boolean = false) extends HasReplyToCommand with ForbiddenCommandDuringQuiescenceNegotiation with ForbiddenCommandWhenQuiescent +sealed trait HtlcSettlementCommand extends HasOptionalReplyToCommand with ForbiddenCommandDuringQuiescenceNegotiation with ForbiddenCommandWhenQuiescent { def id: Long } final case class CMD_FULFILL_HTLC(id: Long, r: ByteVector32, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand final case class CMD_FAIL_HTLC(id: Long, reason: Either[ByteVector, FailureMessage], delay_opt: Option[FiniteDuration] = None, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand final case class CMD_FAIL_MALFORMED_HTLC(id: Long, onionHash: ByteVector32, failureCode: Int, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand -final case class CMD_UPDATE_FEE(feeratePerKw: FeeratePerKw, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HasOptionalReplyToCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence -final case class CMD_SIGN(replyTo_opt: Option[ActorRef] = None) extends HasOptionalReplyToCommand with ForbiddenCommandDuringSplice +final case class CMD_UPDATE_FEE(feeratePerKw: FeeratePerKw, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HasOptionalReplyToCommand with ForbiddenCommandDuringQuiescenceNegotiation with ForbiddenCommandWhenQuiescent +final case class CMD_SIGN(replyTo_opt: Option[ActorRef] = None) extends HasOptionalReplyToCommand with ForbiddenCommandWhenQuiescent final case class ClosingFees(preferred: Satoshi, min: Satoshi, max: Satoshi) final case class ClosingFeerates(preferred: FeeratePerKw, min: FeeratePerKw, max: FeeratePerKw) { @@ -210,19 +210,22 @@ final case class ClosingFeerates(preferred: FeeratePerKw, min: FeeratePerKw, max } sealed trait CloseCommand extends HasReplyToCommand -final case class CMD_CLOSE(replyTo: ActorRef, scriptPubKey: Option[ByteVector], feerates: Option[ClosingFeerates]) extends CloseCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence +final case class CMD_CLOSE(replyTo: ActorRef, scriptPubKey: Option[ByteVector], feerates: Option[ClosingFeerates]) extends CloseCommand with ForbiddenCommandDuringQuiescenceNegotiation with ForbiddenCommandWhenQuiescent final case class CMD_FORCECLOSE(replyTo: ActorRef) extends CloseCommand final case class CMD_BUMP_FORCE_CLOSE_FEE(replyTo: akka.actor.typed.ActorRef[CommandResponse[CMD_BUMP_FORCE_CLOSE_FEE]], confirmationTarget: ConfirmationTarget) extends Command -final case class CMD_BUMP_FUNDING_FEE(replyTo: akka.actor.typed.ActorRef[CommandResponse[CMD_BUMP_FUNDING_FEE]], targetFeerate: FeeratePerKw, fundingFeeBudget: Satoshi, lockTime: Long) extends Command +sealed trait ChannelFundingCommand extends Command { + def replyTo: akka.actor.typed.ActorRef[CommandResponse[ChannelFundingCommand]] +} case class SpliceIn(additionalLocalFunding: Satoshi, pushAmount: MilliSatoshi = 0 msat) case class SpliceOut(amount: Satoshi, scriptPubKey: ByteVector) -final case class CMD_SPLICE(replyTo: akka.actor.typed.ActorRef[CommandResponse[CMD_SPLICE]], spliceIn_opt: Option[SpliceIn], spliceOut_opt: Option[SpliceOut]) extends Command { +final case class CMD_SPLICE(replyTo: akka.actor.typed.ActorRef[CommandResponse[ChannelFundingCommand]], spliceIn_opt: Option[SpliceIn], spliceOut_opt: Option[SpliceOut]) extends ChannelFundingCommand { require(spliceIn_opt.isDefined || spliceOut_opt.isDefined, "there must be a splice-in or a splice-out") val additionalLocalFunding: Satoshi = spliceIn_opt.map(_.additionalLocalFunding).getOrElse(0 sat) val pushAmount: MilliSatoshi = spliceIn_opt.map(_.pushAmount).getOrElse(0 msat) val spliceOutputs: List[TxOut] = spliceOut_opt.toList.map(s => TxOut(s.amount, s.scriptPubKey)) } +final case class CMD_BUMP_FUNDING_FEE(replyTo: akka.actor.typed.ActorRef[CommandResponse[ChannelFundingCommand]], targetFeerate: FeeratePerKw, fundingFeeBudget: Satoshi, lockTime: Long) extends ChannelFundingCommand final case class CMD_UPDATE_RELAY_FEE(replyTo: ActorRef, feeBase: MilliSatoshi, feeProportionalMillionths: Long) extends HasReplyToCommand final case class CMD_GET_CHANNEL_STATE(replyTo: ActorRef) extends HasReplyToCommand final case class CMD_GET_CHANNEL_DATA(replyTo: ActorRef) extends HasReplyToCommand @@ -456,42 +459,61 @@ object RemoteFundingStatus { case object Locked extends RemoteFundingStatus } -sealed trait RbfStatus -object RbfStatus { - case object NoRbf extends RbfStatus - case class RbfRequested(cmd: CMD_BUMP_FUNDING_FEE) extends RbfStatus - case class RbfInProgress(cmd_opt: Option[CMD_BUMP_FUNDING_FEE], rbf: typed.ActorRef[InteractiveTxBuilder.Command], remoteCommitSig: Option[CommitSig]) extends RbfStatus - case class RbfWaitingForSigs(signingSession: InteractiveTxSigningSession.WaitingForSigs) extends RbfStatus - case object RbfAborted extends RbfStatus +sealed trait DualFundingStatus +object DualFundingStatus { + /** We're waiting for one of the funding transactions to confirm. */ + case object WaitingForConfirmations extends DualFundingStatus + /** We told our peer we want to RBF the funding transaction. */ + case class RbfRequested(cmd: CMD_BUMP_FUNDING_FEE) extends DualFundingStatus + /** We both agreed to RBF and are building the new funding transaction. */ + case class RbfInProgress(cmd_opt: Option[CMD_BUMP_FUNDING_FEE], rbf: typed.ActorRef[InteractiveTxBuilder.Command], remoteCommitSig: Option[CommitSig]) extends DualFundingStatus + /** A new funding transaction has been negotiated, we're exchanging signatures. */ + case class RbfWaitingForSigs(signingSession: InteractiveTxSigningSession.WaitingForSigs) extends DualFundingStatus + /** The RBF attempt was aborted by us, we're waiting for our peer to ack. */ + case object RbfAborted extends DualFundingStatus } -sealed trait SpliceStatus /** We're waiting for the channel to be quiescent. */ -sealed trait QuiescenceNegotiation extends SpliceStatus +sealed trait QuiescenceNegotiation object QuiescenceNegotiation { sealed trait Initiator extends QuiescenceNegotiation + object Initiator { + /** We stop sending new updates and wait for our updates to be added to the local and remote commitments. */ + case object QuiescenceRequested extends Initiator + /** Our updates have been added to the local and remote commitments, we wait for our peer to do the same. */ + case class SentStfu(stfu: Stfu) extends Initiator + } + sealed trait NonInitiator extends QuiescenceNegotiation + object NonInitiator { + /** Our peer has asked us to stop sending new updates and wait for our updates to be added to the local and remote commitments. */ + case class ReceivedStfu(stfu: Stfu) extends NonInitiator + } +} + +sealed trait SpliceStatus { + def isNegotiatingQuiescence: Boolean = this.isInstanceOf[SpliceStatus.NegotiatingQuiescence] + def isQuiescent: Boolean = this match { + case SpliceStatus.NoSplice | _: SpliceStatus.NegotiatingQuiescence => false + case _ => true + } } -/** The channel is quiescent and a splice attempt was initiated. */ -sealed trait QuiescentSpliceStatus extends SpliceStatus object SpliceStatus { case object NoSplice extends SpliceStatus - /** We stop sending new updates and wait for our updates to be added to the local and remote commitments. */ - case class QuiescenceRequested(splice: CMD_SPLICE) extends QuiescenceNegotiation.Initiator - /** Our updates have been added to the local and remote commitments, we wait for our peer to do the same. */ - case class InitiatorQuiescent(splice: CMD_SPLICE) extends QuiescenceNegotiation.Initiator - /** Our peer has asked us to stop sending new updates and wait for our updates to be added to the local and remote commitments. */ - case class ReceivedStfu(stfu: Stfu) extends QuiescenceNegotiation.NonInitiator - /** Our updates have been added to the local and remote commitments, we wait for our peer to use the now quiescent channel. */ - case object NonInitiatorQuiescent extends QuiescentSpliceStatus + /** We're trying to quiesce the channel in order to negotiate a splice. */ + case class NegotiatingQuiescence(cmd_opt: Option[ChannelFundingCommand], status: QuiescenceNegotiation) extends SpliceStatus + /** The channel is quiescent, we wait for our peer to send splice_init or tx_init_rbf. */ + case object NonInitiatorQuiescent extends 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], sessionId: ByteVector32, splice: typed.ActorRef[InteractiveTxBuilder.Command], remoteCommitSig: Option[CommitSig]) extends QuiescentSpliceStatus + case class SpliceRequested(cmd: CMD_SPLICE, init: SpliceInit) extends SpliceStatus + /** We told our peer we want to RBF the latest splice transaction. */ + case class RbfRequested(cmd: CMD_BUMP_FUNDING_FEE, rbf: TxInitRbf) extends SpliceStatus + /** We both agreed to splice/rbf and are building the corresponding transaction. */ + case class SpliceInProgress(cmd_opt: Option[ChannelFundingCommand], sessionId: ByteVector32, splice: typed.ActorRef[InteractiveTxBuilder.Command], remoteCommitSig: Option[CommitSig]) extends SpliceStatus /** The splice transaction has been negotiated, we're exchanging signatures. */ - case class SpliceWaitingForSigs(signingSession: InteractiveTxSigningSession.WaitingForSigs) extends QuiescentSpliceStatus + case class SpliceWaitingForSigs(signingSession: InteractiveTxSigningSession.WaitingForSigs) extends SpliceStatus /** The splice attempt was aborted by us, we're waiting for our peer to ack. */ - case object SpliceAborted extends QuiescentSpliceStatus + case object SpliceAborted extends SpliceStatus } sealed trait ChannelData extends PossiblyHarmful { @@ -585,7 +607,7 @@ final case class DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments: Commitments, remotePushAmount: MilliSatoshi, waitingSince: BlockHeight, // how long have we been waiting for a funding tx to confirm lastChecked: BlockHeight, // last time we checked if the channel was double-spent - rbfStatus: RbfStatus, + status: DualFundingStatus, deferred: Option[ChannelReady]) extends ChannelDataWithCommitments { def allFundingTxs: Seq[DualFundedUnconfirmedFundingTx] = commitments.active.map(_.localFundingStatus).collect { case fundingTx: DualFundedUnconfirmedFundingTx => fundingTx } def latestFundingTx: DualFundedUnconfirmedFundingTx = commitments.latest.localFundingStatus.asInstanceOf[DualFundedUnconfirmedFundingTx] @@ -600,7 +622,10 @@ final case class DATA_NORMAL(commitments: Commitments, localShutdown: Option[Shutdown], remoteShutdown: Option[Shutdown], closingFeerates: Option[ClosingFeerates], - spliceStatus: SpliceStatus) extends ChannelDataWithCommitments + spliceStatus: SpliceStatus) extends ChannelDataWithCommitments { + val isNegotiatingQuiescence: Boolean = spliceStatus.isNegotiatingQuiescence + val isQuiescent: Boolean = spliceStatus.isQuiescent +} final case class DATA_SHUTDOWN(commitments: Commitments, localShutdown: Shutdown, remoteShutdown: Shutdown, closingFeerates: Option[ClosingFeerates]) extends ChannelDataWithCommitments final case class DATA_NEGOTIATING(commitments: Commitments, localShutdown: Shutdown, remoteShutdown: Shutdown, diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala index cea5739003..404e31c9f0 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala @@ -85,6 +85,7 @@ case class InvalidRbfAttemptsExhausted (override val channelId: Byte case class InvalidRbfAttemptTooSoon (override val channelId: ByteVector32, previousAttempt: BlockHeight, nextAttempt: BlockHeight) extends ChannelException(channelId, s"invalid rbf attempt: last attempt made at block=$previousAttempt, next attempt available after block=$nextAttempt") case class InvalidSpliceTxAbortNotAcked (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid splice attempt: our previous tx_abort has not been acked") case class InvalidSpliceNotQuiescent (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid splice attempt: the channel is not quiescent") +case class InvalidSpliceWithUnconfirmedRbf (override val channelId: ByteVector32, previousTxs: Seq[TxId]) extends ChannelException(channelId, s"invalid splice attempt: the previous splice was rbf-ed and is still unconfirmed (txIds=${previousTxs.mkString(", ")})") case class InvalidRbfTxConfirmed (override val channelId: ByteVector32) extends ChannelException(channelId, "no need to rbf, transaction is already confirmed") case class InvalidRbfNonInitiator (override val channelId: ByteVector32) extends ChannelException(channelId, "cannot initiate rbf: we're not the initiator of this interactive-tx attempt") case class InvalidRbfZeroConf (override val channelId: ByteVector32) extends ChannelException(channelId, "cannot initiate rbf: we're using zero-conf for this interactive-tx attempt") diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index f600579f31..da246991d6 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -387,7 +387,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with */ when(NORMAL)(handleExceptions { - case Event(c: ForbiddenCommandDuringQuiescence, d: DATA_NORMAL) if d.spliceStatus.isInstanceOf[QuiescenceNegotiation] => + case Event(c: ForbiddenCommandDuringQuiescenceNegotiation, d: DATA_NORMAL) if d.isNegotiatingQuiescence => val error = ForbiddenDuringQuiescence(d.channelId, c.getClass.getSimpleName) c match { case c: CMD_ADD_HTLC => handleAddHtlcCommandError(c, error, Some(d.channelUpdate)) @@ -398,7 +398,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with case _ => handleCommandError(error, c) } - case Event(c: ForbiddenCommandDuringSplice, d: DATA_NORMAL) if d.spliceStatus.isInstanceOf[QuiescentSpliceStatus] => + case Event(c: ForbiddenCommandWhenQuiescent, d: DATA_NORMAL) if d.isQuiescent => val error = ForbiddenDuringSplice(d.channelId, c.getClass.getSimpleName) c match { case c: CMD_ADD_HTLC => handleAddHtlcCommandError(c, error, Some(d.channelUpdate)) @@ -409,7 +409,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with case _ => handleCommandError(error, c) } - case Event(msg: ForbiddenMessageDuringSplice, d: DATA_NORMAL) if d.spliceStatus.isInstanceOf[QuiescentSpliceStatus] => + case Event(msg: ForbiddenMessageWhenQuiescent, d: DATA_NORMAL) if d.isQuiescent => log.warning("received forbidden message {} during splicing with status {}", msg.getClass.getSimpleName, d.spliceStatus.getClass.getSimpleName) val error = ForbiddenDuringSplice(d.channelId, msg.getClass.getSimpleName) // We forward preimages as soon as possible to the upstream channel because it allows us to pull funds. @@ -611,10 +611,11 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with context.system.eventStream.publish(ChannelSignatureReceived(self, commitments1)) // If we're now quiescent, we may send our stfu message. val (d1, toSend) = d.spliceStatus match { - case SpliceStatus.QuiescenceRequested(cmd) if commitments1.localIsQuiescent => + case SpliceStatus.NegotiatingQuiescence(cmd_opt, QuiescenceNegotiation.Initiator.QuiescenceRequested) if commitments1.localIsQuiescent => val stfu = Stfu(d.channelId, initiator = true) - (d.copy(commitments = commitments1, spliceStatus = SpliceStatus.InitiatorQuiescent(cmd)), Seq(revocation, stfu)) - case _: SpliceStatus.ReceivedStfu if commitments1.localIsQuiescent => + val spliceStatus1 = SpliceStatus.NegotiatingQuiescence(cmd_opt, QuiescenceNegotiation.Initiator.SentStfu(stfu)) + (d.copy(commitments = commitments1, spliceStatus = spliceStatus1), Seq(revocation, stfu)) + case SpliceStatus.NegotiatingQuiescence(_, _: QuiescenceNegotiation.NonInitiator.ReceivedStfu) if commitments1.localIsQuiescent => val stfu = Stfu(d.channelId, initiator = false) (d.copy(commitments = commitments1, spliceStatus = SpliceStatus.NonInitiatorQuiescent), Seq(revocation, stfu)) case _ => @@ -851,24 +852,64 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with } case Event(cmd: CMD_SPLICE, d: DATA_NORMAL) => - if (d.commitments.params.remoteParams.initFeatures.hasFeature(Features.Splicing)) { + if (d.commitments.active.count(_.fundingTxIndex == d.commitments.latest.fundingTxIndex) > 1) { + log.warning("cannot initiate splice, the previous splice has unconfirmed rbf attempts") + cmd.replyTo ! RES_FAILURE(cmd, CommandUnavailableInThisState(d.channelId, "splice", NORMAL)) + stay() + } else if (!d.commitments.params.remoteParams.initFeatures.hasFeature(Features.Splicing)) { + log.warning("cannot initiate splice, peer doesn't support splicing") + cmd.replyTo ! RES_FAILURE(cmd, CommandUnavailableInThisState(d.channelId, "splice", NORMAL)) + stay() + } else { d.spliceStatus match { case SpliceStatus.NoSplice => startSingleTimer(QuiescenceTimeout.toString, QuiescenceTimeout(peer), nodeParams.channelConf.quiescenceTimeout) if (d.commitments.localIsQuiescent) { - stay() using d.copy(spliceStatus = SpliceStatus.InitiatorQuiescent(cmd)) sending Stfu(d.channelId, initiator = true) + val stfu = Stfu(d.channelId, initiator = true) + val spliceStatus1 = SpliceStatus.NegotiatingQuiescence(Some(cmd), QuiescenceNegotiation.Initiator.SentStfu(stfu)) + stay() using d.copy(spliceStatus = spliceStatus1) sending stfu } else { - stay() using d.copy(spliceStatus = SpliceStatus.QuiescenceRequested(cmd)) + val spliceStatus1 = SpliceStatus.NegotiatingQuiescence(Some(cmd), QuiescenceNegotiation.Initiator.QuiescenceRequested) + stay() using d.copy(spliceStatus = spliceStatus1) } case _ => log.warning("cannot initiate splice, another one is already in progress") cmd.replyTo ! RES_FAILURE(cmd, InvalidSpliceAlreadyInProgress(d.channelId)) stay() } - } else { - log.warning("cannot initiate splice, peer doesn't support splices") - cmd.replyTo ! RES_FAILURE(cmd, CommandUnavailableInThisState(d.channelId, "splice", NORMAL)) - stay() + } + + case Event(cmd: CMD_BUMP_FUNDING_FEE, d: DATA_NORMAL) => + d.spliceStatus match { + case SpliceStatus.NoSplice => + d.commitments.latest.localFundingStatus match { + case fundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx if fundingTx.fundingParams.minNextFeerate <= cmd.targetFeerate => + startSingleTimer(QuiescenceTimeout.toString, QuiescenceTimeout(peer), nodeParams.channelConf.quiescenceTimeout) + if (d.commitments.localIsQuiescent) { + val stfu = Stfu(d.channelId, initiator = true) + val spliceStatus1 = SpliceStatus.NegotiatingQuiescence(Some(cmd), QuiescenceNegotiation.Initiator.SentStfu(stfu)) + stay() using d.copy(spliceStatus = spliceStatus1) sending stfu + } else { + val spliceStatus1 = SpliceStatus.NegotiatingQuiescence(Some(cmd), QuiescenceNegotiation.Initiator.QuiescenceRequested) + stay() using d.copy(spliceStatus = spliceStatus1) + } + case fundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx => + log.warning("cannot initiate rbf: feerate too low ({} < {})", cmd.targetFeerate, fundingTx.fundingParams.minNextFeerate) + cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfFeerate(d.channelId, cmd.targetFeerate, fundingTx.fundingParams.minNextFeerate)) + stay() + case _: LocalFundingStatus.ZeroconfPublishedFundingTx => + log.warning("cannot initiate rbf: transaction us using 0-conf") + cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfZeroConf(d.channelId)) + stay() + case _ => + log.warning("cannot initiate rbf: transaction already confirmed") + cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfTxConfirmed(d.channelId)) + stay() + } + case _ => + log.warning("cannot initiate rbf, a concurrent splice attempt is already in progress") + cmd.replyTo ! RES_FAILURE(cmd, InvalidSpliceAlreadyInProgress(d.channelId)) + stay() } case Event(_: Stfu, d: DATA_NORMAL) if d.localShutdown.isDefined => @@ -884,24 +925,34 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with if (d.commitments.localIsQuiescent) { stay() using d.copy(spliceStatus = SpliceStatus.NonInitiatorQuiescent) sending Stfu(d.channelId, initiator = false) } else { - stay() using d.copy(spliceStatus = SpliceStatus.ReceivedStfu(msg)) + stay() using d.copy(spliceStatus = SpliceStatus.NegotiatingQuiescence(None, QuiescenceNegotiation.NonInitiator.ReceivedStfu(msg))) } - case SpliceStatus.QuiescenceRequested(cmd) => + case SpliceStatus.NegotiatingQuiescence(cmd_opt, QuiescenceNegotiation.Initiator.QuiescenceRequested) => // We could keep track of our splice attempt and merge it with the remote splice instead of cancelling it. // But this is an edge case that should rarely occur, so it's probably not worth the additional complexity. log.warning("our peer initiated quiescence before us, cancelling our splice attempt") - cmd.replyTo ! RES_FAILURE(cmd, ConcurrentRemoteSplice(d.channelId)) - stay() using d.copy(spliceStatus = SpliceStatus.ReceivedStfu(msg)) - case SpliceStatus.InitiatorQuiescent(cmd) => - // if both sides send stfu at the same time, the quiescence initiator is the channel opener + cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, ConcurrentRemoteSplice(d.channelId))) + stay() using d.copy(spliceStatus = SpliceStatus.NegotiatingQuiescence(None, QuiescenceNegotiation.NonInitiator.ReceivedStfu(msg))) + case SpliceStatus.NegotiatingQuiescence(Some(cmd), QuiescenceNegotiation.Initiator.SentStfu(_)) => + // if both sides send stfu at the same time, the quiescence initiator is the channel opener. if (!msg.initiator || d.commitments.params.localParams.isChannelOpener) { - initiateSplice(cmd, d) match { - case Left(f) => - cmd.replyTo ! RES_FAILURE(cmd, f) - context.system.scheduler.scheduleOnce(2 second, peer, Peer.Disconnect(remoteNodeId)) - stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending Warning(d.channelId, f.getMessage) - case Right(spliceInit) => - stay() using d.copy(spliceStatus = SpliceStatus.SpliceRequested(cmd, spliceInit)) sending spliceInit + cmd match { + case cmd: CMD_SPLICE => initiateSplice(cmd, d) match { + case Left(f) => + cmd.replyTo ! RES_FAILURE(cmd, f) + context.system.scheduler.scheduleOnce(2 second, peer, Peer.Disconnect(remoteNodeId)) + stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending Warning(d.channelId, f.getMessage) + case Right(spliceInit) => + stay() using d.copy(spliceStatus = SpliceStatus.SpliceRequested(cmd, spliceInit)) sending spliceInit + } + case cmd: CMD_BUMP_FUNDING_FEE => initiateSpliceRbf(cmd, d) match { + case Left(f) => + cmd.replyTo ! RES_FAILURE(cmd, f) + context.system.scheduler.scheduleOnce(2 second, peer, Peer.Disconnect(remoteNodeId)) + stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending Warning(d.channelId, f.getMessage) + case Right(txInitRbf) => + stay() using d.copy(spliceStatus = SpliceStatus.RbfRequested(cmd, txInitRbf)) sending txInitRbf + } } } else { log.warning("concurrent stfu received and our peer is the channel initiator, cancelling our splice attempt") @@ -930,6 +981,10 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with } else if (msg.feerate < nodeParams.currentFeerates.minimum) { log.info("rejecting splice request: feerate too low") stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceRequest(d.channelId).getMessage) + } else if (d.commitments.active.count(_.fundingTxIndex == d.commitments.latest.fundingTxIndex) > 1) { + val previousTxs = d.commitments.active.filter(_.fundingTxIndex == d.commitments.latest.fundingTxIndex).map(_.fundingTxId) + log.info("rejecting splice request: the previous splice has unconfirmed rbf attempts ({})", previousTxs.mkString(", ")) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceWithUnconfirmedRbf(d.channelId, previousTxs).getMessage) } else { log.info(s"accepting splice with remote.in.amount=${msg.fundingContribution} remote.in.push=${msg.pushAmount}") val parentCommitment = d.commitments.latest.commitment @@ -1019,6 +1074,100 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with stay() sending Warning(d.channelId, UnexpectedInteractiveTxMessage(d.channelId, msg).getMessage) } + case Event(msg: TxInitRbf, d: DATA_NORMAL) => + d.spliceStatus match { + case SpliceStatus.NonInitiatorQuiescent => + getSpliceRbfContext(None, d) match { + case Right(_) if !d.commitments.isQuiescent => + log.info("rejecting rbf request: channel not quiescent") + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceNotQuiescent(d.channelId).getMessage) + case Right(rbf) if msg.feerate < rbf.latestFundingTx.fundingParams.minNextFeerate => + log.info("rejecting rbf request: the new feerate must be at least {} (proposed={})", rbf.latestFundingTx.fundingParams.minNextFeerate, msg.feerate) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidRbfFeerate(d.channelId, msg.feerate, rbf.latestFundingTx.fundingParams.minNextFeerate).getMessage) + case Right(rbf) if nodeParams.currentBlockHeight < rbf.latestFundingTx.createdAt + nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks => + log.info("rejecting rbf attempt: last attempt was less than {} blocks ago", nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidRbfAttemptTooSoon(d.channelId, rbf.latestFundingTx.createdAt, rbf.latestFundingTx.createdAt + nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks).getMessage) + case Right(rbf) => + log.info("accepting rbf with remote.in.amount={}", msg.fundingContribution) + // We use the same contribution as the previous splice attempt. + val fundingContribution = rbf.latestFundingTx.fundingParams.localContribution + val txAckRbf = TxAckRbf(d.channelId, fundingContribution, rbf.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote) + val fundingParams = InteractiveTxParams( + channelId = d.channelId, + isInitiator = false, + localContribution = fundingContribution, + remoteContribution = msg.fundingContribution, + sharedInput_opt = Some(Multisig2of2Input(rbf.parentCommitment)), + remoteFundingPubKey = rbf.latestFundingTx.fundingParams.remoteFundingPubKey, + localOutputs = rbf.latestFundingTx.fundingParams.localOutputs, + lockTime = msg.lockTime, + dustLimit = rbf.latestFundingTx.fundingParams.dustLimit, + targetFeerate = msg.feerate, + requireConfirmedInputs = RequireConfirmedInputs(forLocal = msg.requireConfirmedInputs, forRemote = txAckRbf.requireConfirmedInputs) + ) + val sessionId = randomBytes32() + val txBuilder = context.spawnAnonymous(InteractiveTxBuilder( + sessionId, + nodeParams, fundingParams, + channelParams = d.commitments.params, + purpose = rbf, + localPushAmount = 0 msat, remotePushAmount = 0 msat, + wallet + )) + txBuilder ! InteractiveTxBuilder.Start(self) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceInProgress(cmd_opt = None, sessionId, txBuilder, remoteCommitSig = None)) sending txAckRbf + case Left(f) => + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, f.getMessage) + } + case SpliceStatus.NoSplice => + log.info("rejecting rbf attempt: quiescence not negotiated") + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceNotQuiescent(d.channelId).getMessage) + case SpliceStatus.SpliceAborted => + log.info("rejecting rbf attempt: our previous tx_abort was not acked") + stay() sending Warning(d.channelId, InvalidRbfTxAbortNotAcked(d.channelId).getMessage) + case _ => + log.info("rejecting rbf attempt: the current attempt must be completed or aborted first") + stay() sending Warning(d.channelId, InvalidRbfAlreadyInProgress(d.channelId).getMessage) + } + + case Event(msg: TxAckRbf, d: DATA_NORMAL) => + d.spliceStatus match { + case SpliceStatus.RbfRequested(cmd, txInitRbf) => + getSpliceRbfContext(Some(cmd), d) match { + case Right(rbf) => + log.info("our peer accepted our rbf request and will contribute {} to the funding transaction", msg.fundingContribution) + val fundingParams = InteractiveTxParams( + channelId = d.channelId, + isInitiator = true, + localContribution = txInitRbf.fundingContribution, + remoteContribution = msg.fundingContribution, + sharedInput_opt = Some(Multisig2of2Input(rbf.parentCommitment)), + remoteFundingPubKey = rbf.latestFundingTx.fundingParams.remoteFundingPubKey, + localOutputs = rbf.latestFundingTx.fundingParams.localOutputs, + lockTime = txInitRbf.lockTime, + dustLimit = rbf.latestFundingTx.fundingParams.dustLimit, + targetFeerate = txInitRbf.feerate, + requireConfirmedInputs = RequireConfirmedInputs(forLocal = msg.requireConfirmedInputs, forRemote = txInitRbf.requireConfirmedInputs) + ) + val sessionId = randomBytes32() + val txBuilder = context.spawnAnonymous(InteractiveTxBuilder( + sessionId, + nodeParams, fundingParams, + channelParams = d.commitments.params, + purpose = rbf, + localPushAmount = 0 msat, remotePushAmount = 0 msat, + wallet + )) + txBuilder ! InteractiveTxBuilder.Start(self) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceInProgress(cmd_opt = Some(cmd), sessionId, txBuilder, remoteCommitSig = None)) + case Left(f) => + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, f.getMessage) + } + case _ => + log.info("ignoring unexpected tx_ack_rbf={}", msg) + stay() + } + case Event(msg: TxAbort, d: DATA_NORMAL) => d.spliceStatus match { case SpliceStatus.SpliceInProgress(cmd_opt, _, txBuilder, _) => @@ -1034,6 +1183,10 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with log.info("our peer rejected our splice attempt: ascii='{}' bin={}", msg.toAscii, msg.data) cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException(s"splice attempt rejected by our peer: ${msg.toAscii}")) stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage) calling endQuiescence(d) + case SpliceStatus.RbfRequested(cmd, _) => + log.info("our peer rejected our rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data) + cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException(s"rbf attempt rejected by our peer: ${msg.toAscii}")) + stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage) calling endQuiescence(d) case SpliceStatus.NonInitiatorQuiescent => log.info("our peer aborted their own splice attempt: ascii='{}' bin={}", msg.toAscii, msg.data) stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage) calling endQuiescence(d) @@ -1044,7 +1197,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with log.info("our peer wants to abort the splice, but we've already negotiated a splice transaction: ascii='{}' bin={}", msg.toAscii, msg.data) // We ack their tx_abort but we keep monitoring the funding transaction until it's confirmed or double-spent. stay() sending TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage) - case _: QuiescenceNegotiation => + case _: SpliceStatus.NegotiatingQuiescence => log.info("our peer aborted the splice during quiescence negotiation, disconnecting: ascii='{}' bin={}", msg.toAscii, msg.data) // NB: we use a small delay to ensure we've sent our warning before disconnecting. context.system.scheduler.scheduleOnce(2 second, peer, Peer.Disconnect(remoteNodeId)) @@ -1876,8 +2029,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with val channelKeyPath = keyManager.keyPath(d.commitments.params.localParams, d.commitments.params.channelConfig) val myCurrentPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, d.commitments.localCommitIndex) val rbfTlv: Set[ChannelReestablishTlv] = d match { - case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => d.rbfStatus match { - case RbfStatus.RbfWaitingForSigs(status) => Set(ChannelReestablishTlv.NextFundingTlv(status.fundingTx.txId)) + case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => d.status match { + case DualFundingStatus.RbfWaitingForSigs(status) => Set(ChannelReestablishTlv.NextFundingTlv(status.fundingTx.txId)) case _ => d.latestFundingTx.sharedTx match { case _: InteractiveTxBuilder.PartiallySignedSharedTransaction => Set(ChannelReestablishTlv.NextFundingTlv(d.latestFundingTx.sharedTx.txId)) case _: InteractiveTxBuilder.FullySignedSharedTransaction => Set.empty @@ -1940,8 +2093,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with case Event(channelReestablish: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => channelReestablish.nextFundingTxId_opt match { case Some(fundingTxId) => - d.rbfStatus match { - case RbfStatus.RbfWaitingForSigs(signingSession) if signingSession.fundingTx.txId == fundingTxId => + d.status match { + case DualFundingStatus.RbfWaitingForSigs(signingSession) if signingSession.fundingTx.txId == fundingTxId => // We retransmit our commit_sig, and will send our tx_signatures once we've received their commit_sig. val commitSig = signingSession.remoteCommit.sign(keyManager, d.commitments.params, signingSession.fundingTxIndex, signingSession.fundingParams.remoteFundingPubKey, signingSession.commitInput) goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) sending commitSig @@ -1959,7 +2112,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with case _ => // The fundingTxId must be for an RBF attempt that we didn't store (we got disconnected before receiving // their tx_complete): we tell them to abort that RBF attempt. - goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) + goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) } case None => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) } @@ -2772,6 +2925,50 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with } } + private def initiateSpliceRbf(cmd: CMD_BUMP_FUNDING_FEE, d: DATA_NORMAL): Either[ChannelException, TxInitRbf] = { + getSpliceRbfContext(Some(cmd), d).flatMap(rbf => { + // We use the same contribution as the previous splice attempt. + val fundingContribution = rbf.latestFundingTx.fundingParams.localContribution + val commitTxFees = if (d.commitments.params.localParams.paysCommitTxFees) { + Transactions.commitTxTotalCost(d.commitments.params.remoteParams.dustLimit, rbf.parentCommitment.remoteCommit.spec, d.commitments.params.commitmentFormat) + } else { + 0.sat + } + if (!d.commitments.isQuiescent) { + log.warning("cannot initiate rbf, channel is not quiescent") + Left(InvalidSpliceNotQuiescent(d.channelId)) + } else if (fundingContribution < 0.sat && rbf.parentCommitment.localCommit.spec.toLocal + fundingContribution < rbf.parentCommitment.localChannelReserve(d.commitments.params).max(commitTxFees)) { + log.warning(s"cannot do rbf: insufficient funds (commitTxFees=$commitTxFees reserve=${rbf.parentCommitment.localChannelReserve(d.commitments.params)})") + Left(InvalidSpliceRequest(d.channelId)) + } else { + val txInitRbf = TxInitRbf(d.channelId, cmd.lockTime, cmd.targetFeerate, fundingContribution, rbf.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote) + Right(txInitRbf) + } + }) + } + + private def getSpliceRbfContext(cmd_opt: Option[CMD_BUMP_FUNDING_FEE], d: DATA_NORMAL): Either[ChannelException, InteractiveTxBuilder.SpliceTxRbf] = { + d.commitments.latest.localFundingStatus match { + case fundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx => + val fundingTxIndex = d.commitments.latest.fundingTxIndex + d.commitments.active.find(_.fundingTxIndex == fundingTxIndex - 1) match { + case Some(parentCommitment) => + val previousTxs = d.commitments.active + .filter(_.fundingTxIndex == fundingTxIndex) + .map(_.localFundingStatus) + // All transactions should be unconfirmed, otherwise we wouldn't be in an RBF context. + .collect { case tx: LocalFundingStatus.DualFundedUnconfirmedFundingTx => tx.sharedTx } + Right(InteractiveTxBuilder.SpliceTxRbf(parentCommitment, fundingTx, previousTxs, cmd_opt.map(_.fundingFeeBudget))) + case None => + log.warning("cannot initiate rbf: cannot find parent commitment") + Left(InvalidRbfAttempt(d.channelId)) + } + case _ => + log.warning("cannot initiate rbf: transaction already confirmed") + Left(InvalidRbfTxConfirmed(d.channelId)) + } + } + private def handleQuiescenceTimeout(d: DATA_NORMAL): State = { if (d.spliceStatus == SpliceStatus.NoSplice) { log.warning("quiescence timed out with no ongoing splice, did we forget to cancel the timer?") @@ -2794,9 +2991,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with private def reportSpliceFailure(spliceStatus: SpliceStatus, f: Throwable): Unit = { val cmd_opt = spliceStatus match { - case SpliceStatus.QuiescenceRequested(cmd) => Some(cmd) - case SpliceStatus.InitiatorQuiescent(cmd) => Some(cmd) + case SpliceStatus.NegotiatingQuiescence(cmd_opt, _) => cmd_opt case SpliceStatus.SpliceRequested(cmd, _) => Some(cmd) + case SpliceStatus.RbfRequested(cmd, _) => Some(cmd) case SpliceStatus.SpliceInProgress(cmd_opt, _, txBuilder, _) => txBuilder ! InteractiveTxBuilder.Abort cmd_opt diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala index a13ba920ef..2f2a5dd449 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala @@ -390,7 +390,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { remotePerCommitmentSecrets = ShaChain.init, originChannels = Map.empty ) - val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, d.localPushAmount, d.remotePushAmount, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, RbfStatus.NoRbf, None) + val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, d.localPushAmount, d.remotePushAmount, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, DualFundingStatus.WaitingForConfirmations, None) goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d1 storing() sending signingSession1.localSigs } } @@ -413,7 +413,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { remotePerCommitmentSecrets = ShaChain.init, originChannels = Map.empty ) - val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, d.localPushAmount, d.remotePushAmount, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, RbfStatus.NoRbf, None) + val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, d.localPushAmount, d.remotePushAmount, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, DualFundingStatus.WaitingForConfirmations, None) goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d1 storing() sending signingSession.localSigs calling publishFundingTx(signingSession.fundingTx) } case msg: TxAbort => @@ -461,17 +461,17 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { stay() using d1 storing() calling publishFundingTx(dfu1) } case _: FullySignedSharedTransaction => - d.rbfStatus match { - case RbfStatus.RbfWaitingForSigs(signingSession) => + d.status match { + case DualFundingStatus.RbfWaitingForSigs(signingSession) => signingSession.receiveTxSigs(nodeParams, d.commitments.params, txSigs) match { case Left(f) => rollbackRbfAttempt(signingSession, d) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, f.getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, f.getMessage) case Right(signingSession1) => val minDepth_opt = d.commitments.params.minDepthDualFunding(nodeParams.channelConf.minDepthBlocks, signingSession1.fundingTx.sharedTx.tx) watchFundingConfirmed(signingSession.fundingTx.txId, minDepth_opt, delay_opt = None) val commitments1 = d.commitments.add(signingSession1.commitment) - val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments1, d.localPushAmount, d.remotePushAmount, d.waitingSince, d.lastChecked, RbfStatus.NoRbf, d.deferred) + val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments1, d.localPushAmount, d.remotePushAmount, d.waitingSince, d.lastChecked, DualFundingStatus.WaitingForConfirmations, d.deferred) stay() using d1 storing() sending signingSession1.localSigs calling publishFundingTx(signingSession1.fundingTx) } case _ if d.commitments.all.exists(_.fundingTxId == txSigs.txId) => @@ -479,8 +479,8 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { stay() case _ => log.debug("rejecting unexpected tx_signatures for txId={}", txSigs.txId) - reportRbfFailure(d.rbfStatus, UnexpectedFundingSignatures(d.channelId)) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, UnexpectedFundingSignatures(d.channelId).getMessage) + reportRbfFailure(d.status, UnexpectedFundingSignatures(d.channelId)) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, UnexpectedFundingSignatures(d.channelId).getMessage) } } @@ -493,15 +493,15 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfZeroConf(d.channelId)) stay() } else { - d.rbfStatus match { - case RbfStatus.NoRbf => + d.status match { + case DualFundingStatus.WaitingForConfirmations => val minNextFeerate = d.latestFundingTx.fundingParams.minNextFeerate if (cmd.targetFeerate < minNextFeerate) { cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfFeerate(d.channelId, cmd.targetFeerate, minNextFeerate)) stay() } else { val txInitRbf = TxInitRbf(d.channelId, cmd.lockTime, cmd.targetFeerate, d.latestFundingTx.fundingParams.localContribution, nodeParams.channelConf.requireConfirmedInputsForDualFunding) - stay() using d.copy(rbfStatus = RbfStatus.RbfRequested(cmd)) sending txInitRbf + stay() using d.copy(status = DualFundingStatus.RbfRequested(cmd)) sending txInitRbf } case _ => log.warning("cannot initiate rbf, another one is already in progress") @@ -518,24 +518,24 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { stay() sending Error(d.channelId, InvalidRbfNonInitiator(d.channelId).getMessage) } else if (zeroConf) { log.info("rejecting tx_init_rbf, we're using zero-conf") - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfZeroConf(d.channelId).getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfZeroConf(d.channelId).getMessage) } else { val minNextFeerate = d.latestFundingTx.fundingParams.minNextFeerate - d.rbfStatus match { - case RbfStatus.NoRbf => + d.status match { + case DualFundingStatus.WaitingForConfirmations => val remainingRbfAttempts = nodeParams.channelConf.remoteRbfLimits.maxAttempts - d.previousFundingTxs.length if (msg.feerate < minNextFeerate) { log.info("rejecting rbf attempt: the new feerate must be at least {} (proposed={})", minNextFeerate, msg.feerate) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfFeerate(d.channelId, msg.feerate, minNextFeerate).getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfFeerate(d.channelId, msg.feerate, minNextFeerate).getMessage) } else if (d.remotePushAmount > msg.fundingContribution) { log.info("rejecting rbf attempt: invalid amount pushed (fundingAmount={}, pushAmount={})", msg.fundingContribution, d.remotePushAmount) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, InvalidPushAmount(d.channelId, d.remotePushAmount, msg.fundingContribution.toMilliSatoshi).getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, InvalidPushAmount(d.channelId, d.remotePushAmount, msg.fundingContribution.toMilliSatoshi).getMessage) } else if (remainingRbfAttempts <= 0) { log.info("rejecting rbf attempt: maximum number of attempts reached (max={})", nodeParams.channelConf.remoteRbfLimits.maxAttempts) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfAttemptsExhausted(d.channelId, nodeParams.channelConf.remoteRbfLimits.maxAttempts).getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfAttemptsExhausted(d.channelId, nodeParams.channelConf.remoteRbfLimits.maxAttempts).getMessage) } else if (nodeParams.currentBlockHeight < d.latestFundingTx.createdAt + nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks) { log.info("rejecting rbf attempt: last attempt was less than {} blocks ago", nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfAttemptTooSoon(d.channelId, d.latestFundingTx.createdAt, d.latestFundingTx.createdAt + nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks).getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfAttemptTooSoon(d.channelId, d.latestFundingTx.createdAt, d.latestFundingTx.createdAt + nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks).getMessage) } else { log.info("our peer wants to raise the feerate of the funding transaction (previous={} target={})", d.latestFundingTx.fundingParams.targetFeerate, msg.feerate) val fundingParams = d.latestFundingTx.fundingParams.copy( @@ -549,7 +549,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { randomBytes32(), nodeParams, fundingParams, channelParams = d.commitments.params, - purpose = InteractiveTxBuilder.PreviousTxRbf(d.commitments.active.head, 0 msat, 0 msat, previousTransactions = d.allFundingTxs.map(_.sharedTx), feeBudget_opt = None), + purpose = InteractiveTxBuilder.FundingTxRbf(d.commitments.active.head, previousTransactions = d.allFundingTxs.map(_.sharedTx), feeBudget_opt = None), localPushAmount = d.localPushAmount, remotePushAmount = d.remotePushAmount, wallet)) txBuilder ! InteractiveTxBuilder.Start(self) @@ -557,25 +557,25 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { Some(TxAckRbf(d.channelId, fundingParams.localContribution, nodeParams.channelConf.requireConfirmedInputsForDualFunding)), if (remainingRbfAttempts <= 3) Some(Warning(d.channelId, s"will accept at most ${remainingRbfAttempts - 1} future rbf attempts")) else None, ).flatten - stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(cmd_opt = None, txBuilder, remoteCommitSig = None)) sending toSend + stay() using d.copy(status = DualFundingStatus.RbfInProgress(cmd_opt = None, txBuilder, remoteCommitSig = None)) sending toSend } - case RbfStatus.RbfAborted => + case DualFundingStatus.RbfAborted => log.info("rejecting rbf attempt: our previous tx_abort was not acked") stay() sending Warning(d.channelId, InvalidRbfTxAbortNotAcked(d.channelId).getMessage) - case _: RbfStatus.RbfRequested | _: RbfStatus.RbfInProgress | _: RbfStatus.RbfWaitingForSigs => + case _: DualFundingStatus.RbfRequested | _: DualFundingStatus.RbfInProgress | _: DualFundingStatus.RbfWaitingForSigs => log.info("rejecting rbf attempt: the current rbf attempt must be completed or aborted first") stay() sending Warning(d.channelId, InvalidRbfAlreadyInProgress(d.channelId).getMessage) } } case Event(msg: TxAckRbf, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - d.rbfStatus match { - case RbfStatus.RbfRequested(cmd) if d.remotePushAmount > msg.fundingContribution => + d.status match { + case DualFundingStatus.RbfRequested(cmd) if d.remotePushAmount > msg.fundingContribution => log.info("rejecting rbf attempt: invalid amount pushed (fundingAmount={}, pushAmount={})", msg.fundingContribution, d.remotePushAmount) val error = InvalidPushAmount(d.channelId, d.remotePushAmount, msg.fundingContribution.toMilliSatoshi) cmd.replyTo ! RES_FAILURE(cmd, error) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, error.getMessage) - case RbfStatus.RbfRequested(cmd) => + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, error.getMessage) + case DualFundingStatus.RbfRequested(cmd) => log.info("our peer accepted our rbf attempt and will contribute {} to the funding transaction", msg.fundingContribution) val fundingParams = d.latestFundingTx.fundingParams.copy( // we don't change our funding contribution @@ -587,19 +587,19 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { randomBytes32(), nodeParams, fundingParams, channelParams = d.commitments.params, - purpose = InteractiveTxBuilder.PreviousTxRbf(d.commitments.active.head, 0 msat, 0 msat, previousTransactions = d.allFundingTxs.map(_.sharedTx), feeBudget_opt = Some(cmd.fundingFeeBudget)), + purpose = InteractiveTxBuilder.FundingTxRbf(d.commitments.active.head, previousTransactions = d.allFundingTxs.map(_.sharedTx), feeBudget_opt = Some(cmd.fundingFeeBudget)), localPushAmount = d.localPushAmount, remotePushAmount = d.remotePushAmount, wallet)) txBuilder ! InteractiveTxBuilder.Start(self) - stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(cmd_opt = Some(cmd), txBuilder, remoteCommitSig = None)) + stay() using d.copy(status = DualFundingStatus.RbfInProgress(cmd_opt = Some(cmd), txBuilder, remoteCommitSig = None)) case _ => log.info("ignoring unexpected tx_ack_rbf") stay() sending Warning(d.channelId, UnexpectedInteractiveTxMessage(d.channelId, msg).getMessage) } case Event(msg: InteractiveTxConstructionMessage, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - d.rbfStatus match { - case RbfStatus.RbfInProgress(_, txBuilder, _) => + d.status match { + case DualFundingStatus.RbfInProgress(_, txBuilder, _) => txBuilder ! InteractiveTxBuilder.ReceiveMessage(msg) stay() case _ => @@ -608,24 +608,24 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { } case Event(commitSig: CommitSig, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - d.rbfStatus match { - case s: RbfStatus.RbfInProgress => + d.status match { + case s: DualFundingStatus.RbfInProgress => log.debug("received their commit_sig, deferring message") - stay() using d.copy(rbfStatus = s.copy(remoteCommitSig = Some(commitSig))) - case RbfStatus.RbfWaitingForSigs(signingSession) => + stay() using d.copy(status = s.copy(remoteCommitSig = Some(commitSig))) + case DualFundingStatus.RbfWaitingForSigs(signingSession) => signingSession.receiveCommitSig(nodeParams, d.commitments.params, commitSig) match { case Left(f) => rollbackRbfAttempt(signingSession, d) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, f.getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, f.getMessage) case Right(signingSession1) => signingSession1 match { case signingSession1: InteractiveTxSigningSession.WaitingForSigs => // No need to store their commit_sig, they will re-send it if we disconnect. - stay() using d.copy(rbfStatus = RbfStatus.RbfWaitingForSigs(signingSession1)) + stay() using d.copy(status = DualFundingStatus.RbfWaitingForSigs(signingSession1)) case signingSession1: InteractiveTxSigningSession.SendingSigs => val minDepth_opt = d.commitments.params.minDepthDualFunding(nodeParams.channelConf.minDepthBlocks, signingSession1.fundingTx.sharedTx.tx) watchFundingConfirmed(signingSession.fundingTx.txId, minDepth_opt, delay_opt = None) val commitments1 = d.commitments.add(signingSession1.commitment) - val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments1, d.localPushAmount, d.remotePushAmount, d.waitingSince, d.lastChecked, RbfStatus.NoRbf, d.deferred) + val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments1, d.localPushAmount, d.remotePushAmount, d.waitingSince, d.lastChecked, DualFundingStatus.WaitingForConfirmations, d.deferred) stay() using d1 storing() sending signingSession1.localSigs } } @@ -635,47 +635,47 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { } case Event(msg: TxAbort, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - d.rbfStatus match { - case RbfStatus.RbfInProgress(cmd_opt, txBuilder, _) => + d.status match { + case DualFundingStatus.RbfInProgress(cmd_opt, txBuilder, _) => log.info("our peer aborted the rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data) cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, RbfAttemptAborted(d.channelId))) txBuilder ! InteractiveTxBuilder.Abort - stay() using d.copy(rbfStatus = RbfStatus.NoRbf) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) - case RbfStatus.RbfWaitingForSigs(signingSession) => + stay() using d.copy(status = DualFundingStatus.WaitingForConfirmations) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) + case DualFundingStatus.RbfWaitingForSigs(signingSession) => log.info("our peer aborted the rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data) rollbackRbfAttempt(signingSession, d) - stay() using d.copy(rbfStatus = RbfStatus.NoRbf) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) - case RbfStatus.RbfRequested(cmd) => + stay() using d.copy(status = DualFundingStatus.WaitingForConfirmations) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) + case DualFundingStatus.RbfRequested(cmd) => log.info("our peer rejected our rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data) cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException(s"rbf attempt rejected by our peer: ${msg.toAscii}")) - stay() using d.copy(rbfStatus = RbfStatus.NoRbf) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) - case RbfStatus.RbfAborted => + stay() using d.copy(status = DualFundingStatus.WaitingForConfirmations) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) + case DualFundingStatus.RbfAborted => log.debug("our peer acked our previous tx_abort") - stay() using d.copy(rbfStatus = RbfStatus.NoRbf) - case RbfStatus.NoRbf => + stay() using d.copy(status = DualFundingStatus.WaitingForConfirmations) + case DualFundingStatus.WaitingForConfirmations => log.info("our peer wants to abort the dual funding flow, but we've already negotiated a funding transaction: ascii='{}' bin={}", msg.toAscii, msg.data) // We ack their tx_abort but we keep monitoring the funding transaction until it's confirmed or double-spent. stay() sending TxAbort(d.channelId, DualFundingAborted(d.channelId).getMessage) } case Event(msg: InteractiveTxBuilder.Response, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - d.rbfStatus match { - case RbfStatus.RbfInProgress(cmd_opt, _, remoteCommitSig_opt) => + d.status match { + case DualFundingStatus.RbfInProgress(cmd_opt, _, remoteCommitSig_opt) => msg match { 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 ! _) - val d1 = d.copy(rbfStatus = RbfStatus.RbfWaitingForSigs(signingSession)) + val d1 = d.copy(status = DualFundingStatus.RbfWaitingForSigs(signingSession)) stay() using d1 storing() sending commitSig case f: InteractiveTxBuilder.Failed => log.info("rbf attempt failed: {}", f.cause.getMessage) cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, f.cause)) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, f.cause.getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, f.cause.getMessage) } case _ => // This can happen if we received a tx_abort right before receiving the interactive-tx result. - log.warning("ignoring interactive-tx result with rbfStatus={}", d.rbfStatus.getClass.getSimpleName) + log.warning("ignoring interactive-tx result with rbfStatus={}", d.status.getClass.getSimpleName) stay() } @@ -700,10 +700,10 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { val realScidStatus = RealScidStatus.Temporary(RealShortChannelId(w.blockHeight, w.txIndex, commitment.commitInput.outPoint.index.toInt)) val shortIds = createShortIds(d.channelId, realScidStatus) val channelReady = createChannelReady(shortIds, d.commitments.params) - reportRbfFailure(d.rbfStatus, InvalidRbfTxConfirmed(d.channelId)) - val toSend = d.rbfStatus match { - case RbfStatus.NoRbf | RbfStatus.RbfAborted => Seq(channelReady) - case _: RbfStatus.RbfRequested | _: RbfStatus.RbfInProgress | _: RbfStatus.RbfWaitingForSigs => Seq(TxAbort(d.channelId, InvalidRbfTxConfirmed(d.channelId).getMessage), channelReady) + reportRbfFailure(d.status, InvalidRbfTxConfirmed(d.channelId)) + val toSend = d.status match { + case DualFundingStatus.WaitingForConfirmations | DualFundingStatus.RbfAborted => Seq(channelReady) + case _: DualFundingStatus.RbfRequested | _: DualFundingStatus.RbfInProgress | _: DualFundingStatus.RbfWaitingForSigs => Seq(TxAbort(d.channelId, InvalidRbfTxConfirmed(d.channelId).getMessage), channelReady) } d.deferred.foreach(self ! _) goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments1, shortIds) storing() sending toSend @@ -713,7 +713,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { case Event(ProcessCurrentBlockHeight(c), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleNewBlockDualFundingUnconfirmed(c, d) case Event(e: BITCOIN_FUNDING_DOUBLE_SPENT, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - reportRbfFailure(d.rbfStatus, FundingTxDoubleSpent(d.channelId)) + reportRbfFailure(d.status, FundingTxDoubleSpent(d.channelId)) handleDualFundingDoubleSpent(e, d) case Event(remoteChannelReady: ChannelReady, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => @@ -729,16 +729,16 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { stay() case Event(INPUT_DISCONNECTED, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - reportRbfFailure(d.rbfStatus, new RuntimeException("rbf attempt failed: disconnected")) - val d1 = d.rbfStatus match { + reportRbfFailure(d.status, new RuntimeException("rbf attempt failed: disconnected")) + val d1 = d.status match { // We keep track of the RBF status: we should be able to complete the signature steps on reconnection. - case _: RbfStatus.RbfWaitingForSigs => d - case _ => d.copy(rbfStatus = RbfStatus.NoRbf) + case _: DualFundingStatus.RbfWaitingForSigs => d + case _ => d.copy(status = DualFundingStatus.WaitingForConfirmations) } goto(OFFLINE) using d1 case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - reportRbfFailure(d.rbfStatus, new RuntimeException(s"remote error: ${e.toAscii}")) + reportRbfFailure(d.status, new RuntimeException(s"remote error: ${e.toAscii}")) handleRemoteError(e, d) }) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala index 953e6b7f94..eb26fa4df9 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala @@ -137,10 +137,10 @@ trait DualFundingHandlers extends CommonFundingHandlers { rollbackFundingAttempt(signingSession.fundingTx.tx, d.allFundingTxs.map(_.sharedTx)) } - def reportRbfFailure(rbfStatus: RbfStatus, f: Throwable): Unit = { - rbfStatus match { - case RbfStatus.RbfRequested(cmd) => cmd.replyTo ! RES_FAILURE(cmd, f) - case RbfStatus.RbfInProgress(cmd_opt, txBuilder, _) => + def reportRbfFailure(fundingStatus: DualFundingStatus, f: Throwable): Unit = { + fundingStatus match { + case DualFundingStatus.RbfRequested(cmd) => cmd.replyTo ! RES_FAILURE(cmd, f) + case DualFundingStatus.RbfInProgress(cmd_opt, txBuilder, _) => txBuilder ! InteractiveTxBuilder.Abort cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, f)) case _ => () diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala index 51a6b1645c..7b1de61349 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala @@ -194,9 +194,10 @@ object InteractiveTxBuilder { * only one of them ends up confirming. We guarantee this by having the latest transaction * always double-spend all its predecessors. */ - case class PreviousTxRbf(replacedCommitment: Commitment, previousLocalBalance: MilliSatoshi, previousRemoteBalance: MilliSatoshi, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], feeBudget_opt: Option[Satoshi]) extends Purpose { - // Note that the truncation is a no-op: the sum of balances in a channel must be a satoshi amount. - override val previousFundingAmount: Satoshi = (previousLocalBalance + previousRemoteBalance).truncateToSatoshi + case class FundingTxRbf(replacedCommitment: Commitment, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], feeBudget_opt: Option[Satoshi]) extends Purpose { + override val previousLocalBalance: MilliSatoshi = 0 msat + override val previousRemoteBalance: MilliSatoshi = 0 msat + override val previousFundingAmount: Satoshi = 0 sat override val localCommitIndex: Long = replacedCommitment.localCommit.index override val remoteCommitIndex: Long = replacedCommitment.remoteCommit.index override val remotePerCommitmentPoint: PublicKey = replacedCommitment.remoteCommit.remotePerCommitmentPoint @@ -204,6 +205,22 @@ object InteractiveTxBuilder { override val fundingTxIndex: Long = replacedCommitment.fundingTxIndex override val localHtlcs: Set[DirectedHtlc] = replacedCommitment.localCommit.spec.htlcs } + + /** + * @param previousTransactions splice RBF attempts all spend the previous funding transaction, so they automatically + * double-spend each other, but we reuse previous inputs as much as possible anyway. + */ + case class SpliceTxRbf(parentCommitment: Commitment, latestFundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], feeBudget_opt: Option[Satoshi]) extends Purpose { + override val previousLocalBalance: MilliSatoshi = parentCommitment.localCommit.spec.toLocal + override val previousRemoteBalance: MilliSatoshi = parentCommitment.remoteCommit.spec.toLocal + override val previousFundingAmount: Satoshi = parentCommitment.capacity + override val localCommitIndex: Long = parentCommitment.localCommit.index + override val remoteCommitIndex: Long = parentCommitment.remoteCommit.index + override val remotePerCommitmentPoint: PublicKey = parentCommitment.remoteCommit.remotePerCommitmentPoint + override val commitTxFeerate: FeeratePerKw = parentCommitment.localCommit.spec.commitTxFeerate + override val fundingTxIndex: Long = parentCommitment.fundingTxIndex + 1 + override val localHtlcs: Set[DirectedHtlc] = parentCommitment.localCommit.spec.htlcs + } // @formatter:on // @formatter:off @@ -401,7 +418,8 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon private val fundingPubkeyScript: ByteVector = Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingPubKey, fundingParams.remoteFundingPubKey))) private val remoteNodeId = channelParams.remoteParams.nodeId private val previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction] = purpose match { - case rbf: PreviousTxRbf => rbf.previousTransactions + case rbf: FundingTxRbf => rbf.previousTransactions + case rbf: SpliceTxRbf => rbf.previousTransactions case _ => Nil } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala index 97a8b7a3d3..6f54ea7d4b 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala @@ -137,7 +137,8 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response private val log = context.log private val previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction] = purpose match { - case rbf: InteractiveTxBuilder.PreviousTxRbf => rbf.previousTransactions + case rbf: InteractiveTxBuilder.FundingTxRbf => rbf.previousTransactions + case rbf: InteractiveTxBuilder.SpliceTxRbf => rbf.previousTransactions case _ => Nil } @@ -191,7 +192,8 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response val sharedInputWeight = fundingParams.sharedInput_opt.toSeq.map(i => i.info.outPoint -> i.weight.toLong).toMap val feeBudget_opt = purpose match { case p: FundingTx => p.feeBudget_opt - case p: PreviousTxRbf => p.feeBudget_opt + case p: FundingTxRbf => p.feeBudget_opt + case p: SpliceTxRbf => p.feeBudget_opt case _ => None } context.pipeToSelf(wallet.fundTransaction(txNotFunded, fundingParams.targetFeerate, replaceable = true, externalInputsWeight = sharedInputWeight, feeBudget_opt = feeBudget_opt)) { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4.scala index ec4fbe1326..ad191d5141 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4.scala @@ -607,9 +607,9 @@ private[channel] object ChannelCodecs4 { waitingForSigsCodec } - val rbfStatusCodec: Codec[RbfStatus] = discriminated[RbfStatus].by(uint8) - .\(0x01) { case status: RbfStatus if !status.isInstanceOf[RbfStatus.RbfWaitingForSigs] => RbfStatus.NoRbf }(provide(RbfStatus.NoRbf)) - .\(0x02) { case status: RbfStatus.RbfWaitingForSigs => status }(interactiveTxWaitingForSigsCodec.as[RbfStatus.RbfWaitingForSigs]) + val dualFundingStatusCodec: Codec[DualFundingStatus] = discriminated[DualFundingStatus].by(uint8) + .\(0x01) { case status: DualFundingStatus if !status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs] => DualFundingStatus.WaitingForConfirmations }(provide(DualFundingStatus.WaitingForConfirmations)) + .\(0x02) { case status: DualFundingStatus.RbfWaitingForSigs => status }(interactiveTxWaitingForSigsCodec.as[DualFundingStatus.RbfWaitingForSigs]) val spliceStatusCodec: Codec[SpliceStatus] = discriminated[SpliceStatus].by(uint8) .\(0x01) { case status: SpliceStatus if !status.isInstanceOf[SpliceStatus.SpliceWaitingForSigs] => SpliceStatus.NoSplice }(provide(SpliceStatus.NoSplice)) @@ -649,7 +649,7 @@ private[channel] object ChannelCodecs4 { ("remotePushAmount" | millisatoshi) :: ("waitingSince" | blockHeight) :: ("lastChecked" | blockHeight) :: - ("rbfStatus" | rbfStatusCodec) :: + ("status" | dualFundingStatusCodec) :: ("deferred" | optional(bool8, lengthDelimited(channelReadyCodec)))).as[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] val DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED_0c_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] = ( @@ -658,7 +658,7 @@ private[channel] object ChannelCodecs4 { ("remotePushAmount" | millisatoshi) :: ("waitingSince" | blockHeight) :: ("lastChecked" | blockHeight) :: - ("rbfStatus" | rbfStatusCodec) :: + ("status" | dualFundingStatusCodec) :: ("deferred" | optional(bool8, lengthDelimited(channelReadyCodec)))).as[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] val DATA_WAIT_FOR_DUAL_FUNDING_READY_03_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_READY] = ( diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala index 4dabd75781..b295110d6d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala @@ -49,8 +49,8 @@ sealed trait HasTemporaryChannelId extends LightningMessage { def temporaryChann sealed trait HasChannelId extends LightningMessage { def channelId: ByteVector32 } // <- not in the spec sealed trait HasChainHash extends LightningMessage { def chainHash: BlockHash } // <- not in the spec sealed trait HasSerialId extends LightningMessage { def serialId: UInt64 } // <- not in the spec -sealed trait ForbiddenMessageDuringSplice extends LightningMessage // <- not in the spec -sealed trait UpdateMessage extends HtlcMessage with ForbiddenMessageDuringSplice // <- not in the spec +sealed trait ForbiddenMessageWhenQuiescent extends LightningMessage // <- not in the spec +sealed trait UpdateMessage extends HtlcMessage with ForbiddenMessageWhenQuiescent // <- not in the spec sealed trait HtlcSettlementMessage extends UpdateMessage { def id: Long } // <- not in the spec sealed trait HtlcFailureMessage extends HtlcSettlementMessage // <- not in the spec // @formatter:on @@ -380,7 +380,7 @@ object ExperimentalSpliceLocked { case class Shutdown(channelId: ByteVector32, scriptPubKey: ByteVector, - tlvStream: TlvStream[ShutdownTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId with ForbiddenMessageDuringSplice + tlvStream: TlvStream[ShutdownTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId with ForbiddenMessageWhenQuiescent case class ClosingSigned(channelId: ByteVector32, feeSatoshis: Satoshi, diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala index 4a146d0094..54a8f1c29e 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala @@ -34,8 +34,8 @@ import fr.acinq.eclair.blockchain.{OnChainWallet, SingleKeyOnChainWallet} import fr.acinq.eclair.channel.fund.InteractiveTxBuilder._ import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession} import fr.acinq.eclair.io.OpenChannelInterceptor.makeChannelParams -import fr.acinq.eclair.transactions.{Scripts, Transactions} import fr.acinq.eclair.transactions.Transactions.InputInfo +import fr.acinq.eclair.transactions.{Scripts, Transactions} import fr.acinq.eclair.wire.protocol._ import fr.acinq.eclair.{Feature, FeatureSupport, Features, InitFeature, MilliSatoshiLong, NodeParams, TestConstants, TestKitBaseClass, ToMilliSatoshiConversion, UInt64, randomBytes32, randomKey} import org.scalatest.BeforeAndAfterAll @@ -132,7 +132,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit 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, feeBudget_opt = None), + FundingTxRbf(commitment, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, wallet)) @@ -143,10 +143,10 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit 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( + def spawnTxBuilderSpliceRbfAlice(fundingParams: InteractiveTxParams, parentCommitment: Commitment, latestFundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx, 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, feeBudget_opt = None), + SpliceTxRbf(parentCommitment, latestFundingTx, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, wallet)) @@ -160,7 +160,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit 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, feeBudget_opt = None), + FundingTxRbf(commitment, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, wallet)) @@ -171,10 +171,10 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit 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( + def spawnTxBuilderSpliceRbfBob(fundingParams: InteractiveTxParams, parentCommitment: Commitment, latestFundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx, 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, feeBudget_opt = None), + SpliceTxRbf(parentCommitment, latestFundingTx, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, wallet)) @@ -1436,8 +1436,8 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit // Alice wants to increase the feerate of the splice transaction. val fundingParamsA2 = spliceFixtureParams.fundingParamsA.copy(targetFeerate = targetFeerate * 2) val fundingParamsB2 = spliceFixtureParams.fundingParamsB.copy(targetFeerate = targetFeerate * 2) - val aliceRbf = fixtureParams.spawnTxBuilderSpliceRbfAlice(fundingParamsA2, parentCommitment = commitmentA1, replacedCommitment = commitmentA2, Seq(spliceTxA1), walletA) - val bobRbf = fixtureParams.spawnTxBuilderSpliceRbfBob(fundingParamsB2, parentCommitment = commitmentB1, replacedCommitment = commitmentB2, Seq(spliceTxB1), walletB) + val aliceRbf = fixtureParams.spawnTxBuilderSpliceRbfAlice(fundingParamsA2, parentCommitment = commitmentA1, commitmentA2.localFundingStatus.asInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx], Seq(spliceTxA1), walletA) + val bobRbf = fixtureParams.spawnTxBuilderSpliceRbfBob(fundingParamsB2, parentCommitment = commitmentB1, commitmentB2.localFundingStatus.asInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx], Seq(spliceTxB1), walletB) val fwdRbf = TypeCheckedForwarder(aliceRbf, bobRbf, alice2bob, bob2alice) aliceRbf ! Start(alice2bob.ref) @@ -1565,8 +1565,8 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit // Alice wants to make a large increase to the feerate of the splice transaction, which requires additional inputs. val fundingParamsA2 = fundingParamsA1.copy(targetFeerate = FeeratePerKw(5_000 sat)) val fundingParamsB2 = fundingParamsB1.copy(targetFeerate = FeeratePerKw(5_000 sat)) - val aliceRbf = fixtureParams.spawnTxBuilderSpliceRbfAlice(fundingParamsA2, parentCommitment = commitmentA1, replacedCommitment = commitmentA2, Seq(spliceTxA1), walletA) - val bobRbf = fixtureParams.spawnTxBuilderSpliceRbfBob(fundingParamsB2, parentCommitment = commitmentB1, replacedCommitment = commitmentB2, Seq(spliceTxB1), walletB) + val aliceRbf = fixtureParams.spawnTxBuilderSpliceRbfAlice(fundingParamsA2, parentCommitment = commitmentA1, commitmentA2.localFundingStatus.asInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx], Seq(spliceTxA1), walletA) + val bobRbf = fixtureParams.spawnTxBuilderSpliceRbfBob(fundingParamsB2, parentCommitment = commitmentB1, commitmentB2.localFundingStatus.asInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx], Seq(spliceTxB1), walletB) val fwdRbf = TypeCheckedForwarder(aliceRbf, bobRbf, alice2bob, bob2alice) aliceRbf ! Start(alice2bob.ref) @@ -1718,8 +1718,8 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit val additionalFundingA2 = 50_000 sat val fundingParamsA2 = fundingParamsA1.copy(targetFeerate = FeeratePerKw(5_000 sat), localContribution = additionalFundingA2, remoteContribution = 0 sat) val fundingParamsB2 = fundingParamsB1.copy(targetFeerate = FeeratePerKw(5_000 sat), localContribution = 0 sat, remoteContribution = additionalFundingA2) - val aliceRbf = fixtureParams.spawnTxBuilderSpliceRbfAlice(fundingParamsA2, parentCommitment = commitmentA1bis, replacedCommitment = commitmentA2bis, Seq(spliceTxA1), walletA) - val bobRbf = fixtureParams.spawnTxBuilderSpliceRbfBob(fundingParamsB2, parentCommitment = commitmentB1bis, replacedCommitment = commitmentB2bis, Seq(spliceTxB1), walletB) + val aliceRbf = fixtureParams.spawnTxBuilderSpliceRbfAlice(fundingParamsA2, parentCommitment = commitmentA1bis, commitmentA2bis.localFundingStatus.asInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx], Seq(spliceTxA1), walletA) + val bobRbf = fixtureParams.spawnTxBuilderSpliceRbfBob(fundingParamsB2, parentCommitment = commitmentB1bis, commitmentB2bis.localFundingStatus.asInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx], Seq(spliceTxB1), walletB) val fwdRbf = TypeCheckedForwarder(aliceRbf, bobRbf, alice2bob, bob2alice) aliceRbf ! Start(alice2bob.ref) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala index 12fe47d53f..9c6eb478f4 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala @@ -391,16 +391,16 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture bob2alice.forward(alice, bobInput.copy(previousTxOutput = 42)) alice2bob.expectMsgType[TxAbort] alice2bob.forward(bob) - awaitAssert(assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.RbfAborted)) + awaitAssert(assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.RbfAborted)) bob2alice.expectMsgType[TxAbort] // bob acks alice's tx_abort bob2alice.forward(alice) alice2bob.expectNoMessage(100 millis) // Alice and Bob clear RBF data from their state. - assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) + assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx == fundingTxAlice) assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.isEmpty) - assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) + assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx == fundingTxBob) assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.isEmpty) } @@ -420,16 +420,16 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture // Alice and Bob build a new version of the funding transaction. alice2bob.expectMsgType[TxAbort] alice2bob.forward(bob) - awaitAssert(assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.RbfAborted)) + awaitAssert(assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.RbfAborted)) bob2alice.expectMsgType[TxAbort] // bob acks alice's tx_abort bob2alice.forward(alice) alice2bob.expectNoMessage(100 millis) // Alice and Bob clear RBF data from their state. - assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) + assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx == fundingTxAlice) assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.isEmpty) - assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) + assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx == fundingTxBob) assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.isEmpty) } @@ -783,16 +783,16 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture alice2bob.expectMsgType[TxComplete] // bob doesn't receive alice's tx_complete alice2bob.expectMsgType[CommitSig] // bob doesn't receive alice's commit_sig - awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs]) - val rbfTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.asInstanceOf[RbfStatus.RbfWaitingForSigs].signingSession.fundingTx - assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfInProgress]) + awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs]) + val rbfTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.asInstanceOf[DualFundingStatus.RbfWaitingForSigs].signingSession.fundingTx + assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfInProgress]) alice ! INPUT_DISCONNECTED awaitCond(alice.stateName == OFFLINE) - assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs]) + assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs]) bob ! INPUT_DISCONNECTED awaitCond(bob.stateName == OFFLINE) - assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) + assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) val aliceInit = Init(alice.underlyingActor.nodeParams.features.initFeatures()) val bobInit = Init(bob.underlyingActor.nodeParams.features.initFeatures()) @@ -808,8 +808,8 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture bob2alice.forward(alice) alice2bob.expectMsgType[TxAbort] alice2bob.forward(bob) - assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) - assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) + assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) + assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) } @@ -844,16 +844,16 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture bob2alice.expectMsgType[CommitSig] // alice doesn't receive bob's commit_sig alice2bob.expectMsgType[CommitSig] // bob doesn't receive alice's commit_sig - awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs]) - awaitCond(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs]) - val rbfTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.asInstanceOf[RbfStatus.RbfWaitingForSigs].signingSession.fundingTx + awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs]) + awaitCond(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs]) + val rbfTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.asInstanceOf[DualFundingStatus.RbfWaitingForSigs].signingSession.fundingTx alice ! INPUT_DISCONNECTED awaitCond(alice.stateName == OFFLINE) - assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs]) + assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs]) bob ! INPUT_DISCONNECTED awaitCond(bob.stateName == OFFLINE) - assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs]) + assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs]) val aliceInit = Init(alice.underlyingActor.nodeParams.features.initFeatures()) val bobInit = Init(bob.underlyingActor.nodeParams.features.initFeatures()) @@ -879,8 +879,8 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture assert(bobListener.expectMsgType[TransactionPublished].tx.txid == nextFundingTx.signedTx.txid) assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == nextFundingTx.signedTx.txid) assert(currentFundingTxId != nextFundingTx.txId) - awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) - awaitCond(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) + awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) + awaitCond(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) } test("recv Error", Tag(ChannelStateTestsTags.DualFunding)) { f => diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/io/PendingChannelsRateLimiterSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/io/PendingChannelsRateLimiterSpec.scala index 0512401433..ea06709f52 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/io/PendingChannelsRateLimiterSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/io/PendingChannelsRateLimiterSpec.scala @@ -82,7 +82,7 @@ class PendingChannelsRateLimiterSpec extends ScalaTestWithActorTestKit(ConfigFac DATA_WAIT_FOR_CHANNEL_READY(commitments(peerAtLimit1, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)), ) val channelsAtLimit2 = Seq( - DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments(peerAtLimit2, channelIdAtLimit2), 0 msat, 0 msat, BlockHeight(0), BlockHeight(0), RbfStatus.NoRbf, None), + DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments(peerAtLimit2, channelIdAtLimit2), 0 msat, 0 msat, BlockHeight(0), BlockHeight(0), DualFundingStatus.WaitingForConfirmations, None), DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(peerAtLimit2, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)), ) val channelsBelowLimit1 = Seq( @@ -101,7 +101,7 @@ class PendingChannelsRateLimiterSpec extends ScalaTestWithActorTestKit(ConfigFac val initiatorChannels = Seq( DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments(peerBelowLimit1, randomBytes32(), isOpener = true), BlockHeight(0), None, Left(FundingCreated(channelIdAtLimit1, TxId(ByteVector32.Zeroes), 3, randomBytes64()))), DATA_WAIT_FOR_CHANNEL_READY(commitments(peerBelowLimit1, randomBytes32(), isOpener = true), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)), - DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments(peerAtLimit1, randomBytes32(), isOpener = true), 0 msat, 0 msat, BlockHeight(0), BlockHeight(0), RbfStatus.NoRbf, None), + DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments(peerAtLimit1, randomBytes32(), isOpener = true), 0 msat, 0 msat, BlockHeight(0), BlockHeight(0), DualFundingStatus.WaitingForConfirmations, None), DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(peerAtLimit1, randomBytes32(), isOpener = true), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)), ) val publicChannels = channelsOnWhitelistAtLimit ++ channelsAtLimit1 ++ channelsAtLimit2 ++ channelsBelowLimit1 ++ channelsBelowLimit2 diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4Spec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4Spec.scala index 4a86251654..b666caa99f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4Spec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4Spec.scala @@ -144,15 +144,15 @@ class ChannelCodecs4Spec extends AnyFunSuite { RemoteCommit(0, CommitmentSpec(Set.empty, FeeratePerKw(1000 sat), 75_000_000 msat, 100_000_000 msat), randomTxId(), randomKey().publicKey) ) val testCases = Map( - RbfStatus.NoRbf -> RbfStatus.NoRbf, - RbfStatus.RbfRequested(CMD_BUMP_FUNDING_FEE(null, FeeratePerKw(750 sat), fundingFeeBudget = 100_000.sat, 0)) -> RbfStatus.NoRbf, - RbfStatus.RbfInProgress(None, null, None) -> RbfStatus.NoRbf, - RbfStatus.RbfWaitingForSigs(waitingForSigs) -> RbfStatus.RbfWaitingForSigs(waitingForSigs), - RbfStatus.RbfAborted -> RbfStatus.NoRbf, + DualFundingStatus.WaitingForConfirmations -> DualFundingStatus.WaitingForConfirmations, + DualFundingStatus.RbfRequested(CMD_BUMP_FUNDING_FEE(null, FeeratePerKw(750 sat), fundingFeeBudget = 100_000.sat, 0)) -> DualFundingStatus.WaitingForConfirmations, + DualFundingStatus.RbfInProgress(None, null, None) -> DualFundingStatus.WaitingForConfirmations, + DualFundingStatus.RbfWaitingForSigs(waitingForSigs) -> DualFundingStatus.RbfWaitingForSigs(waitingForSigs), + DualFundingStatus.RbfAborted -> DualFundingStatus.WaitingForConfirmations, ) testCases.foreach { case (status, expected) => - val encoded = rbfStatusCodec.encode(status).require - val decoded = rbfStatusCodec.decode(encoded).require.value + val encoded = dualFundingStatusCodec.encode(status).require + val decoded = dualFundingStatusCodec.decode(encoded).require.value assert(decoded == expected) } }