Skip to content

Commit

Permalink
Add quiescence negotiation (#2680)
Browse files Browse the repository at this point in the history
This change adds support for the quiescence negotiation protocol via the new `stfu` message. When a channel is quiescent, both sides will have the same set of signed htlc commitments and a splice can be performed without requiring the channel to be idle.

An additional PR is still required to update our splice implementation to properly account for in-flight htlcs. Quiescence should currently only be enabled for compatibility testing.

We send a warning and disconnect when a forbidden messages is received during quiescence; a disconnect ends quiescence. If an htlc is fulfilled/failed while quiescent, any preimage will be relayed immediately and the update will be replayed when quiescence ends.

We also send a warning and disconnect if both quiescence and splice negotiation are not complete before the quiescence timeout.

---------

Co-authored-by: t-bast <bastien@acinq.fr>
  • Loading branch information
remyers and t-bast committed Jul 27, 2023
1 parent 3e43611 commit 47e0b83
Show file tree
Hide file tree
Showing 15 changed files with 827 additions and 99 deletions.
3 changes: 3 additions & 0 deletions eclair-core/src/main/resources/reference.conf
Expand Up @@ -69,6 +69,7 @@ eclair {
option_route_blinding = disabled
option_shutdown_anysegwit = optional
option_dual_fund = disabled
option_quiesce = disabled
option_onion_messages = optional
option_channel_type = optional
option_scid_alias = optional
Expand Down Expand Up @@ -154,6 +155,8 @@ eclair {
max-total-pending-channels-private-nodes = 99 // maximum number of pending channels we will accept from all private nodes
channel-opener-whitelist = [] // a list of public keys; we will ignore rate limits on pending channels from these peers
}

quiescence-timeout = 1 minutes // maximum time we will stay quiescent (or wait to reach quiescence) before disconnecting
}

balance-check-interval = 1 hour
Expand Down
7 changes: 7 additions & 0 deletions eclair-core/src/main/scala/fr/acinq/eclair/Features.scala
Expand Up @@ -247,6 +247,12 @@ object Features {
val mandatory = 28
}

// TODO: this should also extend NodeFeature once the spec is finalized
case object Quiescence extends Feature with InitFeature {
val rfcName = "option_quiesce"
val mandatory = 34
}

case object OnionMessages extends Feature with InitFeature with NodeFeature {
val rfcName = "option_onion_messages"
val mandatory = 38
Expand Down Expand Up @@ -316,6 +322,7 @@ object Features {
RouteBlinding,
ShutdownAnySegwit,
DualFunding,
Quiescence,
OnionMessages,
ChannelType,
ScidAlias,
Expand Down
3 changes: 2 additions & 1 deletion eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala
Expand Up @@ -513,7 +513,8 @@ object NodeParams extends Logging {
channelOpenerWhitelist = channelOpenerWhitelist,
maxPendingChannelsPerPeer = maxPendingChannelsPerPeer,
maxTotalPendingChannelsPrivateNodes = maxTotalPendingChannelsPrivateNodes,
remoteRbfLimits = Channel.RemoteRbfLimits(config.getInt("channel.funding.remote-rbf-limits.max-attempts"), config.getInt("channel.funding.remote-rbf-limits.attempt-delta-blocks"))
remoteRbfLimits = Channel.RemoteRbfLimits(config.getInt("channel.funding.remote-rbf-limits.max-attempts"), config.getInt("channel.funding.remote-rbf-limits.attempt-delta-blocks")),
quiescenceTimeout = FiniteDuration(config.getDuration("channel.quiescence-timeout").getSeconds, TimeUnit.SECONDS),
),
onChainFeeConf = OnChainFeeConf(
feeTargets = feeTargets,
Expand Down
Expand Up @@ -27,7 +27,7 @@ import fr.acinq.eclair.io.Peer
import fr.acinq.eclair.payment.OutgoingPaymentPacket.Upstream
import fr.acinq.eclair.transactions.CommitmentSpec
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelReady, ChannelReestablish, ChannelUpdate, ClosingSigned, CommitSig, FailureMessage, FundingCreated, FundingSigned, Init, OnionRoutingPacket, OpenChannel, OpenDualFundedChannel, Shutdown, SpliceInit, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc}
import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelReady, ChannelReestablish, ChannelUpdate, ClosingSigned, CommitSig, FailureMessage, FundingCreated, FundingSigned, Init, OnionRoutingPacket, OpenChannel, OpenDualFundedChannel, Shutdown, SpliceInit, Stfu, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc}
import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Features, InitFeature, MilliSatoshi, MilliSatoshiLong, RealShortChannelId, UInt64}
import scodec.bits.ByteVector

Expand Down Expand Up @@ -187,13 +187,14 @@ 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

final case class CMD_ADD_HTLC(replyTo: ActorRef, amount: MilliSatoshi, paymentHash: ByteVector32, cltvExpiry: CltvExpiry, onion: OnionRoutingPacket, nextBlindingKey_opt: Option[PublicKey], origin: Origin.Hot, commit: Boolean = false) extends HasReplyToCommand with ForbiddenCommandDuringSplice
sealed trait HtlcSettlementCommand extends HasOptionalReplyToCommand with ForbiddenCommandDuringSplice { def id: Long }
final case class CMD_ADD_HTLC(replyTo: ActorRef, amount: MilliSatoshi, paymentHash: ByteVector32, cltvExpiry: CltvExpiry, onion: OnionRoutingPacket, nextBlindingKey_opt: Option[PublicKey], origin: Origin.Hot, commit: Boolean = false) extends HasReplyToCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence
sealed trait HtlcSettlementCommand extends HasOptionalReplyToCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence { def id: Long }
final case class CMD_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
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 ClosingFees(preferred: Satoshi, min: Satoshi, max: Satoshi)
Expand All @@ -202,7 +203,7 @@ 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
final case class CMD_CLOSE(replyTo: ActorRef, scriptPubKey: Option[ByteVector], feerates: Option[ClosingFeerates]) extends CloseCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence
final case class CMD_FORCECLOSE(replyTo: ActorRef) extends CloseCommand

final case class CMD_BUMP_FUNDING_FEE(replyTo: akka.actor.typed.ActorRef[CommandResponse[CMD_BUMP_FUNDING_FEE]], targetFeerate: FeeratePerKw, lockTime: Long) extends Command
Expand Down Expand Up @@ -450,12 +451,32 @@ object RbfStatus {
}

sealed trait SpliceStatus
/** We're waiting for the channel to be quiescent. */
sealed trait QuiescenceNegotiation extends SpliceStatus
object QuiescenceNegotiation {
sealed trait Initiator extends QuiescenceNegotiation
sealed trait NonInitiator extends QuiescenceNegotiation
}
/** The channel is quiescent and a splice attempt was initiated. */
sealed trait QuiescentSpliceStatus extends SpliceStatus
object SpliceStatus {
case object NoSplice extends SpliceStatus
case class SpliceRequested(cmd: CMD_SPLICE, init: SpliceInit) extends SpliceStatus
case class SpliceInProgress(cmd_opt: Option[CMD_SPLICE], splice: typed.ActorRef[InteractiveTxBuilder.Command], remoteCommitSig: Option[CommitSig]) extends SpliceStatus
case class SpliceWaitingForSigs(signingSession: InteractiveTxSigningSession.WaitingForSigs) extends SpliceStatus
case object SpliceAborted 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 told our peer we want to splice funds in the channel. */
case class SpliceRequested(cmd: CMD_SPLICE, init: SpliceInit) extends QuiescentSpliceStatus
/** We both agreed to splice and are building the splice transaction. */
case class SpliceInProgress(cmd_opt: Option[CMD_SPLICE], splice: typed.ActorRef[InteractiveTxBuilder.Command], remoteCommitSig: Option[CommitSig]) extends QuiescentSpliceStatus
/** The splice transaction has been negotiated, we're exchanging signatures. */
case class SpliceWaitingForSigs(signingSession: InteractiveTxSigningSession.WaitingForSigs) extends QuiescentSpliceStatus
/** The splice attempt was aborted by us, we're waiting for our peer to ack. */
case object SpliceAborted extends QuiescentSpliceStatus
}

sealed trait ChannelData extends PossiblyHarmful {
Expand Down
Expand Up @@ -71,6 +71,7 @@ case class InvalidCompleteInteractiveTx (override val channelId: Byte
case class TooManyInteractiveTxRounds (override val channelId: ByteVector32) extends ChannelException(channelId, "too many messages exchanged during interactive tx construction")
case class RbfAttemptAborted (override val channelId: ByteVector32) extends ChannelException(channelId, "rbf attempt aborted")
case class SpliceAttemptAborted (override val channelId: ByteVector32) extends ChannelException(channelId, "splice attempt aborted")
case class SpliceAttemptTimedOut (override val channelId: ByteVector32) extends ChannelException(channelId, "splice attempt took too long, disconnecting")
case class DualFundingAborted (override val channelId: ByteVector32) extends ChannelException(channelId, "dual funding aborted")
case class UnexpectedInteractiveTxMessage (override val channelId: ByteVector32, msg: InteractiveTxMessage) extends ChannelException(channelId, s"unexpected interactive-tx message (${msg.getClass.getSimpleName})")
case class UnexpectedFundingSignatures (override val channelId: ByteVector32) extends ChannelException(channelId, "unexpected funding signatures (tx_signatures)")
Expand All @@ -84,6 +85,7 @@ case class InvalidRbfTxAbortNotAcked (override val channelId: Byte
case class InvalidRbfAttemptsExhausted (override val channelId: ByteVector32, maxAttempts: Int) extends ChannelException(channelId, s"invalid rbf attempt: $maxAttempts/$maxAttempts attempts already published")
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 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")
Expand Down Expand Up @@ -136,4 +138,6 @@ case class InvalidFailureCode (override val channelId: Byte
case class PleasePublishYourCommitment (override val channelId: ByteVector32) extends ChannelException(channelId, "please publish your local commitment")
case class CommandUnavailableInThisState (override val channelId: ByteVector32, command: String, state: ChannelState) extends ChannelException(channelId, s"cannot execute command=$command in state=$state")
case class ForbiddenDuringSplice (override val channelId: ByteVector32, command: String) extends ChannelException(channelId, s"cannot process $command while splicing")
case class ForbiddenDuringQuiescence (override val channelId: ByteVector32, command: String) extends ChannelException(channelId, s"cannot process $command while quiescent")
case class ConcurrentRemoteSplice (override val channelId: ByteVector32) extends ChannelException(channelId, "splice attempt canceled, remote initiated splice before us")
// @formatter:on
Expand Up @@ -117,7 +117,6 @@ case class ChannelParams(channelId: ByteVector32,
}

/**
*
* @param localScriptPubKey local script pubkey (provided in CMD_CLOSE, as an upfront shutdown script, or set to the current final onchain script)
* @return an exception if the provided script is not valid
*/
Expand All @@ -132,7 +131,6 @@ case class ChannelParams(channelId: ByteVector32,
}

/**
*
* @param remoteScriptPubKey remote script included in a Shutdown message
* @return an exception if the provided script is not valid
*/
Expand All @@ -144,6 +142,9 @@ case class ChannelParams(channelId: ByteVector32,
else Right(remoteScriptPubKey)
}

/** If both peers support quiescence, we have to exchange stfu when splicing. */
def useQuiescence: Boolean = Features.canUseFeature(localParams.initFeatures, remoteParams.initFeatures, Features.Quiescence)

}

object ChannelParams {
Expand Down Expand Up @@ -350,7 +351,7 @@ case class Commitment(fundingTxIndex: Long,
}
}

private def hasNoPendingHtlcs: Boolean = localCommit.spec.htlcs.isEmpty && remoteCommit.spec.htlcs.isEmpty && nextRemoteCommit_opt.isEmpty
def hasNoPendingHtlcs: Boolean = localCommit.spec.htlcs.isEmpty && remoteCommit.spec.htlcs.isEmpty && nextRemoteCommit_opt.isEmpty

def hasNoPendingHtlcsOrFeeUpdate(changes: CommitmentChanges): Boolean = hasNoPendingHtlcs &&
(changes.localChanges.signed ++ changes.localChanges.acked ++ changes.remoteChanges.signed ++ changes.remoteChanges.acked).collectFirst { case _: UpdateFee => true }.isEmpty
Expand All @@ -359,8 +360,6 @@ case class Commitment(fundingTxIndex: Long,
changes.localChanges.all.exists(_.isInstanceOf[UpdateAddHtlc]) ||
changes.remoteChanges.all.exists(_.isInstanceOf[UpdateAddHtlc])

def isIdle(changes: CommitmentChanges): Boolean = hasNoPendingHtlcs && changes.localChanges.all.isEmpty && changes.remoteChanges.all.isEmpty

def timedOutOutgoingHtlcs(currentHeight: BlockHeight): Set[UpdateAddHtlc] = {
def expired(add: UpdateAddHtlc): Boolean = currentHeight >= add.cltvExpiry.blockHeight

Expand Down Expand Up @@ -795,8 +794,10 @@ case class Commitments(params: ChannelParams,
def add(commitment: Commitment): Commitments = copy(active = commitment +: active)

// @formatter:off
def localIsQuiescent: Boolean = changes.localChanges.all.isEmpty
def remoteIsQuiescent: Boolean = changes.remoteChanges.all.isEmpty
// HTLCs and pending changes are the same for all active commitments, so we don't need to loop through all of them.
def isIdle: Boolean = active.head.isIdle(changes)
def isQuiescent: Boolean = (params.useQuiescence || active.head.hasNoPendingHtlcs) && localIsQuiescent && remoteIsQuiescent
def hasNoPendingHtlcsOrFeeUpdate: Boolean = active.head.hasNoPendingHtlcsOrFeeUpdate(changes)
def hasPendingOrProposedHtlcs: Boolean = active.head.hasPendingOrProposedHtlcs(changes)
def timedOutOutgoingHtlcs(currentHeight: BlockHeight): Set[UpdateAddHtlc] = active.head.timedOutOutgoingHtlcs(currentHeight)
Expand Down

0 comments on commit 47e0b83

Please sign in to comment.