1
0
mirror of https://github.com/ACINQ/eclair.git synced 2024-11-19 01:43:22 +01:00

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.
This commit is contained in:
Bastien Teinturier 2024-10-18 04:44:10 +02:00 committed by GitHub
parent f1e07353b9
commit 13d4c9f06c
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
19 changed files with 1414 additions and 536 deletions

View File

@ -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

View File

@ -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))
}
/**

View File

@ -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,

View File

@ -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")

View File

@ -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,23 +853,69 @@ 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()
}
}
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 {
log.warning("cannot initiate splice, peer doesn't support splices")
cmd.replyTo ! RES_FAILURE(cmd, CommandUnavailableInThisState(d.channelId, "splice", NORMAL))
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()
}
@ -885,18 +932,19 @@ 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 {
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))
@ -904,6 +952,15 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
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")
cmd.replyTo ! RES_FAILURE(cmd, ConcurrentRemoteSplice(d.channelId))
@ -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,7 +2953,6 @@ 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(
@ -2803,9 +2985,49 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
)
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, rbf.parentCommitment.remoteCommit.spec, d.commitments.params.commitmentFormat)
} else {
log.warning("cannot initiate splice, channel is not quiescent")
Left(InvalidSpliceNotQuiescent(d.channelId))
0.sat
}
if (fundingContribution < 0.sat && rbf.parentCommitment.localCommit.spec.toLocal + fundingContribution < rbf.parentCommitment.localChannelReserve(d.commitments.params).max(commitTxFees)) {
log.warning(s"cannot do rbf: insufficient funds (commitTxFees=$commitTxFees reserve=${rbf.parentCommitment.localChannelReserve(d.commitments.params)})")
Left(InvalidSpliceRequest(d.channelId))
} else {
val txInitRbf = TxInitRbf(d.channelId, cmd.lockTime, cmd.targetFeerate, fundingContribution, rbf.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote, cmd.requestFunding_opt)
Right(txInitRbf)
}
})
}
private def getSpliceRbfContext(cmd_opt: Option[CMD_BUMP_FUNDING_FEE], d: DATA_NORMAL): Either[ChannelException, InteractiveTxBuilder.SpliceTxRbf] = {
d.commitments.latest.localFundingStatus match {
case fundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx =>
val fundingTxIndex = d.commitments.latest.fundingTxIndex
d.commitments.active.find(_.fundingTxIndex == fundingTxIndex - 1) match {
case Some(parentCommitment) =>
val previousTxs = d.commitments.active
.filter(_.fundingTxIndex == fundingTxIndex)
.map(_.localFundingStatus)
// All transactions should be unconfirmed, otherwise we wouldn't be in an RBF context.
.collect { case tx: LocalFundingStatus.DualFundedUnconfirmedFundingTx => tx.sharedTx }
Right(InteractiveTxBuilder.SpliceTxRbf(parentCommitment, 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

View File

@ -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)
})

View File

@ -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 _ => ()

View File

@ -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
}

View File

@ -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)) {

View File

@ -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] = (

View File

@ -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,

View File

@ -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 {

View File

@ -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,

View File

@ -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)

View File

@ -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
val feerate1 = TestConstants.feeratePerKw
val liquidityFee1 = TestConstants.defaultLiquidityRates.fundingRates.head.fees(feerate1, remoteFunding, remoteFunding, isChannelCreation = true)
// 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)
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)))
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, 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 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)

View File

@ -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

View File

@ -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)
}
}

View File

@ -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
}