From 13d4c9f06c655867d95597359c8d78fbd3c87274 Mon Sep 17 00:00:00 2001 From: Bastien Teinturier <31281497+t-bast@users.noreply.github.com> Date: Fri, 18 Oct 2024 04:44:10 +0200 Subject: [PATCH] Add support for RBF-ing splice transactions (#2925) 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. When 0-conf isn't used, we reject `splice_init` while the previous splice transaction hasn't confirmed. Our peer should either use RBF instead of creating a new splice, or they should wait for our node to receive the block that confirmed the previous transaction. This protects against chains of unconfirmed transactions. When using 0-conf, we reject `tx_init_rbf` and allow creating chains of unconfirmed splice transactions: using RBF with 0-conf can lead to one side stealing funds, which is why we prevent it. If our peer was buying liquidity but tries to cancel the purchase with an RBF attempt, we reject it: this prevents edge cases where the seller may end up adding liquidity to the channel without being paid in return. --- docs/release-notes/eclair-vnext.md | 1 + .../main/scala/fr/acinq/eclair/Eclair.scala | 49 +- .../fr/acinq/eclair/channel/ChannelData.scala | 99 +- .../eclair/channel/ChannelExceptions.scala | 2 + .../fr/acinq/eclair/channel/fsm/Channel.scala | 348 +++++-- .../channel/fsm/ChannelOpenDualFunded.scala | 143 +-- .../channel/fsm/DualFundingHandlers.scala | 8 +- .../channel/fund/InteractiveTxBuilder.scala | 28 +- .../channel/fund/InteractiveTxFunder.scala | 35 +- .../channel/version4/ChannelCodecs4.scala | 12 +- .../wire/protocol/LightningMessageTypes.scala | 6 +- .../eclair/wire/protocol/LiquidityAds.scala | 2 +- .../scala/fr/acinq/eclair/TestConstants.scala | 2 +- .../channel/InteractiveTxBuilderSpec.scala | 24 +- ...WaitForDualFundingConfirmedStateSpec.scala | 206 ++-- .../states/e/NormalSplicesStateSpec.scala | 951 +++++++++++++----- .../io/PendingChannelsRateLimiterSpec.scala | 4 +- .../channel/version4/ChannelCodecs4Spec.scala | 16 +- .../acinq/eclair/api/handlers/Channel.scala | 8 +- 19 files changed, 1411 insertions(+), 533 deletions(-) diff --git a/docs/release-notes/eclair-vnext.md b/docs/release-notes/eclair-vnext.md index bd7013d146..098c63977f 100644 --- a/docs/release-notes/eclair-vnext.md +++ b/docs/release-notes/eclair-vnext.md @@ -38,6 +38,7 @@ Eclair will not allow remote peers to open new obsolete channels that do not sup - `channelstats` now takes optional parameters `--count` and `--skip` to control pagination. By default, it will return first 10 entries. (#2890) - `createinvoice` now takes an optional `--privateChannelIds` parameter that can be used to add routing hints through private channels. (#2909) - `nodes` allows filtering nodes that offer liquidity ads (#2848) +- `rbfsplice` lets any channel participant RBF the current unconfirmed splice transaction (#2887) ### Miscellaneous improvements and bug fixes diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala index 8159a9f109..60be7ba39a 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala @@ -96,6 +96,8 @@ trait Eclair { def spliceOut(channelId: ByteVector32, amountOut: Satoshi, scriptOrAddress: Either[ByteVector, String])(implicit timeout: Timeout): Future[CommandResponse[CMD_SPLICE]] + def rbfSplice(channelId: ByteVector32, targetFeerate: FeeratePerKw, fundingFeeBudget: Satoshi, lockTime_opt: Option[Long])(implicit timeout: Timeout): Future[CommandResponse[CMD_BUMP_FUNDING_FEE]] + def close(channels: List[ApiTypes.ChannelIdentifier], scriptPubKey_opt: Option[ByteVector], closingFeerates_opt: Option[ClosingFeerates])(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_CLOSE]]]] def forceClose(channels: List[ApiTypes.ChannelIdentifier])(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_FORCECLOSE]]]] @@ -232,17 +234,18 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging { } override def rbfOpen(channelId: ByteVector32, targetFeerate: FeeratePerKw, fundingFeeBudget: Satoshi, lockTime_opt: Option[Long])(implicit timeout: Timeout): Future[CommandResponse[CMD_BUMP_FUNDING_FEE]] = { - sendToChannelTyped(channel = Left(channelId), - cmdBuilder = CMD_BUMP_FUNDING_FEE(_, targetFeerate, fundingFeeBudget, lockTime_opt.getOrElse(appKit.nodeParams.currentBlockHeight.toLong), requestFunding_opt = None)) + sendToChannelTyped( + channel = Left(channelId), + cmdBuilder = CMD_BUMP_FUNDING_FEE(_, targetFeerate, fundingFeeBudget, lockTime_opt.getOrElse(appKit.nodeParams.currentBlockHeight.toLong), requestFunding_opt = None) + ) } override def spliceIn(channelId: ByteVector32, amountIn: Satoshi, pushAmount_opt: Option[MilliSatoshi])(implicit timeout: Timeout): Future[CommandResponse[CMD_SPLICE]] = { - sendToChannelTyped(channel = Left(channelId), - cmdBuilder = CMD_SPLICE(_, - spliceIn_opt = Some(SpliceIn(additionalLocalFunding = amountIn, pushAmount = pushAmount_opt.getOrElse(0.msat))), - spliceOut_opt = None, - requestFunding_opt = None, - )) + val spliceIn = SpliceIn(additionalLocalFunding = amountIn, pushAmount = pushAmount_opt.getOrElse(0.msat)) + sendToChannelTyped( + channel = Left(channelId), + cmdBuilder = CMD_SPLICE(_, spliceIn_opt = Some(spliceIn), spliceOut_opt = None, requestFunding_opt = None) + ) } override def spliceOut(channelId: ByteVector32, amountOut: Satoshi, scriptOrAddress: Either[ByteVector, String])(implicit timeout: Timeout): Future[CommandResponse[CMD_SPLICE]] = { @@ -253,12 +256,18 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging { case Right(script) => Script.write(script) } } - sendToChannelTyped(channel = Left(channelId), - cmdBuilder = CMD_SPLICE(_, - spliceIn_opt = None, - spliceOut_opt = Some(SpliceOut(amount = amountOut, scriptPubKey = script)), - requestFunding_opt = None, - )) + val spliceOut = SpliceOut(amount = amountOut, scriptPubKey = script) + sendToChannelTyped( + channel = Left(channelId), + cmdBuilder = CMD_SPLICE(_, spliceIn_opt = None, spliceOut_opt = Some(spliceOut), requestFunding_opt = None) + ) + } + + override def rbfSplice(channelId: ByteVector32, targetFeerate: FeeratePerKw, fundingFeeBudget: Satoshi, lockTime_opt: Option[Long])(implicit timeout: Timeout): Future[CommandResponse[CMD_BUMP_FUNDING_FEE]] = { + sendToChannelTyped( + channel = Left(channelId), + cmdBuilder = CMD_BUMP_FUNDING_FEE(_, targetFeerate, fundingFeeBudget, lockTime_opt.getOrElse(appKit.nodeParams.currentBlockHeight.toLong), requestFunding_opt = None) + ) } override def close(channels: List[ApiTypes.ChannelIdentifier], scriptPubKey_opt: Option[ByteVector], closingFeerates_opt: Option[ClosingFeerates])(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_CLOSE]]]] = { @@ -579,9 +588,9 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging { case Left(channelId) => appKit.register ? Register.Forward(null, channelId, request) case Right(shortChannelId) => appKit.register ? Register.ForwardShortId(null, shortChannelId, request) }).map { - case t: R@unchecked => t - case t: Register.ForwardFailure[C]@unchecked => throw ChannelNotFound(Left(t.fwd.channelId)) - case t: Register.ForwardShortIdFailure[C]@unchecked => throw ChannelNotFound(Right(t.fwd.shortChannelId)) + case t: R @unchecked => t + case t: Register.ForwardFailure[C] @unchecked => throw ChannelNotFound(Left(t.fwd.channelId)) + case t: Register.ForwardShortIdFailure[C] @unchecked => throw ChannelNotFound(Right(t.fwd.shortChannelId)) } private def sendToChannelTyped[C <: Command, R <: CommandResponse[C]](channel: ApiTypes.ChannelIdentifier, cmdBuilder: akka.actor.typed.ActorRef[Any] => C)(implicit timeout: Timeout): Future[R] = @@ -592,9 +601,9 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging { case Right(shortChannelId) => Register.ForwardShortId(replyTo, shortChannelId, cmd) } }.map { - case t: R@unchecked => t - case t: Register.ForwardFailure[C]@unchecked => throw ChannelNotFound(Left(t.fwd.channelId)) - case t: Register.ForwardShortIdFailure[C]@unchecked => throw ChannelNotFound(Right(t.fwd.shortChannelId)) + case t: R @unchecked => t + case t: Register.ForwardFailure[C] @unchecked => throw ChannelNotFound(Left(t.fwd.channelId)) + case t: Register.ForwardShortIdFailure[C] @unchecked => throw ChannelNotFound(Right(t.fwd.shortChannelId)) } /** 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 e18144f1d7..330eb34670 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, LiquidityAds, 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, LiquidityAds, 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 @@ -161,7 +161,7 @@ object Upstream { def apply(hot: Hot): Cold = hot match { case Local(id) => Local(id) case Hot.Channel(add, _, _) => Cold.Channel(add.channelId, add.id, add.amountMsat) - case Hot.Trampoline(received) => Cold.Trampoline(received.map(r => Cold.Channel(r.add.channelId, r.add.id, r.add.amountMsat)).toList) + case Hot.Trampoline(received) => Cold.Trampoline(received.map(r => Cold.Channel(r.add.channelId, r.add.id, r.add.amountMsat))) } /** Our node is forwarding a single incoming HTLC. */ @@ -199,8 +199,8 @@ 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, @@ -211,14 +211,14 @@ final case class CMD_ADD_HTLC(replyTo: ActorRef, confidence: Double, fundingFee_opt: Option[LiquidityAds.FundingFee], origin: Origin.Hot, - commit: Boolean = false) extends HasReplyToCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence + commit: Boolean = false) extends HasReplyToCommand with ForbiddenCommandDuringQuiescenceNegotiation with ForbiddenCommandWhenQuiescent -sealed trait HtlcSettlementCommand extends HasOptionalReplyToCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence { def id: Long } +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) { @@ -226,19 +226,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, requestFunding_opt: Option[LiquidityAds.RequestFunding]) 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], requestFunding_opt: Option[LiquidityAds.RequestFunding]) extends Command { +final case class CMD_SPLICE(replyTo: akka.actor.typed.ActorRef[CommandResponse[ChannelFundingCommand]], spliceIn_opt: Option[SpliceIn], spliceOut_opt: Option[SpliceOut], requestFunding_opt: Option[LiquidityAds.RequestFunding]) 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, requestFunding_opt: Option[LiquidityAds.RequestFunding]) 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 @@ -475,42 +478,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 { @@ -604,7 +626,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] @@ -619,7 +641,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 56c3738ccd..9155c841fa 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 @@ -91,9 +91,11 @@ 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 InvalidSpliceWithUnconfirmedTx (override val channelId: ByteVector32, fundingTx: TxId) extends ChannelException(channelId, s"invalid splice attempt: the current funding transaction is still unconfirmed (txId=$fundingTx), you should use tx_init_rbf instead") 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") +case class InvalidRbfMissingLiquidityPurchase (override val channelId: ByteVector32, expectedAmount: Satoshi) extends ChannelException(channelId, s"cannot accept rbf attempt: the previous attempt contained a liquidity purchase of $expectedAmount but this one doesn't contain any liquidity purchase") case class InvalidRbfAttempt (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid rbf attempt") case class NoMoreHtlcsClosingInProgress (override val channelId: ByteVector32) extends ChannelException(channelId, "cannot send new htlcs, closing in progress") case class ClosingAlreadyInProgress (override val channelId: ByteVector32) extends ChannelException(channelId, "closing already in progress") 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 0383bb96eb..fe0dc8c21b 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 @@ -388,7 +388,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)) @@ -399,7 +399,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)) @@ -410,7 +410,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. @@ -612,10 +612,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 _ => @@ -852,24 +853,70 @@ 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.SplicePrototype)) { + if (!d.commitments.params.remoteParams.initFeatures.hasFeature(Features.SplicePrototype)) { + log.warning("cannot initiate splice, peer doesn't support splicing") + cmd.replyTo ! RES_FAILURE(cmd, CommandUnavailableInThisState(d.channelId, "splice", NORMAL)) + stay() + } else 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, InvalidSpliceWithUnconfirmedTx(d.channelId, d.commitments.latest.fundingTxId)) + 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 (cmd.targetFeerate < fundingTx.fundingParams.minNextFeerate) { + 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() + } else if (cmd.requestFunding_opt.isEmpty && fundingTx.liquidityPurchase_opt.nonEmpty) { + log.warning("cannot initiate rbf: a liquidity purchase was included in the previous transaction but is not included in this one") + cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfMissingLiquidityPurchase(d.channelId, fundingTx.liquidityPurchase_opt.get.amount)) + stay() + } else { + 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 _: LocalFundingStatus.ZeroconfPublishedFundingTx => + log.warning("cannot initiate rbf: transaction is using 0-conf (txId={})", d.commitments.latest.fundingTxId) + cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfZeroConf(d.channelId)) + stay() + case _ => + log.warning("cannot initiate rbf: transaction already confirmed (txId={})", d.commitments.latest.fundingTxId) + 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 => @@ -885,24 +932,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") @@ -931,6 +988,13 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with } else if (msg.feerate < nodeParams.currentBitcoinCoreFeerates.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 (txIds={})", previousTxs.mkString(", ")) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceWithUnconfirmedTx(d.channelId, d.commitments.latest.fundingTxId).getMessage) + } else if (d.commitments.latest.localFundingStatus.isInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx]) { + log.info("rejecting splice request: the previous funding transaction is unconfirmed (txId={})", d.commitments.latest.fundingTxId) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceWithUnconfirmedTx(d.channelId, d.commitments.latest.fundingTxId).getMessage) } else { val parentCommitment = d.commitments.latest.commitment val localFundingPubKey = nodeParams.channelKeyManager.fundingPublicKey(d.commitments.params.localParams.fundingKeyPath, parentCommitment.fundingTxIndex + 1).publicKey @@ -1040,6 +1104,121 @@ 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(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 msg.requestFunding_opt.isEmpty && rbf.latestFundingTx.liquidityPurchase_opt.nonEmpty => + log.info("rejecting rbf attempt: a liquidity purchase was included in the previous transaction but is not included in this one") + // Our peer is trying to trick us into contributing the amount they were previously paying for, but + // without paying for it by leveraging the fact that we'll keep contributing the same amount. + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidRbfMissingLiquidityPurchase(d.channelId, rbf.latestFundingTx.liquidityPurchase_opt.get.amount).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) => + val fundingScript = d.commitments.latest.commitInput.txOut.publicKeyScript + LiquidityAds.validateRequest(nodeParams.privateKey, d.channelId, fundingScript, msg.feerate, isChannelCreation = false, msg.requestFunding_opt, nodeParams.willFundRates_opt, feeCreditUsed_opt = None) match { + case Left(t) => + log.warning("rejecting rbf request with invalid liquidity ads: {}", t.getMessage) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, t.getMessage) + case Right(willFund_opt) => + // We contribute the amount of liquidity requested by our peer, if liquidity ads is active. + // Otherwise we keep the same contribution we made to the previous funding transaction. + val fundingContribution = willFund_opt.map(_.purchase.amount).getOrElse(rbf.latestFundingTx.fundingParams.localContribution) + log.info("accepting rbf with remote.in.amount={} local.in.amount={}", msg.fundingContribution, fundingContribution) + val txAckRbf = TxAckRbf(d.channelId, fundingContribution, rbf.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote, willFund_opt.map(_.willFund)) + 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, + willFund_opt.map(_.purchase), + 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) => + val fundingScript = d.commitments.latest.commitInput.txOut.publicKeyScript + LiquidityAds.validateRemoteFunding(cmd.requestFunding_opt, remoteNodeId, d.channelId, fundingScript, msg.fundingContribution, txInitRbf.feerate, isChannelCreation = false, msg.willFund_opt) match { + case Left(t) => + log.info("rejecting rbf attempt: invalid liquidity ads response ({})", t.getMessage) + cmd.replyTo ! RES_FAILURE(cmd, t) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, t.getMessage) + case Right(liquidityPurchase_opt) => + 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, + liquidityPurchase_opt = liquidityPurchase_opt, + wallet + )) + txBuilder ! InteractiveTxBuilder.Start(self) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceInProgress(cmd_opt = Some(cmd), sessionId, txBuilder, remoteCommitSig = None)) + } + case Left(f) => + cmd.replyTo ! RES_FAILURE(cmd, f) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, f.getMessage) + } + case _ => + log.info("ignoring unexpected tx_ack_rbf (status={})", d.spliceStatus.getClass.getSimpleName) + stay() + } + case Event(msg: TxAbort, d: DATA_NORMAL) => d.spliceStatus match { case SpliceStatus.SpliceInProgress(cmd_opt, _, txBuilder, _) => @@ -1055,6 +1234,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) @@ -1065,7 +1248,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)) @@ -1900,8 +2083,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 @@ -1964,8 +2147,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 @@ -1983,7 +2166,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) } @@ -2770,42 +2953,81 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with } private def initiateSplice(cmd: CMD_SPLICE, d: DATA_NORMAL): Either[ChannelException, SpliceInit] = { - if (d.commitments.isQuiescent) { - val parentCommitment = d.commitments.latest.commitment - val targetFeerate = nodeParams.onChainFeeConf.getFundingFeerate(nodeParams.currentBitcoinCoreFeerates) - val fundingContribution = InteractiveTxFunder.computeSpliceContribution( - isInitiator = true, - sharedInput = Multisig2of2Input(parentCommitment), - spliceInAmount = cmd.additionalLocalFunding, - spliceOut = cmd.spliceOutputs, - targetFeerate = targetFeerate) + val parentCommitment = d.commitments.latest.commitment + val targetFeerate = nodeParams.onChainFeeConf.getFundingFeerate(nodeParams.currentBitcoinCoreFeerates) + val fundingContribution = InteractiveTxFunder.computeSpliceContribution( + isInitiator = true, + sharedInput = Multisig2of2Input(parentCommitment), + spliceInAmount = cmd.additionalLocalFunding, + spliceOut = cmd.spliceOutputs, + targetFeerate = targetFeerate) + val commitTxFees = if (d.commitments.params.localParams.paysCommitTxFees) { + Transactions.commitTxTotalCost(d.commitments.params.remoteParams.dustLimit, parentCommitment.remoteCommit.spec, d.commitments.params.commitmentFormat) + } else { + 0.sat + } + if (fundingContribution < 0.sat && parentCommitment.localCommit.spec.toLocal + fundingContribution < parentCommitment.localChannelReserve(d.commitments.params).max(commitTxFees)) { + log.warning(s"cannot do splice: insufficient funds (commitTxFees=$commitTxFees reserve=${parentCommitment.localChannelReserve(d.commitments.params)})") + Left(InvalidSpliceRequest(d.channelId)) + } else if (cmd.spliceOut_opt.map(_.scriptPubKey).exists(!MutualClose.isValidFinalScriptPubkey(_, allowAnySegwit = true))) { + log.warning("cannot do splice: invalid splice-out script") + Left(InvalidSpliceRequest(d.channelId)) + } else { + log.info(s"initiating splice with local.in.amount=${cmd.additionalLocalFunding} local.in.push=${cmd.pushAmount} local.out.amount=${cmd.spliceOut_opt.map(_.amount).sum}") + val spliceInit = SpliceInit(d.channelId, + fundingContribution = fundingContribution, + lockTime = nodeParams.currentBlockHeight.toLong, + feerate = targetFeerate, + fundingPubKey = keyManager.fundingPublicKey(d.commitments.params.localParams.fundingKeyPath, parentCommitment.fundingTxIndex + 1).publicKey, + pushAmount = cmd.pushAmount, + requireConfirmedInputs = nodeParams.channelConf.requireConfirmedInputsForDualFunding, + requestFunding_opt = cmd.requestFunding_opt + ) + Right(spliceInit) + } + } + + 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, parentCommitment.remoteCommit.spec, d.commitments.params.commitmentFormat) + Transactions.commitTxTotalCost(d.commitments.params.remoteParams.dustLimit, rbf.parentCommitment.remoteCommit.spec, d.commitments.params.commitmentFormat) } else { 0.sat } - if (fundingContribution < 0.sat && parentCommitment.localCommit.spec.toLocal + fundingContribution < parentCommitment.localChannelReserve(d.commitments.params).max(commitTxFees)) { - log.warning(s"cannot do splice: insufficient funds (commitTxFees=$commitTxFees reserve=${parentCommitment.localChannelReserve(d.commitments.params)})") - Left(InvalidSpliceRequest(d.channelId)) - } else if (cmd.spliceOut_opt.map(_.scriptPubKey).exists(!MutualClose.isValidFinalScriptPubkey(_, allowAnySegwit = true))) { - log.warning("cannot do splice: invalid splice-out script") + 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 { - log.info(s"initiating splice with local.in.amount=${cmd.additionalLocalFunding} local.in.push=${cmd.pushAmount} local.out.amount=${cmd.spliceOut_opt.map(_.amount).sum}") - val spliceInit = SpliceInit(d.channelId, - fundingContribution = fundingContribution, - lockTime = nodeParams.currentBlockHeight.toLong, - feerate = targetFeerate, - fundingPubKey = keyManager.fundingPublicKey(d.commitments.params.localParams.fundingKeyPath, parentCommitment.fundingTxIndex + 1).publicKey, - pushAmount = cmd.pushAmount, - requireConfirmedInputs = nodeParams.channelConf.requireConfirmedInputsForDualFunding, - requestFunding_opt = cmd.requestFunding_opt - ) - Right(spliceInit) + val txInitRbf = TxInitRbf(d.channelId, cmd.lockTime, cmd.targetFeerate, fundingContribution, rbf.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote, cmd.requestFunding_opt) + Right(txInitRbf) } - } else { - log.warning("cannot initiate splice, channel is not quiescent") - Left(InvalidSpliceNotQuiescent(d.channelId)) + }) + } + + 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, d.commitments.changes, fundingTx, previousTxs, cmd_opt.map(_.fundingFeeBudget))) + case None => + log.warning("cannot initiate rbf: cannot find parent commitment") + Left(InvalidRbfAttempt(d.channelId)) + } + case _: LocalFundingStatus.ZeroconfPublishedFundingTx => + log.warning("cannot initiate rbf: transaction is using 0-conf (txId={})", d.commitments.latest.fundingTxId) + Left(InvalidRbfZeroConf(d.channelId)) + case _ => + log.warning("cannot initiate rbf: transaction already confirmed (txId={})", d.commitments.latest.fundingTxId) + Left(InvalidRbfTxConfirmed(d.channelId)) } } @@ -2831,9 +3053,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 4e89d3a1df..4cda1931a1 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 @@ -29,7 +29,7 @@ import fr.acinq.eclair.channel.publish.TxPublisher.SetChannelId import fr.acinq.eclair.crypto.ShaChain import fr.acinq.eclair.io.Peer.{LiquidityPurchaseSigned, OpenChannelResponse} import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{MilliSatoshiLong, RealShortChannelId, ToMilliSatoshiConversion, UInt64, randomBytes32} +import fr.acinq.eclair.{RealShortChannelId, ToMilliSatoshiConversion, UInt64, randomBytes32} /** * Created by t-bast on 19/04/2022. @@ -400,7 +400,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 } } @@ -423,7 +423,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 => @@ -471,17 +471,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) => @@ -489,8 +489,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) } } @@ -502,16 +502,19 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { } else if (zeroConf) { cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfZeroConf(d.channelId)) stay() + } else if (cmd.requestFunding_opt.isEmpty && d.latestFundingTx.liquidityPurchase_opt.nonEmpty) { + cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfMissingLiquidityPurchase(d.channelId, d.latestFundingTx.liquidityPurchase_opt.get.amount)) + 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, d.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote, cmd.requestFunding_opt) - 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") @@ -528,35 +531,43 @@ 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 (msg.requestFunding_opt.isEmpty && d.latestFundingTx.liquidityPurchase_opt.nonEmpty) { + log.info("rejecting rbf attempt: a liquidity purchase was included in the previous transaction but is not included in this one") + // Our peer is trying to trick us into contributing the amount they were previously paying for, but + // without paying for it by leveraging the fact that we'll keep contributing the same amount. + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfMissingLiquidityPurchase(d.channelId, d.latestFundingTx.liquidityPurchase_opt.get.amount).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 { val fundingScript = d.commitments.latest.commitInput.txOut.publicKeyScript LiquidityAds.validateRequest(nodeParams.privateKey, d.channelId, fundingScript, msg.feerate, isChannelCreation = true, msg.requestFunding_opt, nodeParams.willFundRates_opt, None) match { case Left(t) => log.warning("rejecting rbf attempt: invalid liquidity ads request ({})", t.getMessage) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, t.getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, t.getMessage) case Right(willFund_opt) => log.info("our peer wants to raise the feerate of the funding transaction (previous={} target={})", d.latestFundingTx.fundingParams.targetFeerate, msg.feerate) // We contribute the amount of liquidity requested by our peer, if liquidity ads is active. + // Otherwise we keep the same contribution we made to the previous funding transaction. + val fundingContribution = willFund_opt.map(_.purchase.amount).getOrElse(d.latestFundingTx.fundingParams.localContribution) + log.info("accepting rbf with remote.in.amount={} local.in.amount={}", msg.fundingContribution, fundingContribution) val fundingParams = d.latestFundingTx.fundingParams.copy( - localContribution = willFund_opt.map(_.purchase.amount).getOrElse(d.latestFundingTx.fundingParams.localContribution), + localContribution = fundingContribution, remoteContribution = msg.fundingContribution, lockTime = msg.lockTime, targetFeerate = msg.feerate, @@ -566,7 +577,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, liquidityPurchase_opt = willFund_opt.map(_.purchase), wallet)) @@ -575,26 +586,26 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { Some(TxAckRbf(d.channelId, fundingParams.localContribution, d.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote, willFund_opt.map(_.willFund))), 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) => val fundingParams = d.latestFundingTx.fundingParams.copy( // we don't change our funding contribution remoteContribution = msg.fundingContribution, @@ -606,19 +617,19 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { case Left(t) => log.warning("rejecting rbf attempt: invalid liquidity ads response ({})", t.getMessage) cmd.replyTo ! RES_FAILURE(cmd, t) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, t.getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, t.getMessage) case Right(liquidityPurchase_opt) => log.info("our peer accepted our rbf attempt and will contribute {} to the funding transaction", msg.fundingContribution) val txBuilder = context.spawnAnonymous(InteractiveTxBuilder( randomBytes32(), nodeParams, fundingParams, channelParams = d.commitments.params, - purpose = InteractiveTxBuilder.PreviousTxRbf(d.commitments.active.head, 0 msat, 0 msat, previousTransactions = d.allFundingTxs.map(_.sharedTx), 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, liquidityPurchase_opt = liquidityPurchase_opt, 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") @@ -626,8 +637,8 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { } 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 _ => @@ -636,24 +647,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 } } @@ -663,32 +674,32 @@ 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, liquidityPurchase_opt) => @@ -697,16 +708,16 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { liquidityPurchase_opt.collect { case purchase if !signingSession.fundingParams.isInitiator => peer ! LiquidityPurchaseSigned(d.channelId, signingSession.fundingTx.txId, signingSession.fundingTxIndex, d.commitments.params.remoteParams.htlcMinimum, purchase) } - 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 funding status={}", d.status.getClass.getSimpleName) stay() } @@ -731,10 +742,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 @@ -744,7 +755,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) => @@ -760,16 +771,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 06afc201ea..ffc2961848 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 @@ -216,9 +216,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 localNextHtlcId: Long = 0 @@ -228,6 +229,24 @@ 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, changes: CommitmentChanges, 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 localNextHtlcId: Long = changes.localNextHtlcId + override val remoteNextHtlcId: Long = changes.remoteNextHtlcId + 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 @@ -440,7 +459,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 bd780a46d6..418ca53cac 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 @@ -94,6 +94,32 @@ object InteractiveTxFunder { spliceInAmount - spliceOut.map(_.amount).sum - fees } + private def needsAdditionalFunding(fundingParams: InteractiveTxParams, purpose: Purpose): Boolean = { + if (fundingParams.isInitiator) { + purpose match { + case _: FundingTx | _: FundingTxRbf => + // We're the initiator, but we may be purchasing liquidity without contributing to the funding transaction if + // we're using on-the-fly funding. In that case it's acceptable that we don't pay the mining fees for the + // shared output. Otherwise, we must contribute funds to pay the mining fees. + fundingParams.localContribution > 0.sat || fundingParams.localOutputs.nonEmpty + case _: SpliceTx | _: SpliceTxRbf => + // We're the initiator, we always have to pay on-chain fees for the shared input and output, even if we don't + // splice in or out. If we're not paying those on-chain fees by lowering our channel contribution, we must add + // more funding. + fundingParams.localContribution + fundingParams.localOutputs.map(_.amount).sum >= 0.sat + } + } else { + // We're not the initiator, so we don't have to pay on-chain fees for the common transaction fields. + if (fundingParams.localOutputs.isEmpty) { + // We're not splicing out: we only need to add funds if we're splicing in. + fundingParams.localContribution > 0.sat + } else { + // We need to add funds if we're not paying on-chain fees by lowering our channel contribution. + fundingParams.localContribution + fundingParams.localOutputs.map(_.amount).sum >= 0.sat + } + } + } + private def canUseInput(fundingParams: InteractiveTxParams, txIn: TxIn, previousTx: Transaction, confirmations: Int): Boolean = { // Wallet input transaction must fit inside the tx_add_input message. val previousTxSizeOk = Transaction.write(previousTx).length <= 65000 @@ -137,7 +163,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 } @@ -150,8 +177,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response // The balances in the shared input may have changed since the previous funding attempt, so we ignore the previous // shared input and will add it explicitly later. val previousWalletInputs = previousTransactions.flatMap(_.tx.localInputs).distinctBy(_.outPoint) - val hasEnoughFunding = fundingParams.localContribution + fundingParams.localOutputs.map(_.amount).sum <= 0.sat - if (hasEnoughFunding) { + if (!needsAdditionalFunding(fundingParams, purpose)) { log.info("we seem to have enough funding, no need to request wallet inputs from bitcoind") // We're not contributing to the shared output or we have enough funds in our shared input, so we don't need to // ask bitcoind for more inputs. When splicing some funds out, we assume that the caller has allocated enough @@ -207,7 +233,8 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response } 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 36b7e93961..afcfbc4cbd 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 @@ -636,10 +636,10 @@ private[channel] object ChannelCodecs4 { (waitingForSigsWithoutLiquidityPurchaseCodec, waitingForSigsCodec) } - val rbfStatusCodec: Codec[RbfStatus] = discriminated[RbfStatus].by(uint8) - .\(0x01) { case status: RbfStatus if !status.isInstanceOf[RbfStatus.RbfWaitingForSigs] => RbfStatus.NoRbf }(provide(RbfStatus.NoRbf)) - .\(0x03) { case status: RbfStatus.RbfWaitingForSigs => status }(interactiveTxWaitingForSigsCodec.as[RbfStatus.RbfWaitingForSigs]) - .\(0x02) { case status: RbfStatus.RbfWaitingForSigs => status }(interactiveTxWaitingForSigsWithoutLiquidityPurchaseCodec.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)) + .\(0x03) { case status: DualFundingStatus.RbfWaitingForSigs => status }(interactiveTxWaitingForSigsCodec.as[DualFundingStatus.RbfWaitingForSigs]) + .\(0x02) { case status: DualFundingStatus.RbfWaitingForSigs => status }(interactiveTxWaitingForSigsWithoutLiquidityPurchaseCodec.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)) @@ -688,7 +688,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] = ( @@ -697,7 +697,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 a4517c2526..ace095ca1a 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 @@ -51,8 +51,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 @@ -353,7 +353,7 @@ case class SpliceLocked(channelId: ByteVector32, 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/main/scala/fr/acinq/eclair/wire/protocol/LiquidityAds.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LiquidityAds.scala index 6543f3d383..cab42821fa 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LiquidityAds.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LiquidityAds.scala @@ -242,7 +242,7 @@ object LiquidityAds { case class WillFundPurchase(willFund: WillFund, purchase: Purchase) - /** Minimal information about a liquidity purchase. */ + /** Minimal information about a liquidity purchase, useful for example when RBF-ing transactions. */ case class PurchaseBasicInfo(isBuyer: Boolean, amount: Satoshi, fees: Fees) object Codecs { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala index c9c2418c6f..e32afd303d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala @@ -319,7 +319,7 @@ object TestConstants { channelOpenerWhitelist = Set.empty, maxPendingChannelsPerPeer = 3, maxTotalPendingChannelsPrivateNodes = 99, - remoteRbfLimits = RemoteRbfLimits(5, 0), + remoteRbfLimits = RemoteRbfLimits(10, 0), quiescenceTimeout = 2 minutes, balanceThresholds = Nil, minTimeBetweenUpdates = 0 hour, 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 341584d6d1..db9ce3817b 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 @@ -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, None, wallet)) @@ -143,10 +143,10 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit 0 msat, 0 msat, liquidityPurchase_opt, 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, CommitmentChanges.init(), latestFundingTx, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, None, 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, None, wallet)) @@ -171,10 +171,10 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit 0 msat, 0 msat, liquidityPurchase_opt, 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, CommitmentChanges.init(), latestFundingTx, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, None, wallet)) @@ -1574,8 +1574,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) @@ -1703,8 +1703,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) @@ -1948,8 +1948,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 0ff6c3beda..101e2de4a0 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 @@ -19,10 +19,11 @@ package fr.acinq.eclair.channel.states.c import akka.actor.typed.scaladsl.adapter.{ClassicActorRefOps, actorRefAdapter} import akka.testkit.{TestFSMRef, TestProbe} import com.softwaremill.quicklens.{ModifyPimp, QuicklensAt} -import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong, Transaction} +import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong, Transaction, TxIn} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.blockchain.{CurrentBlockHeight, SingleKeyOnChainWallet} +import fr.acinq.eclair.channel.LocalFundingStatus.DualFundedUnconfirmedFundingTx import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel import fr.acinq.eclair.channel.fsm.Channel.ProcessCurrentBlockHeight @@ -43,6 +44,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture val bothPushAmount = "both_push_amount" val noFundingContribution = "no_funding_contribution" + val liquidityPurchase = "liquidity_purchase" case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, bob2blockchain: TestProbe, aliceListener: TestProbe, bobListener: TestProbe, wallet: SingleKeyOnChainWallet) @@ -73,16 +75,19 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture } val aliceInit = Init(aliceParams.initFeatures) val bobInit = Init(bobParams.initFeatures) - val (requestFunds_opt, bobContribution) = if (test.tags.contains(noFundingContribution)) { + val (requestFunding_opt, bobContribution) = if (test.tags.contains(noFundingContribution)) { (None, None) - } else { - val requestFunds = LiquidityAds.RequestFunding(TestConstants.nonInitiatorFundingSatoshis, TestConstants.defaultLiquidityRates.fundingRates.head, LiquidityAds.PaymentDetails.FromChannelBalance) + } else if (test.tags.contains(liquidityPurchase)) { + val requestFunding = LiquidityAds.RequestFunding(TestConstants.nonInitiatorFundingSatoshis, TestConstants.defaultLiquidityRates.fundingRates.head, LiquidityAds.PaymentDetails.FromChannelBalance) val addFunding = LiquidityAds.AddFunding(TestConstants.nonInitiatorFundingSatoshis, Some(TestConstants.defaultLiquidityRates)) - (Some(requestFunds), Some(addFunding)) + (Some(requestFunding), Some(addFunding)) + } else { + val addFunding = LiquidityAds.AddFunding(TestConstants.nonInitiatorFundingSatoshis, None) + (None, Some(addFunding)) } val (initiatorPushAmount, nonInitiatorPushAmount) = if (test.tags.contains(bothPushAmount)) (Some(TestConstants.initiatorPushAmount), Some(TestConstants.nonInitiatorPushAmount)) else (None, None) within(30 seconds) { - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, initiatorPushAmount, requireConfirmedInputs = false, requestFunds_opt, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, initiatorPushAmount, requireConfirmedInputs = false, requestFunding_opt, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, bobContribution, dualFunded = true, nonInitiatorPushAmount, requireConfirmedInputs = false, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) alice2blockchain.expectMsgType[SetChannelId] // temporary channel id bob2blockchain.expectMsgType[SetChannelId] // temporary channel id @@ -135,7 +140,11 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture } if (!test.tags.contains(noFundingContribution)) { // Alice pays fees for the liquidity she bought, and push amounts are correctly transferred. - val liquidityFees = TestConstants.defaultLiquidityRates.fundingRates.head.fees(TestConstants.feeratePerKw, TestConstants.nonInitiatorFundingSatoshis, TestConstants.nonInitiatorFundingSatoshis, isChannelCreation = true) + val liquidityFees = if (test.tags.contains(liquidityPurchase)) { + TestConstants.defaultLiquidityRates.fundingRates.head.fees(TestConstants.feeratePerKw, TestConstants.nonInitiatorFundingSatoshis, TestConstants.nonInitiatorFundingSatoshis, isChannelCreation = true) + } else { + LiquidityAds.Fees(0 sat, 0 sat) + } val bobReserve = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.remoteChannelReserve val expectedBalanceBob = bobContribution.map(_.fundingAmount).getOrElse(0 sat) + liquidityFees.total + initiatorPushAmount.getOrElse(0 msat) - nonInitiatorPushAmount.getOrElse(0 msat) - bobReserve assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.availableBalanceForSend == expectedBalanceBob) @@ -144,7 +153,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture } } - test("recv TxSignatures (duplicate)", Tag(ChannelStateTestsTags.DualFunding)) { f => + test("recv TxSignatures (duplicate)", Tag(ChannelStateTestsTags.DualFunding), Tag(liquidityPurchase)) { f => import f._ val aliceSigs = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.localSigs @@ -226,7 +235,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY) } - test("recv WatchFundingConfirmedTriggered (non-initiator)", Tag(ChannelStateTestsTags.DualFunding)) { f => + test("recv WatchFundingConfirmedTriggered (non-initiator)", Tag(ChannelStateTestsTags.DualFunding), Tag(liquidityPurchase)) { f => import f._ val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction].signedTx bob ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx) @@ -294,35 +303,44 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture awaitCond(bob2.stateName == WAIT_FOR_DUAL_FUNDING_READY) } + def testUnusedInputsUnlocked(wallet: SingleKeyOnChainWallet, unusedTxs: Seq[FullySignedSharedTransaction]): Unit = { + val inputs = unusedTxs.flatMap(sharedTx => sharedTx.tx.localInputs ++ sharedTx.tx.sharedInput_opt.toSeq).distinctBy(_.outPoint.txid).map(i => i.outPoint) + awaitCond { + val rollback = wallet.rolledback.flatMap(_.txIn.map(_.outPoint)) + inputs.toSet == rollback.toSet + } + } + test("recv WatchFundingConfirmedTriggered after restart (previous tx)", Tag(ChannelStateTestsTags.DualFunding)) { f => import f._ - val fundingTx1 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction].signedTx - val fundingTx2 = testBumpFundingFees(f).signedTx - assert(fundingTx1.txid != fundingTx2.txid) + val fundingTx1 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction] + val fundingTx2 = testBumpFundingFees(f) + assert(fundingTx1.signedTx.txid != fundingTx2.signedTx.txid) val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] val (alice2, bob2) = restartNodes(f, aliceData, bobData) reconnectNodes(f, alice2, bob2) - alice2 ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx1) - assert(aliceListener.expectMsgType[TransactionConfirmed].tx == fundingTx1) - assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx1.txid) + alice2 ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx1.signedTx) + assert(aliceListener.expectMsgType[TransactionConfirmed].tx == fundingTx1.signedTx) + assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx1.signedTx.txid) alice2bob.expectMsgType[ChannelReady] awaitCond(alice2.stateName == WAIT_FOR_DUAL_FUNDING_READY) assert(alice2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.active.size == 1) assert(alice2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.inactive.isEmpty) - assert(alice2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.latest.fundingTxId == fundingTx1.txid) + assert(alice2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.latest.fundingTxId == fundingTx1.signedTx.txid) + testUnusedInputsUnlocked(wallet, Seq(fundingTx2)) - bob2 ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx1) - assert(bobListener.expectMsgType[TransactionConfirmed].tx == fundingTx1) - assert(bob2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx1.txid) + bob2 ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx1.signedTx) + assert(bobListener.expectMsgType[TransactionConfirmed].tx == fundingTx1.signedTx) + assert(bob2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx1.signedTx.txid) bob2alice.expectMsgType[ChannelReady] awaitCond(bob2.stateName == WAIT_FOR_DUAL_FUNDING_READY) assert(bob2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.active.size == 1) assert(bob2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.inactive.isEmpty) - assert(bob2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.latest.fundingTxId == fundingTx1.txid) + assert(bob2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.latest.fundingTxId == fundingTx1.signedTx.txid) } def testBumpFundingFees(f: FixtureParam, feerate_opt: Option[FeeratePerKw] = None, requestFunding_opt: Option[LiquidityAds.RequestFunding] = None): FullySignedSharedTransaction = { @@ -335,7 +353,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture assert(alice2bob.expectMsgType[TxInitRbf].fundingContribution == TestConstants.fundingSatoshis) alice2bob.forward(bob) val txAckRbf = bob2alice.expectMsgType[TxAckRbf] - assert(txAckRbf.fundingContribution == TestConstants.nonInitiatorFundingSatoshis) + assert(txAckRbf.fundingContribution == requestFunding_opt.map(_.requestedAmount).getOrElse(TestConstants.nonInitiatorFundingSatoshis)) requestFunding_opt.foreach(_ => assert(txAckRbf.willFund_opt.nonEmpty)) bob2alice.forward(alice) @@ -385,24 +403,87 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture test("recv CMD_BUMP_FUNDING_FEE", Tag(ChannelStateTestsTags.DualFunding)) { f => import f._ - val remoteFunding = TestConstants.nonInitiatorFundingSatoshis + // Bob contributed to the funding transaction. + val balanceBob1 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.spec.toLocal + assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.isEmpty) + assert(balanceBob1 == TestConstants.nonInitiatorFundingSatoshis.toMilliSatoshi) + + // Alice RBFs the funding transaction: Bob keeps contributing the same amount. + val fundingTx1 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction] + val feerate2 = FeeratePerKw(12_500 sat) + testBumpFundingFees(f, Some(feerate2)) + val balanceBob2 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.spec.toLocal + assert(balanceBob2 == TestConstants.nonInitiatorFundingSatoshis.toMilliSatoshi) + val fundingTx2 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localFundingStatus.asInstanceOf[DualFundedUnconfirmedFundingTx].sharedTx.asInstanceOf[FullySignedSharedTransaction] + assert(FeeratePerKw(12_500 sat) <= fundingTx2.feerate && fundingTx2.feerate < FeeratePerKw(13_000 sat)) + + // Alice RBFs the funding transaction again: Bob keeps contributing the same amount. + val feerate3 = FeeratePerKw(15_000 sat) + testBumpFundingFees(f, Some(feerate3)) + val balanceBob3 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.spec.toLocal + assert(balanceBob3 == TestConstants.nonInitiatorFundingSatoshis.toMilliSatoshi) + val fundingTx3 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localFundingStatus.asInstanceOf[DualFundedUnconfirmedFundingTx].sharedTx.asInstanceOf[FullySignedSharedTransaction] + assert(FeeratePerKw(15_000 sat) <= fundingTx3.feerate && fundingTx3.feerate < FeeratePerKw(15_500 sat)) + assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.length == 2) + + // The initial funding transaction confirms + alice ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx1.signedTx) + testUnusedInputsUnlocked(wallet, Seq(fundingTx2, fundingTx3)) + } + + test("recv CMD_BUMP_FUNDING_FEE (liquidity ads)", Tag(ChannelStateTestsTags.DualFunding), Tag(liquidityPurchase)) { f => + import f._ + + // Alice initially purchased some inbound liquidity. + val remoteFunding1 = TestConstants.nonInitiatorFundingSatoshis val feerate1 = TestConstants.feeratePerKw - val liquidityFee1 = TestConstants.defaultLiquidityRates.fundingRates.head.fees(feerate1, remoteFunding, remoteFunding, isChannelCreation = true) + val liquidityFee1 = TestConstants.defaultLiquidityRates.fundingRates.head.fees(feerate1, remoteFunding1, remoteFunding1, isChannelCreation = true) val balanceBob1 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.spec.toLocal + assert(balanceBob1 == (remoteFunding1 + liquidityFee1.total).toMilliSatoshi) assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.isEmpty) + // Alice RBFs the funding transaction and purchases the same amount of liquidity. val feerate2 = FeeratePerKw(12_500 sat) - val liquidityFee2 = TestConstants.defaultLiquidityRates.fundingRates.head.fees(feerate2, remoteFunding, remoteFunding, isChannelCreation = true) - testBumpFundingFees(f, Some(feerate2), Some(LiquidityAds.RequestFunding(remoteFunding, TestConstants.defaultLiquidityRates.fundingRates.head, LiquidityAds.PaymentDetails.FromChannelBalance))) + val requestFunding2 = LiquidityAds.RequestFunding(remoteFunding1, TestConstants.defaultLiquidityRates.fundingRates.head, LiquidityAds.PaymentDetails.FromChannelBalance) + val liquidityFee2 = TestConstants.defaultLiquidityRates.fundingRates.head.fees(feerate2, remoteFunding1, remoteFunding1, isChannelCreation = true) + testBumpFundingFees(f, Some(feerate2), Some(requestFunding2)) val balanceBob2 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.spec.toLocal assert(liquidityFee1.total < liquidityFee2.total) - assert(balanceBob1 + liquidityFee2.total - liquidityFee1.total == balanceBob2) + assert(balanceBob2 == (remoteFunding1 + liquidityFee2.total).toMilliSatoshi) + val fundingTx2 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localFundingStatus.asInstanceOf[DualFundedUnconfirmedFundingTx].sharedTx.asInstanceOf[FullySignedSharedTransaction] + assert(FeeratePerKw(12_500 sat) <= fundingTx2.feerate && fundingTx2.feerate < FeeratePerKw(13_000 sat)) - // The second RBF attempt removes the liquidity request. + // Alice RBFs again and purchases more inbound liquidity. + val remoteFunding3 = 750_000.sat val feerate3 = FeeratePerKw(15_000 sat) - testBumpFundingFees(f, Some(feerate3), requestFunding_opt = None) - assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.spec.toLocal.truncateToSatoshi == remoteFunding) - assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.length == 2) + val requestFunding3 = LiquidityAds.RequestFunding(remoteFunding3, TestConstants.defaultLiquidityRates.fundingRates.head, LiquidityAds.PaymentDetails.FromChannelBalance) + val liquidityFee3 = TestConstants.defaultLiquidityRates.fundingRates.head.fees(feerate3, remoteFunding3, remoteFunding3, isChannelCreation = true) + testBumpFundingFees(f, Some(feerate3), Some(requestFunding3)) + val balanceBob3 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.spec.toLocal + assert(balanceBob3 == (remoteFunding3 + liquidityFee3.total).toMilliSatoshi) + val fundingTx3 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localFundingStatus.asInstanceOf[DualFundedUnconfirmedFundingTx].sharedTx.asInstanceOf[FullySignedSharedTransaction] + assert(FeeratePerKw(15_000 sat) <= fundingTx3.feerate && fundingTx3.feerate < FeeratePerKw(15_500 sat)) + + // Alice RBFs again and purchases less inbound liquidity. + val remoteFunding4 = 250_000.sat + val feerate4 = FeeratePerKw(17_500 sat) + val requestFunding4 = LiquidityAds.RequestFunding(remoteFunding4, TestConstants.defaultLiquidityRates.fundingRates.head, LiquidityAds.PaymentDetails.FromChannelBalance) + val liquidityFee4 = TestConstants.defaultLiquidityRates.fundingRates.head.fees(feerate4, remoteFunding4, remoteFunding4, isChannelCreation = true) + testBumpFundingFees(f, Some(feerate4), Some(requestFunding4)) + val balanceBob4 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.spec.toLocal + assert(balanceBob4 == (remoteFunding4 + liquidityFee4.total).toMilliSatoshi) + val fundingTx4 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localFundingStatus.asInstanceOf[DualFundedUnconfirmedFundingTx].sharedTx.asInstanceOf[FullySignedSharedTransaction] + assert(FeeratePerKw(17_500 sat) <= fundingTx4.feerate && fundingTx4.feerate < FeeratePerKw(18_000 sat)) + + // Alice tries to cancel the liquidity purchase. + val sender = TestProbe() + alice ! CMD_BUMP_FUNDING_FEE(sender.ref, FeeratePerKw(20_000 sat), 100_000 sat, 0, requestFunding_opt = None) + assert(sender.expectMsgType[RES_FAILURE[_, ChannelException]].t.isInstanceOf[InvalidRbfMissingLiquidityPurchase]) + alice2bob.forward(bob, TxInitRbf(alice.stateData.channelId, 0, FeeratePerKw(20_000 sat), TestConstants.fundingSatoshis, requireConfirmedInputs = false, requestFunding_opt = None)) + assert(bob2alice.expectMsgType[TxAbort].toAscii.contains("the previous attempt contained a liquidity purchase")) + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAbort] + alice2bob.forward(bob) } test("recv CMD_BUMP_FUNDING_FEE (aborted)", Tag(ChannelStateTestsTags.DualFunding)) { f => @@ -424,16 +505,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) } @@ -450,19 +531,19 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture assert(bob2alice.expectMsgType[TxAckRbf].fundingContribution == TestConstants.nonInitiatorFundingSatoshis) bob2alice.forward(alice) - // Alice and Bob build a new version of the funding transaction. - alice2bob.expectMsgType[TxAbort] + // Alice aborts the funding transaction, because it exceeds its fee budget. + assert(alice2bob.expectMsgType[TxAbort].toAscii == ChannelFundingError(channelId(alice)).getMessage) 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) } @@ -516,7 +597,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) } - test("recv CurrentBlockCount (funding in progress while offline)", Tag(ChannelStateTestsTags.DualFunding)) { f => + test("recv CurrentBlockCount (funding in progress while offline)", Tag(ChannelStateTestsTags.DualFunding), Tag(liquidityPurchase)) { f => import f._ val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction].signedTx val currentBlock = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].waitingSince + 10 @@ -638,7 +719,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) } - test("recv ChannelReady (non-initiator)", Tag(ChannelStateTestsTags.DualFunding)) { f => + test("recv ChannelReady (non-initiator)", Tag(ChannelStateTestsTags.DualFunding), Tag(liquidityPurchase)) { f => import f._ val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction].signedTx alice ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx) @@ -684,8 +765,8 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture val fundingTx1 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction].signedTx val bobCommitTx1 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx - val fundingTx2 = testBumpFundingFees(f).signedTx - assert(fundingTx1.txid != fundingTx2.txid) + val fundingTx2 = testBumpFundingFees(f) + assert(fundingTx1.txid != fundingTx2.signedTx.txid) val bobCommitTx2 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx assert(bobCommitTx1.txid != bobCommitTx2.txid) @@ -700,6 +781,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture alice2bob.expectNoMessage(100 millis) awaitCond(alice.stateData.isInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY]) assert(alice.stateName == OFFLINE) + testUnusedInputsUnlocked(wallet, Seq(fundingTx2)) // Bob broadcasts his commit tx. alice ! WatchFundingSpentTriggered(bobCommitTx1) @@ -750,8 +832,8 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture val fundingTx1 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction].signedTx val aliceCommitTx1 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx val bobCommitTx1 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx - val fundingTx2 = testBumpFundingFees(f).signedTx - assert(fundingTx1.txid != fundingTx2.txid) + val fundingTx2 = testBumpFundingFees(f) + assert(fundingTx1.txid != fundingTx2.signedTx.txid) val bobCommitTx2 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx assert(bobCommitTx1.txid != bobCommitTx2.txid) @@ -769,6 +851,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == bobCommitTx1.txid) assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMainAlice.tx.txid) awaitCond(alice2.stateName == CLOSING) + testUnusedInputsUnlocked(wallet, Seq(fundingTx2)) bob2 ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx1) assert(bobListener.expectMsgType[TransactionConfirmed].tx == fundingTx1) @@ -816,16 +899,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()) @@ -841,8 +924,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) } @@ -877,16 +960,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()) @@ -912,8 +995,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 => @@ -957,10 +1040,10 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture val bobCommitTx1 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx assert(aliceCommitTx1.input.outPoint.txid == fundingTx1.txid) assert(bobCommitTx1.input.outPoint.txid == fundingTx1.txid) - val fundingTx2 = testBumpFundingFees(f).signedTx - assert(fundingTx1.txid != fundingTx2.txid) + val fundingTx2 = testBumpFundingFees(f) + assert(fundingTx1.txid != fundingTx2.signedTx.txid) val aliceCommitTx2 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx - assert(aliceCommitTx2.input.outPoint.txid == fundingTx2.txid) + assert(aliceCommitTx2.input.outPoint.txid == fundingTx2.signedTx.txid) // Alice receives an error and force-closes using the latest funding transaction. alice ! Error(ByteVector32.Zeroes, "dual funding d34d") @@ -984,6 +1067,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture assert(claimMain1.input.txid == aliceCommitTx1.tx.txid) assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == aliceCommitTx1.tx.txid) assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMain1.tx.txid) + testUnusedInputsUnlocked(wallet, Seq(fundingTx2)) // Bob publishes his commit tx, Alice reacts by spending her remote main output. alice ! WatchFundingSpentTriggered(bobCommitTx1.tx) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala index b8148e633d..3fb6adcc4f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala @@ -43,6 +43,7 @@ import fr.acinq.eclair.transactions.DirectedHtlc.{incoming, outgoing} import fr.acinq.eclair.transactions.Transactions import fr.acinq.eclair.transactions.Transactions.ClaimLocalAnchorOutputTx import fr.acinq.eclair.wire.protocol._ +import org.scalatest.Inside.inside import org.scalatest.funsuite.FixtureAnyFunSuiteLike import org.scalatest.time.SpanSugar.convertIntToGrainOfTime import org.scalatest.{Outcome, Tag} @@ -113,6 +114,70 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik private def initiateSpliceWithoutSigs(f: FixtureParam, spliceIn_opt: Option[SpliceIn] = None, spliceOut_opt: Option[SpliceOut] = None): TestProbe = initiateSpliceWithoutSigs(f.alice, f.bob, f.alice2bob, f.bob2alice, spliceIn_opt, spliceOut_opt) + private def initiateRbfWithoutSigs(s: TestFSMRef[ChannelState, ChannelData, Channel], r: TestFSMRef[ChannelState, ChannelData, Channel], s2r: TestProbe, r2s: TestProbe, feerate: FeeratePerKw, sInputsCount: Int, sOutputsCount: Int, rInputsCount: Int, rOutputsCount: Int): TestProbe = { + val sender = TestProbe() + val cmd = CMD_BUMP_FUNDING_FEE(sender.ref, feerate, 100_000 sat, 0, None) + s ! cmd + exchangeStfu(s, r, s2r, r2s) + s2r.expectMsgType[TxInitRbf] + s2r.forward(r) + r2s.expectMsgType[TxAckRbf] + r2s.forward(s) + + // The initiator also adds the shared input and shared output. + var sRemainingInputs = sInputsCount + 1 + var sRemainingOutputs = sOutputsCount + 1 + var rRemainingInputs = rInputsCount + var rRemainingOutputs = rOutputsCount + var sComplete = false + var rComplete = false + + while (sRemainingInputs > 0 || sRemainingOutputs > 0 || rRemainingInputs > 0 || rRemainingOutputs > 0) { + if (sRemainingInputs > 0) { + s2r.expectMsgType[TxAddInput] + s2r.forward(r) + sRemainingInputs -= 1 + } else if (sRemainingOutputs > 0) { + s2r.expectMsgType[TxAddOutput] + s2r.forward(r) + sRemainingOutputs -= 1 + } else { + s2r.expectMsgType[TxComplete] + s2r.forward(r) + sComplete = true + } + + if (rRemainingInputs > 0) { + r2s.expectMsgType[TxAddInput] + r2s.forward(s) + rRemainingInputs -= 1 + } else if (rRemainingOutputs > 0) { + r2s.expectMsgType[TxAddOutput] + r2s.forward(s) + rRemainingOutputs -= 1 + } else { + r2s.expectMsgType[TxComplete] + r2s.forward(s) + rComplete = true + } + } + + if (!sComplete || !rComplete) { + s2r.expectMsgType[TxComplete] + s2r.forward(r) + if (!rComplete) { + r2s.expectMsgType[TxComplete] + r2s.forward(s) + } + } + + sender + } + + private def initiateRbfWithoutSigs(f: FixtureParam, feerate: FeeratePerKw, sInputsCount: Int, sOutputsCount: Int): TestProbe = { + initiateRbfWithoutSigs(f.alice, f.bob, f.alice2bob, f.bob2alice, feerate, sInputsCount, sOutputsCount, rInputsCount = 0, rOutputsCount = 0) + } + private def exchangeSpliceSigs(s: TestFSMRef[ChannelState, ChannelData, Channel], r: TestFSMRef[ChannelState, ChannelData, Channel], s2r: TestProbe, r2s: TestProbe, sender: TestProbe): Transaction = { val commitSigR = r2s.fishForMessage() { case _: CommitSig => true @@ -154,6 +219,11 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik private def initiateSplice(f: FixtureParam, spliceIn_opt: Option[SpliceIn] = None, spliceOut_opt: Option[SpliceOut] = None): Transaction = initiateSplice(f.alice, f.bob, f.alice2bob, f.bob2alice, spliceIn_opt, spliceOut_opt) + private def initiateRbf(f: FixtureParam, feerate: FeeratePerKw, sInputsCount: Int, sOutputsCount: Int): Transaction = { + val sender = initiateRbfWithoutSigs(f, feerate, sInputsCount, sOutputsCount) + exchangeSpliceSigs(f, sender) + } + private def exchangeStfu(s: TestFSMRef[ChannelState, ChannelData, Channel], r: TestFSMRef[ChannelState, ChannelData, Channel], s2r: TestProbe, r2s: TestProbe): Unit = { s2r.expectMsgType[Stfu] s2r.forward(r) @@ -163,6 +233,33 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik private def exchangeStfu(f: FixtureParam): Unit = exchangeStfu(f.alice, f.bob, f.alice2bob, f.bob2alice) + private def checkWatchConfirmed(f: FixtureParam, spliceTx: Transaction): Unit = { + import f._ + + alice2blockchain.expectWatchFundingConfirmed(spliceTx.txid) + alice2blockchain.expectNoMessage(100 millis) + bob2blockchain.expectWatchFundingConfirmed(spliceTx.txid) + bob2blockchain.expectNoMessage(100 millis) + } + + private def confirmSpliceTx(f: FixtureParam, spliceTx: Transaction): Unit = { + import f._ + + val fundingTxIndex = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.find(_.fundingTxId == spliceTx.txid).get.fundingTxIndex + + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, spliceTx) + alice2bob.expectMsgType[SpliceLocked] + alice2bob.forward(bob) + + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, spliceTx) + bob2alice.expectMsgType[SpliceLocked] + bob2alice.forward(alice) + + // Previous commitments have been cleaned up. + awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.forall(c => c.fundingTxIndex > fundingTxIndex || c.fundingTxId == spliceTx.txid), interval = 100 millis) + awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.forall(c => c.fundingTxIndex > fundingTxIndex || c.fundingTxId == spliceTx.txid), interval = 100 millis) + } + case class TestHtlcs(aliceToBob: Seq[(ByteVector32, UpdateAddHtlc)], bobToAlice: Seq[(ByteVector32, UpdateAddHtlc)]) private def setupHtlcs(f: FixtureParam): TestHtlcs = { @@ -488,6 +585,12 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik setupHtlcs(f) + val commitment = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest + assert(commitment.localCommit.spec.toLocal == 770_000_000.msat) + assert(commitment.localChannelReserve == 15_000.sat) + val commitFees = Transactions.commitTxTotalCost(commitment.remoteParams.dustLimit, commitment.remoteCommit.spec, commitment.params.commitmentFormat) + assert(commitFees < 15_000.sat) + val sender = TestProbe() val cmd = CMD_SPLICE(sender.ref, spliceIn_opt = None, Some(SpliceOut(760_000 sat, defaultSpliceOutScriptPubKey)), requestFunding_opt = None) alice ! cmd @@ -495,6 +598,26 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik sender.expectMsgType[RES_FAILURE[_, _]] } + test("recv CMD_SPLICE (splice-out, cannot pay commit fees)", Tag(ChannelStateTestsTags.NoMaxHtlcValueInFlight)) { f => + import f._ + + // We add enough HTLCs to make sure that the commit fees are higher than the reserve. + (0 until 10).foreach(_ => addHtlc(15_000_000 msat, alice, bob, alice2bob, bob2alice)) + crossSign(alice, bob, alice2bob, bob2alice) + + val commitment = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest + assert(commitment.localCommit.spec.toLocal == 650_000_000.msat) + assert(commitment.localChannelReserve == 15_000.sat) + val commitFees = Transactions.commitTxTotalCost(commitment.remoteParams.dustLimit, commitment.remoteCommit.spec, commitment.params.commitmentFormat) + assert(commitFees > 20_000.sat) + + val sender = TestProbe() + val cmd = CMD_SPLICE(sender.ref, spliceIn_opt = None, Some(SpliceOut(630_000 sat, defaultSpliceOutScriptPubKey)), requestFunding_opt = None) + alice ! cmd + exchangeStfu(f) + sender.expectMsgType[RES_FAILURE[_, _]] + } + test("recv CMD_SPLICE (splice-in, feerate too low)") { f => import f._ @@ -566,12 +689,283 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik crossSign(alice, bob, alice2bob, bob2alice) } + test("recv CMD_SPLICE (pending RBF attempts)") { f => + import f._ + + initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + initiateRbf(f, FeeratePerKw(15_000 sat), sInputsCount = 2, sOutputsCount = 1) + + val probe = TestProbe() + alice ! CMD_SPLICE(probe.ref, Some(SpliceIn(250_000 sat)), None, None) + assert(probe.expectMsgType[RES_FAILURE[_, ChannelException]].t.isInstanceOf[InvalidSpliceWithUnconfirmedTx]) + + bob2alice.forward(alice, Stfu(alice.stateData.channelId, initiator = true)) + alice2bob.expectMsgType[Stfu] + bob2alice.forward(alice, SpliceInit(alice.stateData.channelId, 100_000 sat, FeeratePerKw(5000 sat), 0, randomKey().publicKey)) + assert(alice2bob.expectMsgType[TxAbort].toAscii.contains("the current funding transaction is still unconfirmed")) + } + + test("recv CMD_SPLICE (unconfirmed previous tx)") { f => + import f._ + + // We create a first unconfirmed splice. + initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.isInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx]) + + // We allow initiating such splice... + val probe = TestProbe() + alice ! CMD_SPLICE(probe.ref, Some(SpliceIn(250_000 sat)), None, None) + alice2bob.expectMsgType[Stfu] + alice2bob.forward(bob) + bob2alice.expectMsgType[Stfu] + bob2alice.forward(alice) + val spliceInit = alice2bob.expectMsgType[SpliceInit] + + // But we don't allow receiving splice_init if the previous splice is unconfirmed and we're not using 0-conf. + alice2bob.forward(bob, spliceInit) + assert(bob2alice.expectMsgType[TxAbort].toAscii.contains("the current funding transaction is still unconfirmed")) + } + test("recv CMD_SPLICE (splice-in + splice-out)") { f => val htlcs = setupHtlcs(f) initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) resolveHtlcs(f, htlcs, spliceOutFee = 0.sat) } + test("recv CMD_BUMP_FUNDING_FEE (splice-in + splice-out)") { f => + import f._ + + val spliceTx = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(300_000 sat, defaultSpliceOutScriptPubKey))) + val spliceCommitment = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.find(_.fundingTxId == spliceTx.txid).get + + // Alice RBFs the splice transaction. + // Our dummy bitcoin wallet adds an additional input at every funding attempt. + val rbfTx1 = initiateRbf(f, FeeratePerKw(15_000 sat), sInputsCount = 2, sOutputsCount = 2) + assert(rbfTx1.txIn.size == spliceTx.txIn.size + 1) + spliceTx.txIn.foreach(txIn => assert(rbfTx1.txIn.map(_.outPoint).contains(txIn.outPoint))) + assert(rbfTx1.txOut.size == spliceTx.txOut.size) + + // Bob RBFs the splice transaction: he needs to add an input to pay the fees. + // Our dummy bitcoin wallet adds an additional input for Alice: a real bitcoin wallet would simply lower the previous change output. + val sender2 = initiateRbfWithoutSigs(bob, alice, bob2alice, alice2bob, FeeratePerKw(20_000 sat), sInputsCount = 1, sOutputsCount = 1, rInputsCount = 3, rOutputsCount = 2) + val rbfTx2 = exchangeSpliceSigs(alice, bob, alice2bob, bob2alice, sender2) + assert(rbfTx2.txIn.size > rbfTx1.txIn.size) + rbfTx1.txIn.foreach(txIn => assert(rbfTx2.txIn.map(_.outPoint).contains(txIn.outPoint))) + assert(rbfTx2.txOut.size == rbfTx1.txOut.size + 1) + + // There are three pending splice transactions that double-spend each other. + inside(alice.stateData.asInstanceOf[DATA_NORMAL]) { data => + val commitments = data.commitments.active.filter(_.fundingTxIndex == spliceCommitment.fundingTxIndex) + assert(commitments.size == 3) + assert(commitments.map(_.fundingTxId) == Seq(rbfTx2, rbfTx1, spliceTx).map(_.txid)) + // The contributions are the same across RBF attempts. + commitments.foreach(c => assert(c.localCommit.spec.toLocal == spliceCommitment.localCommit.spec.toLocal)) + commitments.foreach(c => assert(c.localCommit.spec.toRemote == spliceCommitment.localCommit.spec.toRemote)) + } + + // The last RBF attempt confirms. + confirmSpliceTx(f, rbfTx2) + inside(alice.stateData.asInstanceOf[DATA_NORMAL]) { data => + assert(data.commitments.active.map(_.fundingTxId) == Seq(rbfTx2.txid)) + data.commitments.active.foreach(c => assert(c.localCommit.spec.toLocal == spliceCommitment.localCommit.spec.toLocal)) + data.commitments.active.foreach(c => assert(c.localCommit.spec.toRemote == spliceCommitment.localCommit.spec.toRemote)) + } + + // We can keep doing more splice transactions now that one of the previous transactions confirmed. + initiateSplice(bob, alice, bob2alice, alice2bob, Some(SpliceIn(100_000 sat)), None) + } + + test("recv CMD_BUMP_FUNDING_FEE (splice-in + splice-out from non-initiator)") { f => + import f._ + + // Alice initiates a first splice. + val spliceTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(2_500_000 sat))) + confirmSpliceTx(f, spliceTx1) + + // Bob initiates a second splice that spends the first splice. + val spliceTx2 = initiateSplice(bob, alice, bob2alice, alice2bob, spliceIn_opt = Some(SpliceIn(50_000 sat)), spliceOut_opt = Some(SpliceOut(25_000 sat, defaultSpliceOutScriptPubKey))) + assert(spliceTx2.txIn.exists(_.outPoint.txid == spliceTx1.txid)) + + // Alice cannot RBF her first splice, so she RBFs Bob's splice instead. + val sender = initiateRbfWithoutSigs(alice, bob, alice2bob, bob2alice, FeeratePerKw(15_000 sat), sInputsCount = 1, sOutputsCount = 1, rInputsCount = 2, rOutputsCount = 2) + val rbfTx = exchangeSpliceSigs(bob, alice, bob2alice, alice2bob, sender) + assert(rbfTx.txIn.size > spliceTx2.txIn.size) + spliceTx2.txIn.foreach(txIn => assert(rbfTx.txIn.map(_.outPoint).contains(txIn.outPoint))) + } + + test("recv CMD_BUMP_FUNDING_FEE (liquidity ads)") { f => + import f._ + + // Alice initiates a splice-in with a liquidity purchase. + val sender = TestProbe() + val fundingRequest = LiquidityAds.RequestFunding(400_000 sat, TestConstants.defaultLiquidityRates.fundingRates.head, LiquidityAds.PaymentDetails.FromChannelBalance) + alice ! CMD_SPLICE(sender.ref, Some(SpliceIn(500_000 sat)), None, Some(fundingRequest)) + exchangeStfu(alice, bob, alice2bob, bob2alice) + inside(alice2bob.expectMsgType[SpliceInit]) { msg => + assert(msg.fundingContribution == 500_000.sat) + assert(msg.requestFunding_opt.nonEmpty) + } + alice2bob.forward(bob) + inside(bob2alice.expectMsgType[SpliceAck]) { msg => + assert(msg.fundingContribution == 400_000.sat) + assert(msg.willFund_opt.nonEmpty) + } + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddInput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxAddInput] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddInput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxAddOutput] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddOutput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxComplete] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddOutput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxComplete] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxComplete] + alice2bob.forward(bob) + exchangeSpliceSigs(alice, bob, alice2bob, bob2alice, sender) + val spliceTx1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.asInstanceOf[DualFundedUnconfirmedFundingTx].sharedTx.asInstanceOf[FullySignedSharedTransaction] + assert(FeeratePerKw(10_000 sat) <= spliceTx1.feerate && spliceTx1.feerate < FeeratePerKw(10_500 sat)) + + // Alice RBFs the previous transaction and purchases less liquidity from Bob. + // Our dummy bitcoin wallet adds an additional input at every funding attempt. + alice ! CMD_BUMP_FUNDING_FEE(sender.ref, FeeratePerKw(12_500 sat), 50_000 sat, 0, Some(fundingRequest.copy(requestedAmount = 300_000 sat))) + exchangeStfu(alice, bob, alice2bob, bob2alice) + inside(alice2bob.expectMsgType[TxInitRbf]) { msg => + assert(msg.fundingContribution == 500_000.sat) + assert(msg.requestFunding_opt.nonEmpty) + } + alice2bob.forward(bob) + inside(bob2alice.expectMsgType[TxAckRbf]) { msg => + assert(msg.fundingContribution == 300_000.sat) + assert(msg.willFund_opt.nonEmpty) + } + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddInput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxAddInput] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddInput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxAddInput] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddInput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxAddOutput] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddOutput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxComplete] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddOutput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxComplete] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxComplete] + alice2bob.forward(bob) + exchangeSpliceSigs(alice, bob, alice2bob, bob2alice, sender) + val spliceTx2 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.asInstanceOf[DualFundedUnconfirmedFundingTx].sharedTx.asInstanceOf[FullySignedSharedTransaction] + spliceTx1.signedTx.txIn.map(_.outPoint).foreach(txIn => assert(spliceTx2.signedTx.txIn.map(_.outPoint).contains(txIn))) + assert(FeeratePerKw(12_500 sat) <= spliceTx2.feerate && spliceTx2.feerate < FeeratePerKw(13_000 sat)) + + // Alice RBFs the previous transaction and purchases more liquidity from Bob. + // Our dummy bitcoin wallet adds an additional input at every funding attempt. + alice ! CMD_BUMP_FUNDING_FEE(sender.ref, FeeratePerKw(15_000 sat), 50_000 sat, 0, Some(fundingRequest.copy(requestedAmount = 500_000 sat))) + exchangeStfu(alice, bob, alice2bob, bob2alice) + inside(alice2bob.expectMsgType[TxInitRbf]) { msg => + assert(msg.fundingContribution == 500_000.sat) + assert(msg.requestFunding_opt.nonEmpty) + } + alice2bob.forward(bob) + inside(bob2alice.expectMsgType[TxAckRbf]) { msg => + assert(msg.fundingContribution == 500_000.sat) + assert(msg.willFund_opt.nonEmpty) + } + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddInput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxAddInput] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddInput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxAddInput] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddInput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxAddInput] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddInput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxAddOutput] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddOutput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxComplete] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddOutput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxComplete] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxComplete] + alice2bob.forward(bob) + exchangeSpliceSigs(alice, bob, alice2bob, bob2alice, sender) + val spliceTx3 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.asInstanceOf[DualFundedUnconfirmedFundingTx].sharedTx.asInstanceOf[FullySignedSharedTransaction] + spliceTx2.signedTx.txIn.map(_.outPoint).foreach(txIn => assert(spliceTx3.signedTx.txIn.map(_.outPoint).contains(txIn))) + assert(FeeratePerKw(15_000 sat) <= spliceTx3.feerate && spliceTx3.feerate < FeeratePerKw(15_500 sat)) + + // Alice RBFs the previous transaction and tries to cancel the liquidity purchase. + alice ! CMD_BUMP_FUNDING_FEE(sender.ref, FeeratePerKw(17_500 sat), 50_000 sat, 0, requestFunding_opt = None) + assert(sender.expectMsgType[RES_FAILURE[_, ChannelException]].t.isInstanceOf[InvalidRbfMissingLiquidityPurchase]) + alice2bob.forward(bob, Stfu(alice.stateData.channelId, initiator = true)) + bob2alice.expectMsgType[Stfu] + alice2bob.forward(bob, TxInitRbf(alice.stateData.channelId, 0, FeeratePerKw(17_500 sat), 500_000 sat, requireConfirmedInputs = false, requestFunding_opt = None)) + inside(bob2alice.expectMsgType[TxAbort]) { msg => + assert(msg.toAscii.contains("the previous attempt contained a liquidity purchase")) + } + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAbort] + alice2bob.forward(bob) + } + + test("recv CMD_BUMP_FUNDING_FEE (transaction already confirmed)") { f => + import f._ + + val spliceTx = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + confirmSpliceTx(f, spliceTx) + + val probe = TestProbe() + alice ! CMD_BUMP_FUNDING_FEE(probe.ref, FeeratePerKw(15_000 sat), 100_000 sat, 0, None) + assert(probe.expectMsgType[RES_FAILURE[_, ChannelException]].t.isInstanceOf[InvalidRbfTxConfirmed]) + + bob2alice.forward(alice, Stfu(alice.stateData.channelId, initiator = true)) + alice2bob.expectMsgType[Stfu] + bob2alice.forward(alice, TxInitRbf(alice.stateData.channelId, 0, FeeratePerKw(15_000 sat), 250_000 sat, requireConfirmedInputs = false, None)) + assert(alice2bob.expectMsgType[TxAbort].toAscii.contains("transaction is already confirmed")) + } + + test("recv CMD_BUMP_FUNDING_FEE (transaction is using 0-conf)", Tag(ChannelStateTestsTags.ZeroConf), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => + import f._ + + val spliceTx = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + alice ! WatchPublishedTriggered(spliceTx) + alice2bob.expectMsgType[SpliceLocked] + + val probe = TestProbe() + alice ! CMD_BUMP_FUNDING_FEE(probe.ref, FeeratePerKw(15_000 sat), 100_000 sat, 0, None) + assert(probe.expectMsgType[RES_FAILURE[_, ChannelException]].t.isInstanceOf[InvalidRbfZeroConf]) + + bob2alice.forward(alice, Stfu(alice.stateData.channelId, initiator = true)) + alice2bob.expectMsgType[Stfu] + bob2alice.forward(alice, TxInitRbf(alice.stateData.channelId, 0, FeeratePerKw(15_000 sat), 250_000 sat, requireConfirmedInputs = false, None)) + assert(alice2bob.expectMsgType[TxAbort].toAscii.contains("we're using zero-conf")) + } + test("recv TxAbort (before TxComplete)") { f => import f._ @@ -726,51 +1120,38 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik bob2alice.forward(alice) } - private def setup2Splices(f: FixtureParam): (Transaction, Transaction) = { + test("splice local/remote locking", Tag(ChannelStateTestsTags.NoMaxHtlcValueInFlight)) { f => import f._ val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) - alice2blockchain.expectWatchFundingConfirmed(fundingTx1.txid) - alice2blockchain.expectNoMessage(100 millis) - bob2blockchain.expectWatchFundingConfirmed(fundingTx1.txid) - bob2blockchain.expectNoMessage(100 millis) + checkWatchConfirmed(f, fundingTx1) + val commitAlice1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.commitTxAndRemoteSig.commitTx.tx + val commitBob1 = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.commitTxAndRemoteSig.commitTx.tx - val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) - alice2blockchain.expectWatchFundingConfirmed(fundingTx2.txid) - alice2blockchain.expectNoMessage(100 millis) - bob2blockchain.expectWatchFundingConfirmed(fundingTx2.txid) - bob2blockchain.expectNoMessage(100 millis) - - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2, 1, 0)) - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2, 1, 0)) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) - - (fundingTx1, fundingTx2) - } - - test("splice local/remote locking", Tag(ChannelStateTestsTags.NoMaxHtlcValueInFlight)) { f => - import f._ + // Bob sees the first splice confirm, but Alice doesn't. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectWatchFundingSpent(fundingTx1.txid, Some(Set(commitAlice1.txid, commitBob1.txid))) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) - val (fundingTx1, fundingTx2) = setup2Splices(f) - val commitAlice1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active(1).localCommit.commitTxAndRemoteSig.commitTx.tx - val commitAlice2 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active(0).localCommit.commitTxAndRemoteSig.commitTx.tx - val commitBob1 = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active(1).localCommit.commitTxAndRemoteSig.commitTx.tx - val commitBob2 = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active(0).localCommit.commitTxAndRemoteSig.commitTx.tx + // Alice creates another splice spending the first splice. + val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + checkWatchConfirmed(f, fundingTx2) + val commitAlice2 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.commitTxAndRemoteSig.commitTx.tx + val commitBob2 = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.commitTxAndRemoteSig.commitTx.tx + assert(commitAlice1.txid != commitAlice2.txid) + assert(commitBob1.txid != commitBob2.txid) - // splice 1 confirms + // Alice sees the first splice confirm. alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) alice2bob.forward(bob) - bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) - bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) - bob2alice.forward(alice) + alice2blockchain.expectWatchFundingSpent(fundingTx1.txid, Some(Set(fundingTx2.txid, commitAlice1.txid, commitBob1.txid))) - bob2blockchain.expectWatchFundingSpent(fundingTx1.txid, Some(Set(fundingTx2.txid, commitAlice1.txid, commitBob1.txid))) assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2, 1)) assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) - // splice 2 confirms + // Alice and Bob see the second splice confirm. alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) alice2bob.forward(bob) @@ -805,70 +1186,67 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.fundingTxId == fundingTx1.txid) awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 1) assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.fundingTxId == fundingTx1.txid) - } - - test("splice local/remote locking (reverse order)", Tag(ChannelStateTestsTags.NoMaxHtlcValueInFlight)) { f => - import f._ - val (fundingTx1, fundingTx2) = setup2Splices(f) - val commitAlice2 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.commitTxAndRemoteSig.commitTx.tx - val commitBob2 = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.commitTxAndRemoteSig.commitTx.tx + // We're using 0-conf, so we can create chains of unconfirmed splice transactions. + val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(100_000 sat))) + assert(fundingTx2.txIn.map(_.outPoint.txid).contains(fundingTx1.txid)) + alice2blockchain.expectWatchPublished(fundingTx2.txid) + alice ! WatchPublishedTriggered(fundingTx2) + bob2blockchain.expectWatchPublished(fundingTx2.txid) + bob ! WatchPublishedTriggered(fundingTx2) - // splice 2 confirms - alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) - alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) + assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) alice2bob.forward(bob) - bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) - bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) + assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) bob2alice.forward(alice) - alice2blockchain.expectWatchFundingSpent(fundingTx2.txid, Some(Set(commitAlice2.txid, commitBob2.txid))) - bob2blockchain.expectWatchFundingSpent(fundingTx2.txid, Some(Set(commitAlice2.txid, commitBob2.txid))) - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2)) - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) - // splice 1 confirms - alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) - // we don't send a splice_locked for the older tx - alice2bob.expectNoMessage(100 millis) - bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) - // we don't send a splice_locked for the older tx - bob2alice.expectNoMessage(100 millis) - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2)) - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) + awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 1) + assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxId).contains(fundingTx1.txid)) + awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 1) + assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxId).contains(fundingTx1.txid)) } test("splice local/remote locking (intermingled)", Tag(ChannelStateTestsTags.NoMaxHtlcValueInFlight)) { f => import f._ - val (fundingTx1, fundingTx2) = setup2Splices(f) + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + checkWatchConfirmed(f, fundingTx1) - // splice 1 confirms on alice, splice 2 confirms on bob - alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) - alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) - alice2bob.forward(bob) - bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) - bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) + // Bob sees the first splice confirm, but Alice doesn't. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) bob2alice.forward(alice) - alice2blockchain.expectWatchFundingSpent(fundingTx1.txid) - bob2blockchain.expectWatchFundingSpent(fundingTx2.txid) - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2, 1)) - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2, 1)) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) - // splice 2 confirms on bob, splice 1 confirms on alice + // Alice creates another splice spending the first splice. + val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + checkWatchConfirmed(f, fundingTx2) + val commitAlice2 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.commitTxAndRemoteSig.commitTx.tx + val commitBob2 = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.commitTxAndRemoteSig.commitTx.tx + + // Alice sees the second splice confirm. alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) alice2bob.forward(bob) - bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + alice2blockchain.expectWatchFundingSpent(fundingTx2.txid, Some(Set(commitAlice2.txid, commitBob2.txid))) bob2alice.expectNoMessage(100 millis) + assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2, 1)) + + // Bob sees the second splice confirm. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) bob2alice.forward(alice) - alice2blockchain.expectWatchFundingSpent(fundingTx2.txid) - bob2blockchain.expectNoMessage(100 millis) + bob2blockchain.expectWatchFundingSpent(fundingTx2.txid, Some(Set(commitAlice2.txid, commitBob2.txid))) + awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2)) + awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2)) + assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) + + // Alice sees the first splice confirm. + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + // Alice doesn't send a splice_locked for the older tx. + alice2bob.expectNoMessage(100 millis) assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2)) assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2)) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) } test("emit post-splice events", Tag(ChannelStateTestsTags.NoMaxHtlcValueInFlight)) { f => @@ -896,24 +1274,24 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik systemB.eventStream.subscribe(bobEvents.ref, classOf[LocalChannelUpdate]) systemB.eventStream.subscribe(bobEvents.ref, classOf[LocalChannelDown]) - val (fundingTx1, fundingTx2) = setup2Splices(f) + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + checkWatchConfirmed(f, fundingTx1) - // splices haven't been locked, so no event is emitted + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) + bobEvents.expectMsg(ForgetHtlcInfos(initialState.channelId, initialState.commitments.localCommitIndex)) aliceEvents.expectNoMessage(100 millis) - bobEvents.expectNoMessage(100 millis) + + val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + checkWatchConfirmed(f, fundingTx2) alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) alice2bob.expectMsgType[SpliceLocked] alice2bob.forward(bob) aliceEvents.expectMsg(ForgetHtlcInfos(initialState.channelId, initialState.commitments.remoteCommitIndex)) - aliceEvents.expectNoMessage(100 millis) - bobEvents.expectNoMessage(100 millis) - - bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) - bob2alice.expectMsgType[SpliceLocked] - bob2alice.forward(alice) aliceEvents.expectAvailableBalanceChanged(balance = 1_275_000_000.msat, capacity = 2_000_000.sat) - bobEvents.expectMsg(ForgetHtlcInfos(initialState.channelId, initialState.commitments.localCommitIndex)) bobEvents.expectAvailableBalanceChanged(balance = 650_000_000.msat, capacity = 2_000_000.sat) aliceEvents.expectNoMessage(100 millis) bobEvents.expectNoMessage(100 millis) @@ -1092,9 +1470,12 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik test("recv UpdateAddHtlc while splice is being locked", Tag(ChannelStateTestsTags.ZeroConf), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => import f._ - initiateSplice(f, spliceOut_opt = Some(SpliceOut(50_000 sat, defaultSpliceOutScriptPubKey))) - val spliceTx = initiateSplice(f, spliceOut_opt = Some(SpliceOut(50_000 sat, defaultSpliceOutScriptPubKey))) - alice ! WatchPublishedTriggered(spliceTx) + val spliceTx1 = initiateSplice(f, spliceOut_opt = Some(SpliceOut(50_000 sat, defaultSpliceOutScriptPubKey))) + bob ! WatchPublishedTriggered(spliceTx1) + bob2alice.expectMsgType[SpliceLocked] // we ignore Bob's splice_locked for the first splice + + val spliceTx2 = initiateSplice(f, spliceOut_opt = Some(SpliceOut(50_000 sat, defaultSpliceOutScriptPubKey))) + alice ! WatchPublishedTriggered(spliceTx2) val spliceLockedAlice = alice2bob.expectMsgType[SpliceLocked] assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 3) @@ -1125,8 +1506,9 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice ! CMD_SIGN() val commitSigsAlice = (1 to 3).map(_ => alice2bob.expectMsgType[CommitSig]) alice2bob.forward(bob, commitSigsAlice(0)) - bob ! WatchPublishedTriggered(spliceTx) + bob ! WatchPublishedTriggered(spliceTx2) val spliceLockedBob = bob2alice.expectMsgType[SpliceLocked] + assert(spliceLockedBob.fundingTxId == spliceTx2.txid) bob2alice.forward(alice, spliceLockedBob) alice2bob.forward(bob, commitSigsAlice(1)) alice2bob.forward(bob, commitSigsAlice(2)) @@ -1415,8 +1797,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik test("don't resend splice_locked when zero-conf channel confirms", Tag(ChannelStateTestsTags.ZeroConf), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => import f._ - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) - val fundingTx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get + val fundingTx = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) alice2blockchain.expectMsgType[WatchPublished] // splice tx gets published, alice sends splice_locked alice ! WatchPublishedTriggered(fundingTx) @@ -1431,17 +1812,21 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik test("re-send splice_locked on reconnection") { f => import f._ - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) - val fundingTx1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get - val watchConfirmed1a = alice2blockchain.expectMsgType[WatchFundingConfirmed] - val watchConfirmed1b = bob2blockchain.expectMsgType[WatchFundingConfirmed] - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) - val fundingTx2 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get - val watchConfirmed2a = alice2blockchain.expectMsgType[WatchFundingConfirmed] - val watchConfirmed2b = bob2blockchain.expectMsgType[WatchFundingConfirmed] + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) + checkWatchConfirmed(f, fundingTx1) + + // The first splice confirms on Bob's side. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) + + val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) + checkWatchConfirmed(f, fundingTx2) alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) - // we now have two unconfirmed splices + + // From Alice's point of view, we now have two unconfirmed splices. alice2bob.ignoreMsg { case _: ChannelUpdate => true } bob2alice.ignoreMsg { case _: ChannelUpdate => true } @@ -1449,22 +1834,13 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik disconnect(f) reconnect(f) - // channel_ready are not re-sent because the channel has already been used (for building splices) + // NB: channel_ready are not re-sent because the channel has already been used (for building splices). + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) alice2bob.expectNoMessage(100 millis) - bob2alice.expectNoMessage(100 millis) - - // splice transactions are not locked yet: we're still at the initial funding index - alicePeer.fishForMessage() { - case e: ChannelReadyForPayments => e.fundingTxIndex == 0 - case _ => false - } - bobPeer.fishForMessage() { - case e: ChannelReadyForPayments => e.fundingTxIndex == 0 - case _ => false - } - // splice 1 confirms on alice's side - watchConfirmed1a.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + // The first splice confirms on Alice's side. + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx1.txid) alice2bob.forward(bob) alice2blockchain.expectMsgType[WatchFundingSpent] @@ -1472,11 +1848,13 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik disconnect(f) reconnect(f) - assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx1.txid) + alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) alice2bob.forward(bob) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) - // splice 2 confirms on alice's side - watchConfirmed2a.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) + // The second splice confirms on Alice's side. + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) alice2bob.forward(bob) alice2blockchain.expectMsgType[WatchFundingSpent] @@ -1484,71 +1862,26 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik disconnect(f) reconnect(f) - assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) - alice2bob.forward(bob) - alice2bob.expectNoMessage(100 millis) - bob2alice.expectNoMessage(100 millis) - - // splice transactions are not locked by bob yet: we're still at the initial funding index - alicePeer.fishForMessage() { - case e: ChannelReadyForPayments => e.fundingTxIndex == 0 - case _ => false - } - - // splice 1 confirms on bob's side - watchConfirmed1b.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) - assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == fundingTx1.txid) - bob2alice.forward(alice) - bob2blockchain.expectMsgType[WatchFundingSpent] - - // splice 1 is locked on both sides - alicePeer.fishForMessage() { - case e: ChannelReadyForPayments => e.fundingTxIndex == 1 - case _ => false - } - bobPeer.fishForMessage() { - case e: ChannelReadyForPayments => e.fundingTxIndex == 1 - case _ => false - } - - disconnect(f) - reconnect(f) - - assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) + alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) alice2bob.forward(bob) - assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) bob2alice.forward(alice) alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) - alicePeer.fishForMessage() { - case e: ChannelReadyForPayments => e.fundingTxIndex == 1 - case _ => false - } - bobPeer.fishForMessage() { - case e: ChannelReadyForPayments => e.fundingTxIndex == 1 - case _ => false - } - - // splice 2 confirms on bob's side - watchConfirmed2b.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) + // The second splice confirms on Bob's side. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) bob2blockchain.expectMsgType[WatchFundingSpent] - // splice 2 is locked on both sides - bobPeer.fishForMessage() { - case e: ChannelReadyForPayments => e.fundingTxIndex == 2 - case _ => false - } - - // NB: we disconnect *before* transmitting the splice_confirmed to alice + // NB: we disconnect *before* transmitting the splice_locked to Alice. disconnect(f) reconnect(f) - assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) + alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) alice2bob.forward(bob) - assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) - // this time alice received the splice_confirmed for funding tx 2 + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) + // This time alice received the splice_locked for the second splice. bob2alice.forward(alice) alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) @@ -1556,9 +1889,9 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik disconnect(f) reconnect(f) - assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) + alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) alice2bob.forward(bob) - assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) bob2alice.forward(alice) alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) @@ -1589,69 +1922,64 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val htlcs = setupHtlcs(f) - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) - val fundingTx1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get - val watchConfirmed1 = alice2blockchain.expectWatchFundingConfirmed(fundingTx1.txid) - initiateSplice(f, spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) - val fundingTx2 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get - val watchConfirmed2 = alice2blockchain.expectWatchFundingConfirmed(fundingTx2.txid) - alice2bob.expectNoMessage(100 millis) - bob2alice.expectNoMessage(100 millis) - alice2blockchain.expectNoMessage(100 millis) - // we now have two unconfirmed splices + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + checkWatchConfirmed(f, fundingTx1) + // The first splice confirms on Bob's side. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) + + val fundingTx2 = initiateSplice(f, spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) + checkWatchConfirmed(f, fundingTx2) + + // From Alice's point of view, we now have two unconfirmed splices. alice ! CMD_FORCECLOSE(ActorRef.noSender) alice2bob.expectMsgType[Error] val commitTx2 = assertPublished(alice2blockchain, "commit-tx") Transaction.correctlySpends(commitTx2, Seq(fundingTx2), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS) val claimMainDelayed2 = assertPublished(alice2blockchain, "local-main-delayed") - // alice publishes her htlc timeout transactions + // Alice publishes her htlc timeout transactions. val htlcsTxsOut = htlcs.aliceToBob.map(_ => assertPublished(alice2blockchain, "htlc-timeout")) htlcsTxsOut.foreach(tx => Transaction.correctlySpends(tx, Seq(commitTx2), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) val watchConfirmedCommit2 = alice2blockchain.expectWatchTxConfirmed(commitTx2.txid) val watchConfirmedClaimMainDelayed2 = alice2blockchain.expectWatchTxConfirmed(claimMainDelayed2.txid) - // watch for all htlc outputs from local commit-tx to be spent val watchHtlcsOut = htlcs.aliceToBob.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) htlcs.bobToAlice.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) - // splice 1 confirms - watchConfirmed1.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) - alice2bob.forward(bob) + // The first splice transaction confirms. + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) alice2blockchain.expectMsgType[WatchFundingSpent] - // splice 2 confirms - watchConfirmed2.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) - alice2bob.forward(bob) + // The second splice transaction confirms. + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) alice2blockchain.expectMsgType[WatchFundingSpent] - // commit tx confirms + // The commit confirms, along with Alice's 2nd-stage transactions. watchConfirmedCommit2.replyTo ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, commitTx2) - // claim-main-delayed tx confirms watchConfirmedClaimMainDelayed2.replyTo ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, claimMainDelayed2) - // alice's htlc-timeout txs confirm watchHtlcsOut.zip(htlcsTxsOut).foreach { case (watch, tx) => watch.replyTo ! WatchOutputSpentTriggered(tx) } htlcsTxsOut.foreach { tx => alice2blockchain.expectWatchTxConfirmed(tx.txid) alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, tx) } - // alice publishes, watches and confirms their 2nd-stage htlc-delayed txs + // Alice publishes 3rd-stage transactions. htlcs.aliceToBob.foreach { _ => val tx = assertPublished(alice2blockchain, "htlc-delayed") alice2blockchain.expectWatchTxConfirmed(tx.txid) alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, tx) } - // confirm bob's htlc-timeout txs + // Bob's htlc-timeout txs confirm. val remoteOutpoints = alice.stateData.asInstanceOf[DATA_CLOSING].localCommitPublished.map(rcp => rcp.htlcTxs.filter(_._2.isEmpty).keys).toSeq.flatten assert(remoteOutpoints.size == htlcs.bobToAlice.size) remoteOutpoints.foreach { out => alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, htlcsTxsOut.head.copy(txIn = Seq(TxIn(out, Nil, 0)))) } alice2blockchain.expectNoMessage(100 millis) checkPostSpliceState(f, spliceOutFee(f, capacity = 1_900_000.sat)) - - // done awaitCond(alice.stateName == CLOSED) assert(Helpers.Closing.isClosed(alice.stateData.asInstanceOf[DATA_CLOSING], None).exists(_.isInstanceOf[LocalClose])) } @@ -1661,60 +1989,60 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val htlcs = setupHtlcs(f) - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) - val fundingTx1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get - val watchConfirmed1 = alice2blockchain.expectMsgType[WatchFundingConfirmed] - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) - alice2blockchain.expectMsgType[WatchFundingConfirmed] + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) + checkWatchConfirmed(f, fundingTx1) + + // The first splice confirms on Bob's side. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) + + val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + checkWatchConfirmed(f, fundingTx2) alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) - alice2blockchain.expectNoMessage(100 millis) - // we now have two unconfirmed splices + // From Alice's point of view, we now have two unconfirmed splices. alice ! CMD_FORCECLOSE(ActorRef.noSender) alice2bob.expectMsgType[Error] val aliceCommitTx2 = assertPublished(alice2blockchain, "commit-tx") assertPublished(alice2blockchain, "local-anchor") assertPublished(alice2blockchain, "local-main-delayed") - // alice publishes her htlc timeout transactions val htlcsTxsOut = htlcs.aliceToBob.map(_ => assertPublished(alice2blockchain, "htlc-timeout")) htlcsTxsOut.foreach(tx => Transaction.correctlySpends(tx, Seq(aliceCommitTx2), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) - alice2blockchain.expectMsgType[WatchTxConfirmed] alice2blockchain.expectMsgType[WatchTxConfirmed] - alice2blockchain.expectMsgType[WatchOutputSpent] - // watch for all htlc outputs from local commit-tx to be spent htlcs.aliceToBob.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) htlcs.bobToAlice.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) - // splice 1 confirms - watchConfirmed1.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + // The first splice transaction confirms. + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) alice2blockchain.expectWatchFundingSpent(fundingTx1.txid) - // bob publishes his commit tx for splice 1 (which double-spends splice 2) + // Bob publishes his commit tx for the first splice transaction (which double-spends the second splice transaction). val bobCommitment1 = bob.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.active.find(_.fundingTxIndex == 1).get val bobCommitTx1 = bobCommitment1.fullySignedLocalCommitTx(bob.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.params, bob.underlyingActor.keyManager).tx Transaction.correctlySpends(bobCommitTx1, Seq(fundingTx1), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS) alice ! WatchFundingSpentTriggered(bobCommitTx1) val watchAlternativeConfirmed = alice2blockchain.expectMsgType[WatchAlternativeCommitTxConfirmed] alice2blockchain.expectNoMessage(100 millis) - // remote commit tx confirms + // Bob's commit tx confirms. watchAlternativeConfirmed.replyTo ! WatchAlternativeCommitTxConfirmedTriggered(BlockHeight(400000), 42, bobCommitTx1) - // we're back to the normal handling of remote commit + // We're back to the normal handling of remote commit. assertPublished(alice2blockchain, "local-anchor") val claimMain = assertPublished(alice2blockchain, "remote-main-delayed") val htlcsTxsOut1 = htlcs.aliceToBob.map(_ => assertPublished(alice2blockchain, "claim-htlc-timeout")) htlcsTxsOut1.foreach(tx => Transaction.correctlySpends(tx, Seq(bobCommitTx1), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) val watchConfirmedRemoteCommit = alice2blockchain.expectWatchTxConfirmed(bobCommitTx1.txid) - // this one fires immediately, tx is already confirmed + // NB: this one fires immediately, tx is already confirmed. watchConfirmedRemoteCommit.replyTo ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, bobCommitTx1) + // Alice's 2nd-stage transactions confirm. val watchConfirmedClaimMain = alice2blockchain.expectWatchTxConfirmed(claimMain.txid) watchConfirmedClaimMain.replyTo ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, claimMain) - - // alice's htlc-timeout transactions confirm val watchHtlcsOut1 = htlcs.aliceToBob.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) htlcs.bobToAlice.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) watchHtlcsOut1.zip(htlcsTxsOut1).foreach { case (watch, tx) => watch.replyTo ! WatchOutputSpentTriggered(tx) } @@ -1723,13 +2051,11 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, tx) } - // bob's htlc-timeout transactions confirm + // Bob's 2nd-stage transactions confirm. bobCommitment1.localCommit.htlcTxsAndRemoteSigs.foreach(txAndSig => alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, txAndSig.htlcTx.tx)) alice2blockchain.expectNoMessage(100 millis) checkPostSpliceState(f, spliceOutFee = 0.sat) - - // done awaitCond(alice.stateName == CLOSED) assert(Helpers.Closing.isClosed(alice.stateData.asInstanceOf[DATA_CLOSING], None).exists(_.isInstanceOf[RemoteClose])) } @@ -1740,32 +2066,37 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val htlcs = setupHtlcs(f) // pay 10_000_000 msat to bob that will be paid back to alice after the splices - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 10_000_000 msat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) - val fundingTx1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get - alice2blockchain.expectWatchFundingConfirmed(fundingTx1.txid) + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 10_000_000 msat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) + checkWatchConfirmed(f, fundingTx1) // remember bob's commitment for later val bobCommit1 = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) - alice2blockchain.expectMsgType[WatchFundingConfirmed] + + // The first splice confirms on Bob's side. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) + + val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) + checkWatchConfirmed(f, fundingTx2) alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) - alice2blockchain.expectNoMessage(100 millis) - // we now have two unconfirmed splices, both active - // bob makes a payment + // From Alice's point of view, We now have two unconfirmed splices, both active. + // Bob makes a payment, that applies to both commitments. val (preimage, add) = addHtlc(10_000_000 msat, bob, alice, bob2alice, alice2bob) crossSign(bob, alice, bob2alice, alice2bob) alice2relayer.expectMsgType[Relayer.RelayForward] fulfillHtlc(add.id, preimage, alice, bob, alice2bob, bob2alice) crossSign(alice, bob, alice2bob, bob2alice) - // funding tx1 confirms + // The first splice transaction confirms. alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) alice2blockchain.expectWatchFundingSpent(fundingTx1.txid) - // bob publishes a revoked commitment for fundingTx1! + // Bob publishes a revoked commitment for fundingTx1! val bobRevokedCommitTx = bobCommit1.localCommit.commitTxAndRemoteSig.commitTx.tx alice ! WatchFundingSpentTriggered(bobRevokedCommitTx) - // alice watches bob's revoked commit tx, and force-closes with her latest commitment + // Alice watches bob's revoked commit tx, and force-closes with her latest commitment. assert(alice2blockchain.expectMsgType[WatchAlternativeCommitTxConfirmed].txId == bobRevokedCommitTx.txid) val aliceCommitTx2 = assertPublished(alice2blockchain, "commit-tx") assertPublished(alice2blockchain, "local-anchor") @@ -1777,9 +2108,9 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik htlcs.aliceToBob.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) htlcs.bobToAlice.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) - // bob's revoked tx wins + // Bob's revoked commit tx wins. alice ! WatchAlternativeCommitTxConfirmedTriggered(BlockHeight(400000), 42, bobRevokedCommitTx) - // alice reacts by punishing bob + // Alice reacts by punishing bob. val aliceClaimMain = assertPublished(alice2blockchain, "remote-main-delayed") val aliceMainPenalty = assertPublished(alice2blockchain, "main-penalty") val aliceHtlcsPenalty = htlcs.aliceToBob.map(_ => assertPublished(alice2blockchain, "htlc-penalty")) ++ htlcs.bobToAlice.map(_ => assertPublished(alice2blockchain, "htlc-penalty")) @@ -1790,18 +2121,15 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik aliceHtlcsPenalty.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) alice2blockchain.expectNoMessage(100 millis) - // alice's main penalty txs confirm + // Alice's penalty txs confirm. alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, bobRevokedCommitTx) alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, aliceClaimMain) alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, aliceMainPenalty) - // alice's htlc-penalty txs confirm aliceHtlcsPenalty.foreach { tx => alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, tx) } val settledOutgoingHtlcs = htlcs.aliceToBob.map(_ => alice2relayer.expectMsgType[RES_ADD_SETTLED[Origin, HtlcResult.OnChainFail]].htlc).toSet assert(settledOutgoingHtlcs == htlcs.aliceToBob.map(_._2).toSet) checkPostSpliceState(f, spliceOutFee = 0.sat) - - // done awaitCond(alice.stateName == CLOSED) assert(Helpers.Closing.isClosed(alice.stateData.asInstanceOf[DATA_CLOSING], None).exists(_.isInstanceOf[RevokedClose])) } @@ -2028,7 +2356,18 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik import f._ val fundingTx0 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get - val (fundingTx1, fundingTx2) = setup2Splices(f) + + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 10_000_000 msat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) + checkWatchConfirmed(f, fundingTx1) + + // The first splice confirms on Bob's side. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) + + val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) + checkWatchConfirmed(f, fundingTx2) val (aliceNodeParams, bobNodeParams) = (alice.underlyingActor.nodeParams, bob.underlyingActor.nodeParams) val (alicePeer, bobPeer) = (alice.getParent, bob.getParent) @@ -2054,7 +2393,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik bob2 ! INPUT_RESTORED(bobData) bob2blockchain.expectMsgType[SetChannelId] bob2blockchain.expectWatchFundingConfirmed(fundingTx2.txid) - bob2blockchain.expectWatchFundingConfirmed(fundingTx1.txid) + bob2blockchain.expectWatchFundingSpent(fundingTx1.txid) bob2blockchain.expectWatchFundingSpent(fundingTx0.txid) bob2blockchain.expectNoMessage(100 millis) } @@ -2175,14 +2514,146 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik } test("recv multiple CMD_SPLICE (splice-in, splice-out)") { f => + import f._ + + // Add some HTLCs before splicing in and out. val htlcs = setupHtlcs(f) - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + // Alice splices some funds in. + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + // The first splice confirms on Bob's side (necessary to allow the second splice transaction). + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2alice.expectMsgType[SpliceLocked] + // Alice splices some funds out. initiateSplice(f, spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) + // The HTLCs complete while multiple commitments are active. resolveHtlcs(f, htlcs, spliceOutFee = spliceOutFee(f, capacity = 1_900_000.sat)) } + test("recv CMD_BUMP_FUNDING_FEE with pre and post rbf htlcs") { f => + import f._ + + // We create two unconfirmed splice transactions spending each other: + // +-----------+ +-----------+ +-----------+ + // | fundingTx |---->| spliceTx1 |---->| spliceTx2 | + // +-----------+ +-----------+ +-----------+ + val spliceTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(200_000 sat))) + checkWatchConfirmed(f, spliceTx1) + val spliceCommitment1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest + assert(spliceCommitment1.fundingTxId == spliceTx1.txid) + + // The first splice confirms on Bob's side (necessary to allow the second splice transaction). + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, spliceTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == spliceTx1.txid) + bob2alice.expectMsgType[SpliceLocked] + bob2alice.forward(alice) + + val spliceTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(100_000 sat))) + checkWatchConfirmed(f, spliceTx2) + val spliceCommitment2 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest + assert(spliceCommitment2.fundingTxId == spliceTx2.txid) + assert(spliceCommitment2.localCommit.spec.toLocal == spliceCommitment1.localCommit.spec.toLocal + 100_000.sat) + + // Bob sends an HTLC while Alice starts an RBF on spliceTx2. + addHtlc(25_000_000 msat, bob, alice, bob2alice, alice2bob) + val rbfTx1 = { + val probe = TestProbe() + alice ! CMD_BUMP_FUNDING_FEE(probe.ref, FeeratePerKw(15_000 sat), 100_000 sat, 250, None) + alice2bob.expectMsgType[Stfu] + alice2bob.forward(bob) + // Bob is waiting to sign its outgoing HTLC before sending stfu. + bob2alice.expectNoMessage(100 millis) + bob ! CMD_SIGN() + (0 until 3).foreach { _ => + bob2alice.expectMsgType[CommitSig] + bob2alice.forward(alice) + } + alice2bob.expectMsgType[RevokeAndAck] + alice2bob.forward(bob) + (0 until 3).foreach { _ => + alice2bob.expectMsgType[CommitSig] + alice2bob.forward(bob) + } + bob2alice.expectMsgType[RevokeAndAck] + bob2alice.forward(alice) + bob2alice.expectMsgType[Stfu] + bob2alice.forward(alice) + + alice2bob.expectMsgType[TxInitRbf] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxAckRbf] + bob2alice.forward(alice) + + // Alice adds three inputs: the shared input, the previous splice input, and an RBF input. + (0 until 3).foreach { _ => + alice2bob.expectMsgType[TxAddInput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxComplete] + bob2alice.forward(alice) + } + // Alice adds two outputs: the shared output and a change output. + (0 until 2).foreach { _ => + alice2bob.expectMsgType[TxAddOutput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxComplete] + bob2alice.forward(alice) + } + // Alice doesn't have anything more to add to the transaction. + alice2bob.expectMsgType[TxComplete] + alice2bob.forward(bob) + val rbfTx = exchangeSpliceSigs(f, probe) + assert(rbfTx.lockTime == 250) + spliceTx2.txIn.foreach(txIn => assert(rbfTx.txIn.map(_.outPoint).contains(txIn.outPoint))) + rbfTx + } + + // Alice and Bob exchange HTLCs while the splice transactions are still unconfirmed. + addHtlc(10_000_000 msat, alice, bob, alice2bob, bob2alice) + addHtlc(5_000_000 msat, bob, alice, bob2alice, alice2bob) + crossSign(alice, bob, alice2bob, bob2alice) + + // Alice initiates another RBF attempt: + // +-----------+ +-----------+ +-----------+ + // | fundingTx |---->| spliceTx1 |---->| spliceTx2 | + // +-----------+ +-----------+ | +-----------+ + // | +-----------+ + // +->| rbfTx1 | + // | +-----------+ + // | +-----------+ + // +->| rbfTx2 | + // +-----------+ + val rbfTx2 = initiateRbf(f, FeeratePerKw(20_000 sat), sInputsCount = 3, sOutputsCount = 1) + assert(rbfTx2.txIn.size == rbfTx1.txIn.size + 1) + rbfTx1.txIn.foreach(txIn => assert(rbfTx2.txIn.map(_.outPoint).contains(txIn.outPoint))) + + // The balance is the same in all RBF attempts. + inside(alice.stateData.asInstanceOf[DATA_NORMAL]) { data => + val commitments = data.commitments.active.filter(_.fundingTxIndex == spliceCommitment2.commitment.fundingTxIndex) + assert(commitments.map(_.fundingTxId) == Seq(rbfTx2, rbfTx1, spliceTx2).map(_.txid)) + assert(commitments.map(_.localCommit.spec.toLocal).toSet.size == 1) + assert(commitments.map(_.localCommit.spec.toRemote).toSet.size == 1) + } + + // The first RBF attempt is confirmed. + confirmSpliceTx(f, rbfTx1) + inside(alice.stateData.asInstanceOf[DATA_NORMAL]) { data => + assert(data.commitments.active.size == 1) + assert(data.commitments.latest.fundingTxId == rbfTx1.txid) + assert(data.commitments.latest.localCommit.spec.toLocal == spliceCommitment1.localCommit.spec.toLocal + 100_000.sat - 10_000.sat) + assert(data.commitments.latest.localCommit.spec.toRemote == spliceCommitment1.localCommit.spec.toRemote - 25_000.sat - 5_000.sat) + assert(data.commitments.latest.localCommit.spec.htlcs.collect(incoming).map(_.amountMsat) == Set(5_000_000 msat, 25_000_000 msat)) + assert(data.commitments.latest.localCommit.spec.htlcs.collect(outgoing).map(_.amountMsat) == Set(10_000_000 msat)) + } + + // The first splice transaction confirms: this was already implied by the RBF attempt confirming. + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, spliceTx1) + inside(alice.stateData.asInstanceOf[DATA_NORMAL]) { data => + assert(data.commitments.active.size == 1) + assert(data.commitments.latest.fundingTxId == rbfTx1.txid) + } + } + test("recv invalid htlc signatures during splice-in") { f => import 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 6ca3b27bcc..ca67359422 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 @@ -145,16 +145,16 @@ class ChannelCodecs4Spec extends AnyFunSuite { Some(LiquidityAds.PurchaseBasicInfo(isBuyer = true, 100_000 sat, LiquidityAds.Fees(1000 sat, 500 sat))), ) val testCases = Map( - RbfStatus.NoRbf -> RbfStatus.NoRbf, - RbfStatus.RbfRequested(CMD_BUMP_FUNDING_FEE(null, FeeratePerKw(750 sat), fundingFeeBudget = 100_000.sat, 0, None)) -> RbfStatus.NoRbf, - RbfStatus.RbfInProgress(None, null, None) -> RbfStatus.NoRbf, - RbfStatus.RbfWaitingForSigs(waitingForSigs) -> RbfStatus.RbfWaitingForSigs(waitingForSigs), - RbfStatus.RbfWaitingForSigs(waitingForSigs.copy(liquidityPurchase_opt = None)) -> RbfStatus.RbfWaitingForSigs(waitingForSigs.copy(liquidityPurchase_opt = None)), - RbfStatus.RbfAborted -> RbfStatus.NoRbf, + DualFundingStatus.WaitingForConfirmations -> DualFundingStatus.WaitingForConfirmations, + DualFundingStatus.RbfRequested(CMD_BUMP_FUNDING_FEE(null, FeeratePerKw(750 sat), fundingFeeBudget = 100_000.sat, 0, None)) -> DualFundingStatus.WaitingForConfirmations, + DualFundingStatus.RbfInProgress(None, null, None) -> DualFundingStatus.WaitingForConfirmations, + DualFundingStatus.RbfWaitingForSigs(waitingForSigs) -> DualFundingStatus.RbfWaitingForSigs(waitingForSigs), + DualFundingStatus.RbfWaitingForSigs(waitingForSigs.copy(liquidityPurchase_opt = None)) -> DualFundingStatus.RbfWaitingForSigs(waitingForSigs.copy(liquidityPurchase_opt = None)), + 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) } } diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Channel.scala b/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Channel.scala index ef1bd0b36e..fd5dd468b5 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Channel.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Channel.scala @@ -90,6 +90,12 @@ trait Channel { } } + val rbfSplice: Route = postRequest("rbfsplice") { implicit f => + formFields(channelIdFormParam, "targetFeerateSatByte".as[FeeratePerByte], "fundingFeeBudgetSatoshis".as[Satoshi], "lockTime".as[Long].?) { + (channelId, targetFeerateSatByte, fundingFeeBudget, lockTime_opt) => complete(eclairApi.rbfSplice(channelId, FeeratePerKw(targetFeerateSatByte), fundingFeeBudget, lockTime_opt)) + } + } + val close: Route = postRequest("close") { implicit t => withChannelsIdentifier { channels => formFields("scriptPubKey".as[ByteVector](bytesUnmarshaller).?, "preferredFeerateSatByte".as[FeeratePerByte].?, "minFeerateSatByte".as[FeeratePerByte].?, "maxFeerateSatByte".as[FeeratePerByte].?) { @@ -165,6 +171,6 @@ trait Channel { complete(eclairApi.channelBalances()) } - val channelRoutes: Route = open ~ rbfOpen ~ spliceIn ~ spliceOut ~ close ~ forceClose ~ bumpForceClose ~ channel ~ channels ~ closedChannels ~ allChannels ~ allUpdates ~ channelStats ~ channelBalances + val channelRoutes: Route = open ~ rbfOpen ~ spliceIn ~ spliceOut ~ rbfSplice ~ close ~ forceClose ~ bumpForceClose ~ channel ~ channels ~ closedChannels ~ allChannels ~ allUpdates ~ channelStats ~ channelBalances }