From addc7c981e2ab0bb1d015ac15aeb20ee45893d73 Mon Sep 17 00:00:00 2001 From: t-bast Date: Mon, 22 Jul 2024 16:00:01 +0200 Subject: [PATCH 1/8] Use official splice messages We replace our experimental version of `splice_init`, `splice_ack` and `splice_locked` by their official version. If our peer is using the experimental feature bit, we convert our outgoing messages to use the experimental encoding and incoming messages to the official messages. We also change the TLV fields added to `tx_add_input`, `tx_signatures` and `splice_locked` to match the spec version. We always write both the official and experimental TLV to updated nodes (because the experimental one is odd and will be ignored) but we drop the official TLV if our peer is using the experimental feature, because it won't understand the even TLV field. This guarantees backwards-compatibility with peers who only support the experimental feature. --- eclair-core/src/main/resources/reference.conf | 1 + .../main/scala/fr/acinq/eclair/Features.scala | 16 +++--- .../fr/acinq/eclair/channel/Commitments.scala | 14 +++-- .../fr/acinq/eclair/channel/fsm/Channel.scala | 4 +- .../fr/acinq/eclair/io/PeerConnection.scala | 27 +++++++--- .../acinq/eclair/wire/protocol/HtlcTlv.scala | 22 +++++--- .../wire/protocol/InteractiveTxTlv.scala | 12 ++++- .../protocol/LightningMessageCodecs.scala | 28 ++++++++-- .../wire/protocol/LightningMessageTypes.scala | 51 +++++++++++++++++-- .../ChannelStateTestsHelperMethods.scala | 4 +- .../acinq/eclair/io/PeerConnectionSpec.scala | 47 ++++++++++++++++- .../protocol/LightningMessageCodecsSpec.scala | 34 +++++++++++-- 12 files changed, 217 insertions(+), 43 deletions(-) diff --git a/eclair-core/src/main/resources/reference.conf b/eclair-core/src/main/resources/reference.conf index fd58844a1d..ca283b18b6 100644 --- a/eclair-core/src/main/resources/reference.conf +++ b/eclair-core/src/main/resources/reference.conf @@ -81,6 +81,7 @@ eclair { // node that you trust using override-init-features (see below). option_zeroconf = disabled keysend = disabled + option_splice = optional trampoline_payment_prototype = disabled async_payment_prototype = disabled } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Features.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Features.scala index 64beeb4303..c403cabdeb 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Features.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Features.scala @@ -264,8 +264,7 @@ object Features { val mandatory = 28 } - // TODO: this should also extend NodeFeature once the spec is finalized - case object Quiescence extends Feature with InitFeature { + case object Quiescence extends Feature with InitFeature with NodeFeature { val rfcName = "option_quiesce" val mandatory = 34 } @@ -300,6 +299,11 @@ object Features { val mandatory = 54 } + case object Splicing extends Feature with InitFeature with NodeFeature { + val rfcName = "option_splice" + val mandatory = 62 + } + // TODO: @t-bast: update feature bits once spec-ed (currently reserved here: https://github.com/lightningnetwork/lightning-rfc/issues/605) // We're not advertising these bits yet in our announcements, clients have to assume support. // This is why we haven't added them yet to `areSupported`. @@ -317,12 +321,6 @@ object Features { val mandatory = 152 } - // TODO: @pm47 custom splices implementation for phoenix, to be replaced once splices is spec-ed (currently reserved here: https://github.com/lightning/bolts/issues/605) - case object SplicePrototype extends Feature with InitFeature { - val rfcName = "splice_prototype" - val mandatory = 154 - } - val knownFeatures: Set[Feature] = Set( DataLossProtect, InitialRoutingSync, @@ -346,9 +344,9 @@ object Features { PaymentMetadata, ZeroConf, KeySend, + Splicing, TrampolinePaymentPrototype, AsyncPaymentPrototype, - SplicePrototype, ) // Features may depend on other features, as specified in Bolt 9. diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala index da1eb5c7f5..5aceab5422 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala @@ -638,9 +638,11 @@ case class Commitment(fundingTxIndex: Long, log.info(s"built remote commit number=${remoteCommit.index + 1} toLocalMsat=${spec.toLocal.toLong} toRemoteMsat=${spec.toRemote.toLong} htlc_in={} htlc_out={} feeratePerKw=${spec.commitTxFeerate} txid=${remoteCommitTx.tx.txid} fundingTxId=$fundingTxId", spec.htlcs.collect(DirectedHtlc.outgoing).map(_.id).mkString(","), spec.htlcs.collect(DirectedHtlc.incoming).map(_.id).mkString(",")) Metrics.recordHtlcsInFlight(spec, remoteCommit.spec) - val commitSig = CommitSig(params.channelId, sig, htlcSigs.toList, TlvStream(Set( - if (batchSize > 1) Some(CommitSigTlv.BatchTlv(batchSize)) else None - ).flatten[CommitSigTlv])) + val tlvs = Set( + if (batchSize > 1) Some(CommitSigTlv.BatchTlv(batchSize, fundingTxId)) else None, + if (batchSize > 1) Some(CommitSigTlv.ExperimentalBatchTlv(batchSize)) else None, + ).flatten[CommitSigTlv] + val commitSig = CommitSig(params.channelId, sig, htlcSigs.toList, TlvStream(tlvs)) val nextRemoteCommit = NextRemoteCommit(commitSig, RemoteCommit(remoteCommit.index + 1, spec, remoteCommitTx.tx.txid, remoteNextPerCommitmentPoint)) (copy(nextRemoteCommit_opt = Some(nextRemoteCommit)), commitSig) } @@ -1021,8 +1023,10 @@ case class Commitments(params: ChannelParams, } val channelKeyPath = keyManager.keyPath(params.localParams, params.channelConfig) val localPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, localCommitIndex + 1) - // Signatures are sent in order (most recent first), calling `zip` will drop trailing sigs that are for deactivated/pruned commitments. - val active1 = active.zip(commits).map { case (commitment, commit) => + val active1 = active.zipWithIndex.map { case (commitment, idx) => + // If the funding_txid isn't provided, we assume that signatures are sent in order (most recent first). + // This matches the behavior of peers who only support the experimental version of splicing. + val commit = commits.find(_.fundingTxId_opt.contains(commitment.fundingTxId)).getOrElse(commits(idx)) commitment.receiveCommit(keyManager, params, changes, localPerCommitmentPoint, commit) match { case Left(f) => return Left(f) case Right(commitment1) => commitment1 diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index e55d86d612..d254cff03e 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -851,7 +851,7 @@ 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.Splicing)) { d.spliceStatus match { case SpliceStatus.NoSplice if d.commitments.params.useQuiescence => startSingleTimer(QuiescenceTimeout.toString, QuiescenceTimeout(peer), nodeParams.channelConf.quiescenceTimeout) @@ -2526,7 +2526,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with /** For splices we will send one commit_sig per active commitments. */ private def aggregateSigs(commit: CommitSig): Option[Seq[CommitSig]] = { sigStash = sigStash :+ commit - log.debug("received sig for batch of size={}", commit.batchSize) + log.debug("received sig for batch of size={} for fundingTxId={}", commit.batchSize, commit.fundingTxId_opt) if (sigStash.size == commit.batchSize) { val sigs = sigStash sigStash = Nil diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/io/PeerConnection.scala b/eclair-core/src/main/scala/fr/acinq/eclair/io/PeerConnection.scala index be93ac38a2..7fdb50f7d1 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/io/PeerConnection.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/io/PeerConnection.scala @@ -18,7 +18,7 @@ package fr.acinq.eclair.io import akka.actor.{ActorRef, FSM, OneForOneStrategy, PoisonPill, Props, Stash, SupervisorStrategy, Terminated} import akka.event.Logging.MDC -import fr.acinq.bitcoin.scalacompat.{BlockHash, ByteVector32} +import fr.acinq.bitcoin.scalacompat.BlockHash import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.eclair.Logs.LogCategory import fr.acinq.eclair.crypto.Noise.KeyPair @@ -28,7 +28,7 @@ import fr.acinq.eclair.remote.EclairInternalsSerializer.RemoteTypes import fr.acinq.eclair.router.Router._ import fr.acinq.eclair.wire.protocol import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{FSMDiagnosticActorLogging, FeatureCompatibilityResult, Features, InitFeature, Logs, TimestampMilli, TimestampSecond} +import fr.acinq.eclair.{FSMDiagnosticActorLogging, Features, InitFeature, Logs, TimestampMilli, TimestampSecond} import scodec.Attempt import scodec.bits.ByteVector @@ -200,11 +200,20 @@ class PeerConnection(keyPair: KeyPair, conf: PeerConnection.Conf, switchboard: A stay() case Event(msg: LightningMessage, d: ConnectedData) if sender() != d.transport => // if the message doesn't originate from the transport, it is an outgoing message - d.transport forward msg + val useExperimentalSplice = d.remoteInit.features.unknown.map(_.bitIndex).contains(155) + msg match { + // If our peer is using the experimental splice version, we convert splice messages. + case msg: SpliceInit if useExperimentalSplice => d.transport forward ExperimentalSpliceInit.from(msg) + case msg: SpliceAck if useExperimentalSplice => d.transport forward ExperimentalSpliceAck.from(msg) + case msg: SpliceLocked if useExperimentalSplice => d.transport forward ExperimentalSpliceLocked.from(msg) + case msg: TxAddInput if useExperimentalSplice => d.transport forward msg.copy(tlvStream = TlvStream(msg.tlvStream.records.filterNot(_.isInstanceOf[TxAddInputTlv.SharedInputTxId]))) + case msg: CommitSig if useExperimentalSplice => d.transport forward msg.copy(tlvStream = TlvStream(msg.tlvStream.records.filterNot(_.isInstanceOf[CommitSigTlv.BatchTlv]))) + case msg: TxSignatures if useExperimentalSplice => d.transport forward msg.copy(tlvStream = TlvStream(msg.tlvStream.records.filterNot(_.isInstanceOf[TxSignaturesTlv.PreviousFundingTxSig]))) + case _ => d.transport forward msg + } msg match { // If we send any channel management message to this peer, the connection should be persistent. - case _: ChannelMessage if !d.isPersistent => - stay() using d.copy(isPersistent = true) + case _: ChannelMessage if !d.isPersistent => stay() using d.copy(isPersistent = true) case _ => stay() } @@ -337,7 +346,13 @@ class PeerConnection(keyPair: KeyPair, conf: PeerConnection.Conf, switchboard: A case Event(msg: LightningMessage, d: ConnectedData) => // we acknowledge and pass all other messages to the peer d.transport ! TransportHandler.ReadAck(msg) - d.peer ! msg + msg match { + // If our peer is using the experimental splice version, we convert splice messages. + case msg: ExperimentalSpliceInit => d.peer ! msg.toSpliceInit() + case msg: ExperimentalSpliceAck => d.peer ! msg.toSpliceAck() + case msg: ExperimentalSpliceLocked => d.peer ! msg.toSpliceLocked() + case _ => d.peer ! msg + } stay() case Event(readAck: TransportHandler.ReadAck, d: ConnectedData) => diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/HtlcTlv.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/HtlcTlv.scala index 30e4895534..b95d7fb43f 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/HtlcTlv.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/HtlcTlv.scala @@ -17,6 +17,7 @@ package fr.acinq.eclair.wire.protocol import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey +import fr.acinq.bitcoin.scalacompat.TxId import fr.acinq.eclair.UInt64 import fr.acinq.eclair.wire.protocol.CommonCodecs._ import fr.acinq.eclair.wire.protocol.TlvCodecs.{tlvField, tlvStream, tu16} @@ -66,16 +67,25 @@ object UpdateFailMalformedHtlcTlv { sealed trait CommitSigTlv extends Tlv object CommitSigTlv { + /** + * While a splice is ongoing and not locked, we have multiple valid commitments. + * We send one [[CommitSig]] message for each valid commitment. + * + * @param size the number of [[CommitSig]] messages in the batch. + * @param fundingTxId the funding transaction spent by this commitment. + */ + case class BatchTlv(size: Int, fundingTxId: TxId) extends CommitSigTlv - /** @param size the number of [[CommitSig]] messages in the batch */ - case class BatchTlv(size: Int) extends CommitSigTlv + private val batchTlv: Codec[BatchTlv] = tlvField(uint16 :: txIdAsHash) - object BatchTlv { - val codec: Codec[BatchTlv] = tlvField(tu16) - } + /** Similar to [[BatchTlv]] for peers who only support the experimental version of splicing. */ + case class ExperimentalBatchTlv(size: Int) extends CommitSigTlv + + private val experimentalBatchTlv: Codec[ExperimentalBatchTlv] = tlvField(tu16) val commitSigTlvCodec: Codec[TlvStream[CommitSigTlv]] = tlvStream(discriminated[CommitSigTlv].by(varint) - .typecase(UInt64(0x47010005), BatchTlv.codec) + .typecase(UInt64(0), batchTlv) + .typecase(UInt64(0x47010005), experimentalBatchTlv) ) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/InteractiveTxTlv.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/InteractiveTxTlv.scala index 96696d8356..97c77e5ae5 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/InteractiveTxTlv.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/InteractiveTxTlv.scala @@ -33,9 +33,13 @@ object TxAddInputTlv { /** When doing a splice, the initiator must provide the previous funding txId instead of the whole transaction. */ case class SharedInputTxId(txId: TxId) extends TxAddInputTlv + /** Same as [[SharedInputTxId]] for peers who only support the experimental version of splicing. */ + case class ExperimentalSharedInputTxId(txId: TxId) extends TxAddInputTlv + val txAddInputTlvCodec: Codec[TlvStream[TxAddInputTlv]] = tlvStream(discriminated[TxAddInputTlv].by(varint) // Note that we actually encode as a tx_hash to be consistent with other lightning messages. - .typecase(UInt64(1105), tlvField(txIdAsHash.as[SharedInputTxId])) + .typecase(UInt64(0), tlvField(txIdAsHash.as[SharedInputTxId])) + .typecase(UInt64(1105), tlvField(txIdAsHash.as[ExperimentalSharedInputTxId])) ) } @@ -69,8 +73,12 @@ object TxSignaturesTlv { /** When doing a splice, each peer must provide their signature for the previous 2-of-2 funding output. */ case class PreviousFundingTxSig(sig: ByteVector64) extends TxSignaturesTlv + /** Same as [[PreviousFundingTxSig]] for peers who only support the experimental version of splicing. */ + case class ExperimentalPreviousFundingTxSig(sig: ByteVector64) extends TxSignaturesTlv + val txSignaturesTlvCodec: Codec[TlvStream[TxSignaturesTlv]] = tlvStream(discriminated[TxSignaturesTlv].by(varint) - .typecase(UInt64(601), tlvField(bytes64.as[PreviousFundingTxSig])) + .typecase(UInt64(0), tlvField(bytes64.as[PreviousFundingTxSig])) + .typecase(UInt64(601), tlvField(bytes64.as[ExperimentalPreviousFundingTxSig])) ) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageCodecs.scala index f7790558a0..d599ebc7e2 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageCodecs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageCodecs.scala @@ -410,17 +410,36 @@ object LightningMessageCodecs { ("fundingPubkey" | publicKey) :: ("tlvStream" | SpliceInitTlv.spliceInitTlvCodec)).as[SpliceInit] + val experimentalSpliceInitCodec: Codec[ExperimentalSpliceInit] = ( + ("channelId" | bytes32) :: + ("fundingContribution" | satoshiSigned) :: + ("feerate" | feeratePerKw) :: + ("lockTime" | uint32) :: + ("fundingPubkey" | publicKey) :: + ("tlvStream" | SpliceInitTlv.spliceInitTlvCodec)).as[ExperimentalSpliceInit] + val spliceAckCodec: Codec[SpliceAck] = ( ("channelId" | bytes32) :: ("fundingContribution" | satoshiSigned) :: ("fundingPubkey" | publicKey) :: ("tlvStream" | SpliceAckTlv.spliceAckTlvCodec)).as[SpliceAck] + val experimentalSpliceAckCodec: Codec[ExperimentalSpliceAck] = ( + ("channelId" | bytes32) :: + ("fundingContribution" | satoshiSigned) :: + ("fundingPubkey" | publicKey) :: + ("tlvStream" | SpliceAckTlv.spliceAckTlvCodec)).as[ExperimentalSpliceAck] + val spliceLockedCodec: Codec[SpliceLocked] = ( ("channelId" | bytes32) :: ("fundingTxHash" | txIdAsHash) :: ("tlvStream" | SpliceLockedTlv.spliceLockedTlvCodec)).as[SpliceLocked] + val experimentalSpliceLockedCodec: Codec[ExperimentalSpliceLocked] = ( + ("channelId" | bytes32) :: + ("fundingTxHash" | txIdAsHash) :: + ("tlvStream" | SpliceLockedTlv.spliceLockedTlvCodec)).as[ExperimentalSpliceLocked] + val stfuCodec: Codec[Stfu] = ( ("channelId" | bytes32) :: ("initiator" | byte.xmap[Boolean](b => b != 0, b => if (b) 1 else 0))).as[Stfu] @@ -459,6 +478,9 @@ object LightningMessageCodecs { .typecase(72, txInitRbfCodec) .typecase(73, txAckRbfCodec) .typecase(74, txAbortCodec) + .typecase(77, spliceLockedCodec) + .typecase(80, spliceInitCodec) + .typecase(81, spliceAckCodec) .typecase(128, updateAddHtlcCodec) .typecase(130, updateFulfillHtlcCodec) .typecase(131, updateFailHtlcCodec) @@ -480,9 +502,9 @@ object LightningMessageCodecs { // NB: blank lines to minimize merge conflicts // - .typecase(37000, spliceInitCodec) - .typecase(37002, spliceAckCodec) - .typecase(37004, spliceLockedCodec) + .typecase(37000, experimentalSpliceInitCodec) + .typecase(37002, experimentalSpliceAckCodec) + .typecase(37004, experimentalSpliceLockedCodec) // // diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala index c0d43d2749..4dabd75781 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala @@ -90,11 +90,16 @@ case class TxAddInput(channelId: ByteVector32, sequence: Long, tlvStream: TlvStream[TxAddInputTlv] = TlvStream.empty) extends InteractiveTxConstructionMessage with HasChannelId with HasSerialId { val sharedInput_opt: Option[OutPoint] = tlvStream.get[TxAddInputTlv.SharedInputTxId].map(i => OutPoint(i.txId, previousTxOutput)) + .orElse(tlvStream.get[TxAddInputTlv.ExperimentalSharedInputTxId].map(i => OutPoint(i.txId, previousTxOutput))) } object TxAddInput { def apply(channelId: ByteVector32, serialId: UInt64, sharedInput: OutPoint, sequence: Long): TxAddInput = { - TxAddInput(channelId, serialId, None, sharedInput.index, sequence, TlvStream(TxAddInputTlv.SharedInputTxId(sharedInput.txid))) + val tlvs = Set[TxAddInputTlv]( + TxAddInputTlv.SharedInputTxId(sharedInput.txid), + TxAddInputTlv.ExperimentalSharedInputTxId(sharedInput.txid), + ) + TxAddInput(channelId, serialId, None, sharedInput.index, sequence, TlvStream(tlvs)) } } @@ -120,11 +125,16 @@ case class TxSignatures(channelId: ByteVector32, witnesses: Seq[ScriptWitness], tlvStream: TlvStream[TxSignaturesTlv] = TlvStream.empty) extends InteractiveTxMessage with HasChannelId { val previousFundingTxSig_opt: Option[ByteVector64] = tlvStream.get[TxSignaturesTlv.PreviousFundingTxSig].map(_.sig) + .orElse(tlvStream.get[TxSignaturesTlv.ExperimentalPreviousFundingTxSig].map(_.sig)) } object TxSignatures { def apply(channelId: ByteVector32, tx: Transaction, witnesses: Seq[ScriptWitness], previousFundingSig_opt: Option[ByteVector64]): TxSignatures = { - TxSignatures(channelId, tx.txid, witnesses, TlvStream(previousFundingSig_opt.map(TxSignaturesTlv.PreviousFundingTxSig).toSet[TxSignaturesTlv])) + val tlvs = Set( + previousFundingSig_opt.map(TxSignaturesTlv.PreviousFundingTxSig), + previousFundingSig_opt.map(TxSignaturesTlv.ExperimentalPreviousFundingTxSig), + ).flatten[TxSignaturesTlv] + TxSignatures(channelId, tx.txid, witnesses, TlvStream(tlvs)) } } @@ -311,6 +321,19 @@ object SpliceInit { } } +case class ExperimentalSpliceInit(channelId: ByteVector32, + fundingContribution: Satoshi, + feerate: FeeratePerKw, + lockTime: Long, + fundingPubKey: PublicKey, + tlvStream: TlvStream[SpliceInitTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId { + def toSpliceInit(): SpliceInit = SpliceInit(channelId, fundingContribution, feerate, lockTime, fundingPubKey, tlvStream) +} + +object ExperimentalSpliceInit { + def from(msg: SpliceInit): ExperimentalSpliceInit = ExperimentalSpliceInit(msg.channelId, msg.fundingContribution, msg.feerate, msg.lockTime, msg.fundingPubKey, msg.tlvStream) +} + case class SpliceAck(channelId: ByteVector32, fundingContribution: Satoshi, fundingPubKey: PublicKey, @@ -329,11 +352,32 @@ object SpliceAck { } } +case class ExperimentalSpliceAck(channelId: ByteVector32, + fundingContribution: Satoshi, + fundingPubKey: PublicKey, + tlvStream: TlvStream[SpliceAckTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId { + def toSpliceAck(): SpliceAck = SpliceAck(channelId, fundingContribution, fundingPubKey, tlvStream) +} + +object ExperimentalSpliceAck { + def from(msg: SpliceAck): ExperimentalSpliceAck = ExperimentalSpliceAck(msg.channelId, msg.fundingContribution, msg.fundingPubKey, msg.tlvStream) +} + case class SpliceLocked(channelId: ByteVector32, fundingTxId: TxId, tlvStream: TlvStream[SpliceLockedTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId { } +case class ExperimentalSpliceLocked(channelId: ByteVector32, + fundingTxId: TxId, + tlvStream: TlvStream[SpliceLockedTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId { + def toSpliceLocked(): SpliceLocked = SpliceLocked(channelId, fundingTxId, tlvStream) +} + +object ExperimentalSpliceLocked { + def from(msg: SpliceLocked): ExperimentalSpliceLocked = ExperimentalSpliceLocked(msg.channelId, msg.fundingTxId, msg.tlvStream) +} + case class Shutdown(channelId: ByteVector32, scriptPubKey: ByteVector, tlvStream: TlvStream[ShutdownTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId with ForbiddenMessageDuringSplice @@ -394,7 +438,8 @@ case class CommitSig(channelId: ByteVector32, signature: ByteVector64, htlcSignatures: List[ByteVector64], tlvStream: TlvStream[CommitSigTlv] = TlvStream.empty) extends HtlcMessage with HasChannelId { - val batchSize: Int = tlvStream.get[CommitSigTlv.BatchTlv].map(_.size).getOrElse(1) + val fundingTxId_opt: Option[TxId] = tlvStream.get[CommitSigTlv.BatchTlv].map(_.fundingTxId) + val batchSize: Int = tlvStream.get[CommitSigTlv.BatchTlv].map(_.size).orElse(tlvStream.get[CommitSigTlv.ExperimentalBatchTlv].map(_.size)).getOrElse(1) } case class RevokeAndAck(channelId: ByteVector32, diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala index ebc97c182a..d32838a674 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala @@ -190,7 +190,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually { .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.ZeroConf))(_.updated(Features.ZeroConf, FeatureSupport.Optional)) .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.ScidAlias))(_.updated(Features.ScidAlias, FeatureSupport.Optional)) .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.DualFunding))(_.updated(Features.DualFunding, FeatureSupport.Optional)) - .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.Splicing))(_.updated(Features.SplicePrototype, FeatureSupport.Optional)) + .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.Splicing))(_.updated(Features.Splicing, FeatureSupport.Optional)) .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.Quiescence))(_.updated(Features.Quiescence, FeatureSupport.Optional)) .initFeatures() val bobInitFeatures = Bob.nodeParams.features @@ -204,7 +204,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually { .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.ZeroConf))(_.updated(Features.ZeroConf, FeatureSupport.Optional)) .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.ScidAlias))(_.updated(Features.ScidAlias, FeatureSupport.Optional)) .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.DualFunding))(_.updated(Features.DualFunding, FeatureSupport.Optional)) - .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.Splicing))(_.updated(Features.SplicePrototype, FeatureSupport.Optional)) + .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.Splicing))(_.updated(Features.Splicing, FeatureSupport.Optional)) .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.Quiescence))(_.updated(Features.Quiescence, FeatureSupport.Optional)) .initFeatures() diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerConnectionSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerConnectionSpec.scala index 06ae1ba234..c9367de721 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerConnectionSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerConnectionSpec.scala @@ -19,10 +19,11 @@ package fr.acinq.eclair.io import akka.actor.PoisonPill import akka.testkit.{TestFSMRef, TestProbe} import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} -import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32} +import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, OutPoint, SatoshiLong, Transaction, TxId} import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional} -import fr.acinq.eclair.Features.{BasicMultiPartPayment, ChannelRangeQueries, PaymentSecret, StaticRemoteKey, VariableLengthOnion} +import fr.acinq.eclair.Features._ import fr.acinq.eclair.TestConstants._ +import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.crypto.TransportHandler import fr.acinq.eclair.io.Peer.ConnectionDown import fr.acinq.eclair.message.OnionMessages.{Recipient, buildMessage} @@ -496,5 +497,47 @@ class PeerConnectionSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wi } } + test("convert experimental splice messages") { f => + import f._ + val remoteInit = protocol.Init(Bob.nodeParams.features.initFeatures().copy(unknown = Set(UnknownFeature(155)))) + connect(nodeParams, remoteNodeId, switchboard, router, connection, transport, peerConnection, peer, remoteInit) + + val spliceInit = SpliceInit(randomBytes32(), 100_000 sat, FeeratePerKw(5000 sat), 0, randomKey().publicKey) + val spliceAck = SpliceAck(randomBytes32(), 50_000 sat, randomKey().publicKey) + val spliceLocked = SpliceLocked(randomBytes32(), TxId(randomBytes32())) + + // Outgoing messages use the experimental version of splicing. + peer.send(peerConnection, spliceInit) + transport.expectMsg(ExperimentalSpliceInit.from(spliceInit)) + peer.send(peerConnection, spliceAck) + transport.expectMsg(ExperimentalSpliceAck.from(spliceAck)) + peer.send(peerConnection, spliceLocked) + transport.expectMsg(ExperimentalSpliceLocked.from(spliceLocked)) + + // Incoming messages are converted from their experimental version. + transport.send(peerConnection, ExperimentalSpliceInit.from(spliceInit)) + peer.expectMsg(spliceInit) + transport.expectMsgType[TransportHandler.ReadAck] + transport.send(peerConnection, ExperimentalSpliceAck.from(spliceAck)) + peer.expectMsg(spliceAck) + transport.expectMsgType[TransportHandler.ReadAck] + transport.send(peerConnection, ExperimentalSpliceLocked.from(spliceLocked)) + peer.expectMsg(spliceLocked) + transport.expectMsgType[TransportHandler.ReadAck] + + // Incompatible TLVs are dropped when sending messages to peers using the experimental version. + val txAddInput = TxAddInput(randomBytes32(), UInt64(0), OutPoint(TxId(randomBytes32()), 3), 0) + assert(txAddInput.tlvStream.get[TxAddInputTlv.SharedInputTxId].nonEmpty) + peer.send(peerConnection, txAddInput) + assert(transport.expectMsgType[TxAddInput].tlvStream.get[TxAddInputTlv.SharedInputTxId].isEmpty) + val txSignatures = TxSignatures(randomBytes32(), Transaction(2, Nil, Nil, 0), Nil, Some(randomBytes64())) + assert(txSignatures.tlvStream.get[TxSignaturesTlv.PreviousFundingTxSig].nonEmpty) + peer.send(peerConnection, txSignatures) + assert(transport.expectMsgType[TxSignatures].tlvStream.get[TxSignaturesTlv.PreviousFundingTxSig].isEmpty) + val commitSig = CommitSig(randomBytes32(), randomBytes64(), Nil, TlvStream(CommitSigTlv.BatchTlv(2, TxId(randomBytes32())), CommitSigTlv.ExperimentalBatchTlv(2))) + peer.send(peerConnection, commitSig) + assert(transport.expectMsgType[CommitSig].tlvStream.get[CommitSigTlv.BatchTlv].isEmpty) + } + } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/LightningMessageCodecsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/LightningMessageCodecsSpec.scala index 83ae806361..6a2b9f7f3f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/LightningMessageCodecsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/protocol/LightningMessageCodecsSpec.scala @@ -18,7 +18,7 @@ package fr.acinq.eclair.wire.protocol import com.google.common.base.Charsets import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} -import fr.acinq.bitcoin.scalacompat.{Block, BlockHash, ByteVector32, ByteVector64, OutPoint, SatoshiLong, ScriptWitness, Transaction, TxId} +import fr.acinq.bitcoin.scalacompat.{Block, BlockHash, ByteVector32, ByteVector64, OutPoint, SatoshiLong, ScriptWitness, Transaction, TxHash, TxId} import fr.acinq.eclair.FeatureSupport.Optional import fr.acinq.eclair.Features.DataLossProtect import fr.acinq.eclair.TestUtils.randomTxId @@ -138,6 +138,7 @@ class LightningMessageCodecsSpec extends AnyFunSuite { hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"fe47010000 07 bbbbbbbbbbbbbb" -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream[ChannelReestablishTlv](Set.empty[ChannelReestablishTlv], Set(GenericTlv(tlvTag, hex"bbbbbbbbbbbbbb")))), hex"0084" ++ channelId ++ signature ++ hex"0000" -> CommitSig(channelId, signature, Nil), + hex"0084" ++ channelId ++ signature ++ hex"0000" ++ hex"00 22 0003" ++ txId.value.reverse -> CommitSig(channelId, signature, Nil, TlvStream(CommitSigTlv.BatchTlv(3, txId))), hex"0084" ++ channelId ++ signature ++ hex"0000 fe47010000 00" -> CommitSig(channelId, signature, Nil, TlvStream[CommitSigTlv](Set.empty[CommitSigTlv], Set(GenericTlv(tlvTag, ByteVector.empty)))), hex"0084" ++ channelId ++ signature ++ hex"0000 fe47010000 07 cccccccccccccc" -> CommitSig(channelId, signature, Nil, TlvStream[CommitSigTlv](Set.empty[CommitSigTlv], Set(GenericTlv(tlvTag, hex"cccccccccccccc")))), @@ -183,7 +184,7 @@ class LightningMessageCodecsSpec extends AnyFunSuite { TxAddInput(channelId1, UInt64(561), Some(tx1), 1, 5) -> hex"0042 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000000231 00f7 020000000001014ade359c5deb7c1cde2e94f401854658f97d7fa31c17ce9a831db253120a0a410100000017160014eb9a5bd79194a23d19d6ec473c768fb74f9ed32cffffffff021ca408000000000017a914946118f24bb7b37d5e9e39579e4a411e70f5b6a08763e703000000000017a9143638b2602d11f934c04abc6adb1494f69d1f14af8702473044022059ddd943b399211e4266a349f26b3289979e29f9b067792c6cfa8cc5ae25f44602204d627a5a5b603d0562e7969011fb3d64908af90a3ec7c876eaa9baf61e1958af012102f5188df1da92ed818581c29778047800ed6635788aa09d9469f7d17628f7323300000000 00000001 00000005", TxAddInput(channelId2, UInt64(0), Some(tx2), 2, 0) -> hex"0042 bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb 0000000000000000 0100 0200000000010142180a8812fc79a3da7fb2471eff3e22d7faee990604c2ba7f2fc8dfb15b550a0200000000feffffff030f241800000000001976a9146774040642a78ca3b8b395e70f8391b21ec026fc88ac4a155801000000001600148d2e0b57adcb8869e603fd35b5179caf053361253b1d010000000000160014e032f4f4b9f8611df0d30a20648c190c263bbc33024730440220506005aa347f5b698542cafcb4f1a10250aeb52a609d6fd67ef68f9c1a5d954302206b9bb844343f4012bccd9d08a0f5430afb9549555a3252e499be7df97aae477a012103976d6b3eea3de4b056cd88cdfd50a22daf121e0fb5c6e45ba0f40e1effbd275a00000000 00000002 00000000", TxAddInput(channelId1, UInt64(561), Some(tx1), 0, 0) -> hex"0042 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000000231 00f7 020000000001014ade359c5deb7c1cde2e94f401854658f97d7fa31c17ce9a831db253120a0a410100000017160014eb9a5bd79194a23d19d6ec473c768fb74f9ed32cffffffff021ca408000000000017a914946118f24bb7b37d5e9e39579e4a411e70f5b6a08763e703000000000017a9143638b2602d11f934c04abc6adb1494f69d1f14af8702473044022059ddd943b399211e4266a349f26b3289979e29f9b067792c6cfa8cc5ae25f44602204d627a5a5b603d0562e7969011fb3d64908af90a3ec7c876eaa9baf61e1958af012102f5188df1da92ed818581c29778047800ed6635788aa09d9469f7d17628f7323300000000 00000000 00000000", - TxAddInput(channelId1, UInt64(561), OutPoint(tx1, 1), 5) -> hex"0042 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000000231 0000 00000001 00000005 fd0451201f2ec025a33e39ef8e177afcdc1adc855bf128dc906182255aeb64efa825f106", + TxAddInput(channelId1, UInt64(561), OutPoint(tx1, 1), 5) -> hex"0042 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000000231 0000 00000001 00000005 00201f2ec025a33e39ef8e177afcdc1adc855bf128dc906182255aeb64efa825f106 fd0451201f2ec025a33e39ef8e177afcdc1adc855bf128dc906182255aeb64efa825f106", TxAddOutput(channelId1, UInt64(1105), 2047 sat, hex"00149357014afd0ccd265658c9ae81efa995e771f472") -> hex"0043 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000000451 00000000000007ff 0016 00149357014afd0ccd265658c9ae81efa995e771f472", TxAddOutput(channelId1, UInt64(1105), 2047 sat, hex"00149357014afd0ccd265658c9ae81efa995e771f472", TlvStream(Set.empty[TxAddOutputTlv], Set(GenericTlv(UInt64(301), hex"2a")))) -> hex"0043 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000000451 00000000000007ff 0016 00149357014afd0ccd265658c9ae81efa995e771f472 fd012d012a", TxRemoveInput(channelId2, UInt64(561)) -> hex"0044 bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb 0000000000000231", @@ -192,7 +193,7 @@ class LightningMessageCodecsSpec extends AnyFunSuite { TxComplete(channelId1, TlvStream(Set.empty[TxCompleteTlv], Set(GenericTlv(UInt64(231), hex"deadbeef"), GenericTlv(UInt64(507), hex"")))) -> hex"0046 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa e704deadbeef fd01fb00", TxSignatures(channelId1, tx2, Seq(ScriptWitness(Seq(hex"68656c6c6f2074686572652c2074686973206973206120626974636f6e212121", hex"82012088a820add57dfe5277079d069ca4ad4893c96de91f88ffb981fdc6a2a34d5336c66aff87")), ScriptWitness(Seq(hex"304402207de9ba56bb9f641372e805782575ee840a899e61021c8b1572b3ec1d5b5950e9022069e9ba998915dae193d3c25cb89b5e64370e6a3a7755e7f31cf6d7cbc2a49f6d01", hex"034695f5b7864c580bf11f9f8cb1a94eb336f2ce9ef872d2ae1a90ee276c772484"))), None) -> hex"0047 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa fc7aa8845f192959202c1b7ff704e7cbddded463c05e844676a94ccb4bed69f1 0002 004a 022068656c6c6f2074686572652c2074686973206973206120626974636f6e2121212782012088a820add57dfe5277079d069ca4ad4893c96de91f88ffb981fdc6a2a34d5336c66aff87 006b 0247304402207de9ba56bb9f641372e805782575ee840a899e61021c8b1572b3ec1d5b5950e9022069e9ba998915dae193d3c25cb89b5e64370e6a3a7755e7f31cf6d7cbc2a49f6d0121034695f5b7864c580bf11f9f8cb1a94eb336f2ce9ef872d2ae1a90ee276c772484", TxSignatures(channelId2, tx1, Nil, None) -> hex"0047 bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb 1f2ec025a33e39ef8e177afcdc1adc855bf128dc906182255aeb64efa825f106 0000", - TxSignatures(channelId2, tx1, Nil, Some(signature)) -> hex"0047 bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb 1f2ec025a33e39ef8e177afcdc1adc855bf128dc906182255aeb64efa825f106 0000 fd0259 40 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb", + TxSignatures(channelId2, tx1, Nil, Some(signature)) -> hex"0047 bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb 1f2ec025a33e39ef8e177afcdc1adc855bf128dc906182255aeb64efa825f106 0000 0040aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb fd025940aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb", TxInitRbf(channelId1, 8388607, FeeratePerKw(4000 sat)) -> hex"0048 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 007fffff 00000fa0", TxInitRbf(channelId1, 0, FeeratePerKw(4000 sat), 1_500_000 sat, requireConfirmedInputs = true) -> hex"0048 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000 00000fa0 0008000000000016e360 0200", TxInitRbf(channelId1, 0, FeeratePerKw(4000 sat), 0 sat, requireConfirmedInputs = false) -> hex"0048 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000 00000fa0 00080000000000000000", @@ -360,6 +361,33 @@ class LightningMessageCodecsSpec extends AnyFunSuite { } } + test("encode/decode splice messages") { + val channelId = ByteVector32(hex"aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa") + val fundingTxId = TxId(TxHash(ByteVector32(hex"24e1b2c94c4e734dd5b9c5f3c910fbb6b3b436ced6382c7186056a5a23f14566"))) + val fundingPubkey = PublicKey(hex"0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798") + val testCases = Seq( + // @formatter:off + SpliceInit(channelId, 100_000 sat, FeeratePerKw(2500 sat), 100, fundingPubkey) -> hex"0050 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000186a0 000009c4 00000064 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798", + SpliceInit(channelId, 150_000 sat, 100, FeeratePerKw(2500 sat), fundingPubkey, 25_000_000 msat, requireConfirmedInputs = false) -> hex"0050 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000249f0 000009c4 00000064 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fe4700000704017d7840", + SpliceInit(channelId, 150_000 sat, 100, FeeratePerKw(2500 sat), fundingPubkey, 25_000_000 msat, requireConfirmedInputs = true) -> hex"0050 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000249f0 000009c4 00000064 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 0200 fe4700000704017d7840", + SpliceInit(channelId, 0 sat, FeeratePerKw(500 sat), 0, fundingPubkey) -> hex"0050 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000000000 000001f4 00000000 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798", + SpliceInit(channelId, (-50_000).sat, FeeratePerKw(500 sat), 0, fundingPubkey) -> hex"0050 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa ffffffffffff3cb0 000001f4 00000000 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798", + SpliceAck(channelId, 25_000 sat, fundingPubkey) -> hex"0051 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000061a8 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798", + SpliceAck(channelId, 40_000 sat, fundingPubkey, 10_000_000 msat, requireConfirmedInputs = false) -> hex"0051 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000009c40 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fe4700000703989680", + SpliceAck(channelId, 40_000 sat, fundingPubkey, 10_000_000 msat, requireConfirmedInputs = true) -> hex"0051 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000009c40 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 0200 fe4700000703989680", + SpliceAck(channelId, 0.sat, fundingPubkey) -> hex"0051 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000000000 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798", + SpliceAck(channelId, (-25_000).sat, fundingPubkey) -> hex"0051 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa ffffffffffff9e58 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798", + SpliceLocked(channelId, fundingTxId) -> hex"004d aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 24e1b2c94c4e734dd5b9c5f3c910fbb6b3b436ced6382c7186056a5a23f14566", + // @formatter:on + ) + testCases.foreach { case (msg, bin) => + val decoded = lightningMessageCodec.decode(bin.bits).require.value + assert(decoded == msg) + val encoded = lightningMessageCodec.encode(msg).require.bytes + assert(encoded == bin) + } + } + test("encode/decode closing_signed") { val defaultSig = ByteVector64(hex"01010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101") val testCases = Seq( From 64b4879df5b8dd4dcc238e255837faaa886eb074 Mon Sep 17 00:00:00 2001 From: t-bast Date: Mon, 22 Jul 2024 16:36:29 +0200 Subject: [PATCH 2/8] Remove support for splicing without quiescence We initially supported splicing with a poor man's quiescence, where we allowed splice messages if the commitments were already quiescent. We've shipped support for quiescence since then, which means that new even nodes relying on experimental splicing should support quiescence. We can thus remove support for the non-quiescent version. --- .../fr/acinq/eclair/channel/Commitments.scala | 5 +- .../fr/acinq/eclair/channel/fsm/Channel.scala | 96 +++---- .../scala/fr/acinq/eclair/TestConstants.scala | 8 +- .../ChannelStateTestsHelperMethods.scala | 10 +- .../states/e/NormalQuiescentStateSpec.scala | 2 +- .../states/e/NormalSplicesStateSpec.scala | 272 +++++------------- 6 files changed, 120 insertions(+), 273 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala index 5aceab5422..d6687fb5e6 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala @@ -133,9 +133,6 @@ 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 { @@ -824,7 +821,7 @@ case class Commitments(params: ChannelParams, 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 isQuiescent: Boolean = (params.useQuiescence || active.head.hasNoPendingHtlcs) && localIsQuiescent && remoteIsQuiescent + def isQuiescent: Boolean = 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) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index d254cff03e..f600579f31 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -853,21 +853,13 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with case Event(cmd: CMD_SPLICE, d: DATA_NORMAL) => if (d.commitments.params.remoteParams.initFeatures.hasFeature(Features.Splicing)) { d.spliceStatus match { - case SpliceStatus.NoSplice if d.commitments.params.useQuiescence => + 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) } else { stay() using d.copy(spliceStatus = SpliceStatus.QuiescenceRequested(cmd)) } - case SpliceStatus.NoSplice if !d.commitments.params.useQuiescence => - initiateSplice(cmd, d) match { - case Left(f) => - cmd.replyTo ! RES_FAILURE(cmd, f) - stay() - case Right(spliceInit) => - stay() using d.copy(spliceStatus = SpliceStatus.SpliceRequested(cmd, spliceInit)) sending spliceInit - } case _ => log.warning("cannot initiate splice, another one is already in progress") cmd.replyTo ! RES_FAILURE(cmd, InvalidSpliceAlreadyInProgress(d.channelId)) @@ -885,62 +877,53 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with stay() case Event(msg: Stfu, d: DATA_NORMAL) => - if (d.commitments.params.useQuiescence) { - if (d.commitments.remoteIsQuiescent) { - d.spliceStatus match { - case SpliceStatus.NoSplice => - startSingleTimer(QuiescenceTimeout.toString, QuiescenceTimeout(peer), nodeParams.channelConf.quiescenceTimeout) - 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)) - } - case SpliceStatus.QuiescenceRequested(cmd) => - // 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)) + if (d.commitments.remoteIsQuiescent) { + d.spliceStatus match { + case SpliceStatus.NoSplice => + startSingleTimer(QuiescenceTimeout.toString, QuiescenceTimeout(peer), nodeParams.channelConf.quiescenceTimeout) + 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)) - case SpliceStatus.InitiatorQuiescent(cmd) => - // if both sides send stfu at the same time, the quiescence initiator is the channel opener - if (!msg.initiator || d.commitments.params.localParams.isChannelOpener) { - initiateSplice(cmd, d) match { - case Left(f) => - cmd.replyTo ! RES_FAILURE(cmd, f) - context.system.scheduler.scheduleOnce(2 second, peer, Peer.Disconnect(remoteNodeId)) - stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending Warning(d.channelId, f.getMessage) - case Right(spliceInit) => - stay() using d.copy(spliceStatus = SpliceStatus.SpliceRequested(cmd, spliceInit)) sending spliceInit - } - } 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)) - stay() using d.copy(spliceStatus = SpliceStatus.NonInitiatorQuiescent) + } + case SpliceStatus.QuiescenceRequested(cmd) => + // 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 + if (!msg.initiator || d.commitments.params.localParams.isChannelOpener) { + initiateSplice(cmd, d) match { + case Left(f) => + cmd.replyTo ! RES_FAILURE(cmd, f) + context.system.scheduler.scheduleOnce(2 second, peer, Peer.Disconnect(remoteNodeId)) + stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending Warning(d.channelId, f.getMessage) + case Right(spliceInit) => + stay() using d.copy(spliceStatus = SpliceStatus.SpliceRequested(cmd, spliceInit)) sending spliceInit } - case _ => - log.warning("ignoring duplicate stfu") - stay() - } - } else { - log.warning("our peer sent stfu but is not quiescent") - // 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)) - stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending Warning(d.channelId, InvalidSpliceNotQuiescent(d.channelId).getMessage) + } 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)) + stay() using d.copy(spliceStatus = SpliceStatus.NonInitiatorQuiescent) + } + case _ => + log.warning("ignoring duplicate stfu") + stay() } } else { - log.warning("ignoring stfu because both peers do not advertise quiescence") - stay() + log.warning("our peer sent stfu but is not quiescent") + // 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)) + stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending Warning(d.channelId, InvalidSpliceNotQuiescent(d.channelId).getMessage) } case Event(_: QuiescenceTimeout, d: DATA_NORMAL) => handleQuiescenceTimeout(d) - case Event(_: SpliceInit, d: DATA_NORMAL) if d.spliceStatus == SpliceStatus.NoSplice && d.commitments.params.useQuiescence => - log.info("rejecting splice attempt: quiescence not negotiated") - stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceNotQuiescent(d.channelId).getMessage) - case Event(msg: SpliceInit, d: DATA_NORMAL) => d.spliceStatus match { - case SpliceStatus.NoSplice | SpliceStatus.NonInitiatorQuiescent => + case SpliceStatus.NonInitiatorQuiescent => if (!d.commitments.isQuiescent) { log.info("rejecting splice request: channel not quiescent") stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceNotQuiescent(d.channelId).getMessage) @@ -981,6 +964,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with txBuilder ! InteractiveTxBuilder.Start(self) stay() using d.copy(spliceStatus = SpliceStatus.SpliceInProgress(cmd_opt = None, sessionId, txBuilder, remoteCommitSig = None)) sending spliceAck } + case SpliceStatus.NoSplice => + log.info("rejecting splice 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 splice attempt: our previous tx_abort was not acked") stay() sending Warning(d.channelId, InvalidSpliceTxAbortNotAcked(d.channelId).getMessage) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala index 0b0483b437..6f520c1fb6 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala @@ -102,7 +102,9 @@ object TestConstants { Wumbo -> Optional, PaymentMetadata -> Optional, RouteBlinding -> Optional, - StaticRemoteKey -> Mandatory + StaticRemoteKey -> Mandatory, + Quiescence -> Optional, + Splicing -> Optional, ), unknown = Set(UnknownFeature(TestFeature.optional)) ), @@ -274,7 +276,9 @@ object TestConstants { PaymentMetadata -> Optional, RouteBlinding -> Optional, StaticRemoteKey -> Mandatory, - AnchorOutputsZeroFeeHtlcTx -> Optional + AnchorOutputsZeroFeeHtlcTx -> Optional, + Quiescence -> Optional, + Splicing -> Optional, ), pluginParams = Nil, overrideInitFeatures = Map.empty, diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala index d32838a674..97afafb6a0 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala @@ -51,8 +51,6 @@ object ChannelStateTestsTags { val DisableWumbo = "disable_wumbo" /** If set, channels will use option_dual_fund. */ val DualFunding = "dual_funding" - /** If set, peers will support splicing. */ - val Splicing = "splicing" /** If set, channels will use option_static_remotekey. */ val StaticRemoteKey = "static_remotekey" /** If set, channels will use option_anchor_outputs. */ @@ -91,8 +89,6 @@ object ChannelStateTestsTags { val RejectRbfAttempts = "reject_rbf_attempts" /** If set, the non-initiator will require a 1-block delay between RBF attempts. */ val DelayRbfAttempts = "delay_rbf_attempts" - /** If set, peers will support the quiesce protocol. */ - val Quiescence = "quiescence" /** If set, channels will adapt their max HTLC amount to the available balance */ val AdaptMaxHtlcAmount = "adapt-max-htlc-amount" } @@ -163,7 +159,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually { .modify(_.channelConf.balanceThresholds).setToIf(tags.contains(ChannelStateTestsTags.AdaptMaxHtlcAmount))(Seq(Channel.BalanceThreshold(1_000 sat, 0 sat), Channel.BalanceThreshold(5_000 sat, 1_000 sat), Channel.BalanceThreshold(10_000 sat, 5_000 sat))) val wallet = wallet_opt match { case Some(wallet) => wallet - case None => if (tags.contains(ChannelStateTestsTags.DualFunding) || tags.contains(ChannelStateTestsTags.Splicing)) new SingleKeyOnChainWallet() else new DummyOnChainWallet() + case None => if (tags.contains(ChannelStateTestsTags.DualFunding)) new SingleKeyOnChainWallet() else new DummyOnChainWallet() } val alice: TestFSMRef[ChannelState, ChannelData, Channel] = { implicit val system: ActorSystem = systemA @@ -190,8 +186,6 @@ trait ChannelStateTestsBase extends Assertions with Eventually { .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.ZeroConf))(_.updated(Features.ZeroConf, FeatureSupport.Optional)) .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.ScidAlias))(_.updated(Features.ScidAlias, FeatureSupport.Optional)) .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.DualFunding))(_.updated(Features.DualFunding, FeatureSupport.Optional)) - .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.Splicing))(_.updated(Features.Splicing, FeatureSupport.Optional)) - .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.Quiescence))(_.updated(Features.Quiescence, FeatureSupport.Optional)) .initFeatures() val bobInitFeatures = Bob.nodeParams.features .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.DisableWumbo))(_.removed(Features.Wumbo)) @@ -204,8 +198,6 @@ trait ChannelStateTestsBase extends Assertions with Eventually { .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.ZeroConf))(_.updated(Features.ZeroConf, FeatureSupport.Optional)) .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.ScidAlias))(_.updated(Features.ScidAlias, FeatureSupport.Optional)) .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.DualFunding))(_.updated(Features.DualFunding, FeatureSupport.Optional)) - .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.Splicing))(_.updated(Features.Splicing, FeatureSupport.Optional)) - .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.Quiescence))(_.updated(Features.Quiescence, FeatureSupport.Optional)) .initFeatures() val channelType = ChannelTypes.defaultFromFeatures(aliceInitFeatures, bobInitFeatures, announceChannel = channelFlags.announceChannel) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalQuiescentStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalQuiescentStateSpec.scala index 8cc310007a..037e11784a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalQuiescentStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalQuiescentStateSpec.scala @@ -45,7 +45,7 @@ class NormalQuiescentStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteL implicit val log: akka.event.LoggingAdapter = akka.event.NoLogging override def withFixture(test: OneArgTest): Outcome = { - val tags = test.tags + ChannelStateTestsTags.DualFunding + ChannelStateTestsTags.Splicing + ChannelStateTestsTags.Quiescence + val tags = test.tags + ChannelStateTestsTags.DualFunding val setup = init(tags = tags) import setup._ reachNormal(setup, tags) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala index f690f8e1eb..ec7ad8f719 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala @@ -58,7 +58,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik implicit val log: akka.event.LoggingAdapter = akka.event.NoLogging override def withFixture(test: OneArgTest): Outcome = { - val tags = test.tags + DualFunding + Splicing + val tags = test.tags + DualFunding val setup = init(tags = tags) import setup._ reachNormal(setup, tags) @@ -71,17 +71,11 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik private val defaultSpliceOutScriptPubKey = hex"0020aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa" - private def useQuiescence(s: TestFSMRef[ChannelState, ChannelData, Channel]): Boolean = s.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.params.useQuiescence - - private def useQuiescence(f: FixtureParam): Boolean = useQuiescence(f.alice) - private def initiateSpliceWithoutSigs(s: TestFSMRef[ChannelState, ChannelData, Channel], r: TestFSMRef[ChannelState, ChannelData, Channel], s2r: TestProbe, r2s: TestProbe, spliceIn_opt: Option[SpliceIn], spliceOut_opt: Option[SpliceOut]): TestProbe = { val sender = TestProbe() val cmd = CMD_SPLICE(sender.ref, spliceIn_opt, spliceOut_opt) s ! cmd - if (useQuiescence(s)) { - exchangeStfu(s, r, s2r, r2s) - } + exchangeStfu(s, r, s2r, r2s) s2r.expectMsgType[SpliceInit] s2r.forward(r) r2s.expectMsgType[SpliceAck] @@ -173,37 +167,29 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik private def setupHtlcs(f: FixtureParam): TestHtlcs = { import f._ - if (useQuiescence(f)) { - // add htlcs in both directions - val htlcsAliceToBob = Seq( - addHtlc(15_000_000 msat, alice, bob, alice2bob, bob2alice), - addHtlc(15_000_000 msat, alice, bob, alice2bob, bob2alice) - ) - crossSign(alice, bob, alice2bob, bob2alice) - val htlcsBobToAlice = Seq( - addHtlc(20_000_000 msat, bob, alice, bob2alice, alice2bob), - addHtlc(15_000_000 msat, bob, alice, bob2alice, alice2bob) - ) - crossSign(bob, alice, bob2alice, alice2bob) - - val initialState = alice.stateData.asInstanceOf[DATA_NORMAL] - assert(initialState.commitments.latest.capacity == 1_500_000.sat) - assert(initialState.commitments.latest.localCommit.spec.toLocal == 770_000_000.msat) - assert(initialState.commitments.latest.localCommit.spec.toRemote == 665_000_000.msat) - - alice2relayer.expectMsgType[Relayer.RelayForward] - alice2relayer.expectMsgType[Relayer.RelayForward] - bob2relayer.expectMsgType[Relayer.RelayForward] - bob2relayer.expectMsgType[Relayer.RelayForward] - - TestHtlcs(htlcsAliceToBob, htlcsBobToAlice) - } else { - val initialState = alice.stateData.asInstanceOf[DATA_NORMAL] - assert(initialState.commitments.latest.capacity == 1_500_000.sat) - assert(initialState.commitments.latest.localCommit.spec.toLocal == 800_000_000.msat) - assert(initialState.commitments.latest.localCommit.spec.toRemote == 700_000_000.msat) - TestHtlcs(Seq.empty, Seq.empty) - } + // add htlcs in both directions + val htlcsAliceToBob = Seq( + addHtlc(15_000_000 msat, alice, bob, alice2bob, bob2alice), + addHtlc(15_000_000 msat, alice, bob, alice2bob, bob2alice) + ) + crossSign(alice, bob, alice2bob, bob2alice) + val htlcsBobToAlice = Seq( + addHtlc(20_000_000 msat, bob, alice, bob2alice, alice2bob), + addHtlc(15_000_000 msat, bob, alice, bob2alice, alice2bob) + ) + crossSign(bob, alice, bob2alice, alice2bob) + + val initialState = alice.stateData.asInstanceOf[DATA_NORMAL] + assert(initialState.commitments.latest.capacity == 1_500_000.sat) + assert(initialState.commitments.latest.localCommit.spec.toLocal == 770_000_000.msat) + assert(initialState.commitments.latest.localCommit.spec.toRemote == 665_000_000.msat) + + alice2relayer.expectMsgType[Relayer.RelayForward] + alice2relayer.expectMsgType[Relayer.RelayForward] + bob2relayer.expectMsgType[Relayer.RelayForward] + bob2relayer.expectMsgType[Relayer.RelayForward] + + TestHtlcs(htlcsAliceToBob, htlcsBobToAlice) } def spliceOutFee(f: FixtureParam, capacity: Satoshi): Satoshi = { @@ -223,7 +209,8 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik import f._ // if the swap includes a splice-in, swap-out fees will be paid from bitcoind so final capacity is predictable - val (outgoingHtlcs, incomingHtlcs) = if (useQuiescence(f)) (30_000_000.msat, 35_000_000.msat) else (0.msat, 0.msat) + val outgoingHtlcs = 30_000_000.msat + val incomingHtlcs = 35_000_000.msat val postSpliceState = alice.stateData.asInstanceOf[ChannelDataWithCommitments] assert(postSpliceState.commitments.latest.capacity == 1_900_000.sat - spliceOutFee) assert(postSpliceState.commitments.latest.localCommit.spec.toLocal == 1_200_000_000.msat - spliceOutFee - outgoingHtlcs) @@ -237,23 +224,21 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik checkPostSpliceState(f, spliceOutFee) - if (useQuiescence(f)) { - // resolve pre-splice HTLCs after splice - val Seq((preimage1a, htlc1a), (preimage2a, htlc2a)) = htlcs.aliceToBob - val Seq((preimage1b, htlc1b), (preimage2b, htlc2b)) = htlcs.bobToAlice - fulfillHtlc(htlc1a.id, preimage1a, bob, alice, bob2alice, alice2bob) - fulfillHtlc(htlc2a.id, preimage2a, bob, alice, bob2alice, alice2bob) - crossSign(bob, alice, bob2alice, alice2bob) - fulfillHtlc(htlc1b.id, preimage1b, alice, bob, alice2bob, bob2alice) - fulfillHtlc(htlc2b.id, preimage2b, alice, bob, alice2bob, bob2alice) - crossSign(alice, bob, alice2bob, bob2alice) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.spec.htlcs.collect(outgoing).isEmpty) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.remoteCommit.spec.htlcs.collect(outgoing).isEmpty) - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.spec.htlcs.collect(outgoing).isEmpty) - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.remoteCommit.spec.htlcs.collect(outgoing).isEmpty) - } + // resolve pre-splice HTLCs after splice + val Seq((preimage1a, htlc1a), (preimage2a, htlc2a)) = htlcs.aliceToBob + val Seq((preimage1b, htlc1b), (preimage2b, htlc2b)) = htlcs.bobToAlice + fulfillHtlc(htlc1a.id, preimage1a, bob, alice, bob2alice, alice2bob) + fulfillHtlc(htlc2a.id, preimage2a, bob, alice, bob2alice, alice2bob) + crossSign(bob, alice, bob2alice, alice2bob) + fulfillHtlc(htlc1b.id, preimage1b, alice, bob, alice2bob, bob2alice) + fulfillHtlc(htlc2b.id, preimage2b, alice, bob, alice2bob, bob2alice) + crossSign(alice, bob, alice2bob, bob2alice) + assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.spec.htlcs.collect(outgoing).isEmpty) + assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.remoteCommit.spec.htlcs.collect(outgoing).isEmpty) + assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.spec.htlcs.collect(outgoing).isEmpty) + assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.remoteCommit.spec.htlcs.collect(outgoing).isEmpty) - val settledHtlcs = if (useQuiescence(f)) 5_000_000.msat else 0.msat + val settledHtlcs = 5_000_000.msat val finalState = alice.stateData.asInstanceOf[DATA_NORMAL] assert(finalState.commitments.latest.capacity == 1_900_000.sat - spliceOutFee) assert(finalState.commitments.latest.localCommit.spec.toLocal == 1_200_000_000.msat - spliceOutFee + settledHtlcs) @@ -276,10 +261,10 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik } test("recv CMD_SPLICE (splice-in, non dual-funded channel)") { () => - val f = init(tags = Set(DualFunding, Splicing)) + val f = init(tags = Set(DualFunding)) import f._ - reachNormal(f, tags = Set(Splicing)) // we open a non dual-funded channel + reachNormal(f, tags = Set.empty) // we open a non dual-funded channel alice2bob.ignoreMsg { case _: ChannelUpdate => true } bob2alice.ignoreMsg { case _: ChannelUpdate => true } awaitCond(alice.stateName == NORMAL && bob.stateName == NORMAL) @@ -304,7 +289,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik assert(postSpliceState.commitments.latest.remoteChannelReserve == 15_000.sat) } - test("recv CMD_SPLICE (splice-in, local and remote commit index mismatch)", Tag(Quiescence)) { f => + test("recv CMD_SPLICE (splice-in, local and remote commit index mismatch)") { f => import f._ // Alice and Bob asynchronously exchange HTLCs, which makes their commit indices diverge. @@ -368,18 +353,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.spec.toRemote == 700_000_000.msat) } - test("recv CMD_SPLICE (splice-out, would go below reserve)") { f => - import f._ - - setupHtlcs(f) - - val sender = TestProbe() - val cmd = CMD_SPLICE(sender.ref, spliceIn_opt = None, Some(SpliceOut(780_000.sat, defaultSpliceOutScriptPubKey))) - alice ! cmd - sender.expectMsgType[RES_FAILURE[_, _]] - } - - test("recv CMD_SPLICE (splice-out, would go below reserve, quiescent)", Tag(Quiescence), Tag(NoMaxHtlcValueInFlight)) { f => + test("recv CMD_SPLICE (splice-out, would go below reserve)", Tag(NoMaxHtlcValueInFlight)) { f => import f._ setupHtlcs(f) @@ -397,6 +371,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val sender = TestProbe() val cmd = CMD_SPLICE(sender.ref, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = None) alice ! cmd + exchangeStfu(f) // we tweak the feerate val spliceInit = alice2bob.expectMsgType[SpliceInit].copy(feerate = FeeratePerKw(100.sat)) bob.setFeerates(alice.nodeParams.currentFeerates.copy(minimum = FeeratePerKw(101.sat))) @@ -416,6 +391,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val sender = TestProbe() val bobBalance = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.spec.toLocal alice ! CMD_SPLICE(sender.ref, spliceIn_opt = Some(SpliceIn(100_000 sat)), spliceOut_opt = None) + exchangeStfu(f) val spliceInit = alice2bob.expectMsgType[SpliceInit] alice2bob.forward(bob, spliceInit) val spliceAck = bob2alice.expectMsgType[SpliceAck] @@ -460,27 +436,18 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik crossSign(alice, bob, alice2bob, bob2alice) } - def testSpliceInAndOutCmd(f: FixtureParam): Unit = { + test("recv CMD_SPLICE (splice-in + splice-out)") { f => val htlcs = setupHtlcs(f) - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) - resolveHtlcs(f, htlcs, spliceOutFee = 0.sat) } - test("recv CMD_SPLICE (splice-in + splice-out)") { f => - testSpliceInAndOutCmd(f) - } - - test("recv CMD_SPLICE (splice-in + splice-out, quiescence)", Tag(Quiescence)) { f => - testSpliceInAndOutCmd(f) - } - test("recv TxAbort (before TxComplete)") { f => import f._ val sender = TestProbe() alice ! CMD_SPLICE(sender.ref, spliceIn_opt = None, spliceOut_opt = Some(SpliceOut(50_000 sat, defaultSpliceOutScriptPubKey))) + exchangeStfu(f) alice2bob.expectMsgType[SpliceInit] alice2bob.forward(bob) bob2alice.expectMsgType[SpliceAck] @@ -505,6 +472,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val sender = TestProbe() alice ! CMD_SPLICE(sender.ref, spliceIn_opt = None, spliceOut_opt = Some(SpliceOut(50_000 sat, defaultSpliceOutScriptPubKey))) + exchangeStfu(f) alice2bob.expectMsgType[SpliceInit] alice2bob.forward(bob) bob2alice.expectMsgType[SpliceAck] @@ -540,6 +508,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val sender = TestProbe() alice ! CMD_SPLICE(sender.ref, spliceIn_opt = Some(SpliceIn(50_000 sat)), spliceOut_opt = None) + exchangeStfu(f) alice2bob.expectMsgType[SpliceInit] alice2bob.forward(bob) bob2alice.expectMsgType[SpliceAck] @@ -772,7 +741,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) } - test("emit post-splice events", Tag(NoMaxHtlcValueInFlight), Tag(Quiescence)) { f => + test("emit post-splice events", Tag(NoMaxHtlcValueInFlight)) { f => import f._ // Alice and Bob asynchronously exchange HTLCs, which makes their commit indices diverge. @@ -902,6 +871,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val sender = TestProbe() val cmd = CMD_SPLICE(sender.ref, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat)), spliceOut_opt = None) alice ! cmd + exchangeStfu(f) alice2bob.expectMsgType[SpliceInit] alice ! CMD_ADD_HTLC(sender.ref, 500000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, localOrigin(sender.ref)) sender.expectMsgType[RES_ADD_FAILED[_]] @@ -913,6 +883,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val sender = TestProbe() val cmd = CMD_SPLICE(sender.ref, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat)), spliceOut_opt = None) alice ! cmd + exchangeStfu(f) alice2bob.expectMsgType[SpliceInit] alice2bob.forward(bob) bob2alice.expectMsgType[SpliceAck] @@ -923,36 +894,12 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice2bob.expectNoMessage(100 millis) } - test("recv UpdateAddHtlc while a splice is requested") { f => - import f._ - val sender = TestProbe() - val cmd = CMD_SPLICE(sender.ref, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat)), spliceOut_opt = None) - alice ! cmd - alice2bob.expectMsgType[SpliceInit] - // we're holding the splice_init to create a race - - val (_, cmdAdd: CMD_ADD_HTLC) = makeCmdAdd(5_000_000 msat, bob.underlyingActor.remoteNodeId, bob.underlyingActor.nodeParams.currentBlockHeight) - bob ! cmdAdd - bob2alice.expectMsgType[UpdateAddHtlc] - bob2alice.forward(alice) - // now we forward the splice_init - alice2bob.forward(bob) - // bob rejects the SpliceInit because they have a pending htlc - bob2alice.expectMsgType[TxAbort] - bob2alice.forward(alice) - // alice returns a warning and schedules a disconnect after receiving UpdateAddHtlc - alice2bob.expectMsg(Warning(channelId(alice), ForbiddenDuringSplice(channelId(alice), "UpdateAddHtlc").getMessage)) - // alice confirms the splice abort - alice2bob.expectMsgType[TxAbort] - // the htlc is not added - assert(!alice.stateData.asInstanceOf[DATA_NORMAL].commitments.hasPendingOrProposedHtlcs) - } - test("recv UpdateAddHtlc while a splice is in progress") { f => import f._ val sender = TestProbe() val cmd = CMD_SPLICE(sender.ref, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat)), spliceOut_opt = None) alice ! cmd + exchangeStfu(f) alice2bob.expectMsgType[SpliceInit] alice2bob.forward(bob) bob2alice.expectMsgType[SpliceAck] @@ -1091,6 +1038,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val sender = TestProbe() val cmd = CMD_SPLICE(sender.ref, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat)), spliceOut_opt = None) alice ! cmd + exchangeStfu(f) alice2bob.expectMsgType[SpliceInit] alice2bob.forward(bob) bob2alice.expectMsgType[SpliceAck] @@ -1102,7 +1050,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik assert(alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus == SpliceStatus.NoSplice) } - def testDisconnectCommitSigNotReceived(f: FixtureParam): Unit = { + test("disconnect (commit_sig not received)") { f => import f._ val htlcs = setupHtlcs(f) @@ -1133,15 +1081,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik resolveHtlcs(f, htlcs, 0.sat) } - test("disconnect (commit_sig not received)") { f => - testDisconnectCommitSigNotReceived(f) - } - - test("disconnect (commit_sig not received, quiescence)", Tag(Quiescence)) { f => - testDisconnectCommitSigNotReceived(f) - } - - def testDisconnectCommitSigReceivedByAlice(f: FixtureParam): Unit = { + test("disconnect (commit_sig received by alice)") { f => import f._ val htlcs = setupHtlcs(f) @@ -1173,15 +1113,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik resolveHtlcs(f, htlcs, spliceOutFee = 0.sat) } - test("disconnect (commit_sig received by alice)") { f => - testDisconnectCommitSigReceivedByAlice(f) - } - - test("disconnect (commit_sig received by alice, quiescence)", Tag(Quiescence)) { f => - testDisconnectCommitSigReceivedByAlice(f) - } - - def testDisconnectTxSignaturesSentByBob(f: FixtureParam): Unit = { + test("disconnect (tx_signatures sent by bob)") { f => import f._ val htlcs = setupHtlcs(f) @@ -1215,15 +1147,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik resolveHtlcs(f, htlcs, spliceOutFee = 0.sat) } - test("disconnect (tx_signatures sent by bob)") { f => - testDisconnectTxSignaturesSentByBob(f) - } - - test("disconnect (tx_signatures sent by bob, quiescence)", Tag(Quiescence)) { f => - testDisconnectTxSignaturesSentByBob(f) - } - - def testDisconnectTxSignaturesReceivedByAlice(f: FixtureParam): Unit = { + test("disconnect (tx_signatures received by alice)") { f => import f._ val htlcs = setupHtlcs(f) @@ -1264,15 +1188,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik resolveHtlcs(f, htlcs, spliceOutFee = 0.sat) } - test("disconnect (tx_signatures received by alice)") { f => - testDisconnectTxSignaturesReceivedByAlice(f) - } - - test("disconnect (tx_signatures received by alice, quiescence)", Tag(Quiescence)) { f => - testDisconnectTxSignaturesReceivedByAlice(f) - } - - def testDisconnectTxSignaturesReceivedByAliceZeroConf(f: FixtureParam): Unit = { + test("disconnect (tx_signatures received by alice, zero-conf)", Tag(ZeroConf), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f => import f._ val htlcs = setupHtlcs(f) @@ -1314,14 +1230,6 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik resolveHtlcs(f, htlcs, spliceOutFee = 0.sat) } - test("disconnect (tx_signatures received by alice, zero-conf)", Tag(ZeroConf), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f => - testDisconnectTxSignaturesReceivedByAliceZeroConf(f) - } - - test("disconnect (tx_signatures received by alice, zero-conf, quiescence)", Tag(ZeroConf), Tag(AnchorOutputsZeroFeeHtlcTxs), Tag(Quiescence)) { f => - testDisconnectTxSignaturesReceivedByAliceZeroConf(f) - } - test("disconnect (tx_signatures sent by alice, splice confirms while bob is offline)") { f => import f._ @@ -1475,7 +1383,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik } } - def testForceCloseWithMultipleSplicesSimple(f: FixtureParam): Unit = { + test("force-close with multiple splices (simple)") { f => import f._ val htlcs = setupHtlcs(f) @@ -1547,15 +1455,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik assert(Helpers.Closing.isClosed(alice.stateData.asInstanceOf[DATA_CLOSING], None).exists(_.isInstanceOf[LocalClose])) } - test("force-close with multiple splices (simple)") { f => - testForceCloseWithMultipleSplicesSimple(f) - } - - test("force-close with multiple splices (simple, quiescence)", Tag(Quiescence)) { f => - testForceCloseWithMultipleSplicesSimple(f) - } - - def testForceCloseWithMultipleSplicesPreviousActiveRemote(f: FixtureParam): Unit = { + test("force-close with multiple splices (previous active remote)", Tag(ChannelStateTestsTags.StaticRemoteKey), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => import f._ val htlcs = setupHtlcs(f) @@ -1633,15 +1533,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik assert(Helpers.Closing.isClosed(alice.stateData.asInstanceOf[DATA_CLOSING], None).exists(_.isInstanceOf[RemoteClose])) } - test("force-close with multiple splices (previous active remote)", Tag(ChannelStateTestsTags.StaticRemoteKey), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => - testForceCloseWithMultipleSplicesPreviousActiveRemote(f) - } - - test("force-close with multiple splices (previous active remote, quiescence)", Tag(Quiescence), Tag(ChannelStateTestsTags.StaticRemoteKey), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => - testForceCloseWithMultipleSplicesPreviousActiveRemote(f) - } - - def testForceCloseWithMultipleSplicesPreviousActiveRevoked(f: FixtureParam): Unit = { + test("force-close with multiple splices (previous active revoked)", Tag(ChannelStateTestsTags.StaticRemoteKey), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => import f._ val htlcs = setupHtlcs(f) @@ -1713,15 +1605,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik assert(Helpers.Closing.isClosed(alice.stateData.asInstanceOf[DATA_CLOSING], None).exists(_.isInstanceOf[RevokedClose])) } - test("force-close with multiple splices (previous active revoked)", Tag(ChannelStateTestsTags.StaticRemoteKey), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => - testForceCloseWithMultipleSplicesPreviousActiveRevoked(f) - } - - test("force-close with multiple splices (previous active revoked, quiescent)", Tag(Quiescence), Tag(ChannelStateTestsTags.StaticRemoteKey), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => - testForceCloseWithMultipleSplicesPreviousActiveRevoked(f) - } - - def testForceCloseWithMultipleSplicesInactiveRemote(f: FixtureParam): Unit = { + test("force-close with multiple splices (inactive remote)", Tag(ZeroConf), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f => import f._ val htlcs = setupHtlcs(f) @@ -1828,15 +1712,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik assert(Helpers.Closing.isClosed(alice.stateData.asInstanceOf[DATA_CLOSING], None).exists(_.isInstanceOf[RemoteClose])) } - test("force-close with multiple splices (inactive remote)", Tag(ZeroConf), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f => - testForceCloseWithMultipleSplicesInactiveRemote(f) - } - - test("force-close with multiple splices (inactive remote, quiescence)", Tag(Quiescence), Tag(ZeroConf), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f => - testForceCloseWithMultipleSplicesInactiveRemote(f) - } - - def testForceCloseWithMultipleSplicesInactiveRevoked(f: FixtureParam): Unit = { + test("force-close with multiple splices (inactive revoked)", Tag(ZeroConf), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f => import f._ val htlcs = setupHtlcs(f) @@ -1947,14 +1823,6 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik assert(Helpers.Closing.isClosed(alice.stateData.asInstanceOf[DATA_CLOSING], None).exists(_.isInstanceOf[RevokedClose])) } - test("force-close with multiple splices (inactive revoked)", Tag(ZeroConf), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f => - testForceCloseWithMultipleSplicesInactiveRevoked(f) - } - - test("force-close with multiple splices (inactive revoked, quiescence)", Tag(Quiescence), Tag(ZeroConf), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f => - testForceCloseWithMultipleSplicesInactiveRevoked(f) - } - test("put back watches after restart") { f => import f._ @@ -2049,7 +1917,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik bob2blockchain.expectNoMessage(100 millis) } - test("recv CMD_SPLICE (splice-in + splice-out) with pre and post splice htlcs", Tag(Quiescence)) { f => + test("recv CMD_SPLICE (splice-in + splice-out) with pre and post splice htlcs") { f => import f._ val htlcs = setupHtlcs(f) @@ -2086,7 +1954,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik resolveHtlcs(f, htlcs, spliceOutFee = 0.sat) } - test("recv CMD_SPLICE (splice-in + splice-out) with pending htlcs, resolved after splice locked", Tag(Quiescence), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f => + test("recv CMD_SPLICE (splice-in + splice-out) with pending htlcs, resolved after splice locked", Tag(AnchorOutputsZeroFeeHtlcTxs)) { f => import f._ val htlcs = setupHtlcs(f) @@ -2105,7 +1973,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik resolveHtlcs(f, htlcs, spliceOutFee = 0.sat) } - test("recv multiple CMD_SPLICE (splice-in, splice-out, quiescence)", Tag(Quiescence)) { f => + test("recv multiple CMD_SPLICE (splice-in, splice-out)") { f => val htlcs = setupHtlcs(f) initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) @@ -2114,7 +1982,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik resolveHtlcs(f, htlcs, spliceOutFee = spliceOutFee(f, capacity = 1_900_000.sat)) } - test("recv invalid htlc signatures during splice-in", Tag(Quiescence)) { f => + test("recv invalid htlc signatures during splice-in") { f => import f._ val htlcs = setupHtlcs(f) From 2d350e51ebac19e26aab3be0c484ea47cf5a116f Mon Sep 17 00:00:00 2001 From: t-bast Date: Wed, 31 Jul 2024 16:45:30 +0200 Subject: [PATCH 3/8] Add support for RBF-ing splice transactions If the latest splice transaction doesn't confirm, we allow exchanging `tx_init_rbf` and `tx_ack_rbf` to create another splice transaction to replace it. We use the same funding contribution as the previous splice. We disallow creating another splice transaction using `splice_init` if we have several RBF attempts for the latest splice: we cannot know which one of them will confirm and should be spent by the new splice. --- docs/release-notes/eclair-vnext.md | 33 +- .../main/scala/fr/acinq/eclair/Eclair.scala | 47 ++- .../fr/acinq/eclair/channel/ChannelData.scala | 97 ++++-- .../eclair/channel/ChannelExceptions.scala | 1 + .../fr/acinq/eclair/channel/fsm/Channel.scala | 265 +++++++++++++-- .../channel/fsm/ChannelOpenDualFunded.scala | 124 +++---- .../channel/fsm/DualFundingHandlers.scala | 8 +- .../channel/fund/InteractiveTxBuilder.scala | 26 +- .../channel/fund/InteractiveTxFunder.scala | 26 +- .../channel/version4/ChannelCodecs4.scala | 10 +- .../wire/protocol/LightningMessageTypes.scala | 6 +- .../channel/InteractiveTxBuilderSpec.scala | 26 +- ...WaitForDualFundingConfirmedStateSpec.scala | 40 +-- .../states/e/NormalSplicesStateSpec.scala | 311 ++++++++++++++++++ .../io/PendingChannelsRateLimiterSpec.scala | 4 +- .../channel/version4/ChannelCodecs4Spec.scala | 14 +- .../acinq/eclair/api/handlers/Channel.scala | 8 +- 17 files changed, 832 insertions(+), 214 deletions(-) diff --git a/docs/release-notes/eclair-vnext.md b/docs/release-notes/eclair-vnext.md index ede7884a1a..2a3be9effc 100644 --- a/docs/release-notes/eclair-vnext.md +++ b/docs/release-notes/eclair-vnext.md @@ -4,6 +4,36 @@ ## Major changes +### Channel Splicing + +With this release, we add support for the final version of [splicing](https://github.com/lightning/bolts/pull/1160) that was recently added to the BOLTs. +Splicing allows node operators to change the size of their existing channels, which makes it easier and more efficient to allocate liquidity where it is most needed. +Most node operators can now have a single channel with each of their peer, which costs less on-chain fees and resources, and makes path-finding easier. + +The size of an existing channel can be increased with the `splicein` API: + +```sh +eclair-cli splicein --channelId= --amountIn= +``` + +Once that transaction confirms, the additional liquidity can be used to send outgoing payments. +If the transaction doesn't confirm, the node operator can speed up confirmation with the `rbfsplice` API: + +```sh +eclair-cli rbfsplice --channelId= --targetFeerateSatByte= --fundingFeeBudgetSatoshis= +``` + +If the node operator wants to reduce the size of a channel, or send some of the channel funds to an on-chain address, they can use the `spliceout` API: + +```sh +eclair-cli spliceout --channelId= --amountOut= --scriptPubKey= +``` + +That operation can also be RBF-ed with the `rbfsplice` API to speed up confirmation if necessary. + +Note that eclair had already introduced support for a splicing prototype in v0.9.0, which helped improve the BOLT proposal. +We're removing support for the previous splicing prototype feature: users that depended on this protocol must upgrade to create official splice transactions. + ### Update minimal version of Bitcoin Core With this release, eclair requires using Bitcoin Core 27.1. @@ -26,7 +56,8 @@ Eclair will not allow remote peers to open new obsolete channels that do not sup ### API changes -- `channelstats` now takes optional parameters `--count` and `--skip` to control pagination. By default, it will return first 10 entries. (#2890) +- `channelstats` now accept `--count` and `--skip` parameters to limit the number of retrieved items (#2890) +- `rbfsplice` lets any channel participant RBF the current unconfirmed splice transaction (#2887) ### Miscellaneous improvements and bug fixes diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala index 9aeec55496..518f0c1127 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala @@ -94,6 +94,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]]]] @@ -227,16 +229,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))) + sendToChannelTyped( + channel = Left(channelId), + cmdBuilder = CMD_BUMP_FUNDING_FEE(_, targetFeerate, fundingFeeBudget, lockTime_opt.getOrElse(appKit.nodeParams.currentBlockHeight.toLong)) + ) } 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 - )) + 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) + ) } override def spliceOut(channelId: ByteVector32, amountOut: Satoshi, scriptOrAddress: Either[ByteVector, String])(implicit timeout: Timeout): Future[CommandResponse[CMD_SPLICE]] = { @@ -247,11 +251,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)) - )) + val spliceOut = SpliceOut(amount = amountOut, scriptPubKey = script) + sendToChannelTyped( + channel = Left(channelId), + cmdBuilder = CMD_SPLICE(_, spliceIn_opt = None, spliceOut_opt = Some(spliceOut)) + ) + } + + 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)) + ) } 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]]]] = { @@ -558,9 +569,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] = @@ -571,9 +582,9 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging { case Right(shortChannelId) => Register.ForwardShortId(replyTo, shortChannelId, cmd) } }.map { - case t: R@unchecked => t - case t: Register.ForwardFailure[C]@unchecked => throw ChannelNotFound(Left(t.fwd.channelId)) - case t: Register.ForwardShortIdFailure[C]@unchecked => throw ChannelNotFound(Right(t.fwd.shortChannelId)) + case t: R @unchecked => t + case t: Register.ForwardFailure[C] @unchecked => throw ChannelNotFound(Left(t.fwd.channelId)) + case t: Register.ForwardShortIdFailure[C] @unchecked => throw ChannelNotFound(Right(t.fwd.shortChannelId)) } /** diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala index 05aeb6f985..2495e161f0 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala @@ -26,7 +26,7 @@ import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningS import fr.acinq.eclair.io.Peer import fr.acinq.eclair.transactions.CommitmentSpec import fr.acinq.eclair.transactions.Transactions._ -import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelReady, ChannelReestablish, ChannelUpdate, ClosingSigned, CommitSig, FailureMessage, FundingCreated, FundingSigned, Init, OnionRoutingPacket, OpenChannel, OpenDualFundedChannel, Shutdown, SpliceInit, Stfu, TxSignatures, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc} +import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelReady, ChannelReestablish, ChannelUpdate, ClosingSigned, CommitSig, FailureMessage, FundingCreated, FundingSigned, Init, OnionRoutingPacket, OpenChannel, OpenDualFundedChannel, Shutdown, SpliceInit, Stfu, TxInitRbf, TxSignatures, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc} import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Features, InitFeature, MilliSatoshi, MilliSatoshiLong, RealShortChannelId, TimestampMilli, UInt64} import scodec.bits.ByteVector @@ -193,16 +193,16 @@ sealed trait Command extends PossiblyHarmful sealed trait HasReplyToCommand extends Command { def replyTo: ActorRef } sealed trait HasOptionalReplyToCommand extends Command { def replyTo_opt: Option[ActorRef] } -sealed trait ForbiddenCommandDuringSplice extends Command -sealed trait ForbiddenCommandDuringQuiescence extends Command +sealed trait ForbiddenCommandDuringQuiescenceNegotiation extends Command +sealed trait ForbiddenCommandWhenQuiescent extends Command -final case class CMD_ADD_HTLC(replyTo: ActorRef, amount: MilliSatoshi, paymentHash: ByteVector32, cltvExpiry: CltvExpiry, onion: OnionRoutingPacket, nextBlindingKey_opt: Option[PublicKey], confidence: Double, origin: Origin.Hot, commit: Boolean = false) extends HasReplyToCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence -sealed trait HtlcSettlementCommand extends HasOptionalReplyToCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence { def id: Long } +final case class CMD_ADD_HTLC(replyTo: ActorRef, amount: MilliSatoshi, paymentHash: ByteVector32, cltvExpiry: CltvExpiry, onion: OnionRoutingPacket, nextBlindingKey_opt: Option[PublicKey], confidence: Double, origin: Origin.Hot, commit: Boolean = false) extends HasReplyToCommand with ForbiddenCommandDuringQuiescenceNegotiation with ForbiddenCommandWhenQuiescent +sealed trait HtlcSettlementCommand extends HasOptionalReplyToCommand with ForbiddenCommandDuringQuiescenceNegotiation with ForbiddenCommandWhenQuiescent { def id: Long } final case class CMD_FULFILL_HTLC(id: Long, r: ByteVector32, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand final case class CMD_FAIL_HTLC(id: Long, reason: Either[ByteVector, FailureMessage], delay_opt: Option[FiniteDuration] = None, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand final case class CMD_FAIL_MALFORMED_HTLC(id: Long, onionHash: ByteVector32, failureCode: Int, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand -final case class CMD_UPDATE_FEE(feeratePerKw: FeeratePerKw, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HasOptionalReplyToCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence -final case class CMD_SIGN(replyTo_opt: Option[ActorRef] = None) extends HasOptionalReplyToCommand with ForbiddenCommandDuringSplice +final case class CMD_UPDATE_FEE(feeratePerKw: FeeratePerKw, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HasOptionalReplyToCommand with ForbiddenCommandDuringQuiescenceNegotiation with ForbiddenCommandWhenQuiescent +final case class CMD_SIGN(replyTo_opt: Option[ActorRef] = None) extends HasOptionalReplyToCommand with ForbiddenCommandWhenQuiescent final case class ClosingFees(preferred: Satoshi, min: Satoshi, max: Satoshi) final case class ClosingFeerates(preferred: FeeratePerKw, min: FeeratePerKw, max: FeeratePerKw) { @@ -210,19 +210,22 @@ final case class ClosingFeerates(preferred: FeeratePerKw, min: FeeratePerKw, max } sealed trait CloseCommand extends HasReplyToCommand -final case class CMD_CLOSE(replyTo: ActorRef, scriptPubKey: Option[ByteVector], feerates: Option[ClosingFeerates]) extends CloseCommand with ForbiddenCommandDuringSplice with ForbiddenCommandDuringQuiescence +final case class CMD_CLOSE(replyTo: ActorRef, scriptPubKey: Option[ByteVector], feerates: Option[ClosingFeerates]) extends CloseCommand with ForbiddenCommandDuringQuiescenceNegotiation with ForbiddenCommandWhenQuiescent final case class CMD_FORCECLOSE(replyTo: ActorRef) extends CloseCommand final case class CMD_BUMP_FORCE_CLOSE_FEE(replyTo: akka.actor.typed.ActorRef[CommandResponse[CMD_BUMP_FORCE_CLOSE_FEE]], confirmationTarget: ConfirmationTarget) extends Command -final case class CMD_BUMP_FUNDING_FEE(replyTo: akka.actor.typed.ActorRef[CommandResponse[CMD_BUMP_FUNDING_FEE]], targetFeerate: FeeratePerKw, fundingFeeBudget: Satoshi, lockTime: Long) extends Command +sealed trait ChannelFundingCommand extends Command { + def replyTo: akka.actor.typed.ActorRef[CommandResponse[ChannelFundingCommand]] +} case class SpliceIn(additionalLocalFunding: Satoshi, pushAmount: MilliSatoshi = 0 msat) case class SpliceOut(amount: Satoshi, scriptPubKey: ByteVector) -final case class CMD_SPLICE(replyTo: akka.actor.typed.ActorRef[CommandResponse[CMD_SPLICE]], spliceIn_opt: Option[SpliceIn], spliceOut_opt: Option[SpliceOut]) extends Command { +final case class CMD_SPLICE(replyTo: akka.actor.typed.ActorRef[CommandResponse[ChannelFundingCommand]], spliceIn_opt: Option[SpliceIn], spliceOut_opt: Option[SpliceOut]) extends ChannelFundingCommand { require(spliceIn_opt.isDefined || spliceOut_opt.isDefined, "there must be a splice-in or a splice-out") val additionalLocalFunding: Satoshi = spliceIn_opt.map(_.additionalLocalFunding).getOrElse(0 sat) val pushAmount: MilliSatoshi = spliceIn_opt.map(_.pushAmount).getOrElse(0 msat) val spliceOutputs: List[TxOut] = spliceOut_opt.toList.map(s => TxOut(s.amount, s.scriptPubKey)) } +final case class CMD_BUMP_FUNDING_FEE(replyTo: akka.actor.typed.ActorRef[CommandResponse[ChannelFundingCommand]], targetFeerate: FeeratePerKw, fundingFeeBudget: Satoshi, lockTime: Long) extends ChannelFundingCommand final case class CMD_UPDATE_RELAY_FEE(replyTo: ActorRef, feeBase: MilliSatoshi, feeProportionalMillionths: Long) extends HasReplyToCommand final case class CMD_GET_CHANNEL_STATE(replyTo: ActorRef) extends HasReplyToCommand final case class CMD_GET_CHANNEL_DATA(replyTo: ActorRef) extends HasReplyToCommand @@ -456,42 +459,61 @@ object RemoteFundingStatus { case object Locked extends RemoteFundingStatus } -sealed trait RbfStatus -object RbfStatus { - case object NoRbf extends RbfStatus - case class RbfRequested(cmd: CMD_BUMP_FUNDING_FEE) extends RbfStatus - case class RbfInProgress(cmd_opt: Option[CMD_BUMP_FUNDING_FEE], rbf: typed.ActorRef[InteractiveTxBuilder.Command], remoteCommitSig: Option[CommitSig]) extends RbfStatus - case class RbfWaitingForSigs(signingSession: InteractiveTxSigningSession.WaitingForSigs) extends RbfStatus - case object RbfAborted extends RbfStatus +sealed trait DualFundingStatus +object DualFundingStatus { + /** We're waiting for one of the funding transactions to confirm. */ + case object WaitingForConfirmations extends DualFundingStatus + /** We told our peer we want to RBF the funding transaction. */ + case class RbfRequested(cmd: CMD_BUMP_FUNDING_FEE) extends DualFundingStatus + /** We both agreed to RBF and are building the new funding transaction. */ + case class RbfInProgress(cmd_opt: Option[CMD_BUMP_FUNDING_FEE], rbf: typed.ActorRef[InteractiveTxBuilder.Command], remoteCommitSig: Option[CommitSig]) extends DualFundingStatus + /** A new funding transaction has been negotiated, we're exchanging signatures. */ + case class RbfWaitingForSigs(signingSession: InteractiveTxSigningSession.WaitingForSigs) extends DualFundingStatus + /** The RBF attempt was aborted by us, we're waiting for our peer to ack. */ + case object RbfAborted extends DualFundingStatus } -sealed trait SpliceStatus /** We're waiting for the channel to be quiescent. */ -sealed trait QuiescenceNegotiation extends SpliceStatus +sealed trait QuiescenceNegotiation object QuiescenceNegotiation { sealed trait Initiator extends QuiescenceNegotiation + object Initiator { + /** We stop sending new updates and wait for our updates to be added to the local and remote commitments. */ + case object QuiescenceRequested extends Initiator + /** Our updates have been added to the local and remote commitments, we wait for our peer to do the same. */ + case class SentStfu(stfu: Stfu) extends Initiator + } + sealed trait NonInitiator extends QuiescenceNegotiation + object NonInitiator { + /** Our peer has asked us to stop sending new updates and wait for our updates to be added to the local and remote commitments. */ + case class ReceivedStfu(stfu: Stfu) extends NonInitiator + } +} + +sealed trait SpliceStatus { + def isNegotiatingQuiescence: Boolean = this.isInstanceOf[SpliceStatus.NegotiatingQuiescence] + def isQuiescent: Boolean = this match { + case SpliceStatus.NoSplice | _: SpliceStatus.NegotiatingQuiescence => false + case _ => true + } } -/** The channel is quiescent and a splice attempt was initiated. */ -sealed trait QuiescentSpliceStatus extends SpliceStatus object SpliceStatus { case object NoSplice extends SpliceStatus - /** We stop sending new updates and wait for our updates to be added to the local and remote commitments. */ - case class QuiescenceRequested(splice: CMD_SPLICE) extends QuiescenceNegotiation.Initiator - /** Our updates have been added to the local and remote commitments, we wait for our peer to do the same. */ - case class InitiatorQuiescent(splice: CMD_SPLICE) extends QuiescenceNegotiation.Initiator - /** Our peer has asked us to stop sending new updates and wait for our updates to be added to the local and remote commitments. */ - case class ReceivedStfu(stfu: Stfu) extends QuiescenceNegotiation.NonInitiator - /** Our updates have been added to the local and remote commitments, we wait for our peer to use the now quiescent channel. */ - case object NonInitiatorQuiescent extends QuiescentSpliceStatus + /** We're trying to quiesce the channel in order to negotiate a splice. */ + case class NegotiatingQuiescence(cmd_opt: Option[ChannelFundingCommand], status: QuiescenceNegotiation) extends SpliceStatus + /** The channel is quiescent, we wait for our peer to send splice_init or tx_init_rbf. */ + case object NonInitiatorQuiescent extends SpliceStatus /** We told our peer we want to splice funds in the channel. */ - case class SpliceRequested(cmd: CMD_SPLICE, init: SpliceInit) extends QuiescentSpliceStatus - /** We both agreed to splice and are building the splice transaction. */ - case class SpliceInProgress(cmd_opt: Option[CMD_SPLICE], sessionId: ByteVector32, splice: typed.ActorRef[InteractiveTxBuilder.Command], remoteCommitSig: Option[CommitSig]) extends QuiescentSpliceStatus + case class SpliceRequested(cmd: CMD_SPLICE, init: SpliceInit) extends SpliceStatus + /** We told our peer we want to RBF the latest splice transaction. */ + case class RbfRequested(cmd: CMD_BUMP_FUNDING_FEE, rbf: TxInitRbf) extends SpliceStatus + /** We both agreed to splice/rbf and are building the corresponding transaction. */ + case class SpliceInProgress(cmd_opt: Option[ChannelFundingCommand], sessionId: ByteVector32, splice: typed.ActorRef[InteractiveTxBuilder.Command], remoteCommitSig: Option[CommitSig]) extends SpliceStatus /** The splice transaction has been negotiated, we're exchanging signatures. */ - case class SpliceWaitingForSigs(signingSession: InteractiveTxSigningSession.WaitingForSigs) extends QuiescentSpliceStatus + case class SpliceWaitingForSigs(signingSession: InteractiveTxSigningSession.WaitingForSigs) extends SpliceStatus /** The splice attempt was aborted by us, we're waiting for our peer to ack. */ - case object SpliceAborted extends QuiescentSpliceStatus + case object SpliceAborted extends SpliceStatus } sealed trait ChannelData extends PossiblyHarmful { @@ -585,7 +607,7 @@ final case class DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments: Commitments, remotePushAmount: MilliSatoshi, waitingSince: BlockHeight, // how long have we been waiting for a funding tx to confirm lastChecked: BlockHeight, // last time we checked if the channel was double-spent - rbfStatus: RbfStatus, + status: DualFundingStatus, deferred: Option[ChannelReady]) extends ChannelDataWithCommitments { def allFundingTxs: Seq[DualFundedUnconfirmedFundingTx] = commitments.active.map(_.localFundingStatus).collect { case fundingTx: DualFundedUnconfirmedFundingTx => fundingTx } def latestFundingTx: DualFundedUnconfirmedFundingTx = commitments.latest.localFundingStatus.asInstanceOf[DualFundedUnconfirmedFundingTx] @@ -600,7 +622,10 @@ final case class DATA_NORMAL(commitments: Commitments, localShutdown: Option[Shutdown], remoteShutdown: Option[Shutdown], closingFeerates: Option[ClosingFeerates], - spliceStatus: SpliceStatus) extends ChannelDataWithCommitments + spliceStatus: SpliceStatus) extends ChannelDataWithCommitments { + val isNegotiatingQuiescence: Boolean = spliceStatus.isNegotiatingQuiescence + val isQuiescent: Boolean = spliceStatus.isQuiescent +} final case class DATA_SHUTDOWN(commitments: Commitments, localShutdown: Shutdown, remoteShutdown: Shutdown, closingFeerates: Option[ClosingFeerates]) extends ChannelDataWithCommitments final case class DATA_NEGOTIATING(commitments: Commitments, localShutdown: Shutdown, remoteShutdown: Shutdown, diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala index cea5739003..404e31c9f0 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala @@ -85,6 +85,7 @@ case class InvalidRbfAttemptsExhausted (override val channelId: Byte case class InvalidRbfAttemptTooSoon (override val channelId: ByteVector32, previousAttempt: BlockHeight, nextAttempt: BlockHeight) extends ChannelException(channelId, s"invalid rbf attempt: last attempt made at block=$previousAttempt, next attempt available after block=$nextAttempt") case class InvalidSpliceTxAbortNotAcked (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid splice attempt: our previous tx_abort has not been acked") case class InvalidSpliceNotQuiescent (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid splice attempt: the channel is not quiescent") +case class InvalidSpliceWithUnconfirmedRbf (override val channelId: ByteVector32, previousTxs: Seq[TxId]) extends ChannelException(channelId, s"invalid splice attempt: the previous splice was rbf-ed and is still unconfirmed (txIds=${previousTxs.mkString(", ")})") case class InvalidRbfTxConfirmed (override val channelId: ByteVector32) extends ChannelException(channelId, "no need to rbf, transaction is already confirmed") case class InvalidRbfNonInitiator (override val channelId: ByteVector32) extends ChannelException(channelId, "cannot initiate rbf: we're not the initiator of this interactive-tx attempt") case class InvalidRbfZeroConf (override val channelId: ByteVector32) extends ChannelException(channelId, "cannot initiate rbf: we're using zero-conf for this interactive-tx attempt") diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index f600579f31..c8ff14e4df 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -387,7 +387,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with */ when(NORMAL)(handleExceptions { - case Event(c: ForbiddenCommandDuringQuiescence, d: DATA_NORMAL) if d.spliceStatus.isInstanceOf[QuiescenceNegotiation] => + case Event(c: ForbiddenCommandDuringQuiescenceNegotiation, d: DATA_NORMAL) if d.isNegotiatingQuiescence => val error = ForbiddenDuringQuiescence(d.channelId, c.getClass.getSimpleName) c match { case c: CMD_ADD_HTLC => handleAddHtlcCommandError(c, error, Some(d.channelUpdate)) @@ -398,7 +398,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with case _ => handleCommandError(error, c) } - case Event(c: ForbiddenCommandDuringSplice, d: DATA_NORMAL) if d.spliceStatus.isInstanceOf[QuiescentSpliceStatus] => + case Event(c: ForbiddenCommandWhenQuiescent, d: DATA_NORMAL) if d.isQuiescent => val error = ForbiddenDuringSplice(d.channelId, c.getClass.getSimpleName) c match { case c: CMD_ADD_HTLC => handleAddHtlcCommandError(c, error, Some(d.channelUpdate)) @@ -409,7 +409,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with case _ => handleCommandError(error, c) } - case Event(msg: ForbiddenMessageDuringSplice, d: DATA_NORMAL) if d.spliceStatus.isInstanceOf[QuiescentSpliceStatus] => + case Event(msg: ForbiddenMessageWhenQuiescent, d: DATA_NORMAL) if d.isQuiescent => log.warning("received forbidden message {} during splicing with status {}", msg.getClass.getSimpleName, d.spliceStatus.getClass.getSimpleName) val error = ForbiddenDuringSplice(d.channelId, msg.getClass.getSimpleName) // We forward preimages as soon as possible to the upstream channel because it allows us to pull funds. @@ -611,10 +611,11 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with context.system.eventStream.publish(ChannelSignatureReceived(self, commitments1)) // If we're now quiescent, we may send our stfu message. val (d1, toSend) = d.spliceStatus match { - case SpliceStatus.QuiescenceRequested(cmd) if commitments1.localIsQuiescent => + case SpliceStatus.NegotiatingQuiescence(cmd_opt, QuiescenceNegotiation.Initiator.QuiescenceRequested) if commitments1.localIsQuiescent => val stfu = Stfu(d.channelId, initiator = true) - (d.copy(commitments = commitments1, spliceStatus = SpliceStatus.InitiatorQuiescent(cmd)), Seq(revocation, stfu)) - case _: SpliceStatus.ReceivedStfu if commitments1.localIsQuiescent => + val spliceStatus1 = SpliceStatus.NegotiatingQuiescence(cmd_opt, QuiescenceNegotiation.Initiator.SentStfu(stfu)) + (d.copy(commitments = commitments1, spliceStatus = spliceStatus1), Seq(revocation, stfu)) + case SpliceStatus.NegotiatingQuiescence(_, _: QuiescenceNegotiation.NonInitiator.ReceivedStfu) if commitments1.localIsQuiescent => val stfu = Stfu(d.channelId, initiator = false) (d.copy(commitments = commitments1, spliceStatus = SpliceStatus.NonInitiatorQuiescent), Seq(revocation, stfu)) case _ => @@ -851,24 +852,64 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with } case Event(cmd: CMD_SPLICE, d: DATA_NORMAL) => - if (d.commitments.params.remoteParams.initFeatures.hasFeature(Features.Splicing)) { + if (d.commitments.active.count(_.fundingTxIndex == d.commitments.latest.fundingTxIndex) > 1) { + log.warning("cannot initiate splice, the previous splice has unconfirmed rbf attempts") + cmd.replyTo ! RES_FAILURE(cmd, InvalidSpliceWithUnconfirmedRbf(d.channelId, d.commitments.active.filter(_.fundingTxIndex == d.commitments.latest.fundingTxIndex).map(_.fundingTxId))) + stay() + } else if (!d.commitments.params.remoteParams.initFeatures.hasFeature(Features.Splicing)) { + log.warning("cannot initiate splice, peer doesn't support splicing") + cmd.replyTo ! RES_FAILURE(cmd, CommandUnavailableInThisState(d.channelId, "splice", NORMAL)) + stay() + } else { d.spliceStatus match { case SpliceStatus.NoSplice => startSingleTimer(QuiescenceTimeout.toString, QuiescenceTimeout(peer), nodeParams.channelConf.quiescenceTimeout) if (d.commitments.localIsQuiescent) { - stay() using d.copy(spliceStatus = SpliceStatus.InitiatorQuiescent(cmd)) sending Stfu(d.channelId, initiator = true) + val stfu = Stfu(d.channelId, initiator = true) + val spliceStatus1 = SpliceStatus.NegotiatingQuiescence(Some(cmd), QuiescenceNegotiation.Initiator.SentStfu(stfu)) + stay() using d.copy(spliceStatus = spliceStatus1) sending stfu } else { - stay() using d.copy(spliceStatus = SpliceStatus.QuiescenceRequested(cmd)) + val spliceStatus1 = SpliceStatus.NegotiatingQuiescence(Some(cmd), QuiescenceNegotiation.Initiator.QuiescenceRequested) + stay() using d.copy(spliceStatus = spliceStatus1) } case _ => log.warning("cannot initiate splice, another one is already in progress") cmd.replyTo ! RES_FAILURE(cmd, InvalidSpliceAlreadyInProgress(d.channelId)) stay() } - } else { - log.warning("cannot initiate splice, peer doesn't support splices") - cmd.replyTo ! RES_FAILURE(cmd, CommandUnavailableInThisState(d.channelId, "splice", NORMAL)) - stay() + } + + case Event(cmd: CMD_BUMP_FUNDING_FEE, d: DATA_NORMAL) => + d.spliceStatus match { + case SpliceStatus.NoSplice => + d.commitments.latest.localFundingStatus match { + case fundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx if fundingTx.fundingParams.minNextFeerate <= cmd.targetFeerate => + startSingleTimer(QuiescenceTimeout.toString, QuiescenceTimeout(peer), nodeParams.channelConf.quiescenceTimeout) + if (d.commitments.localIsQuiescent) { + val stfu = Stfu(d.channelId, initiator = true) + val spliceStatus1 = SpliceStatus.NegotiatingQuiescence(Some(cmd), QuiescenceNegotiation.Initiator.SentStfu(stfu)) + stay() using d.copy(spliceStatus = spliceStatus1) sending stfu + } else { + val spliceStatus1 = SpliceStatus.NegotiatingQuiescence(Some(cmd), QuiescenceNegotiation.Initiator.QuiescenceRequested) + stay() using d.copy(spliceStatus = spliceStatus1) + } + case fundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx => + log.warning("cannot initiate rbf: feerate too low ({} < {})", cmd.targetFeerate, fundingTx.fundingParams.minNextFeerate) + cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfFeerate(d.channelId, cmd.targetFeerate, fundingTx.fundingParams.minNextFeerate)) + stay() + case _: LocalFundingStatus.ZeroconfPublishedFundingTx => + log.warning("cannot initiate rbf: transaction is using 0-conf") + cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfZeroConf(d.channelId)) + stay() + case _ => + log.warning("cannot initiate rbf: transaction already confirmed") + cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfTxConfirmed(d.channelId)) + stay() + } + case _ => + log.warning("cannot initiate rbf, a concurrent splice attempt is already in progress") + cmd.replyTo ! RES_FAILURE(cmd, InvalidSpliceAlreadyInProgress(d.channelId)) + stay() } case Event(_: Stfu, d: DATA_NORMAL) if d.localShutdown.isDefined => @@ -884,24 +925,34 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with if (d.commitments.localIsQuiescent) { stay() using d.copy(spliceStatus = SpliceStatus.NonInitiatorQuiescent) sending Stfu(d.channelId, initiator = false) } else { - stay() using d.copy(spliceStatus = SpliceStatus.ReceivedStfu(msg)) + stay() using d.copy(spliceStatus = SpliceStatus.NegotiatingQuiescence(None, QuiescenceNegotiation.NonInitiator.ReceivedStfu(msg))) } - case SpliceStatus.QuiescenceRequested(cmd) => + case SpliceStatus.NegotiatingQuiescence(cmd_opt, QuiescenceNegotiation.Initiator.QuiescenceRequested) => // We could keep track of our splice attempt and merge it with the remote splice instead of cancelling it. // But this is an edge case that should rarely occur, so it's probably not worth the additional complexity. log.warning("our peer initiated quiescence before us, cancelling our splice attempt") - cmd.replyTo ! RES_FAILURE(cmd, ConcurrentRemoteSplice(d.channelId)) - stay() using d.copy(spliceStatus = SpliceStatus.ReceivedStfu(msg)) - case SpliceStatus.InitiatorQuiescent(cmd) => - // if both sides send stfu at the same time, the quiescence initiator is the channel opener + cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, ConcurrentRemoteSplice(d.channelId))) + stay() using d.copy(spliceStatus = SpliceStatus.NegotiatingQuiescence(None, QuiescenceNegotiation.NonInitiator.ReceivedStfu(msg))) + case SpliceStatus.NegotiatingQuiescence(Some(cmd), QuiescenceNegotiation.Initiator.SentStfu(_)) => + // if both sides send stfu at the same time, the quiescence initiator is the channel opener. if (!msg.initiator || d.commitments.params.localParams.isChannelOpener) { - initiateSplice(cmd, d) match { - case Left(f) => - cmd.replyTo ! RES_FAILURE(cmd, f) - context.system.scheduler.scheduleOnce(2 second, peer, Peer.Disconnect(remoteNodeId)) - stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending Warning(d.channelId, f.getMessage) - case Right(spliceInit) => - stay() using d.copy(spliceStatus = SpliceStatus.SpliceRequested(cmd, spliceInit)) sending spliceInit + cmd match { + case cmd: CMD_SPLICE => initiateSplice(cmd, d) match { + case Left(f) => + cmd.replyTo ! RES_FAILURE(cmd, f) + context.system.scheduler.scheduleOnce(2 second, peer, Peer.Disconnect(remoteNodeId)) + stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending Warning(d.channelId, f.getMessage) + case Right(spliceInit) => + stay() using d.copy(spliceStatus = SpliceStatus.SpliceRequested(cmd, spliceInit)) sending spliceInit + } + case cmd: CMD_BUMP_FUNDING_FEE => initiateSpliceRbf(cmd, d) match { + case Left(f) => + cmd.replyTo ! RES_FAILURE(cmd, f) + context.system.scheduler.scheduleOnce(2 second, peer, Peer.Disconnect(remoteNodeId)) + stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending Warning(d.channelId, f.getMessage) + case Right(txInitRbf) => + stay() using d.copy(spliceStatus = SpliceStatus.RbfRequested(cmd, txInitRbf)) sending txInitRbf + } } } else { log.warning("concurrent stfu received and our peer is the channel initiator, cancelling our splice attempt") @@ -930,6 +981,10 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with } else if (msg.feerate < nodeParams.currentFeerates.minimum) { log.info("rejecting splice request: feerate too low") stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceRequest(d.channelId).getMessage) + } else if (d.commitments.active.count(_.fundingTxIndex == d.commitments.latest.fundingTxIndex) > 1) { + val previousTxs = d.commitments.active.filter(_.fundingTxIndex == d.commitments.latest.fundingTxIndex).map(_.fundingTxId) + log.info("rejecting splice request: the previous splice has unconfirmed rbf attempts ({})", previousTxs.mkString(", ")) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceWithUnconfirmedRbf(d.channelId, previousTxs).getMessage) } else { log.info(s"accepting splice with remote.in.amount=${msg.fundingContribution} remote.in.push=${msg.pushAmount}") val parentCommitment = d.commitments.latest.commitment @@ -1019,6 +1074,100 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with stay() sending Warning(d.channelId, UnexpectedInteractiveTxMessage(d.channelId, msg).getMessage) } + case Event(msg: TxInitRbf, d: DATA_NORMAL) => + d.spliceStatus match { + case SpliceStatus.NonInitiatorQuiescent => + getSpliceRbfContext(None, d) match { + case Right(_) if !d.commitments.isQuiescent => + log.info("rejecting rbf request: channel not quiescent") + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceNotQuiescent(d.channelId).getMessage) + case Right(rbf) if msg.feerate < rbf.latestFundingTx.fundingParams.minNextFeerate => + log.info("rejecting rbf request: the new feerate must be at least {} (proposed={})", rbf.latestFundingTx.fundingParams.minNextFeerate, msg.feerate) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidRbfFeerate(d.channelId, msg.feerate, rbf.latestFundingTx.fundingParams.minNextFeerate).getMessage) + case Right(rbf) if nodeParams.currentBlockHeight < rbf.latestFundingTx.createdAt + nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks => + log.info("rejecting rbf attempt: last attempt was less than {} blocks ago", nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidRbfAttemptTooSoon(d.channelId, rbf.latestFundingTx.createdAt, rbf.latestFundingTx.createdAt + nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks).getMessage) + case Right(rbf) => + log.info("accepting rbf with remote.in.amount={}", msg.fundingContribution) + // We use the same contribution as the previous splice attempt. + val fundingContribution = rbf.latestFundingTx.fundingParams.localContribution + val txAckRbf = TxAckRbf(d.channelId, fundingContribution, rbf.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote) + val fundingParams = InteractiveTxParams( + channelId = d.channelId, + isInitiator = false, + localContribution = fundingContribution, + remoteContribution = msg.fundingContribution, + sharedInput_opt = Some(Multisig2of2Input(rbf.parentCommitment)), + remoteFundingPubKey = rbf.latestFundingTx.fundingParams.remoteFundingPubKey, + localOutputs = rbf.latestFundingTx.fundingParams.localOutputs, + lockTime = msg.lockTime, + dustLimit = rbf.latestFundingTx.fundingParams.dustLimit, + targetFeerate = msg.feerate, + requireConfirmedInputs = RequireConfirmedInputs(forLocal = msg.requireConfirmedInputs, forRemote = txAckRbf.requireConfirmedInputs) + ) + val sessionId = randomBytes32() + val txBuilder = context.spawnAnonymous(InteractiveTxBuilder( + sessionId, + nodeParams, fundingParams, + channelParams = d.commitments.params, + purpose = rbf, + localPushAmount = 0 msat, remotePushAmount = 0 msat, + wallet + )) + txBuilder ! InteractiveTxBuilder.Start(self) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceInProgress(cmd_opt = None, sessionId, txBuilder, remoteCommitSig = None)) sending txAckRbf + case Left(f) => + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, f.getMessage) + } + case SpliceStatus.NoSplice => + log.info("rejecting rbf attempt: quiescence not negotiated") + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceNotQuiescent(d.channelId).getMessage) + case SpliceStatus.SpliceAborted => + log.info("rejecting rbf attempt: our previous tx_abort was not acked") + stay() sending Warning(d.channelId, InvalidRbfTxAbortNotAcked(d.channelId).getMessage) + case _ => + log.info("rejecting rbf attempt: the current attempt must be completed or aborted first") + stay() sending Warning(d.channelId, InvalidRbfAlreadyInProgress(d.channelId).getMessage) + } + + case Event(msg: TxAckRbf, d: DATA_NORMAL) => + d.spliceStatus match { + case SpliceStatus.RbfRequested(cmd, txInitRbf) => + getSpliceRbfContext(Some(cmd), d) match { + case Right(rbf) => + log.info("our peer accepted our rbf request and will contribute {} to the funding transaction", msg.fundingContribution) + val fundingParams = InteractiveTxParams( + channelId = d.channelId, + isInitiator = true, + localContribution = txInitRbf.fundingContribution, + remoteContribution = msg.fundingContribution, + sharedInput_opt = Some(Multisig2of2Input(rbf.parentCommitment)), + remoteFundingPubKey = rbf.latestFundingTx.fundingParams.remoteFundingPubKey, + localOutputs = rbf.latestFundingTx.fundingParams.localOutputs, + lockTime = txInitRbf.lockTime, + dustLimit = rbf.latestFundingTx.fundingParams.dustLimit, + targetFeerate = txInitRbf.feerate, + requireConfirmedInputs = RequireConfirmedInputs(forLocal = msg.requireConfirmedInputs, forRemote = txInitRbf.requireConfirmedInputs) + ) + val sessionId = randomBytes32() + val txBuilder = context.spawnAnonymous(InteractiveTxBuilder( + sessionId, + nodeParams, fundingParams, + channelParams = d.commitments.params, + purpose = rbf, + localPushAmount = 0 msat, remotePushAmount = 0 msat, + wallet + )) + txBuilder ! InteractiveTxBuilder.Start(self) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceInProgress(cmd_opt = Some(cmd), sessionId, txBuilder, remoteCommitSig = None)) + case Left(f) => + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, f.getMessage) + } + case _ => + log.info("ignoring unexpected tx_ack_rbf={}", msg) + stay() + } + case Event(msg: TxAbort, d: DATA_NORMAL) => d.spliceStatus match { case SpliceStatus.SpliceInProgress(cmd_opt, _, txBuilder, _) => @@ -1034,6 +1183,10 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with log.info("our peer rejected our splice attempt: ascii='{}' bin={}", msg.toAscii, msg.data) cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException(s"splice attempt rejected by our peer: ${msg.toAscii}")) stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage) calling endQuiescence(d) + case SpliceStatus.RbfRequested(cmd, _) => + log.info("our peer rejected our rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data) + cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException(s"rbf attempt rejected by our peer: ${msg.toAscii}")) + stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage) calling endQuiescence(d) case SpliceStatus.NonInitiatorQuiescent => log.info("our peer aborted their own splice attempt: ascii='{}' bin={}", msg.toAscii, msg.data) stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage) calling endQuiescence(d) @@ -1044,7 +1197,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with log.info("our peer wants to abort the splice, but we've already negotiated a splice transaction: ascii='{}' bin={}", msg.toAscii, msg.data) // We ack their tx_abort but we keep monitoring the funding transaction until it's confirmed or double-spent. stay() sending TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage) - case _: QuiescenceNegotiation => + case _: SpliceStatus.NegotiatingQuiescence => log.info("our peer aborted the splice during quiescence negotiation, disconnecting: ascii='{}' bin={}", msg.toAscii, msg.data) // NB: we use a small delay to ensure we've sent our warning before disconnecting. context.system.scheduler.scheduleOnce(2 second, peer, Peer.Disconnect(remoteNodeId)) @@ -1876,8 +2029,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with val channelKeyPath = keyManager.keyPath(d.commitments.params.localParams, d.commitments.params.channelConfig) val myCurrentPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, d.commitments.localCommitIndex) val rbfTlv: Set[ChannelReestablishTlv] = d match { - case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => d.rbfStatus match { - case RbfStatus.RbfWaitingForSigs(status) => Set(ChannelReestablishTlv.NextFundingTlv(status.fundingTx.txId)) + case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => d.status match { + case DualFundingStatus.RbfWaitingForSigs(status) => Set(ChannelReestablishTlv.NextFundingTlv(status.fundingTx.txId)) case _ => d.latestFundingTx.sharedTx match { case _: InteractiveTxBuilder.PartiallySignedSharedTransaction => Set(ChannelReestablishTlv.NextFundingTlv(d.latestFundingTx.sharedTx.txId)) case _: InteractiveTxBuilder.FullySignedSharedTransaction => Set.empty @@ -1940,8 +2093,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with case Event(channelReestablish: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => channelReestablish.nextFundingTxId_opt match { case Some(fundingTxId) => - d.rbfStatus match { - case RbfStatus.RbfWaitingForSigs(signingSession) if signingSession.fundingTx.txId == fundingTxId => + d.status match { + case DualFundingStatus.RbfWaitingForSigs(signingSession) if signingSession.fundingTx.txId == fundingTxId => // We retransmit our commit_sig, and will send our tx_signatures once we've received their commit_sig. val commitSig = signingSession.remoteCommit.sign(keyManager, d.commitments.params, signingSession.fundingTxIndex, signingSession.fundingParams.remoteFundingPubKey, signingSession.commitInput) goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) sending commitSig @@ -1959,7 +2112,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with case _ => // The fundingTxId must be for an RBF attempt that we didn't store (we got disconnected before receiving // their tx_complete): we tell them to abort that RBF attempt. - goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) + goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) } case None => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) } @@ -2772,6 +2925,50 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with } } + private def initiateSpliceRbf(cmd: CMD_BUMP_FUNDING_FEE, d: DATA_NORMAL): Either[ChannelException, TxInitRbf] = { + getSpliceRbfContext(Some(cmd), d).flatMap(rbf => { + // We use the same contribution as the previous splice attempt. + val fundingContribution = rbf.latestFundingTx.fundingParams.localContribution + val commitTxFees = if (d.commitments.params.localParams.paysCommitTxFees) { + Transactions.commitTxTotalCost(d.commitments.params.remoteParams.dustLimit, rbf.parentCommitment.remoteCommit.spec, d.commitments.params.commitmentFormat) + } else { + 0.sat + } + if (!d.commitments.isQuiescent) { + log.warning("cannot initiate rbf, channel is not quiescent") + Left(InvalidSpliceNotQuiescent(d.channelId)) + } else if (fundingContribution < 0.sat && rbf.parentCommitment.localCommit.spec.toLocal + fundingContribution < rbf.parentCommitment.localChannelReserve(d.commitments.params).max(commitTxFees)) { + log.warning(s"cannot do rbf: insufficient funds (commitTxFees=$commitTxFees reserve=${rbf.parentCommitment.localChannelReserve(d.commitments.params)})") + Left(InvalidSpliceRequest(d.channelId)) + } else { + val txInitRbf = TxInitRbf(d.channelId, cmd.lockTime, cmd.targetFeerate, fundingContribution, rbf.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote) + Right(txInitRbf) + } + }) + } + + private def getSpliceRbfContext(cmd_opt: Option[CMD_BUMP_FUNDING_FEE], d: DATA_NORMAL): Either[ChannelException, InteractiveTxBuilder.SpliceTxRbf] = { + d.commitments.latest.localFundingStatus match { + case fundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx => + val fundingTxIndex = d.commitments.latest.fundingTxIndex + d.commitments.active.find(_.fundingTxIndex == fundingTxIndex - 1) match { + case Some(parentCommitment) => + val previousTxs = d.commitments.active + .filter(_.fundingTxIndex == fundingTxIndex) + .map(_.localFundingStatus) + // All transactions should be unconfirmed, otherwise we wouldn't be in an RBF context. + .collect { case tx: LocalFundingStatus.DualFundedUnconfirmedFundingTx => tx.sharedTx } + Right(InteractiveTxBuilder.SpliceTxRbf(parentCommitment, fundingTx, previousTxs, cmd_opt.map(_.fundingFeeBudget))) + case None => + log.warning("cannot initiate rbf: cannot find parent commitment") + Left(InvalidRbfAttempt(d.channelId)) + } + case _ => + log.warning("cannot initiate rbf: transaction already confirmed") + Left(InvalidRbfTxConfirmed(d.channelId)) + } + } + private def handleQuiescenceTimeout(d: DATA_NORMAL): State = { if (d.spliceStatus == SpliceStatus.NoSplice) { log.warning("quiescence timed out with no ongoing splice, did we forget to cancel the timer?") @@ -2794,9 +2991,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with private def reportSpliceFailure(spliceStatus: SpliceStatus, f: Throwable): Unit = { val cmd_opt = spliceStatus match { - case SpliceStatus.QuiescenceRequested(cmd) => Some(cmd) - case SpliceStatus.InitiatorQuiescent(cmd) => Some(cmd) + case SpliceStatus.NegotiatingQuiescence(cmd_opt, _) => cmd_opt case SpliceStatus.SpliceRequested(cmd, _) => Some(cmd) + case SpliceStatus.RbfRequested(cmd, _) => Some(cmd) case SpliceStatus.SpliceInProgress(cmd_opt, _, txBuilder, _) => txBuilder ! InteractiveTxBuilder.Abort cmd_opt diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala index a13ba920ef..2f2a5dd449 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala @@ -390,7 +390,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { remotePerCommitmentSecrets = ShaChain.init, originChannels = Map.empty ) - val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, d.localPushAmount, d.remotePushAmount, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, RbfStatus.NoRbf, None) + val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, d.localPushAmount, d.remotePushAmount, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, DualFundingStatus.WaitingForConfirmations, None) goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d1 storing() sending signingSession1.localSigs } } @@ -413,7 +413,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { remotePerCommitmentSecrets = ShaChain.init, originChannels = Map.empty ) - val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, d.localPushAmount, d.remotePushAmount, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, RbfStatus.NoRbf, None) + val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, d.localPushAmount, d.remotePushAmount, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, DualFundingStatus.WaitingForConfirmations, None) goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d1 storing() sending signingSession.localSigs calling publishFundingTx(signingSession.fundingTx) } case msg: TxAbort => @@ -461,17 +461,17 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { stay() using d1 storing() calling publishFundingTx(dfu1) } case _: FullySignedSharedTransaction => - d.rbfStatus match { - case RbfStatus.RbfWaitingForSigs(signingSession) => + d.status match { + case DualFundingStatus.RbfWaitingForSigs(signingSession) => signingSession.receiveTxSigs(nodeParams, d.commitments.params, txSigs) match { case Left(f) => rollbackRbfAttempt(signingSession, d) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, f.getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, f.getMessage) case Right(signingSession1) => val minDepth_opt = d.commitments.params.minDepthDualFunding(nodeParams.channelConf.minDepthBlocks, signingSession1.fundingTx.sharedTx.tx) watchFundingConfirmed(signingSession.fundingTx.txId, minDepth_opt, delay_opt = None) val commitments1 = d.commitments.add(signingSession1.commitment) - val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments1, d.localPushAmount, d.remotePushAmount, d.waitingSince, d.lastChecked, RbfStatus.NoRbf, d.deferred) + val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments1, d.localPushAmount, d.remotePushAmount, d.waitingSince, d.lastChecked, DualFundingStatus.WaitingForConfirmations, d.deferred) stay() using d1 storing() sending signingSession1.localSigs calling publishFundingTx(signingSession1.fundingTx) } case _ if d.commitments.all.exists(_.fundingTxId == txSigs.txId) => @@ -479,8 +479,8 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { stay() case _ => log.debug("rejecting unexpected tx_signatures for txId={}", txSigs.txId) - reportRbfFailure(d.rbfStatus, UnexpectedFundingSignatures(d.channelId)) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, UnexpectedFundingSignatures(d.channelId).getMessage) + reportRbfFailure(d.status, UnexpectedFundingSignatures(d.channelId)) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, UnexpectedFundingSignatures(d.channelId).getMessage) } } @@ -493,15 +493,15 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfZeroConf(d.channelId)) stay() } else { - d.rbfStatus match { - case RbfStatus.NoRbf => + d.status match { + case DualFundingStatus.WaitingForConfirmations => val minNextFeerate = d.latestFundingTx.fundingParams.minNextFeerate if (cmd.targetFeerate < minNextFeerate) { cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfFeerate(d.channelId, cmd.targetFeerate, minNextFeerate)) stay() } else { val txInitRbf = TxInitRbf(d.channelId, cmd.lockTime, cmd.targetFeerate, d.latestFundingTx.fundingParams.localContribution, nodeParams.channelConf.requireConfirmedInputsForDualFunding) - stay() using d.copy(rbfStatus = RbfStatus.RbfRequested(cmd)) sending txInitRbf + stay() using d.copy(status = DualFundingStatus.RbfRequested(cmd)) sending txInitRbf } case _ => log.warning("cannot initiate rbf, another one is already in progress") @@ -518,24 +518,24 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { stay() sending Error(d.channelId, InvalidRbfNonInitiator(d.channelId).getMessage) } else if (zeroConf) { log.info("rejecting tx_init_rbf, we're using zero-conf") - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfZeroConf(d.channelId).getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfZeroConf(d.channelId).getMessage) } else { val minNextFeerate = d.latestFundingTx.fundingParams.minNextFeerate - d.rbfStatus match { - case RbfStatus.NoRbf => + d.status match { + case DualFundingStatus.WaitingForConfirmations => val remainingRbfAttempts = nodeParams.channelConf.remoteRbfLimits.maxAttempts - d.previousFundingTxs.length if (msg.feerate < minNextFeerate) { log.info("rejecting rbf attempt: the new feerate must be at least {} (proposed={})", minNextFeerate, msg.feerate) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfFeerate(d.channelId, msg.feerate, minNextFeerate).getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfFeerate(d.channelId, msg.feerate, minNextFeerate).getMessage) } else if (d.remotePushAmount > msg.fundingContribution) { log.info("rejecting rbf attempt: invalid amount pushed (fundingAmount={}, pushAmount={})", msg.fundingContribution, d.remotePushAmount) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, InvalidPushAmount(d.channelId, d.remotePushAmount, msg.fundingContribution.toMilliSatoshi).getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, InvalidPushAmount(d.channelId, d.remotePushAmount, msg.fundingContribution.toMilliSatoshi).getMessage) } else if (remainingRbfAttempts <= 0) { log.info("rejecting rbf attempt: maximum number of attempts reached (max={})", nodeParams.channelConf.remoteRbfLimits.maxAttempts) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfAttemptsExhausted(d.channelId, nodeParams.channelConf.remoteRbfLimits.maxAttempts).getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfAttemptsExhausted(d.channelId, nodeParams.channelConf.remoteRbfLimits.maxAttempts).getMessage) } else if (nodeParams.currentBlockHeight < d.latestFundingTx.createdAt + nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks) { log.info("rejecting rbf attempt: last attempt was less than {} blocks ago", nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfAttemptTooSoon(d.channelId, d.latestFundingTx.createdAt, d.latestFundingTx.createdAt + nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks).getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, InvalidRbfAttemptTooSoon(d.channelId, d.latestFundingTx.createdAt, d.latestFundingTx.createdAt + nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks).getMessage) } else { log.info("our peer wants to raise the feerate of the funding transaction (previous={} target={})", d.latestFundingTx.fundingParams.targetFeerate, msg.feerate) val fundingParams = d.latestFundingTx.fundingParams.copy( @@ -549,7 +549,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { randomBytes32(), nodeParams, fundingParams, channelParams = d.commitments.params, - purpose = InteractiveTxBuilder.PreviousTxRbf(d.commitments.active.head, 0 msat, 0 msat, previousTransactions = d.allFundingTxs.map(_.sharedTx), feeBudget_opt = None), + purpose = InteractiveTxBuilder.FundingTxRbf(d.commitments.active.head, previousTransactions = d.allFundingTxs.map(_.sharedTx), feeBudget_opt = None), localPushAmount = d.localPushAmount, remotePushAmount = d.remotePushAmount, wallet)) txBuilder ! InteractiveTxBuilder.Start(self) @@ -557,25 +557,25 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { Some(TxAckRbf(d.channelId, fundingParams.localContribution, nodeParams.channelConf.requireConfirmedInputsForDualFunding)), if (remainingRbfAttempts <= 3) Some(Warning(d.channelId, s"will accept at most ${remainingRbfAttempts - 1} future rbf attempts")) else None, ).flatten - stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(cmd_opt = None, txBuilder, remoteCommitSig = None)) sending toSend + stay() using d.copy(status = DualFundingStatus.RbfInProgress(cmd_opt = None, txBuilder, remoteCommitSig = None)) sending toSend } - case RbfStatus.RbfAborted => + case DualFundingStatus.RbfAborted => log.info("rejecting rbf attempt: our previous tx_abort was not acked") stay() sending Warning(d.channelId, InvalidRbfTxAbortNotAcked(d.channelId).getMessage) - case _: RbfStatus.RbfRequested | _: RbfStatus.RbfInProgress | _: RbfStatus.RbfWaitingForSigs => + case _: DualFundingStatus.RbfRequested | _: DualFundingStatus.RbfInProgress | _: DualFundingStatus.RbfWaitingForSigs => log.info("rejecting rbf attempt: the current rbf attempt must be completed or aborted first") stay() sending Warning(d.channelId, InvalidRbfAlreadyInProgress(d.channelId).getMessage) } } case Event(msg: TxAckRbf, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - d.rbfStatus match { - case RbfStatus.RbfRequested(cmd) if d.remotePushAmount > msg.fundingContribution => + d.status match { + case DualFundingStatus.RbfRequested(cmd) if d.remotePushAmount > msg.fundingContribution => log.info("rejecting rbf attempt: invalid amount pushed (fundingAmount={}, pushAmount={})", msg.fundingContribution, d.remotePushAmount) val error = InvalidPushAmount(d.channelId, d.remotePushAmount, msg.fundingContribution.toMilliSatoshi) cmd.replyTo ! RES_FAILURE(cmd, error) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, error.getMessage) - case RbfStatus.RbfRequested(cmd) => + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, error.getMessage) + case DualFundingStatus.RbfRequested(cmd) => log.info("our peer accepted our rbf attempt and will contribute {} to the funding transaction", msg.fundingContribution) val fundingParams = d.latestFundingTx.fundingParams.copy( // we don't change our funding contribution @@ -587,19 +587,19 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { randomBytes32(), nodeParams, fundingParams, channelParams = d.commitments.params, - purpose = InteractiveTxBuilder.PreviousTxRbf(d.commitments.active.head, 0 msat, 0 msat, previousTransactions = d.allFundingTxs.map(_.sharedTx), feeBudget_opt = Some(cmd.fundingFeeBudget)), + purpose = InteractiveTxBuilder.FundingTxRbf(d.commitments.active.head, previousTransactions = d.allFundingTxs.map(_.sharedTx), feeBudget_opt = Some(cmd.fundingFeeBudget)), localPushAmount = d.localPushAmount, remotePushAmount = d.remotePushAmount, wallet)) txBuilder ! InteractiveTxBuilder.Start(self) - stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(cmd_opt = Some(cmd), txBuilder, remoteCommitSig = None)) + stay() using d.copy(status = DualFundingStatus.RbfInProgress(cmd_opt = Some(cmd), txBuilder, remoteCommitSig = None)) case _ => log.info("ignoring unexpected tx_ack_rbf") stay() sending Warning(d.channelId, UnexpectedInteractiveTxMessage(d.channelId, msg).getMessage) } case Event(msg: InteractiveTxConstructionMessage, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - d.rbfStatus match { - case RbfStatus.RbfInProgress(_, txBuilder, _) => + d.status match { + case DualFundingStatus.RbfInProgress(_, txBuilder, _) => txBuilder ! InteractiveTxBuilder.ReceiveMessage(msg) stay() case _ => @@ -608,24 +608,24 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { } case Event(commitSig: CommitSig, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - d.rbfStatus match { - case s: RbfStatus.RbfInProgress => + d.status match { + case s: DualFundingStatus.RbfInProgress => log.debug("received their commit_sig, deferring message") - stay() using d.copy(rbfStatus = s.copy(remoteCommitSig = Some(commitSig))) - case RbfStatus.RbfWaitingForSigs(signingSession) => + stay() using d.copy(status = s.copy(remoteCommitSig = Some(commitSig))) + case DualFundingStatus.RbfWaitingForSigs(signingSession) => signingSession.receiveCommitSig(nodeParams, d.commitments.params, commitSig) match { case Left(f) => rollbackRbfAttempt(signingSession, d) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, f.getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, f.getMessage) case Right(signingSession1) => signingSession1 match { case signingSession1: InteractiveTxSigningSession.WaitingForSigs => // No need to store their commit_sig, they will re-send it if we disconnect. - stay() using d.copy(rbfStatus = RbfStatus.RbfWaitingForSigs(signingSession1)) + stay() using d.copy(status = DualFundingStatus.RbfWaitingForSigs(signingSession1)) case signingSession1: InteractiveTxSigningSession.SendingSigs => val minDepth_opt = d.commitments.params.minDepthDualFunding(nodeParams.channelConf.minDepthBlocks, signingSession1.fundingTx.sharedTx.tx) watchFundingConfirmed(signingSession.fundingTx.txId, minDepth_opt, delay_opt = None) val commitments1 = d.commitments.add(signingSession1.commitment) - val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments1, d.localPushAmount, d.remotePushAmount, d.waitingSince, d.lastChecked, RbfStatus.NoRbf, d.deferred) + val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments1, d.localPushAmount, d.remotePushAmount, d.waitingSince, d.lastChecked, DualFundingStatus.WaitingForConfirmations, d.deferred) stay() using d1 storing() sending signingSession1.localSigs } } @@ -635,47 +635,47 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { } case Event(msg: TxAbort, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - d.rbfStatus match { - case RbfStatus.RbfInProgress(cmd_opt, txBuilder, _) => + d.status match { + case DualFundingStatus.RbfInProgress(cmd_opt, txBuilder, _) => log.info("our peer aborted the rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data) cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, RbfAttemptAborted(d.channelId))) txBuilder ! InteractiveTxBuilder.Abort - stay() using d.copy(rbfStatus = RbfStatus.NoRbf) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) - case RbfStatus.RbfWaitingForSigs(signingSession) => + stay() using d.copy(status = DualFundingStatus.WaitingForConfirmations) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) + case DualFundingStatus.RbfWaitingForSigs(signingSession) => log.info("our peer aborted the rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data) rollbackRbfAttempt(signingSession, d) - stay() using d.copy(rbfStatus = RbfStatus.NoRbf) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) - case RbfStatus.RbfRequested(cmd) => + stay() using d.copy(status = DualFundingStatus.WaitingForConfirmations) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) + case DualFundingStatus.RbfRequested(cmd) => log.info("our peer rejected our rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data) cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException(s"rbf attempt rejected by our peer: ${msg.toAscii}")) - stay() using d.copy(rbfStatus = RbfStatus.NoRbf) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) - case RbfStatus.RbfAborted => + stay() using d.copy(status = DualFundingStatus.WaitingForConfirmations) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage) + case DualFundingStatus.RbfAborted => log.debug("our peer acked our previous tx_abort") - stay() using d.copy(rbfStatus = RbfStatus.NoRbf) - case RbfStatus.NoRbf => + stay() using d.copy(status = DualFundingStatus.WaitingForConfirmations) + case DualFundingStatus.WaitingForConfirmations => log.info("our peer wants to abort the dual funding flow, but we've already negotiated a funding transaction: ascii='{}' bin={}", msg.toAscii, msg.data) // We ack their tx_abort but we keep monitoring the funding transaction until it's confirmed or double-spent. stay() sending TxAbort(d.channelId, DualFundingAborted(d.channelId).getMessage) } case Event(msg: InteractiveTxBuilder.Response, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - d.rbfStatus match { - case RbfStatus.RbfInProgress(cmd_opt, _, remoteCommitSig_opt) => + d.status match { + case DualFundingStatus.RbfInProgress(cmd_opt, _, remoteCommitSig_opt) => msg match { case InteractiveTxBuilder.SendMessage(_, msg) => stay() sending msg case InteractiveTxBuilder.Succeeded(signingSession, commitSig) => cmd_opt.foreach(cmd => cmd.replyTo ! RES_BUMP_FUNDING_FEE(rbfIndex = d.previousFundingTxs.length, signingSession.fundingTx.txId, signingSession.fundingTx.tx.localFees.truncateToSatoshi)) remoteCommitSig_opt.foreach(self ! _) - val d1 = d.copy(rbfStatus = RbfStatus.RbfWaitingForSigs(signingSession)) + val d1 = d.copy(status = DualFundingStatus.RbfWaitingForSigs(signingSession)) stay() using d1 storing() sending commitSig case f: InteractiveTxBuilder.Failed => log.info("rbf attempt failed: {}", f.cause.getMessage) cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, f.cause)) - stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, f.cause.getMessage) + stay() using d.copy(status = DualFundingStatus.RbfAborted) sending TxAbort(d.channelId, f.cause.getMessage) } case _ => // This can happen if we received a tx_abort right before receiving the interactive-tx result. - log.warning("ignoring interactive-tx result with rbfStatus={}", d.rbfStatus.getClass.getSimpleName) + log.warning("ignoring interactive-tx result with rbfStatus={}", d.status.getClass.getSimpleName) stay() } @@ -700,10 +700,10 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { val realScidStatus = RealScidStatus.Temporary(RealShortChannelId(w.blockHeight, w.txIndex, commitment.commitInput.outPoint.index.toInt)) val shortIds = createShortIds(d.channelId, realScidStatus) val channelReady = createChannelReady(shortIds, d.commitments.params) - reportRbfFailure(d.rbfStatus, InvalidRbfTxConfirmed(d.channelId)) - val toSend = d.rbfStatus match { - case RbfStatus.NoRbf | RbfStatus.RbfAborted => Seq(channelReady) - case _: RbfStatus.RbfRequested | _: RbfStatus.RbfInProgress | _: RbfStatus.RbfWaitingForSigs => Seq(TxAbort(d.channelId, InvalidRbfTxConfirmed(d.channelId).getMessage), channelReady) + reportRbfFailure(d.status, InvalidRbfTxConfirmed(d.channelId)) + val toSend = d.status match { + case DualFundingStatus.WaitingForConfirmations | DualFundingStatus.RbfAborted => Seq(channelReady) + case _: DualFundingStatus.RbfRequested | _: DualFundingStatus.RbfInProgress | _: DualFundingStatus.RbfWaitingForSigs => Seq(TxAbort(d.channelId, InvalidRbfTxConfirmed(d.channelId).getMessage), channelReady) } d.deferred.foreach(self ! _) goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments1, shortIds) storing() sending toSend @@ -713,7 +713,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { case Event(ProcessCurrentBlockHeight(c), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleNewBlockDualFundingUnconfirmed(c, d) case Event(e: BITCOIN_FUNDING_DOUBLE_SPENT, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - reportRbfFailure(d.rbfStatus, FundingTxDoubleSpent(d.channelId)) + reportRbfFailure(d.status, FundingTxDoubleSpent(d.channelId)) handleDualFundingDoubleSpent(e, d) case Event(remoteChannelReady: ChannelReady, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => @@ -729,16 +729,16 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { stay() case Event(INPUT_DISCONNECTED, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - reportRbfFailure(d.rbfStatus, new RuntimeException("rbf attempt failed: disconnected")) - val d1 = d.rbfStatus match { + reportRbfFailure(d.status, new RuntimeException("rbf attempt failed: disconnected")) + val d1 = d.status match { // We keep track of the RBF status: we should be able to complete the signature steps on reconnection. - case _: RbfStatus.RbfWaitingForSigs => d - case _ => d.copy(rbfStatus = RbfStatus.NoRbf) + case _: DualFundingStatus.RbfWaitingForSigs => d + case _ => d.copy(status = DualFundingStatus.WaitingForConfirmations) } goto(OFFLINE) using d1 case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - reportRbfFailure(d.rbfStatus, new RuntimeException(s"remote error: ${e.toAscii}")) + reportRbfFailure(d.status, new RuntimeException(s"remote error: ${e.toAscii}")) handleRemoteError(e, d) }) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala index 953e6b7f94..eb26fa4df9 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala @@ -137,10 +137,10 @@ trait DualFundingHandlers extends CommonFundingHandlers { rollbackFundingAttempt(signingSession.fundingTx.tx, d.allFundingTxs.map(_.sharedTx)) } - def reportRbfFailure(rbfStatus: RbfStatus, f: Throwable): Unit = { - rbfStatus match { - case RbfStatus.RbfRequested(cmd) => cmd.replyTo ! RES_FAILURE(cmd, f) - case RbfStatus.RbfInProgress(cmd_opt, txBuilder, _) => + def reportRbfFailure(fundingStatus: DualFundingStatus, f: Throwable): Unit = { + fundingStatus match { + case DualFundingStatus.RbfRequested(cmd) => cmd.replyTo ! RES_FAILURE(cmd, f) + case DualFundingStatus.RbfInProgress(cmd_opt, txBuilder, _) => txBuilder ! InteractiveTxBuilder.Abort cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, f)) case _ => () diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala index 51a6b1645c..7b1de61349 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala @@ -194,9 +194,10 @@ object InteractiveTxBuilder { * only one of them ends up confirming. We guarantee this by having the latest transaction * always double-spend all its predecessors. */ - case class PreviousTxRbf(replacedCommitment: Commitment, previousLocalBalance: MilliSatoshi, previousRemoteBalance: MilliSatoshi, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], feeBudget_opt: Option[Satoshi]) extends Purpose { - // Note that the truncation is a no-op: the sum of balances in a channel must be a satoshi amount. - override val previousFundingAmount: Satoshi = (previousLocalBalance + previousRemoteBalance).truncateToSatoshi + case class FundingTxRbf(replacedCommitment: Commitment, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], feeBudget_opt: Option[Satoshi]) extends Purpose { + override val previousLocalBalance: MilliSatoshi = 0 msat + override val previousRemoteBalance: MilliSatoshi = 0 msat + override val previousFundingAmount: Satoshi = 0 sat override val localCommitIndex: Long = replacedCommitment.localCommit.index override val remoteCommitIndex: Long = replacedCommitment.remoteCommit.index override val remotePerCommitmentPoint: PublicKey = replacedCommitment.remoteCommit.remotePerCommitmentPoint @@ -204,6 +205,22 @@ object InteractiveTxBuilder { override val fundingTxIndex: Long = replacedCommitment.fundingTxIndex override val localHtlcs: Set[DirectedHtlc] = replacedCommitment.localCommit.spec.htlcs } + + /** + * @param previousTransactions splice RBF attempts all spend the previous funding transaction, so they automatically + * double-spend each other, but we reuse previous inputs as much as possible anyway. + */ + case class SpliceTxRbf(parentCommitment: Commitment, latestFundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], feeBudget_opt: Option[Satoshi]) extends Purpose { + override val previousLocalBalance: MilliSatoshi = parentCommitment.localCommit.spec.toLocal + override val previousRemoteBalance: MilliSatoshi = parentCommitment.remoteCommit.spec.toLocal + override val previousFundingAmount: Satoshi = parentCommitment.capacity + override val localCommitIndex: Long = parentCommitment.localCommit.index + override val remoteCommitIndex: Long = parentCommitment.remoteCommit.index + override val remotePerCommitmentPoint: PublicKey = parentCommitment.remoteCommit.remotePerCommitmentPoint + override val commitTxFeerate: FeeratePerKw = parentCommitment.localCommit.spec.commitTxFeerate + override val fundingTxIndex: Long = parentCommitment.fundingTxIndex + 1 + override val localHtlcs: Set[DirectedHtlc] = parentCommitment.localCommit.spec.htlcs + } // @formatter:on // @formatter:off @@ -401,7 +418,8 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon private val fundingPubkeyScript: ByteVector = Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingPubKey, fundingParams.remoteFundingPubKey))) private val remoteNodeId = channelParams.remoteParams.nodeId private val previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction] = purpose match { - case rbf: PreviousTxRbf => rbf.previousTransactions + case rbf: FundingTxRbf => rbf.previousTransactions + case rbf: SpliceTxRbf => rbf.previousTransactions case _ => Nil } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala index 97a8b7a3d3..be5d540f1c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala @@ -94,6 +94,23 @@ object InteractiveTxFunder { spliceInAmount - spliceOut.map(_.amount).sum - fees } + private def needsAdditionalFunding(fundingParams: InteractiveTxParams): Boolean = { + if (fundingParams.isInitiator) { + // We're the initiator, we always have to pay on-chain fees, 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 +154,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 } @@ -148,8 +166,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)) { 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 @@ -191,7 +208,8 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response val sharedInputWeight = fundingParams.sharedInput_opt.toSeq.map(i => i.info.outPoint -> i.weight.toLong).toMap val feeBudget_opt = purpose match { case p: FundingTx => p.feeBudget_opt - case p: PreviousTxRbf => p.feeBudget_opt + case p: FundingTxRbf => p.feeBudget_opt + case p: SpliceTxRbf => p.feeBudget_opt case _ => None } context.pipeToSelf(wallet.fundTransaction(txNotFunded, fundingParams.targetFeerate, replaceable = true, externalInputsWeight = sharedInputWeight, feeBudget_opt = feeBudget_opt)) { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4.scala index ec4fbe1326..ad191d5141 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4.scala @@ -607,9 +607,9 @@ private[channel] object ChannelCodecs4 { waitingForSigsCodec } - val rbfStatusCodec: Codec[RbfStatus] = discriminated[RbfStatus].by(uint8) - .\(0x01) { case status: RbfStatus if !status.isInstanceOf[RbfStatus.RbfWaitingForSigs] => RbfStatus.NoRbf }(provide(RbfStatus.NoRbf)) - .\(0x02) { case status: RbfStatus.RbfWaitingForSigs => status }(interactiveTxWaitingForSigsCodec.as[RbfStatus.RbfWaitingForSigs]) + val dualFundingStatusCodec: Codec[DualFundingStatus] = discriminated[DualFundingStatus].by(uint8) + .\(0x01) { case status: DualFundingStatus if !status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs] => DualFundingStatus.WaitingForConfirmations }(provide(DualFundingStatus.WaitingForConfirmations)) + .\(0x02) { case status: DualFundingStatus.RbfWaitingForSigs => status }(interactiveTxWaitingForSigsCodec.as[DualFundingStatus.RbfWaitingForSigs]) val spliceStatusCodec: Codec[SpliceStatus] = discriminated[SpliceStatus].by(uint8) .\(0x01) { case status: SpliceStatus if !status.isInstanceOf[SpliceStatus.SpliceWaitingForSigs] => SpliceStatus.NoSplice }(provide(SpliceStatus.NoSplice)) @@ -649,7 +649,7 @@ private[channel] object ChannelCodecs4 { ("remotePushAmount" | millisatoshi) :: ("waitingSince" | blockHeight) :: ("lastChecked" | blockHeight) :: - ("rbfStatus" | rbfStatusCodec) :: + ("status" | dualFundingStatusCodec) :: ("deferred" | optional(bool8, lengthDelimited(channelReadyCodec)))).as[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] val DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED_0c_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] = ( @@ -658,7 +658,7 @@ private[channel] object ChannelCodecs4 { ("remotePushAmount" | millisatoshi) :: ("waitingSince" | blockHeight) :: ("lastChecked" | blockHeight) :: - ("rbfStatus" | rbfStatusCodec) :: + ("status" | dualFundingStatusCodec) :: ("deferred" | optional(bool8, lengthDelimited(channelReadyCodec)))).as[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] val DATA_WAIT_FOR_DUAL_FUNDING_READY_03_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_READY] = ( diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala index 4dabd75781..b295110d6d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageTypes.scala @@ -49,8 +49,8 @@ sealed trait HasTemporaryChannelId extends LightningMessage { def temporaryChann sealed trait HasChannelId extends LightningMessage { def channelId: ByteVector32 } // <- not in the spec sealed trait HasChainHash extends LightningMessage { def chainHash: BlockHash } // <- not in the spec sealed trait HasSerialId extends LightningMessage { def serialId: UInt64 } // <- not in the spec -sealed trait ForbiddenMessageDuringSplice extends LightningMessage // <- not in the spec -sealed trait UpdateMessage extends HtlcMessage with ForbiddenMessageDuringSplice // <- not in the spec +sealed trait ForbiddenMessageWhenQuiescent extends LightningMessage // <- not in the spec +sealed trait UpdateMessage extends HtlcMessage with ForbiddenMessageWhenQuiescent // <- not in the spec sealed trait HtlcSettlementMessage extends UpdateMessage { def id: Long } // <- not in the spec sealed trait HtlcFailureMessage extends HtlcSettlementMessage // <- not in the spec // @formatter:on @@ -380,7 +380,7 @@ object ExperimentalSpliceLocked { case class Shutdown(channelId: ByteVector32, scriptPubKey: ByteVector, - tlvStream: TlvStream[ShutdownTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId with ForbiddenMessageDuringSplice + tlvStream: TlvStream[ShutdownTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId with ForbiddenMessageWhenQuiescent case class ClosingSigned(channelId: ByteVector32, feeSatoshis: Satoshi, diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala index 4a146d0094..54a8f1c29e 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala @@ -34,8 +34,8 @@ import fr.acinq.eclair.blockchain.{OnChainWallet, SingleKeyOnChainWallet} import fr.acinq.eclair.channel.fund.InteractiveTxBuilder._ import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession} import fr.acinq.eclair.io.OpenChannelInterceptor.makeChannelParams -import fr.acinq.eclair.transactions.{Scripts, Transactions} import fr.acinq.eclair.transactions.Transactions.InputInfo +import fr.acinq.eclair.transactions.{Scripts, Transactions} import fr.acinq.eclair.wire.protocol._ import fr.acinq.eclair.{Feature, FeatureSupport, Features, InitFeature, MilliSatoshiLong, NodeParams, TestConstants, TestKitBaseClass, ToMilliSatoshiConversion, UInt64, randomBytes32, randomKey} import org.scalatest.BeforeAndAfterAll @@ -132,7 +132,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit def spawnTxBuilderRbfAlice(fundingParams: InteractiveTxParams, commitment: Commitment, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( ByteVector32.Zeroes, nodeParamsA, fundingParams, channelParamsA, - PreviousTxRbf(commitment, 0 msat, 0 msat, previousTransactions, feeBudget_opt = None), + FundingTxRbf(commitment, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, wallet)) @@ -143,10 +143,10 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit 0 msat, 0 msat, wallet)) - def spawnTxBuilderSpliceRbfAlice(fundingParams: InteractiveTxParams, parentCommitment: Commitment, replacedCommitment: Commitment, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( + def spawnTxBuilderSpliceRbfAlice(fundingParams: InteractiveTxParams, parentCommitment: Commitment, latestFundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( ByteVector32.Zeroes, nodeParamsA, fundingParams, channelParamsA, - PreviousTxRbf(replacedCommitment, parentCommitment.localCommit.spec.toLocal, parentCommitment.remoteCommit.spec.toLocal, previousTransactions, feeBudget_opt = None), + SpliceTxRbf(parentCommitment, latestFundingTx, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, wallet)) @@ -160,7 +160,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit def spawnTxBuilderRbfBob(fundingParams: InteractiveTxParams, commitment: Commitment, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( ByteVector32.Zeroes, nodeParamsB, fundingParams, channelParamsB, - PreviousTxRbf(commitment, 0 msat, 0 msat, previousTransactions, feeBudget_opt = None), + FundingTxRbf(commitment, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, wallet)) @@ -171,10 +171,10 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit 0 msat, 0 msat, wallet)) - def spawnTxBuilderSpliceRbfBob(fundingParams: InteractiveTxParams, parentCommitment: Commitment, replacedCommitment: Commitment, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( + def spawnTxBuilderSpliceRbfBob(fundingParams: InteractiveTxParams, parentCommitment: Commitment, latestFundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( ByteVector32.Zeroes, nodeParamsB, fundingParams, channelParamsB, - PreviousTxRbf(replacedCommitment, parentCommitment.localCommit.spec.toLocal, parentCommitment.remoteCommit.spec.toLocal, previousTransactions, feeBudget_opt = None), + SpliceTxRbf(parentCommitment, latestFundingTx, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, wallet)) @@ -1436,8 +1436,8 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit // Alice wants to increase the feerate of the splice transaction. val fundingParamsA2 = spliceFixtureParams.fundingParamsA.copy(targetFeerate = targetFeerate * 2) val fundingParamsB2 = spliceFixtureParams.fundingParamsB.copy(targetFeerate = targetFeerate * 2) - val aliceRbf = fixtureParams.spawnTxBuilderSpliceRbfAlice(fundingParamsA2, parentCommitment = commitmentA1, replacedCommitment = commitmentA2, Seq(spliceTxA1), walletA) - val bobRbf = fixtureParams.spawnTxBuilderSpliceRbfBob(fundingParamsB2, parentCommitment = commitmentB1, replacedCommitment = commitmentB2, Seq(spliceTxB1), walletB) + val aliceRbf = fixtureParams.spawnTxBuilderSpliceRbfAlice(fundingParamsA2, parentCommitment = commitmentA1, commitmentA2.localFundingStatus.asInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx], Seq(spliceTxA1), walletA) + val bobRbf = fixtureParams.spawnTxBuilderSpliceRbfBob(fundingParamsB2, parentCommitment = commitmentB1, commitmentB2.localFundingStatus.asInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx], Seq(spliceTxB1), walletB) val fwdRbf = TypeCheckedForwarder(aliceRbf, bobRbf, alice2bob, bob2alice) aliceRbf ! Start(alice2bob.ref) @@ -1565,8 +1565,8 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit // Alice wants to make a large increase to the feerate of the splice transaction, which requires additional inputs. val fundingParamsA2 = fundingParamsA1.copy(targetFeerate = FeeratePerKw(5_000 sat)) val fundingParamsB2 = fundingParamsB1.copy(targetFeerate = FeeratePerKw(5_000 sat)) - val aliceRbf = fixtureParams.spawnTxBuilderSpliceRbfAlice(fundingParamsA2, parentCommitment = commitmentA1, replacedCommitment = commitmentA2, Seq(spliceTxA1), walletA) - val bobRbf = fixtureParams.spawnTxBuilderSpliceRbfBob(fundingParamsB2, parentCommitment = commitmentB1, replacedCommitment = commitmentB2, Seq(spliceTxB1), walletB) + val aliceRbf = fixtureParams.spawnTxBuilderSpliceRbfAlice(fundingParamsA2, parentCommitment = commitmentA1, commitmentA2.localFundingStatus.asInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx], Seq(spliceTxA1), walletA) + val bobRbf = fixtureParams.spawnTxBuilderSpliceRbfBob(fundingParamsB2, parentCommitment = commitmentB1, commitmentB2.localFundingStatus.asInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx], Seq(spliceTxB1), walletB) val fwdRbf = TypeCheckedForwarder(aliceRbf, bobRbf, alice2bob, bob2alice) aliceRbf ! Start(alice2bob.ref) @@ -1718,8 +1718,8 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit val additionalFundingA2 = 50_000 sat val fundingParamsA2 = fundingParamsA1.copy(targetFeerate = FeeratePerKw(5_000 sat), localContribution = additionalFundingA2, remoteContribution = 0 sat) val fundingParamsB2 = fundingParamsB1.copy(targetFeerate = FeeratePerKw(5_000 sat), localContribution = 0 sat, remoteContribution = additionalFundingA2) - val aliceRbf = fixtureParams.spawnTxBuilderSpliceRbfAlice(fundingParamsA2, parentCommitment = commitmentA1bis, replacedCommitment = commitmentA2bis, Seq(spliceTxA1), walletA) - val bobRbf = fixtureParams.spawnTxBuilderSpliceRbfBob(fundingParamsB2, parentCommitment = commitmentB1bis, replacedCommitment = commitmentB2bis, Seq(spliceTxB1), walletB) + val aliceRbf = fixtureParams.spawnTxBuilderSpliceRbfAlice(fundingParamsA2, parentCommitment = commitmentA1bis, commitmentA2bis.localFundingStatus.asInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx], Seq(spliceTxA1), walletA) + val bobRbf = fixtureParams.spawnTxBuilderSpliceRbfBob(fundingParamsB2, parentCommitment = commitmentB1bis, commitmentB2bis.localFundingStatus.asInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx], Seq(spliceTxB1), walletB) val fwdRbf = TypeCheckedForwarder(aliceRbf, bobRbf, alice2bob, bob2alice) aliceRbf ! Start(alice2bob.ref) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala index 12fe47d53f..9c6eb478f4 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala @@ -391,16 +391,16 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture bob2alice.forward(alice, bobInput.copy(previousTxOutput = 42)) alice2bob.expectMsgType[TxAbort] alice2bob.forward(bob) - awaitAssert(assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.RbfAborted)) + awaitAssert(assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.RbfAborted)) bob2alice.expectMsgType[TxAbort] // bob acks alice's tx_abort bob2alice.forward(alice) alice2bob.expectNoMessage(100 millis) // Alice and Bob clear RBF data from their state. - assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) + assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx == fundingTxAlice) assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.isEmpty) - assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) + assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx == fundingTxBob) assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.isEmpty) } @@ -420,16 +420,16 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture // Alice and Bob build a new version of the funding transaction. alice2bob.expectMsgType[TxAbort] alice2bob.forward(bob) - awaitAssert(assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.RbfAborted)) + awaitAssert(assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.RbfAborted)) bob2alice.expectMsgType[TxAbort] // bob acks alice's tx_abort bob2alice.forward(alice) alice2bob.expectNoMessage(100 millis) // Alice and Bob clear RBF data from their state. - assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) + assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx == fundingTxAlice) assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.isEmpty) - assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) + assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx == fundingTxBob) assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.isEmpty) } @@ -783,16 +783,16 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture alice2bob.expectMsgType[TxComplete] // bob doesn't receive alice's tx_complete alice2bob.expectMsgType[CommitSig] // bob doesn't receive alice's commit_sig - awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs]) - val rbfTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.asInstanceOf[RbfStatus.RbfWaitingForSigs].signingSession.fundingTx - assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfInProgress]) + awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs]) + val rbfTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.asInstanceOf[DualFundingStatus.RbfWaitingForSigs].signingSession.fundingTx + assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfInProgress]) alice ! INPUT_DISCONNECTED awaitCond(alice.stateName == OFFLINE) - assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs]) + assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs]) bob ! INPUT_DISCONNECTED awaitCond(bob.stateName == OFFLINE) - assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) + assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) val aliceInit = Init(alice.underlyingActor.nodeParams.features.initFeatures()) val bobInit = Init(bob.underlyingActor.nodeParams.features.initFeatures()) @@ -808,8 +808,8 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture bob2alice.forward(alice) alice2bob.expectMsgType[TxAbort] alice2bob.forward(bob) - assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) - assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) + assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) + assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) } @@ -844,16 +844,16 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture bob2alice.expectMsgType[CommitSig] // alice doesn't receive bob's commit_sig alice2bob.expectMsgType[CommitSig] // bob doesn't receive alice's commit_sig - awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs]) - awaitCond(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs]) - val rbfTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.asInstanceOf[RbfStatus.RbfWaitingForSigs].signingSession.fundingTx + awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs]) + awaitCond(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs]) + val rbfTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.asInstanceOf[DualFundingStatus.RbfWaitingForSigs].signingSession.fundingTx alice ! INPUT_DISCONNECTED awaitCond(alice.stateName == OFFLINE) - assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs]) + assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs]) bob ! INPUT_DISCONNECTED awaitCond(bob.stateName == OFFLINE) - assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs]) + assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs]) val aliceInit = Init(alice.underlyingActor.nodeParams.features.initFeatures()) val bobInit = Init(bob.underlyingActor.nodeParams.features.initFeatures()) @@ -879,8 +879,8 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture assert(bobListener.expectMsgType[TransactionPublished].tx.txid == nextFundingTx.signedTx.txid) assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == nextFundingTx.signedTx.txid) assert(currentFundingTxId != nextFundingTx.txId) - awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) - awaitCond(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf) + awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) + awaitCond(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations) } test("recv Error", Tag(ChannelStateTestsTags.DualFunding)) { f => diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala index ec7ad8f719..6dfb0de959 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala @@ -42,6 +42,7 @@ import fr.acinq.eclair.transactions.DirectedHtlc.{incoming, outgoing} import fr.acinq.eclair.transactions.Transactions import fr.acinq.eclair.transactions.Transactions.ClaimLocalAnchorOutputTx import fr.acinq.eclair.wire.protocol._ +import org.scalatest.Inside.inside import org.scalatest.funsuite.FixtureAnyFunSuiteLike import org.scalatest.time.SpanSugar.convertIntToGrainOfTime import org.scalatest.{Outcome, Tag} @@ -112,6 +113,70 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik private def initiateSpliceWithoutSigs(f: FixtureParam, spliceIn_opt: Option[SpliceIn] = None, spliceOut_opt: Option[SpliceOut] = None): TestProbe = initiateSpliceWithoutSigs(f.alice, f.bob, f.alice2bob, f.bob2alice, spliceIn_opt, spliceOut_opt) + private def initiateRbfWithoutSigs(s: TestFSMRef[ChannelState, ChannelData, Channel], r: TestFSMRef[ChannelState, ChannelData, Channel], s2r: TestProbe, r2s: TestProbe, feerate: FeeratePerKw, sInputsCount: Int, sOutputsCount: Int, rInputsCount: Int, rOutputsCount: Int): TestProbe = { + val sender = TestProbe() + val cmd = CMD_BUMP_FUNDING_FEE(sender.ref, feerate, 100_000 sat, 0) + s ! cmd + exchangeStfu(s, r, s2r, r2s) + s2r.expectMsgType[TxInitRbf] + s2r.forward(r) + r2s.expectMsgType[TxAckRbf] + r2s.forward(s) + + // The initiator also adds the shared input and shared output. + var sRemainingInputs = sInputsCount + 1 + var sRemainingOutputs = sOutputsCount + 1 + var rRemainingInputs = rInputsCount + var rRemainingOutputs = rOutputsCount + var sComplete = false + var rComplete = false + + while (sRemainingInputs > 0 || sRemainingOutputs > 0 || rRemainingInputs > 0 || rRemainingOutputs > 0) { + if (sRemainingInputs > 0) { + s2r.expectMsgType[TxAddInput] + s2r.forward(r) + sRemainingInputs -= 1 + } else if (sRemainingOutputs > 0) { + s2r.expectMsgType[TxAddOutput] + s2r.forward(r) + sRemainingOutputs -= 1 + } else { + s2r.expectMsgType[TxComplete] + s2r.forward(r) + sComplete = true + } + + if (rRemainingInputs > 0) { + r2s.expectMsgType[TxAddInput] + r2s.forward(s) + rRemainingInputs -= 1 + } else if (rRemainingOutputs > 0) { + r2s.expectMsgType[TxAddOutput] + r2s.forward(s) + rRemainingOutputs -= 1 + } else { + r2s.expectMsgType[TxComplete] + r2s.forward(s) + rComplete = true + } + } + + if (!sComplete || !rComplete) { + s2r.expectMsgType[TxComplete] + s2r.forward(r) + if (!rComplete) { + r2s.expectMsgType[TxComplete] + r2s.forward(s) + } + } + + sender + } + + private def initiateRbfWithoutSigs(f: FixtureParam, feerate: FeeratePerKw, sInputsCount: Int, sOutputsCount: Int): TestProbe = { + initiateRbfWithoutSigs(f.alice, f.bob, f.alice2bob, f.bob2alice, feerate, sInputsCount, sOutputsCount, rInputsCount = 0, rOutputsCount = 0) + } + private def exchangeSpliceSigs(s: TestFSMRef[ChannelState, ChannelData, Channel], r: TestFSMRef[ChannelState, ChannelData, Channel], s2r: TestProbe, r2s: TestProbe, sender: TestProbe): Transaction = { val commitSigR = r2s.fishForMessage() { case _: CommitSig => true @@ -153,6 +218,11 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik private def initiateSplice(f: FixtureParam, spliceIn_opt: Option[SpliceIn] = None, spliceOut_opt: Option[SpliceOut] = None): Transaction = initiateSplice(f.alice, f.bob, f.alice2bob, f.bob2alice, spliceIn_opt, spliceOut_opt) + private def initiateRbf(f: FixtureParam, feerate: FeeratePerKw, sInputsCount: Int, sOutputsCount: Int): Transaction = { + val sender = initiateRbfWithoutSigs(f, feerate, sInputsCount, sOutputsCount) + exchangeSpliceSigs(f, sender) + } + private def exchangeStfu(s: TestFSMRef[ChannelState, ChannelData, Channel], r: TestFSMRef[ChannelState, ChannelData, Channel], s2r: TestProbe, r2s: TestProbe): Unit = { s2r.expectMsgType[Stfu] s2r.forward(r) @@ -162,6 +232,24 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik private def exchangeStfu(f: FixtureParam): Unit = exchangeStfu(f.alice, f.bob, f.alice2bob, f.bob2alice) + private def confirmSpliceTx(f: FixtureParam, spliceTx: Transaction): Unit = { + import f._ + + val fundingTxIndex = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.find(_.fundingTxId == spliceTx.txid).get.fundingTxIndex + + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, spliceTx) + alice2bob.expectMsgType[SpliceLocked] + alice2bob.forward(bob) + + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, spliceTx) + bob2alice.expectMsgType[SpliceLocked] + bob2alice.forward(alice) + + // Previous commitments have been cleaned up. + awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.forall(c => c.fundingTxIndex > fundingTxIndex || c.fundingTxId == spliceTx.txid), interval = 100 millis) + awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.forall(c => c.fundingTxIndex > fundingTxIndex || c.fundingTxId == spliceTx.txid), interval = 100 millis) + } + case class TestHtlcs(aliceToBob: Seq[(ByteVector32, UpdateAddHtlc)], bobToAlice: Seq[(ByteVector32, UpdateAddHtlc)]) private def setupHtlcs(f: FixtureParam): TestHtlcs = { @@ -436,12 +524,121 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik crossSign(alice, bob, alice2bob, bob2alice) } + test("recv CMD_SPLICE (pending RBF attempts)") { f => + import f._ + + initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + initiateRbf(f, FeeratePerKw(15_000 sat), sInputsCount = 2, sOutputsCount = 1) + + val probe = TestProbe() + alice ! CMD_SPLICE(probe.ref, Some(SpliceIn(250_000 sat)), None) + assert(probe.expectMsgType[RES_FAILURE[_, ChannelException]].t.isInstanceOf[InvalidSpliceWithUnconfirmedRbf]) + + bob2alice.forward(alice, Stfu(alice.stateData.channelId, initiator = true)) + alice2bob.expectMsgType[Stfu] + bob2alice.forward(alice, SpliceInit(alice.stateData.channelId, 100_000 sat, FeeratePerKw(5000 sat), 0, randomKey().publicKey)) + assert(alice2bob.expectMsgType[TxAbort].toAscii.contains("the previous splice was rbf-ed and is still unconfirmed")) + } + test("recv CMD_SPLICE (splice-in + splice-out)") { f => val htlcs = setupHtlcs(f) initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) resolveHtlcs(f, htlcs, spliceOutFee = 0.sat) } + test("recv CMD_BUMP_FUNDING_FEE (splice-in + splice-out)") { f => + import f._ + + val spliceTx = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(300_000 sat, defaultSpliceOutScriptPubKey))) + val spliceCommitment = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.find(_.fundingTxId == spliceTx.txid).get + + // Alice RBFs the splice transaction. + // Our dummy bitcoin wallet adds an additional input at every funding attempt. + val rbfTx1 = initiateRbf(f, FeeratePerKw(15_000 sat), sInputsCount = 2, sOutputsCount = 2) + assert(rbfTx1.txIn.size == spliceTx.txIn.size + 1) + spliceTx.txIn.foreach(txIn => assert(rbfTx1.txIn.map(_.outPoint).contains(txIn.outPoint))) + assert(rbfTx1.txOut.size == spliceTx.txOut.size) + + // Bob RBFs the splice transaction: he needs to add an input to pay the fees. + // Our dummy bitcoin wallet adds an additional input for Alice: a real bitcoin wallet would simply lower the previous change output. + val sender2 = initiateRbfWithoutSigs(bob, alice, bob2alice, alice2bob, FeeratePerKw(20_000 sat), sInputsCount = 1, sOutputsCount = 1, rInputsCount = 3, rOutputsCount = 2) + val rbfTx2 = exchangeSpliceSigs(alice, bob, alice2bob, bob2alice, sender2) + assert(rbfTx2.txIn.size > rbfTx1.txIn.size) + rbfTx1.txIn.foreach(txIn => assert(rbfTx2.txIn.map(_.outPoint).contains(txIn.outPoint))) + assert(rbfTx2.txOut.size == rbfTx1.txOut.size + 1) + + // There are three pending splice transactions that double-spend each other. + inside(alice.stateData.asInstanceOf[DATA_NORMAL]) { data => + val commitments = data.commitments.active.filter(_.fundingTxIndex == spliceCommitment.fundingTxIndex) + assert(commitments.size == 3) + assert(commitments.map(_.fundingTxId) == Seq(rbfTx2, rbfTx1, spliceTx).map(_.txid)) + // The contributions are the same across RBF attempts. + commitments.foreach(c => assert(c.localCommit.spec.toLocal == spliceCommitment.localCommit.spec.toLocal)) + commitments.foreach(c => assert(c.localCommit.spec.toRemote == spliceCommitment.localCommit.spec.toRemote)) + } + + // The last RBF attempt confirms. + confirmSpliceTx(f, rbfTx2) + inside(alice.stateData.asInstanceOf[DATA_NORMAL]) { data => + assert(data.commitments.active.map(_.fundingTxId) == Seq(rbfTx2.txid)) + data.commitments.active.foreach(c => assert(c.localCommit.spec.toLocal == spliceCommitment.localCommit.spec.toLocal)) + data.commitments.active.foreach(c => assert(c.localCommit.spec.toRemote == spliceCommitment.localCommit.spec.toRemote)) + } + + // We can keep doing more splice transactions now that one of the previous transactions confirmed. + initiateSplice(bob, alice, bob2alice, alice2bob, Some(SpliceIn(100_000 sat)), None) + } + + test("recv CMD_BUMP_FUNDING_FEE (splice-in + splice-out from non-initiator)") { f => + import f._ + + // Alice initiates a first splice. + val spliceTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(2_500_000 sat))) + + // Bob initiates a second splice that spends the first splice. + val spliceTx2 = initiateSplice(bob, alice, bob2alice, alice2bob, spliceIn_opt = Some(SpliceIn(50_000 sat)), spliceOut_opt = Some(SpliceOut(25_000 sat, defaultSpliceOutScriptPubKey))) + assert(spliceTx2.txIn.exists(_.outPoint.txid == spliceTx1.txid)) + + // Alice cannot RBF her first splice, so she RBFs Bob's splice instead. + val sender = initiateRbfWithoutSigs(alice, bob, alice2bob, bob2alice, FeeratePerKw(15_000 sat), sInputsCount = 1, sOutputsCount = 1, rInputsCount = 2, rOutputsCount = 2) + val rbfTx = exchangeSpliceSigs(bob, alice, bob2alice, alice2bob, sender) + assert(rbfTx.txIn.size > spliceTx2.txIn.size) + spliceTx2.txIn.foreach(txIn => assert(rbfTx.txIn.map(_.outPoint).contains(txIn.outPoint))) + } + + test("recv CMD_BUMP_FUNDING_FEE (transaction already confirmed)") { f => + import f._ + + val spliceTx = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + confirmSpliceTx(f, spliceTx) + + val probe = TestProbe() + alice ! CMD_BUMP_FUNDING_FEE(probe.ref, FeeratePerKw(15_000 sat), 100_000 sat, 0) + assert(probe.expectMsgType[RES_FAILURE[_, ChannelException]].t.isInstanceOf[InvalidRbfTxConfirmed]) + + bob2alice.forward(alice, Stfu(alice.stateData.channelId, initiator = true)) + alice2bob.expectMsgType[Stfu] + bob2alice.forward(alice, TxInitRbf(alice.stateData.channelId, 0, FeeratePerKw(15_000 sat), 250_000 sat, requireConfirmedInputs = false)) + assert(alice2bob.expectMsgType[TxAbort].toAscii.contains("transaction is already confirmed")) + } + + test("recv CMD_BUMP_FUNDING_FEE (transaction is using 0-conf)", Tag(ZeroConf), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f => + import f._ + + val spliceTx = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + alice ! WatchPublishedTriggered(spliceTx) + alice2bob.expectMsgType[SpliceLocked] + + val probe = TestProbe() + alice ! CMD_BUMP_FUNDING_FEE(probe.ref, FeeratePerKw(15_000 sat), 100_000 sat, 0) + assert(probe.expectMsgType[RES_FAILURE[_, ChannelException]].t.isInstanceOf[InvalidRbfZeroConf]) + + bob2alice.forward(alice, Stfu(alice.stateData.channelId, initiator = true)) + alice2bob.expectMsgType[Stfu] + bob2alice.forward(alice, TxInitRbf(alice.stateData.channelId, 0, FeeratePerKw(15_000 sat), 250_000 sat, requireConfirmedInputs = false)) + assert(alice2bob.expectMsgType[TxAbort].toAscii.contains("transaction is already confirmed")) + } + test("recv TxAbort (before TxComplete)") { f => import f._ @@ -1982,6 +2179,120 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik resolveHtlcs(f, htlcs, spliceOutFee = spliceOutFee(f, capacity = 1_900_000.sat)) } + test("recv CMD_BUMP_FUNDING_FEE with pre and post rbf htlcs") { f => + import f._ + + // We create two unconfirmed splice transactions spending each other: + // +-----------+ +-----------+ +-----------+ + // | fundingTx |---->| spliceTx1 |---->| spliceTx2 | + // +-----------+ +-----------+ +-----------+ + val spliceTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(200_000 sat))) + val spliceCommitment1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest + assert(spliceCommitment1.fundingTxId == spliceTx1.txid) + val spliceTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(100_000 sat))) + val spliceCommitment2 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest + assert(spliceCommitment2.fundingTxId == spliceTx2.txid) + assert(spliceCommitment2.localCommit.spec.toLocal == spliceCommitment1.localCommit.spec.toLocal + 100_000.sat) + + // Bob sends an HTLC while Alice starts an RBF on spliceTx2. + addHtlc(25_000_000 msat, bob, alice, bob2alice, alice2bob) + val rbfTx1 = { + val probe = TestProbe() + alice ! CMD_BUMP_FUNDING_FEE(probe.ref, FeeratePerKw(15_000 sat), 100_000 sat, 250) + alice2bob.expectMsgType[Stfu] + alice2bob.forward(bob) + // Bob is waiting to sign its outgoing HTLC before sending stfu. + bob2alice.expectNoMessage(100 millis) + bob ! CMD_SIGN() + (0 until 3).foreach { _ => + bob2alice.expectMsgType[CommitSig] + bob2alice.forward(alice) + } + alice2bob.expectMsgType[RevokeAndAck] + alice2bob.forward(bob) + (0 until 3).foreach { _ => + alice2bob.expectMsgType[CommitSig] + alice2bob.forward(bob) + } + bob2alice.expectMsgType[RevokeAndAck] + bob2alice.forward(alice) + bob2alice.expectMsgType[Stfu] + bob2alice.forward(alice) + + alice2bob.expectMsgType[TxInitRbf] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxAckRbf] + bob2alice.forward(alice) + + // Alice adds three inputs: the shared input, the previous splice input, and an RBF input. + (0 until 3).foreach { _ => + alice2bob.expectMsgType[TxAddInput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxComplete] + bob2alice.forward(alice) + } + // Alice adds two outputs: the shared output and a change output. + (0 until 2).foreach { _ => + alice2bob.expectMsgType[TxAddOutput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxComplete] + bob2alice.forward(alice) + } + // Alice doesn't have anything more to add to the transaction. + alice2bob.expectMsgType[TxComplete] + alice2bob.forward(bob) + val rbfTx = exchangeSpliceSigs(f, probe) + assert(rbfTx.lockTime == 250) + spliceTx2.txIn.foreach(txIn => assert(rbfTx.txIn.map(_.outPoint).contains(txIn.outPoint))) + rbfTx + } + + // Alice and Bob exchange HTLCs while the splice transactions are still unconfirmed. + addHtlc(10_000_000 msat, alice, bob, alice2bob, bob2alice) + addHtlc(5_000_000 msat, bob, alice, bob2alice, alice2bob) + crossSign(alice, bob, alice2bob, bob2alice) + + // Alice initiates another RBF attempt: + // +-----------+ +-----------+ +-----------+ + // | fundingTx |---->| spliceTx1 |---->| spliceTx2 | + // +-----------+ +-----------+ | +-----------+ + // | +-----------+ + // +->| rbfTx1 | + // | +-----------+ + // | +-----------+ + // +->| rbfTx2 | + // +-----------+ + val rbfTx2 = initiateRbf(f, FeeratePerKw(20_000 sat), sInputsCount = 3, sOutputsCount = 1) + assert(rbfTx2.txIn.size == rbfTx1.txIn.size + 1) + rbfTx1.txIn.foreach(txIn => assert(rbfTx2.txIn.map(_.outPoint).contains(txIn.outPoint))) + + // The balance is the same in all RBF attempts. + inside(alice.stateData.asInstanceOf[DATA_NORMAL]) { data => + val commitments = data.commitments.active.filter(_.fundingTxIndex == spliceCommitment2.commitment.fundingTxIndex) + assert(commitments.map(_.fundingTxId) == Seq(rbfTx2, rbfTx1, spliceTx2).map(_.txid)) + assert(commitments.map(_.localCommit.spec.toLocal).toSet.size == 1) + assert(commitments.map(_.localCommit.spec.toRemote).toSet.size == 1) + } + + // The first RBF attempt is confirmed. + confirmSpliceTx(f, rbfTx1) + inside(alice.stateData.asInstanceOf[DATA_NORMAL]) { data => + assert(data.commitments.active.size == 1) + assert(data.commitments.latest.fundingTxId == rbfTx1.txid) + assert(data.commitments.latest.localCommit.spec.toLocal == spliceCommitment1.localCommit.spec.toLocal + 100_000.sat - 10_000.sat) + assert(data.commitments.latest.localCommit.spec.toRemote == spliceCommitment1.localCommit.spec.toRemote - 25_000.sat - 5_000.sat) + assert(data.commitments.latest.localCommit.spec.htlcs.collect(incoming).map(_.amountMsat) == Set(5_000_000 msat, 25_000_000 msat)) + assert(data.commitments.latest.localCommit.spec.htlcs.collect(outgoing).map(_.amountMsat) == Set(10_000_000 msat)) + } + + // The first splice transaction confirms: this was already implied by the RBF attempt confirming. + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, spliceTx1) + inside(alice.stateData.asInstanceOf[DATA_NORMAL]) { data => + assert(data.commitments.active.size == 1) + assert(data.commitments.latest.fundingTxId == rbfTx1.txid) + } + } + test("recv invalid htlc signatures during splice-in") { f => import f._ diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/io/PendingChannelsRateLimiterSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/io/PendingChannelsRateLimiterSpec.scala index 0512401433..ea06709f52 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/io/PendingChannelsRateLimiterSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/io/PendingChannelsRateLimiterSpec.scala @@ -82,7 +82,7 @@ class PendingChannelsRateLimiterSpec extends ScalaTestWithActorTestKit(ConfigFac DATA_WAIT_FOR_CHANNEL_READY(commitments(peerAtLimit1, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)), ) val channelsAtLimit2 = Seq( - DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments(peerAtLimit2, channelIdAtLimit2), 0 msat, 0 msat, BlockHeight(0), BlockHeight(0), RbfStatus.NoRbf, None), + DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments(peerAtLimit2, channelIdAtLimit2), 0 msat, 0 msat, BlockHeight(0), BlockHeight(0), DualFundingStatus.WaitingForConfirmations, None), DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(peerAtLimit2, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)), ) val channelsBelowLimit1 = Seq( @@ -101,7 +101,7 @@ class PendingChannelsRateLimiterSpec extends ScalaTestWithActorTestKit(ConfigFac val initiatorChannels = Seq( DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments(peerBelowLimit1, randomBytes32(), isOpener = true), BlockHeight(0), None, Left(FundingCreated(channelIdAtLimit1, TxId(ByteVector32.Zeroes), 3, randomBytes64()))), DATA_WAIT_FOR_CHANNEL_READY(commitments(peerBelowLimit1, randomBytes32(), isOpener = true), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)), - DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments(peerAtLimit1, randomBytes32(), isOpener = true), 0 msat, 0 msat, BlockHeight(0), BlockHeight(0), RbfStatus.NoRbf, None), + DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments(peerAtLimit1, randomBytes32(), isOpener = true), 0 msat, 0 msat, BlockHeight(0), BlockHeight(0), DualFundingStatus.WaitingForConfirmations, None), DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(peerAtLimit1, randomBytes32(), isOpener = true), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)), ) val publicChannels = channelsOnWhitelistAtLimit ++ channelsAtLimit1 ++ channelsAtLimit2 ++ channelsBelowLimit1 ++ channelsBelowLimit2 diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4Spec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4Spec.scala index 4a86251654..b666caa99f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4Spec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4Spec.scala @@ -144,15 +144,15 @@ class ChannelCodecs4Spec extends AnyFunSuite { RemoteCommit(0, CommitmentSpec(Set.empty, FeeratePerKw(1000 sat), 75_000_000 msat, 100_000_000 msat), randomTxId(), randomKey().publicKey) ) val testCases = Map( - RbfStatus.NoRbf -> RbfStatus.NoRbf, - RbfStatus.RbfRequested(CMD_BUMP_FUNDING_FEE(null, FeeratePerKw(750 sat), fundingFeeBudget = 100_000.sat, 0)) -> RbfStatus.NoRbf, - RbfStatus.RbfInProgress(None, null, None) -> RbfStatus.NoRbf, - RbfStatus.RbfWaitingForSigs(waitingForSigs) -> RbfStatus.RbfWaitingForSigs(waitingForSigs), - RbfStatus.RbfAborted -> RbfStatus.NoRbf, + DualFundingStatus.WaitingForConfirmations -> DualFundingStatus.WaitingForConfirmations, + DualFundingStatus.RbfRequested(CMD_BUMP_FUNDING_FEE(null, FeeratePerKw(750 sat), fundingFeeBudget = 100_000.sat, 0)) -> DualFundingStatus.WaitingForConfirmations, + DualFundingStatus.RbfInProgress(None, null, None) -> DualFundingStatus.WaitingForConfirmations, + DualFundingStatus.RbfWaitingForSigs(waitingForSigs) -> DualFundingStatus.RbfWaitingForSigs(waitingForSigs), + DualFundingStatus.RbfAborted -> DualFundingStatus.WaitingForConfirmations, ) testCases.foreach { case (status, expected) => - val encoded = rbfStatusCodec.encode(status).require - val decoded = rbfStatusCodec.decode(encoded).require.value + val encoded = dualFundingStatusCodec.encode(status).require + val decoded = dualFundingStatusCodec.decode(encoded).require.value assert(decoded == expected) } } diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Channel.scala b/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Channel.scala index ef1bd0b36e..fd5dd468b5 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Channel.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Channel.scala @@ -90,6 +90,12 @@ trait Channel { } } + val rbfSplice: Route = postRequest("rbfsplice") { implicit f => + formFields(channelIdFormParam, "targetFeerateSatByte".as[FeeratePerByte], "fundingFeeBudgetSatoshis".as[Satoshi], "lockTime".as[Long].?) { + (channelId, targetFeerateSatByte, fundingFeeBudget, lockTime_opt) => complete(eclairApi.rbfSplice(channelId, FeeratePerKw(targetFeerateSatByte), fundingFeeBudget, lockTime_opt)) + } + } + val close: Route = postRequest("close") { implicit t => withChannelsIdentifier { channels => formFields("scriptPubKey".as[ByteVector](bytesUnmarshaller).?, "preferredFeerateSatByte".as[FeeratePerByte].?, "minFeerateSatByte".as[FeeratePerByte].?, "maxFeerateSatByte".as[FeeratePerByte].?) { @@ -165,6 +171,6 @@ trait Channel { complete(eclairApi.channelBalances()) } - val channelRoutes: Route = open ~ rbfOpen ~ spliceIn ~ spliceOut ~ close ~ forceClose ~ bumpForceClose ~ channel ~ channels ~ closedChannels ~ allChannels ~ allUpdates ~ channelStats ~ channelBalances + val channelRoutes: Route = open ~ rbfOpen ~ spliceIn ~ spliceOut ~ rbfSplice ~ close ~ forceClose ~ bumpForceClose ~ channel ~ channels ~ closedChannels ~ allChannels ~ allUpdates ~ channelStats ~ channelBalances } From cb02ea994b43150332e917edef56f088c00c0cea Mon Sep 17 00:00:00 2001 From: t-bast Date: Tue, 27 Aug 2024 14:24:38 +0200 Subject: [PATCH 4/8] Disallow chains of unconfirmed splice transactions 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. --- docs/release-notes/eclair-vnext.md | 3 + .../eclair/channel/ChannelExceptions.scala | 1 + .../fr/acinq/eclair/channel/fsm/Channel.scala | 3 + .../channel/fsm/ChannelOpenDualFunded.scala | 2 +- .../states/e/NormalSplicesStateSpec.scala | 415 +++++++++--------- 5 files changed, 224 insertions(+), 200 deletions(-) diff --git a/docs/release-notes/eclair-vnext.md b/docs/release-notes/eclair-vnext.md index 2a3be9effc..e0a3e3bbb4 100644 --- a/docs/release-notes/eclair-vnext.md +++ b/docs/release-notes/eclair-vnext.md @@ -31,6 +31,9 @@ eclair-cli spliceout --channelId= --amountOut= --sc That operation can also be RBF-ed with the `rbfsplice` API to speed up confirmation if necessary. +Note that when 0-conf is used for the channel, it is not possible to RBF splice transactions. +Node operators should instead create a new splice transaction (with `splicein` or `spliceout`) to CPFP the previous transaction. + Note that eclair had already introduced support for a splicing prototype in v0.9.0, which helped improve the BOLT proposal. We're removing support for the previous splicing prototype feature: users that depended on this protocol must upgrade to create official splice transactions. diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala index 404e31c9f0..4aa36e158c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala @@ -86,6 +86,7 @@ case class InvalidRbfAttemptTooSoon (override val channelId: Byte case class InvalidSpliceTxAbortNotAcked (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid splice attempt: our previous tx_abort has not been acked") case class InvalidSpliceNotQuiescent (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid splice attempt: the channel is not quiescent") case class InvalidSpliceWithUnconfirmedRbf (override val channelId: ByteVector32, previousTxs: Seq[TxId]) extends ChannelException(channelId, s"invalid splice attempt: the previous splice was rbf-ed and is still unconfirmed (txIds=${previousTxs.mkString(", ")})") +case class 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") diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index c8ff14e4df..757c804031 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -985,6 +985,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with val previousTxs = d.commitments.active.filter(_.fundingTxIndex == d.commitments.latest.fundingTxIndex).map(_.fundingTxId) log.info("rejecting splice request: the previous splice has unconfirmed rbf attempts ({})", previousTxs.mkString(", ")) stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceWithUnconfirmedRbf(d.channelId, previousTxs).getMessage) + } else if (d.commitments.latest.localFundingStatus.isInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx]) { + log.info("rejecting splice request: the previous funding transaction is unconfirmed ({})", d.commitments.latest.fundingTxId) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceWithUnconfirmedTx(d.channelId, d.commitments.latest.fundingTxId).getMessage) } else { log.info(s"accepting splice with remote.in.amount=${msg.fundingContribution} remote.in.push=${msg.pushAmount}") val parentCommitment = d.commitments.latest.commitment diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala index 2f2a5dd449..35f7cf6297 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala @@ -675,7 +675,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { } 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.status.getClass.getSimpleName) + log.warning("ignoring interactive-tx result with funding status={}", d.status.getClass.getSimpleName) stay() } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala index 6dfb0de959..c8abe3e2fe 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala @@ -232,6 +232,15 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik private def exchangeStfu(f: FixtureParam): Unit = exchangeStfu(f.alice, f.bob, f.alice2bob, f.bob2alice) + private def checkWatchConfirmed(f: FixtureParam, spliceTx: Transaction): Unit = { + import f._ + + alice2blockchain.expectWatchFundingConfirmed(spliceTx.txid) + alice2blockchain.expectNoMessage(100 millis) + bob2blockchain.expectWatchFundingConfirmed(spliceTx.txid) + bob2blockchain.expectNoMessage(100 millis) + } + private def confirmSpliceTx(f: FixtureParam, spliceTx: Transaction): Unit = { import f._ @@ -540,6 +549,27 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik assert(alice2bob.expectMsgType[TxAbort].toAscii.contains("the previous splice was rbf-ed and is still unconfirmed")) } + test("recv CMD_SPLICE (unconfirmed previous tx)") { f => + import f._ + + // We create a first unconfirmed splice. + initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.isInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx]) + + // We allow initiating such splice... + val probe = TestProbe() + alice ! CMD_SPLICE(probe.ref, Some(SpliceIn(250_000 sat)), None) + alice2bob.expectMsgType[Stfu] + alice2bob.forward(bob) + bob2alice.expectMsgType[Stfu] + bob2alice.forward(alice) + val spliceInit = alice2bob.expectMsgType[SpliceInit] + + // But we don't allow receiving splice_init if the previous splice is unconfirmed and we're not using 0-conf. + alice2bob.forward(bob, spliceInit) + assert(bob2alice.expectMsgType[TxAbort].toAscii.contains("the current funding transaction is still unconfirmed")) + } + test("recv CMD_SPLICE (splice-in + splice-out)") { f => val htlcs = setupHtlcs(f) initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) @@ -594,6 +624,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik // Alice initiates a first splice. val spliceTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(2_500_000 sat))) + confirmSpliceTx(f, spliceTx1) // Bob initiates a second splice that spends the first splice. val spliceTx2 = initiateSplice(bob, alice, bob2alice, alice2bob, spliceIn_opt = Some(SpliceIn(50_000 sat)), spliceOut_opt = Some(SpliceOut(25_000 sat, defaultSpliceOutScriptPubKey))) @@ -793,51 +824,38 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik bob2alice.forward(alice) } - private def setup2Splices(f: FixtureParam): (Transaction, Transaction) = { + test("splice local/remote locking", Tag(NoMaxHtlcValueInFlight)) { f => import f._ val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) - alice2blockchain.expectWatchFundingConfirmed(fundingTx1.txid) - alice2blockchain.expectNoMessage(100 millis) - bob2blockchain.expectWatchFundingConfirmed(fundingTx1.txid) - bob2blockchain.expectNoMessage(100 millis) + checkWatchConfirmed(f, fundingTx1) + val commitAlice1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.commitTxAndRemoteSig.commitTx.tx + val commitBob1 = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.commitTxAndRemoteSig.commitTx.tx - val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) - alice2blockchain.expectWatchFundingConfirmed(fundingTx2.txid) - alice2blockchain.expectNoMessage(100 millis) - bob2blockchain.expectWatchFundingConfirmed(fundingTx2.txid) - bob2blockchain.expectNoMessage(100 millis) - - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2, 1, 0)) - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2, 1, 0)) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) - - (fundingTx1, fundingTx2) - } - - test("splice local/remote locking", Tag(NoMaxHtlcValueInFlight)) { f => - import f._ + // Bob sees the first splice confirm, but Alice doesn't. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectWatchFundingSpent(fundingTx1.txid, Some(Set(commitAlice1.txid, commitBob1.txid))) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) - val (fundingTx1, fundingTx2) = setup2Splices(f) - val commitAlice1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active(1).localCommit.commitTxAndRemoteSig.commitTx.tx - val commitAlice2 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active(0).localCommit.commitTxAndRemoteSig.commitTx.tx - val commitBob1 = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active(1).localCommit.commitTxAndRemoteSig.commitTx.tx - val commitBob2 = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active(0).localCommit.commitTxAndRemoteSig.commitTx.tx + // Alice creates another splice spending the first splice. + val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + checkWatchConfirmed(f, fundingTx2) + val commitAlice2 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.commitTxAndRemoteSig.commitTx.tx + val commitBob2 = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.commitTxAndRemoteSig.commitTx.tx + assert(commitAlice1.txid != commitAlice2.txid) + assert(commitBob1.txid != commitBob2.txid) - // splice 1 confirms + // Alice sees the first splice confirm. alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) alice2bob.forward(bob) - bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) - bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) - bob2alice.forward(alice) + alice2blockchain.expectWatchFundingSpent(fundingTx1.txid, Some(Set(fundingTx2.txid, commitAlice1.txid, commitBob1.txid))) - bob2blockchain.expectWatchFundingSpent(fundingTx1.txid, Some(Set(fundingTx2.txid, commitAlice1.txid, commitBob1.txid))) assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2, 1)) assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) - // splice 2 confirms + // Alice and Bob see the second splice confirm. alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) alice2bob.forward(bob) @@ -874,68 +892,47 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.fundingTxId == fundingTx1.txid) } - test("splice local/remote locking (reverse order)", Tag(NoMaxHtlcValueInFlight)) { f => + test("splice local/remote locking (intermingled)", Tag(NoMaxHtlcValueInFlight)) { f => import f._ - val (fundingTx1, fundingTx2) = setup2Splices(f) + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + checkWatchConfirmed(f, fundingTx1) + + // Bob sees the first splice confirm, but Alice doesn't. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) + + // Alice creates another splice spending the first splice. + val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + checkWatchConfirmed(f, fundingTx2) val commitAlice2 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.commitTxAndRemoteSig.commitTx.tx val commitBob2 = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.commitTxAndRemoteSig.commitTx.tx - // splice 2 confirms + // Alice sees the second splice confirm. alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) alice2bob.forward(bob) - bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) - bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) - bob2alice.forward(alice) alice2blockchain.expectWatchFundingSpent(fundingTx2.txid, Some(Set(commitAlice2.txid, commitBob2.txid))) - bob2blockchain.expectWatchFundingSpent(fundingTx2.txid, Some(Set(commitAlice2.txid, commitBob2.txid))) - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2)) - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) - - // splice 1 confirms - alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) - // we don't send a splice_locked for the older tx - alice2bob.expectNoMessage(100 millis) - bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) - // we don't send a splice_locked for the older tx bob2alice.expectNoMessage(100 millis) - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2)) - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) - } - - test("splice local/remote locking (intermingled)", Tag(NoMaxHtlcValueInFlight)) { f => - import f._ - - val (fundingTx1, fundingTx2) = setup2Splices(f) + assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2, 1)) - // splice 1 confirms on alice, splice 2 confirms on bob - alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) - alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) - alice2bob.forward(bob) + // Bob sees the second splice confirm. bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) bob2alice.forward(alice) - alice2blockchain.expectWatchFundingSpent(fundingTx1.txid) - bob2blockchain.expectWatchFundingSpent(fundingTx2.txid) - assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2, 1)) + bob2blockchain.expectWatchFundingSpent(fundingTx2.txid, Some(Set(commitAlice2.txid, commitBob2.txid))) + awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2)) + awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2)) assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2, 1)) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) - // splice 2 confirms on bob, splice 1 confirms on alice - alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) - alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) - alice2bob.forward(bob) - bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) - bob2alice.expectNoMessage(100 millis) - bob2alice.forward(alice) - alice2blockchain.expectWatchFundingSpent(fundingTx2.txid) - bob2blockchain.expectNoMessage(100 millis) + // Alice sees the first splice confirm. + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + // Alice doesn't send a splice_locked for the older tx. + alice2bob.expectNoMessage(100 millis) assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2)) assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.map(_.fundingTxIndex) == Seq(2)) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.map(_.fundingTxIndex) == Seq.empty) } test("emit post-splice events", Tag(NoMaxHtlcValueInFlight)) { f => @@ -963,24 +960,24 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik systemB.eventStream.subscribe(bobEvents.ref, classOf[LocalChannelUpdate]) systemB.eventStream.subscribe(bobEvents.ref, classOf[LocalChannelDown]) - val (fundingTx1, fundingTx2) = setup2Splices(f) + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + checkWatchConfirmed(f, fundingTx1) - // splices haven't been locked, so no event is emitted + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) + bobEvents.expectMsg(ForgetHtlcInfos(initialState.channelId, initialState.commitments.localCommitIndex)) aliceEvents.expectNoMessage(100 millis) - bobEvents.expectNoMessage(100 millis) + + val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + checkWatchConfirmed(f, fundingTx2) alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) alice2bob.expectMsgType[SpliceLocked] alice2bob.forward(bob) aliceEvents.expectMsg(ForgetHtlcInfos(initialState.channelId, initialState.commitments.remoteCommitIndex)) - aliceEvents.expectNoMessage(100 millis) - bobEvents.expectNoMessage(100 millis) - - bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) - bob2alice.expectMsgType[SpliceLocked] - bob2alice.forward(alice) aliceEvents.expectAvailableBalanceChanged(balance = 1_275_000_000.msat, capacity = 2_000_000.sat) - bobEvents.expectMsg(ForgetHtlcInfos(initialState.channelId, initialState.commitments.localCommitIndex)) bobEvents.expectAvailableBalanceChanged(balance = 650_000_000.msat, capacity = 2_000_000.sat) aliceEvents.expectNoMessage(100 millis) bobEvents.expectNoMessage(100 millis) @@ -1139,9 +1136,12 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik test("recv UpdateAddHtlc while splice is being locked", Tag(ZeroConf), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f => import f._ - initiateSplice(f, spliceOut_opt = Some(SpliceOut(50_000 sat, defaultSpliceOutScriptPubKey))) - val spliceTx = initiateSplice(f, spliceOut_opt = Some(SpliceOut(50_000 sat, defaultSpliceOutScriptPubKey))) - alice ! WatchPublishedTriggered(spliceTx) + val spliceTx1 = initiateSplice(f, spliceOut_opt = Some(SpliceOut(50_000 sat, defaultSpliceOutScriptPubKey))) + bob ! WatchPublishedTriggered(spliceTx1) + bob2alice.expectMsgType[SpliceLocked] // we ignore Bob's splice_locked for the first splice + + val spliceTx2 = initiateSplice(f, spliceOut_opt = Some(SpliceOut(50_000 sat, defaultSpliceOutScriptPubKey))) + alice ! WatchPublishedTriggered(spliceTx2) val spliceLockedAlice = alice2bob.expectMsgType[SpliceLocked] assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 3) @@ -1172,8 +1172,9 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice ! CMD_SIGN() val commitSigsAlice = (1 to 3).map(_ => alice2bob.expectMsgType[CommitSig]) alice2bob.forward(bob, commitSigsAlice(0)) - bob ! WatchPublishedTriggered(spliceTx) + bob ! WatchPublishedTriggered(spliceTx2) val spliceLockedBob = bob2alice.expectMsgType[SpliceLocked] + assert(spliceLockedBob.fundingTxId == spliceTx2.txid) bob2alice.forward(alice, spliceLockedBob) alice2bob.forward(bob, commitSigsAlice(1)) alice2bob.forward(bob, commitSigsAlice(2)) @@ -1462,8 +1463,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik test("don't resend splice_locked when zero-conf channel confirms", Tag(ZeroConf), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f => import f._ - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) - val fundingTx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get + val fundingTx = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) alice2blockchain.expectMsgType[WatchPublished] // splice tx gets published, alice sends splice_locked alice ! WatchPublishedTriggered(fundingTx) @@ -1478,17 +1478,21 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik test("re-send splice_locked on reconnection") { f => import f._ - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) - val fundingTx1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get - val watchConfirmed1a = alice2blockchain.expectMsgType[WatchFundingConfirmed] - val watchConfirmed1b = bob2blockchain.expectMsgType[WatchFundingConfirmed] - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) - val fundingTx2 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get - val watchConfirmed2a = alice2blockchain.expectMsgType[WatchFundingConfirmed] - val watchConfirmed2b = bob2blockchain.expectMsgType[WatchFundingConfirmed] + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) + checkWatchConfirmed(f, fundingTx1) + + // The first splice confirms on Bob's side. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) + + val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) + checkWatchConfirmed(f, fundingTx2) alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) - // we now have two unconfirmed splices + + // From Alice's point of view, we now have two unconfirmed splices. alice2bob.ignoreMsg { case _: ChannelUpdate => true } bob2alice.ignoreMsg { case _: ChannelUpdate => true } @@ -1496,12 +1500,13 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik disconnect(f) reconnect(f) - // channel_ready are not re-sent because the channel has already been used (for building splices) + // NB: channel_ready are not re-sent because the channel has already been used (for building splices). + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) alice2bob.expectNoMessage(100 millis) - bob2alice.expectNoMessage(100 millis) - // splice 1 confirms on alice's side - watchConfirmed1a.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + // The first splice confirms on Alice's side. + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx1.txid) alice2bob.forward(bob) alice2blockchain.expectMsgType[WatchFundingSpent] @@ -1509,11 +1514,13 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik disconnect(f) reconnect(f) - assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx1.txid) + alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) alice2bob.forward(bob) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) - // splice 2 confirms on alice's side - watchConfirmed2a.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) + // The second splice confirms on Alice's side. + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) alice2bob.forward(bob) alice2blockchain.expectMsgType[WatchFundingSpent] @@ -1521,40 +1528,26 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik disconnect(f) reconnect(f) - assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) - alice2bob.forward(bob) - alice2bob.expectNoMessage(100 millis) - bob2alice.expectNoMessage(100 millis) - - // splice 1 confirms on bob's side - watchConfirmed1b.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) - assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == fundingTx1.txid) - bob2alice.forward(alice) - bob2blockchain.expectMsgType[WatchFundingSpent] - - disconnect(f) - reconnect(f) - - assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) + alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) alice2bob.forward(bob) - assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) bob2alice.forward(alice) alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) - // splice 2 confirms on bob's side - watchConfirmed2b.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) + // The second splice confirms on Bob's side. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) bob2blockchain.expectMsgType[WatchFundingSpent] - // NB: we disconnect *before* transmitting the splice_confirmed to alice + // NB: we disconnect *before* transmitting the splice_locked to Alice. disconnect(f) reconnect(f) - assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) + alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) alice2bob.forward(bob) - assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) - // this time alice received the splice_confirmed for funding tx 2 + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) + // This time alice received the splice_locked for the second splice. bob2alice.forward(alice) alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) @@ -1562,9 +1555,9 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik disconnect(f) reconnect(f) - assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) + alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) alice2bob.forward(bob) - assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == fundingTx2.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx2.txid) bob2alice.forward(alice) alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) @@ -1585,69 +1578,64 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val htlcs = setupHtlcs(f) - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) - val fundingTx1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get - val watchConfirmed1 = alice2blockchain.expectWatchFundingConfirmed(fundingTx1.txid) - initiateSplice(f, spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) - val fundingTx2 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get - val watchConfirmed2 = alice2blockchain.expectWatchFundingConfirmed(fundingTx2.txid) - alice2bob.expectNoMessage(100 millis) - bob2alice.expectNoMessage(100 millis) - alice2blockchain.expectNoMessage(100 millis) - // we now have two unconfirmed splices + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + checkWatchConfirmed(f, fundingTx1) + + // The first splice confirms on Bob's side. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) + + val fundingTx2 = initiateSplice(f, spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) + checkWatchConfirmed(f, fundingTx2) + // From Alice's point of view, we now have two unconfirmed splices. alice ! CMD_FORCECLOSE(ActorRef.noSender) alice2bob.expectMsgType[Error] val commitTx2 = assertPublished(alice2blockchain, "commit-tx") Transaction.correctlySpends(commitTx2, Seq(fundingTx2), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS) val claimMainDelayed2 = assertPublished(alice2blockchain, "local-main-delayed") - // alice publishes her htlc timeout transactions + // Alice publishes her htlc timeout transactions. val htlcsTxsOut = htlcs.aliceToBob.map(_ => assertPublished(alice2blockchain, "htlc-timeout")) htlcsTxsOut.foreach(tx => Transaction.correctlySpends(tx, Seq(commitTx2), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) val watchConfirmedCommit2 = alice2blockchain.expectWatchTxConfirmed(commitTx2.txid) val watchConfirmedClaimMainDelayed2 = alice2blockchain.expectWatchTxConfirmed(claimMainDelayed2.txid) - // watch for all htlc outputs from local commit-tx to be spent val watchHtlcsOut = htlcs.aliceToBob.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) htlcs.bobToAlice.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) - // splice 1 confirms - watchConfirmed1.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) - alice2bob.forward(bob) + // The first splice transaction confirms. + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) alice2blockchain.expectMsgType[WatchFundingSpent] - // splice 2 confirms - watchConfirmed2.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) - alice2bob.forward(bob) + // The second splice transaction confirms. + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) alice2blockchain.expectMsgType[WatchFundingSpent] - // commit tx confirms + // The commit confirms, along with Alice's 2nd-stage transactions. watchConfirmedCommit2.replyTo ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, commitTx2) - // claim-main-delayed tx confirms watchConfirmedClaimMainDelayed2.replyTo ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, claimMainDelayed2) - // alice's htlc-timeout txs confirm watchHtlcsOut.zip(htlcsTxsOut).foreach { case (watch, tx) => watch.replyTo ! WatchOutputSpentTriggered(tx) } htlcsTxsOut.foreach { tx => alice2blockchain.expectWatchTxConfirmed(tx.txid) alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, tx) } - // alice publishes, watches and confirms their 2nd-stage htlc-delayed txs + // Alice publishes 3rd-stage transactions. htlcs.aliceToBob.foreach { _ => val tx = assertPublished(alice2blockchain, "htlc-delayed") alice2blockchain.expectWatchTxConfirmed(tx.txid) alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, tx) } - // confirm bob's htlc-timeout txs + // Bob's htlc-timeout txs confirm. val remoteOutpoints = alice.stateData.asInstanceOf[DATA_CLOSING].localCommitPublished.map(rcp => rcp.htlcTxs.filter(_._2.isEmpty).keys).toSeq.flatten assert(remoteOutpoints.size == htlcs.bobToAlice.size) remoteOutpoints.foreach { out => alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, htlcsTxsOut.head.copy(txIn = Seq(TxIn(out, Nil, 0)))) } alice2blockchain.expectNoMessage(100 millis) checkPostSpliceState(f, spliceOutFee(f, capacity = 1_900_000.sat)) - - // done awaitCond(alice.stateName == CLOSED) assert(Helpers.Closing.isClosed(alice.stateData.asInstanceOf[DATA_CLOSING], None).exists(_.isInstanceOf[LocalClose])) } @@ -1657,60 +1645,60 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val htlcs = setupHtlcs(f) - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) - val fundingTx1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get - val watchConfirmed1 = alice2blockchain.expectMsgType[WatchFundingConfirmed] - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) - alice2blockchain.expectMsgType[WatchFundingConfirmed] + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) + checkWatchConfirmed(f, fundingTx1) + + // The first splice confirms on Bob's side. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) + + val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + checkWatchConfirmed(f, fundingTx2) alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) - alice2blockchain.expectNoMessage(100 millis) - // we now have two unconfirmed splices + // From Alice's point of view, we now have two unconfirmed splices. alice ! CMD_FORCECLOSE(ActorRef.noSender) alice2bob.expectMsgType[Error] val aliceCommitTx2 = assertPublished(alice2blockchain, "commit-tx") assertPublished(alice2blockchain, "local-anchor") assertPublished(alice2blockchain, "local-main-delayed") - // alice publishes her htlc timeout transactions val htlcsTxsOut = htlcs.aliceToBob.map(_ => assertPublished(alice2blockchain, "htlc-timeout")) htlcsTxsOut.foreach(tx => Transaction.correctlySpends(tx, Seq(aliceCommitTx2), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) - alice2blockchain.expectMsgType[WatchTxConfirmed] alice2blockchain.expectMsgType[WatchTxConfirmed] - alice2blockchain.expectMsgType[WatchOutputSpent] - // watch for all htlc outputs from local commit-tx to be spent htlcs.aliceToBob.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) htlcs.bobToAlice.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) - // splice 1 confirms - watchConfirmed1.replyTo ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + // The first splice transaction confirms. + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) alice2blockchain.expectWatchFundingSpent(fundingTx1.txid) - // bob publishes his commit tx for splice 1 (which double-spends splice 2) + // Bob publishes his commit tx for the first splice transaction (which double-spends the second splice transaction). val bobCommitment1 = bob.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.active.find(_.fundingTxIndex == 1).get val bobCommitTx1 = bobCommitment1.fullySignedLocalCommitTx(bob.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.params, bob.underlyingActor.keyManager).tx Transaction.correctlySpends(bobCommitTx1, Seq(fundingTx1), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS) alice ! WatchFundingSpentTriggered(bobCommitTx1) val watchAlternativeConfirmed = alice2blockchain.expectMsgType[WatchAlternativeCommitTxConfirmed] alice2blockchain.expectNoMessage(100 millis) - // remote commit tx confirms + // Bob's commit tx confirms. watchAlternativeConfirmed.replyTo ! WatchAlternativeCommitTxConfirmedTriggered(BlockHeight(400000), 42, bobCommitTx1) - // we're back to the normal handling of remote commit + // We're back to the normal handling of remote commit. assertPublished(alice2blockchain, "local-anchor") val claimMain = assertPublished(alice2blockchain, "remote-main-delayed") val htlcsTxsOut1 = htlcs.aliceToBob.map(_ => assertPublished(alice2blockchain, "claim-htlc-timeout")) htlcsTxsOut1.foreach(tx => Transaction.correctlySpends(tx, Seq(bobCommitTx1), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) val watchConfirmedRemoteCommit = alice2blockchain.expectWatchTxConfirmed(bobCommitTx1.txid) - // this one fires immediately, tx is already confirmed + // NB: this one fires immediately, tx is already confirmed. watchConfirmedRemoteCommit.replyTo ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, bobCommitTx1) + // Alice's 2nd-stage transactions confirm. val watchConfirmedClaimMain = alice2blockchain.expectWatchTxConfirmed(claimMain.txid) watchConfirmedClaimMain.replyTo ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, claimMain) - - // alice's htlc-timeout transactions confirm val watchHtlcsOut1 = htlcs.aliceToBob.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) htlcs.bobToAlice.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) watchHtlcsOut1.zip(htlcsTxsOut1).foreach { case (watch, tx) => watch.replyTo ! WatchOutputSpentTriggered(tx) } @@ -1719,13 +1707,11 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, tx) } - // bob's htlc-timeout transactions confirm + // Bob's 2nd-stage transactions confirm. bobCommitment1.localCommit.htlcTxsAndRemoteSigs.foreach(txAndSig => alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, txAndSig.htlcTx.tx)) alice2blockchain.expectNoMessage(100 millis) checkPostSpliceState(f, spliceOutFee = 0.sat) - - // done awaitCond(alice.stateName == CLOSED) assert(Helpers.Closing.isClosed(alice.stateData.asInstanceOf[DATA_CLOSING], None).exists(_.isInstanceOf[RemoteClose])) } @@ -1736,32 +1722,37 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val htlcs = setupHtlcs(f) // pay 10_000_000 msat to bob that will be paid back to alice after the splices - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 10_000_000 msat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) - val fundingTx1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get - alice2blockchain.expectWatchFundingConfirmed(fundingTx1.txid) + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 10_000_000 msat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) + checkWatchConfirmed(f, fundingTx1) // remember bob's commitment for later val bobCommit1 = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) - alice2blockchain.expectMsgType[WatchFundingConfirmed] + + // The first splice confirms on Bob's side. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) + + val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) + checkWatchConfirmed(f, fundingTx2) alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) - alice2blockchain.expectNoMessage(100 millis) - // we now have two unconfirmed splices, both active - // bob makes a payment + // From Alice's point of view, We now have two unconfirmed splices, both active. + // Bob makes a payment, that applies to both commitments. val (preimage, add) = addHtlc(10_000_000 msat, bob, alice, bob2alice, alice2bob) crossSign(bob, alice, bob2alice, alice2bob) alice2relayer.expectMsgType[Relayer.RelayForward] fulfillHtlc(add.id, preimage, alice, bob, alice2bob, bob2alice) crossSign(alice, bob, alice2bob, bob2alice) - // funding tx1 confirms + // The first splice transaction confirms. alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) alice2blockchain.expectWatchFundingSpent(fundingTx1.txid) - // bob publishes a revoked commitment for fundingTx1! + // Bob publishes a revoked commitment for fundingTx1! val bobRevokedCommitTx = bobCommit1.localCommit.commitTxAndRemoteSig.commitTx.tx alice ! WatchFundingSpentTriggered(bobRevokedCommitTx) - // alice watches bob's revoked commit tx, and force-closes with her latest commitment + // Alice watches bob's revoked commit tx, and force-closes with her latest commitment. assert(alice2blockchain.expectMsgType[WatchAlternativeCommitTxConfirmed].txId == bobRevokedCommitTx.txid) val aliceCommitTx2 = assertPublished(alice2blockchain, "commit-tx") assertPublished(alice2blockchain, "local-anchor") @@ -1773,9 +1764,9 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik htlcs.aliceToBob.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) htlcs.bobToAlice.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) - // bob's revoked tx wins + // Bob's revoked commit tx wins. alice ! WatchAlternativeCommitTxConfirmedTriggered(BlockHeight(400000), 42, bobRevokedCommitTx) - // alice reacts by punishing bob + // Alice reacts by punishing bob. val aliceClaimMain = assertPublished(alice2blockchain, "remote-main-delayed") val aliceMainPenalty = assertPublished(alice2blockchain, "main-penalty") val aliceHtlcsPenalty = htlcs.aliceToBob.map(_ => assertPublished(alice2blockchain, "htlc-penalty")) ++ htlcs.bobToAlice.map(_ => assertPublished(alice2blockchain, "htlc-penalty")) @@ -1786,18 +1777,15 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik aliceHtlcsPenalty.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent]) alice2blockchain.expectNoMessage(100 millis) - // alice's main penalty txs confirm + // Alice's penalty txs confirm. alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, bobRevokedCommitTx) alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, aliceClaimMain) alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, aliceMainPenalty) - // alice's htlc-penalty txs confirm aliceHtlcsPenalty.foreach { tx => alice ! WatchTxConfirmedTriggered(BlockHeight(400000), 42, tx) } val settledOutgoingHtlcs = htlcs.aliceToBob.map(_ => alice2relayer.expectMsgType[RES_ADD_SETTLED[Origin, HtlcResult.OnChainFail]].htlc).toSet assert(settledOutgoingHtlcs == htlcs.aliceToBob.map(_._2).toSet) checkPostSpliceState(f, spliceOutFee = 0.sat) - - // done awaitCond(alice.stateName == CLOSED) assert(Helpers.Closing.isClosed(alice.stateData.asInstanceOf[DATA_CLOSING], None).exists(_.isInstanceOf[RevokedClose])) } @@ -2024,7 +2012,18 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik import f._ val fundingTx0 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get - val (fundingTx1, fundingTx2) = setup2Splices(f) + + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 10_000_000 msat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) + checkWatchConfirmed(f, fundingTx1) + + // The first splice confirms on Bob's side. + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx1.txid) + bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx1.txid) + bob2alice.forward(alice) + + val fundingTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) + checkWatchConfirmed(f, fundingTx2) val (aliceNodeParams, bobNodeParams) = (alice.underlyingActor.nodeParams, bob.underlyingActor.nodeParams) val (alicePeer, bobPeer) = (alice.getParent, bob.getParent) @@ -2050,7 +2049,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik bob2 ! INPUT_RESTORED(bobData) bob2blockchain.expectMsgType[SetChannelId] bob2blockchain.expectWatchFundingConfirmed(fundingTx2.txid) - bob2blockchain.expectWatchFundingConfirmed(fundingTx1.txid) + bob2blockchain.expectWatchFundingSpent(fundingTx1.txid) bob2blockchain.expectWatchFundingSpent(fundingTx0.txid) bob2blockchain.expectNoMessage(100 millis) } @@ -2171,11 +2170,20 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik } test("recv multiple CMD_SPLICE (splice-in, splice-out)") { f => + import f._ + + // Add some HTLCs before splicing in and out. val htlcs = setupHtlcs(f) - initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + // Alice splices some funds in. + val fundingTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat))) + // The first splice confirms on Bob's side (necessary to allow the second splice transaction). + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) + bob2alice.expectMsgType[SpliceLocked] + // Alice splices some funds out. initiateSplice(f, spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) + // The HTLCs complete while multiple commitments are active. resolveHtlcs(f, htlcs, spliceOutFee = spliceOutFee(f, capacity = 1_900_000.sat)) } @@ -2187,9 +2195,18 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik // | fundingTx |---->| spliceTx1 |---->| spliceTx2 | // +-----------+ +-----------+ +-----------+ val spliceTx1 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(200_000 sat))) + checkWatchConfirmed(f, spliceTx1) val spliceCommitment1 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest assert(spliceCommitment1.fundingTxId == spliceTx1.txid) + + // The first splice confirms on Bob's side (necessary to allow the second splice transaction). + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, spliceTx1) + bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == spliceTx1.txid) + bob2alice.expectMsgType[SpliceLocked] + bob2alice.forward(alice) + val spliceTx2 = initiateSplice(f, spliceIn_opt = Some(SpliceIn(100_000 sat))) + checkWatchConfirmed(f, spliceTx2) val spliceCommitment2 = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest assert(spliceCommitment2.fundingTxId == spliceTx2.txid) assert(spliceCommitment2.localCommit.spec.toLocal == spliceCommitment1.localCommit.spec.toLocal + 100_000.sat) From 5003dc0ef1b54e12701ddb8bdcf5704d2e9bd299 Mon Sep 17 00:00:00 2001 From: Richard Myers Date: Mon, 22 Jul 2024 10:37:02 +0200 Subject: [PATCH 5/8] Add optional `maxExcess` parameter to channel `open` API This change passes `maxExcess` to Peer via OpenChannel and from Peer to Channel via INPUT_INIT_CHANNEL_INITIATOR. The `maxExcess` parameter will be used by bitcoind funding calls but is not currently used. --- .../main/scala/fr/acinq/eclair/Eclair.scala | 5 ++-- .../fr/acinq/eclair/channel/ChannelData.scala | 1 + .../main/scala/fr/acinq/eclair/io/Peer.scala | 4 +++- .../fr/acinq/eclair/EclairImplSpec.scala | 12 +++++----- .../fr/acinq/eclair/channel/FuzzySpec.scala | 2 +- .../ChannelStateTestsHelperMethods.scala | 2 +- .../a/WaitForAcceptChannelStateSpec.scala | 4 ++-- ...tForAcceptDualFundedChannelStateSpec.scala | 2 +- .../a/WaitForOpenChannelStateSpec.scala | 2 +- ...aitForOpenDualFundedChannelStateSpec.scala | 2 +- .../WaitForDualFundingCreatedStateSpec.scala | 2 +- .../b/WaitForDualFundingSignedStateSpec.scala | 2 +- .../b/WaitForFundingCreatedStateSpec.scala | 2 +- .../b/WaitForFundingInternalStateSpec.scala | 2 +- .../b/WaitForFundingSignedStateSpec.scala | 2 +- .../c/WaitForChannelReadyStateSpec.scala | 2 +- ...WaitForDualFundingConfirmedStateSpec.scala | 2 +- .../c/WaitForDualFundingReadyStateSpec.scala | 2 +- .../c/WaitForFundingConfirmedStateSpec.scala | 2 +- .../channel/states/h/ClosingStateSpec.scala | 2 +- .../eclair/integration/IntegrationSpec.scala | 1 + .../basic/fixtures/MinimalNodeFixture.scala | 2 +- .../interop/rustytests/RustyTestsSpec.scala | 2 +- .../io/OpenChannelInterceptorSpec.scala | 4 ++-- .../scala/fr/acinq/eclair/io/PeerSpec.scala | 24 +++++++++---------- .../acinq/eclair/api/handlers/Channel.scala | 6 ++--- .../fr/acinq/eclair/api/ApiServiceSpec.scala | 18 +++++++------- 27 files changed, 59 insertions(+), 54 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala index 518f0c1127..6511a4957c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala @@ -86,7 +86,7 @@ trait Eclair { def disconnect(nodeId: PublicKey)(implicit timeout: Timeout): Future[String] - def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], channelType_opt: Option[SupportedChannelType], fundingFeerate_opt: Option[FeeratePerByte], fundingFeeBudget_opt: Option[Satoshi], announceChannel_opt: Option[Boolean], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[OpenChannelResponse] + def open(nodeId: PublicKey, fundingAmount: Satoshi, maxExcess_opt: Option[Satoshi], pushAmount_opt: Option[MilliSatoshi], channelType_opt: Option[SupportedChannelType], fundingFeerate_opt: Option[FeeratePerByte], fundingFeeBudget_opt: Option[Satoshi], announceChannel_opt: Option[Boolean], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[OpenChannelResponse] def rbfOpen(channelId: ByteVector32, targetFeerate: FeeratePerKw, fundingFeeBudget: Satoshi, lockTime_opt: Option[Long])(implicit timeout: Timeout): Future[CommandResponse[CMD_BUMP_FUNDING_FEE]] @@ -208,7 +208,7 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging { (appKit.switchboard ? Peer.Disconnect(nodeId)).mapTo[Peer.DisconnectResponse].map(_.toString) } - override def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], channelType_opt: Option[SupportedChannelType], fundingFeerate_opt: Option[FeeratePerByte], fundingFeeBudget_opt: Option[Satoshi], announceChannel_opt: Option[Boolean], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[OpenChannelResponse] = { + override def open(nodeId: PublicKey, fundingAmount: Satoshi, maxExcess_opt: Option[Satoshi], pushAmount_opt: Option[MilliSatoshi], channelType_opt: Option[SupportedChannelType], fundingFeerate_opt: Option[FeeratePerByte], fundingFeeBudget_opt: Option[Satoshi], announceChannel_opt: Option[Boolean], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[OpenChannelResponse] = { // we want the open timeout to expire *before* the default ask timeout, otherwise user will get a generic response val openTimeout = openTimeout_opt.getOrElse(Timeout(20 seconds)) // if no budget is provided for the mining fee of the funding tx, we use a default of 0.1% of the funding amount as a safety measure @@ -218,6 +218,7 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging { open = Peer.OpenChannel( remoteNodeId = nodeId, fundingAmount = fundingAmount, + maxExcess_opt = maxExcess_opt, channelType_opt = channelType_opt, pushAmount_opt = pushAmount_opt, fundingTxFeerate_opt = fundingFeerate_opt.map(FeeratePerKw(_)), diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala index 2495e161f0..ff3809ff0d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala @@ -92,6 +92,7 @@ case object ERR_INFORMATION_LEAK extends ChannelState case class INPUT_INIT_CHANNEL_INITIATOR(temporaryChannelId: ByteVector32, fundingAmount: Satoshi, + maxExcess_opt: Option[Satoshi], dualFunded: Boolean, commitTxFeerate: FeeratePerKw, fundingTxFeerate: FeeratePerKw, diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/io/Peer.scala b/eclair-core/src/main/scala/fr/acinq/eclair/io/Peer.scala index 296ae08f26..673918dd26 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/io/Peer.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/io/Peer.scala @@ -173,7 +173,7 @@ class Peer(val nodeParams: NodeParams, val fundingTxFeerate = c.fundingTxFeerate_opt.getOrElse(nodeParams.onChainFeeConf.getFundingFeerate(nodeParams.currentFeerates)) val commitTxFeerate = nodeParams.onChainFeeConf.getCommitmentFeerate(nodeParams.currentFeerates, remoteNodeId, channelType.commitmentFormat, c.fundingAmount) log.info(s"requesting a new channel with type=$channelType fundingAmount=${c.fundingAmount} dualFunded=$dualFunded pushAmount=${c.pushAmount_opt} fundingFeerate=$fundingTxFeerate temporaryChannelId=$temporaryChannelId localParams=$localParams") - channel ! INPUT_INIT_CHANNEL_INITIATOR(temporaryChannelId, c.fundingAmount, dualFunded, commitTxFeerate, fundingTxFeerate, c.fundingTxFeeBudget_opt, c.pushAmount_opt, requireConfirmedInputs, localParams, d.peerConnection, d.remoteInit, c.channelFlags_opt.getOrElse(nodeParams.channelConf.channelFlags), channelConfig, channelType, c.channelOrigin, replyTo) + channel ! INPUT_INIT_CHANNEL_INITIATOR(temporaryChannelId, c.fundingAmount, c.maxExcess_opt, dualFunded, commitTxFeerate, fundingTxFeerate, c.fundingTxFeeBudget_opt, c.pushAmount_opt, requireConfirmedInputs, localParams, d.peerConnection, d.remoteInit, c.channelFlags_opt.getOrElse(nodeParams.channelConf.channelFlags), channelConfig, channelType, c.channelOrigin, replyTo) stay() using d.copy(channels = d.channels + (TemporaryChannelId(temporaryChannelId) -> channel)) case Event(open: protocol.OpenChannel, d: ConnectedData) => @@ -512,6 +512,7 @@ object Peer { case class OpenChannel(remoteNodeId: PublicKey, fundingAmount: Satoshi, + maxExcess_opt: Option[Satoshi], channelType_opt: Option[SupportedChannelType], pushAmount_opt: Option[MilliSatoshi], fundingTxFeerate_opt: Option[FeeratePerKw], @@ -523,6 +524,7 @@ object Peer { channelOrigin: ChannelOrigin = ChannelOrigin.Default) extends PossiblyHarmful { require(!(channelType_opt.exists(_.features.contains(Features.ScidAlias)) && channelFlags_opt.exists(_.announceChannel)), "option_scid_alias is not compatible with public channels") require(fundingAmount > 0.sat, s"funding amount must be positive") + maxExcess_opt.foreach(maxExcess => { require(maxExcess >= 0.sat, s"maxExcess must be positive") }) pushAmount_opt.foreach(pushAmount => { require(pushAmount >= 0.msat, s"pushAmount must be positive") require(pushAmount <= fundingAmount, s"pushAmount must be less than or equal to funding amount") diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala index cb2a6b1fb3..d2de66d0d9 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala @@ -100,12 +100,12 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I val nodeId = PublicKey(hex"030bb6a5e0c6b203c7e2180fb78c7ba4bdce46126761d8201b91ddac089cdecc87") // standard conversion - eclair.open(nodeId, fundingAmount = 10000000L sat, pushAmount_opt = None, channelType_opt = None, fundingFeerate_opt = Some(FeeratePerByte(5 sat)), fundingFeeBudget_opt = None, announceChannel_opt = None, openTimeout_opt = None) + eclair.open(nodeId, fundingAmount = 10000000L sat, maxExcess_opt = None, pushAmount_opt = None, channelType_opt = None, fundingFeerate_opt = Some(FeeratePerByte(5 sat)), fundingFeeBudget_opt = None, announceChannel_opt = None, openTimeout_opt = None) val open = switchboard.expectMsgType[OpenChannel] assert(open.fundingTxFeerate_opt.contains(FeeratePerKw(1250 sat))) // check that minimum fee rate of 253 sat/bw is used - eclair.open(nodeId, fundingAmount = 10000000L sat, pushAmount_opt = None, channelType_opt = Some(ChannelTypes.StaticRemoteKey()), fundingFeerate_opt = Some(FeeratePerByte(1 sat)), fundingFeeBudget_opt = None, announceChannel_opt = None, openTimeout_opt = None) + eclair.open(nodeId, fundingAmount = 10000000L sat, maxExcess_opt = None, pushAmount_opt = None, channelType_opt = Some(ChannelTypes.StaticRemoteKey()), fundingFeerate_opt = Some(FeeratePerByte(1 sat)), fundingFeeBudget_opt = None, announceChannel_opt = None, openTimeout_opt = None) val open1 = switchboard.expectMsgType[OpenChannel] assert(open1.fundingTxFeerate_opt.contains(FeeratePerKw.MinimumFeeratePerKw)) assert(open1.channelType_opt.contains(ChannelTypes.StaticRemoteKey())) @@ -251,16 +251,16 @@ class EclairImplSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with I val eclair = new EclairImpl(kit) // option_scid_alias is not compatible with public channels - eclair.open(randomKey().publicKey, 123456 sat, None, Some(ChannelTypes.AnchorOutputsZeroFeeHtlcTx(scidAlias = true, zeroConf = true)), None, None, announceChannel_opt = Some(true), None).pipeTo(sender.ref) + eclair.open(randomKey().publicKey, 123456 sat, None, None, Some(ChannelTypes.AnchorOutputsZeroFeeHtlcTx(scidAlias = true, zeroConf = true)), None, None, announceChannel_opt = Some(true), None).pipeTo(sender.ref) assert(sender.expectMsgType[Status.Failure].cause.getMessage.contains("option_scid_alias is not compatible with public channels")) - eclair.open(randomKey().publicKey, 123456 sat, None, Some(ChannelTypes.AnchorOutputsZeroFeeHtlcTx(scidAlias = true, zeroConf = true)), None, None, announceChannel_opt = Some(false), None).pipeTo(sender.ref) + eclair.open(randomKey().publicKey, 123456 sat, None, None, Some(ChannelTypes.AnchorOutputsZeroFeeHtlcTx(scidAlias = true, zeroConf = true)), None, None, announceChannel_opt = Some(false), None).pipeTo(sender.ref) switchboard.expectMsgType[Peer.OpenChannel] - eclair.open(randomKey().publicKey, 123456 sat, None, Some(ChannelTypes.AnchorOutputsZeroFeeHtlcTx(zeroConf = true)), None, None, announceChannel_opt = Some(true), None).pipeTo(sender.ref) + eclair.open(randomKey().publicKey, 123456 sat, None, None, Some(ChannelTypes.AnchorOutputsZeroFeeHtlcTx(zeroConf = true)), None, None, announceChannel_opt = Some(true), None).pipeTo(sender.ref) switchboard.expectMsgType[Peer.OpenChannel] - eclair.open(randomKey().publicKey, 123456 sat, None, Some(ChannelTypes.AnchorOutputsZeroFeeHtlcTx(zeroConf = true)), None, None, announceChannel_opt = Some(false), None).pipeTo(sender.ref) + eclair.open(randomKey().publicKey, 123456 sat, None, None, Some(ChannelTypes.AnchorOutputsZeroFeeHtlcTx(zeroConf = true)), None, None, announceChannel_opt = Some(false), None).pipeTo(sender.ref) switchboard.expectMsgType[Peer.OpenChannel] } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/FuzzySpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/FuzzySpec.scala index dc25ecedc6..16e0518049 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/FuzzySpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/FuzzySpec.scala @@ -77,7 +77,7 @@ class FuzzySpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with Channe aliceRegister ! alice bobRegister ! bob // no announcements - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, Alice.channelParams, pipe, bobInit, channelFlags, ChannelConfig.standard, ChannelTypes.Standard(), replyTo = system.deadLetters) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, maxExcess_opt = None, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, Alice.channelParams, pipe, bobInit, channelFlags, ChannelConfig.standard, ChannelTypes.Standard(), replyTo = system.deadLetters) alice2blockchain.expectMsgType[TxPublisher.SetChannelId] bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, Bob.channelParams, pipe, aliceInit, ChannelConfig.standard, ChannelTypes.Standard()) bob2blockchain.expectMsgType[TxPublisher.SetChannelId] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala index 97afafb6a0..d6142fda40 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala @@ -247,7 +247,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually { val aliceInit = Init(aliceParams.initFeatures) val bobInit = Init(bobParams.initFeatures) - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingAmount, dualFunded, commitTxFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, initiatorPushAmount, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingAmount, maxExcess_opt = None, dualFunded, commitTxFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, initiatorPushAmount, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) assert(alice2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId == ByteVector32.Zeroes) bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, nonInitiatorFundingAmount, dualFunded, nonInitiatorPushAmount, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) assert(bob2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId == ByteVector32.Zeroes) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForAcceptChannelStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForAcceptChannelStateSpec.scala index 0f94af78b0..6a25ac5c40 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForAcceptChannelStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForAcceptChannelStateSpec.scala @@ -66,7 +66,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS within(30 seconds) { alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted]) val fundingAmount = if (test.tags.contains(LargeChannel)) Btc(5).toSatoshi else TestConstants.fundingSatoshis - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingAmount, dualFunded = false, commitTxFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingAmount, maxExcess_opt = None, dualFunded = false, commitTxFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) alice2bob.expectMsgType[OpenChannel] alice2bob.forward(bob) @@ -172,7 +172,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS // Bob advertises support for anchor outputs, but Alice doesn't. val aliceParams = Alice.channelParams val bobParams = Bob.channelParams.copy(initFeatures = Features(Features.StaticRemoteKey -> FeatureSupport.Optional, Features.AnchorOutputs -> FeatureSupport.Optional)) - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, Init(bobParams.initFeatures), channelFlags, channelConfig, ChannelTypes.AnchorOutputs(), replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, maxExcess_opt = None, dualFunded = false, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, Init(bobParams.initFeatures), channelFlags, channelConfig, ChannelTypes.AnchorOutputs(), replyTo = aliceOpenReplyTo.ref.toTyped) bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, Init(bobParams.initFeatures), channelConfig, ChannelTypes.AnchorOutputs()) val open = alice2bob.expectMsgType[OpenChannel] assert(open.channelType_opt.contains(ChannelTypes.AnchorOutputs())) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForAcceptDualFundedChannelStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForAcceptDualFundedChannelStateSpec.scala index 34c505afb3..c3f4d30531 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForAcceptDualFundedChannelStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForAcceptDualFundedChannelStateSpec.scala @@ -59,7 +59,7 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt val listener = TestProbe() within(30 seconds) { alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted]) - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, None, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, maxExcess_opt = None, dualFunded = true, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, None, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, nonInitiatorContribution, dualFunded = true, nonInitiatorPushAmount, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) val open = alice2bob.expectMsgType[OpenDualFundedChannel] alice2bob.forward(bob, open) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForOpenChannelStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForOpenChannelStateSpec.scala index ca8a8672a6..ab0495da97 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForOpenChannelStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForOpenChannelStateSpec.scala @@ -57,7 +57,7 @@ class WaitForOpenChannelStateSpec extends TestKitBaseClass with FixtureAnyFunSui val listener = TestProbe() within(30 seconds) { bob.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted]) - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, commitTxFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, maxExcess_opt = None, dualFunded = false, commitTxFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) awaitCond(bob.stateName == WAIT_FOR_OPEN_CHANNEL) withFixture(test.toNoArgTest(FixtureParam(alice, bob, alice2bob, bob2alice, bob2blockchain, listener))) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForOpenDualFundedChannelStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForOpenDualFundedChannelStateSpec.scala index fc1cec79b2..a0faa13322 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForOpenDualFundedChannelStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForOpenDualFundedChannelStateSpec.scala @@ -57,7 +57,7 @@ class WaitForOpenDualFundedChannelStateSpec extends TestKitBaseClass with Fixtur val bobInit = Init(bobParams.initFeatures) val requireConfirmedInputs = test.tags.contains(aliceRequiresConfirmedInputs) within(30 seconds) { - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, pushAmount, requireConfirmedInputs, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, maxExcess_opt = None, dualFunded = true, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, pushAmount, requireConfirmedInputs, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = true, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) awaitCond(bob.stateName == WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL) withFixture(test.toNoArgTest(FixtureParam(alice, bob, alice2bob, bob2alice, aliceListener, bobListener))) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala index 9458661e32..239dea6d4c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala @@ -53,7 +53,7 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn within(30 seconds) { alice.underlying.system.eventStream.subscribe(aliceListener.ref, classOf[ChannelAborted]) bob.underlying.system.eventStream.subscribe(bobListener.ref, classOf[ChannelAborted]) - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, None, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, maxExcess_opt = None, dualFunded = true, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, None, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, Some(TestConstants.nonInitiatorFundingSatoshis), dualFunded = true, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) alice2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id bob2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingSignedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingSignedStateSpec.scala index 75e26ecfd9..72b18f3a07 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingSignedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingSignedStateSpec.scala @@ -61,7 +61,7 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny within(30 seconds) { alice.underlying.system.eventStream.subscribe(aliceListener.ref, classOf[ChannelAborted]) bob.underlying.system.eventStream.subscribe(bobListener.ref, classOf[ChannelAborted]) - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, initiatorPushAmount, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, maxExcess_opt = None, dualFunded = true, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, initiatorPushAmount, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, bobContribution, dualFunded = true, nonInitiatorPushAmount, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) alice2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id bob2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingCreatedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingCreatedStateSpec.scala index 9ee675f56e..85764379b0 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingCreatedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingCreatedStateSpec.scala @@ -65,7 +65,7 @@ class WaitForFundingCreatedStateSpec extends TestKitBaseClass with FixtureAnyFun val listener = TestProbe() within(30 seconds) { bob.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted]) - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(pushMsat), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingSatoshis, maxExcess_opt = None, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(pushMsat), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) alice2blockchain.expectMsgType[TxPublisher.SetChannelId] bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) bob2blockchain.expectMsgType[TxPublisher.SetChannelId] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingInternalStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingInternalStateSpec.scala index 301609f613..e4565e2f59 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingInternalStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingInternalStateSpec.scala @@ -52,7 +52,7 @@ class WaitForFundingInternalStateSpec extends TestKitBaseClass with FixtureAnyFu val listener = TestProbe() within(30 seconds) { alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted]) - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, maxExcess_opt = None, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) alice2bob.expectMsgType[OpenChannel] alice2bob.forward(bob) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingSignedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingSignedStateSpec.scala index 6abc907f0a..93a31599cc 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingSignedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingSignedStateSpec.scala @@ -68,7 +68,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS val listener = TestProbe() within(30 seconds) { alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted]) - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingSatoshis, dualFunded = false, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(pushMsat), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, fundingSatoshis, maxExcess_opt = None, dualFunded = false, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(pushMsat), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) alice2blockchain.expectMsgType[TxPublisher.SetChannelId] bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) bob2blockchain.expectMsgType[TxPublisher.SetChannelId] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForChannelReadyStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForChannelReadyStateSpec.scala index 5e2cb53505..0557ceedb7 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForChannelReadyStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForChannelReadyStateSpec.scala @@ -65,7 +65,7 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu alice.underlying.system.eventStream.subscribe(aliceListener.ref, classOf[ChannelAborted]) bob.underlying.system.eventStream.subscribe(bobListener.ref, classOf[ChannelAborted]) alice.underlyingActor.nodeParams.db.peers.addOrUpdateRelayFees(bobParams.nodeId, relayFees) - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, pushMsat, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, maxExcess_opt = None, dualFunded = false, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, pushMsat, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) alice2blockchain.expectMsgType[TxPublisher.SetChannelId] bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) bob2blockchain.expectMsgType[TxPublisher.SetChannelId] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala index 9c6eb478f4..ca772d62aa 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala @@ -72,7 +72,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture val bobContribution = if (test.tags.contains("no-funding-contribution")) None else Some(TestConstants.nonInitiatorFundingSatoshis) val (initiatorPushAmount, nonInitiatorPushAmount) = if (test.tags.contains("both_push_amount")) (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, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, maxExcess_opt = None, dualFunded = true, commitFeerate, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, initiatorPushAmount, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, bobContribution, dualFunded = true, nonInitiatorPushAmount, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) alice2blockchain.expectMsgType[SetChannelId] // temporary channel id bob2blockchain.expectMsgType[SetChannelId] // temporary channel id diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala index d260fc9bde..8b0d8f4e4a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala @@ -52,7 +52,7 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF val listener = TestProbe() within(30 seconds) { alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted]) - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, None, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, maxExcess_opt = None, dualFunded = true, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, None, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, Some(TestConstants.nonInitiatorFundingSatoshis), dualFunded = true, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) alice2blockchain.expectMsgType[SetChannelId] // temporary channel id bob2blockchain.expectMsgType[SetChannelId] // temporary channel id diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingConfirmedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingConfirmedStateSpec.scala index f111398296..88e3e5fc7d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingConfirmedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingConfirmedStateSpec.scala @@ -61,7 +61,7 @@ class WaitForFundingConfirmedStateSpec extends TestKitBaseClass with FixtureAnyF bob.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted]) bob.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelClosed]) val commitTxFeerate = if (test.tags.contains(ChannelStateTestsTags.AnchorOutputs) || test.tags.contains(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) TestConstants.anchorOutputsFeeratePerKw else TestConstants.feeratePerKw - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, commitTxFeerate, commitTxFeerate, fundingTxFeeBudget_opt = None, Some(pushMsat), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, maxExcess_opt = None, dualFunded = false, commitTxFeerate, commitTxFeerate, fundingTxFeeBudget_opt = None, Some(pushMsat), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) alice2blockchain.expectMsgType[TxPublisher.SetChannelId] bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) bob2blockchain.expectMsgType[TxPublisher.SetChannelId] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/h/ClosingStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/h/ClosingStateSpec.scala index 4b2e0a4ad7..d32251e49c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/h/ClosingStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/h/ClosingStateSpec.scala @@ -75,7 +75,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with val (aliceParams, bobParams, channelType) = computeFeatures(setup, test.tags, channelFlags) val aliceInit = Init(aliceParams.initFeatures) val bobInit = Init(bobParams.initFeatures) - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, maxExcess_opt = None, dualFunded = false, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) alice2blockchain.expectMsgType[SetChannelId] bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) bob2blockchain.expectMsgType[SetChannelId] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala index f4fdd37510..966d4106b9 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala @@ -180,6 +180,7 @@ abstract class IntegrationSpec extends TestKitBaseClass with BitcoindService wit sender.send(node1.switchboard, Peer.OpenChannel( remoteNodeId = node2.nodeParams.nodeId, fundingAmount = fundingAmount, + maxExcess_opt = None, channelType_opt = None, pushAmount_opt = Some(pushMsat), fundingTxFeerate_opt = None, diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala index 1fcbada4a6..27a4cd58de 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala @@ -182,7 +182,7 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat def openChannel(node1: MinimalNodeFixture, node2: MinimalNodeFixture, funding: Satoshi, channelType_opt: Option[SupportedChannelType] = None)(implicit system: ActorSystem): OpenChannelResponse.Created = { val sender = TestProbe("sender") - sender.send(node1.switchboard, Peer.OpenChannel(node2.nodeParams.nodeId, funding, channelType_opt, None, None, None, None, None)) + sender.send(node1.switchboard, Peer.OpenChannel(node2.nodeParams.nodeId, funding, maxExcess_opt = None, channelType_opt, None, None, None, None, None)) sender.expectMsgType[OpenChannelResponse.Created] } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/interop/rustytests/RustyTestsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/interop/rustytests/RustyTestsSpec.scala index 69ae7dfedc..b2bc436741 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/interop/rustytests/RustyTestsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/interop/rustytests/RustyTestsSpec.scala @@ -72,7 +72,7 @@ class RustyTestsSpec extends TestKitBaseClass with Matchers with FixtureAnyFunSu val aliceInit = Init(Alice.channelParams.initFeatures) val bobInit = Init(Bob.channelParams.initFeatures) // alice and bob will both have 1 000 000 sat - alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, 2000000 sat, dualFunded = false, commitTxFeerate = feeratePerKw, fundingTxFeerate = feeratePerKw, fundingTxFeeBudget_opt = None, Some(1000000000 msat), requireConfirmedInputs = false, Alice.channelParams, pipe, bobInit, ChannelFlags(announceChannel = false), channelConfig, channelType, replyTo = system.deadLetters) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, 2000000 sat, maxExcess_opt = None, dualFunded = false, commitTxFeerate = feeratePerKw, fundingTxFeerate = feeratePerKw, fundingTxFeeBudget_opt = None, Some(1000000000 msat), requireConfirmedInputs = false, Alice.channelParams, pipe, bobInit, ChannelFlags(announceChannel = false), channelConfig, channelType, replyTo = system.deadLetters) alice2blockchain.expectMsgType[TxPublisher.SetChannelId] bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, None, dualFunded = false, None, Bob.channelParams, pipe, aliceInit, channelConfig, channelType) bob2blockchain.expectMsgType[TxPublisher.SetChannelId] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/io/OpenChannelInterceptorSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/io/OpenChannelInterceptorSpec.scala index 3231ce2ad2..9a8e74a406 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/io/OpenChannelInterceptorSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/io/OpenChannelInterceptorSpec.scala @@ -190,7 +190,7 @@ class OpenChannelInterceptorSpec extends ScalaTestWithActorTestKit(ConfigFactory val probe = TestProbe[Any]() val fundingAmountBig = Channel.MAX_FUNDING_WITHOUT_WUMBO + 10_000.sat - openChannelInterceptor ! OpenChannelInitiator(probe.ref, remoteNodeId, Peer.OpenChannel(remoteNodeId, fundingAmountBig, None, None, None, None, None, None), defaultFeatures, defaultFeatures.add(Wumbo, Optional)) + openChannelInterceptor ! OpenChannelInitiator(probe.ref, remoteNodeId, Peer.OpenChannel(remoteNodeId, fundingAmountBig, None, None, None, None, None, None, None), defaultFeatures, defaultFeatures.add(Wumbo, Optional)) assert(probe.expectMessageType[OpenChannelResponse.Rejected].reason.contains("you must enable large channels support")) } @@ -199,7 +199,7 @@ class OpenChannelInterceptorSpec extends ScalaTestWithActorTestKit(ConfigFactory val probe = TestProbe[Any]() val fundingAmountBig = Channel.MAX_FUNDING_WITHOUT_WUMBO + 10_000.sat - openChannelInterceptor ! OpenChannelInitiator(probe.ref, remoteNodeId, Peer.OpenChannel(remoteNodeId, fundingAmountBig, None, None, None, None, None, None), defaultFeatures.add(Wumbo, Optional), defaultFeatures) + openChannelInterceptor ! OpenChannelInitiator(probe.ref, remoteNodeId, Peer.OpenChannel(remoteNodeId, fundingAmountBig, None, None, None, None, None, None, None), defaultFeatures.add(Wumbo, Optional), defaultFeatures) assert(probe.expectMessageType[OpenChannelResponse.Rejected].reason == s"fundingAmount=$fundingAmountBig is too big, the remote peer doesn't support wumbo") } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpec.scala index eea78d52ca..52aacd8b96 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpec.scala @@ -363,7 +363,7 @@ class PeerSpec extends FixtureSpec { connect(remoteNodeId, peer, peerConnection, switchboard) assert(peer.stateData.channels.isEmpty) - val open = Peer.OpenChannel(remoteNodeId, 10000 sat, None, None, None, None, None, None) + val open = Peer.OpenChannel(remoteNodeId, 10000 sat, None, None, None, None, None, None, None) peerConnection.send(peer, open) channel.expectMsgType[INPUT_INIT_CHANNEL_INITIATOR] } @@ -384,7 +384,7 @@ class PeerSpec extends FixtureSpec { // Both peers support option_dual_fund, so it is automatically used. connect(remoteNodeId, peer, peerConnection, switchboard, remoteInit = protocol.Init(Features(StaticRemoteKey -> Optional, AnchorOutputsZeroFeeHtlcTx -> Optional, DualFunding -> Optional))) assert(peer.stateData.channels.isEmpty) - probe.send(peer, Peer.OpenChannel(remoteNodeId, 25000 sat, None, None, None, None, None, None)) + probe.send(peer, Peer.OpenChannel(remoteNodeId, 25000 sat, None, None, None, None, None, None, None)) assert(channel.expectMsgType[INPUT_INIT_CHANNEL_INITIATOR].dualFunded) } @@ -427,15 +427,15 @@ class PeerSpec extends FixtureSpec { connect(remoteNodeId, peer, peerConnection, switchboard, remoteInit = protocol.Init(Features(StaticRemoteKey -> Mandatory))) assert(peer.stateData.channels.isEmpty) - probe.send(peer, Peer.OpenChannel(remoteNodeId, 15000 sat, None, None, None, None, None, None)) + probe.send(peer, Peer.OpenChannel(remoteNodeId, 15000 sat, None, None, None, None, None, None, None)) assert(channel.expectMsgType[INPUT_INIT_CHANNEL_INITIATOR].channelType == ChannelTypes.StaticRemoteKey()) // We can create channels that don't use the features we have enabled. - probe.send(peer, Peer.OpenChannel(remoteNodeId, 15000 sat, Some(ChannelTypes.Standard()), None, None, None, None, None)) + probe.send(peer, Peer.OpenChannel(remoteNodeId, 15000 sat, None, Some(ChannelTypes.Standard()), None, None, None, None, None)) assert(channel.expectMsgType[INPUT_INIT_CHANNEL_INITIATOR].channelType == ChannelTypes.Standard()) // We can create channels that use features that we haven't enabled. - probe.send(peer, Peer.OpenChannel(remoteNodeId, 15000 sat, Some(ChannelTypes.AnchorOutputs()), None, None, None, None, None)) + probe.send(peer, Peer.OpenChannel(remoteNodeId, 15000 sat, None, Some(ChannelTypes.AnchorOutputs()), None, None, None, None, None)) assert(channel.expectMsgType[INPUT_INIT_CHANNEL_INITIATOR].channelType == ChannelTypes.AnchorOutputs()) } @@ -470,7 +470,7 @@ class PeerSpec extends FixtureSpec { // We ensure the current network feerate is higher than the default anchor output feerate. nodeParams.setFeerates(FeeratesPerKw.single(TestConstants.anchorOutputsFeeratePerKw * 2).copy(minimum = FeeratePerKw(250 sat))) - probe.send(peer, Peer.OpenChannel(remoteNodeId, 15000 sat, None, None, None, None, None, None)) + probe.send(peer, Peer.OpenChannel(remoteNodeId, 15000 sat, None, None, None, None, None, None, None)) val init = channel.expectMsgType[INPUT_INIT_CHANNEL_INITIATOR] assert(init.channelType == ChannelTypes.AnchorOutputs()) assert(!init.dualFunded) @@ -488,7 +488,7 @@ class PeerSpec extends FixtureSpec { // We ensure the current network feerate is higher than the default anchor output feerate. nodeParams.setFeerates(FeeratesPerKw.single(TestConstants.anchorOutputsFeeratePerKw * 2).copy(minimum = FeeratePerKw(250 sat))) - probe.send(peer, Peer.OpenChannel(remoteNodeId, 15000 sat, None, None, None, None, None, None)) + probe.send(peer, Peer.OpenChannel(remoteNodeId, 15000 sat, None, None, None, None, None, None, None)) val init = channel.expectMsgType[INPUT_INIT_CHANNEL_INITIATOR] assert(init.channelType == ChannelTypes.AnchorOutputsZeroFeeHtlcTx()) assert(!init.dualFunded) @@ -502,7 +502,7 @@ class PeerSpec extends FixtureSpec { val probe = TestProbe() connect(remoteNodeId, peer, peerConnection, switchboard, remoteInit = protocol.Init(Features(StaticRemoteKey -> Mandatory))) - probe.send(peer, Peer.OpenChannel(remoteNodeId, 24000 sat, None, None, None, None, None, None)) + probe.send(peer, Peer.OpenChannel(remoteNodeId, 24000 sat, None, None, None, None, None, None, None)) val init = channel.expectMsgType[INPUT_INIT_CHANNEL_INITIATOR] assert(init.channelType == ChannelTypes.StaticRemoteKey()) assert(!init.dualFunded) @@ -519,12 +519,12 @@ class PeerSpec extends FixtureSpec { assert(peer.underlyingActor.nodeParams.channelConf.maxHtlcValueInFlightMsat == 100_000_000.msat) { - probe.send(peer, Peer.OpenChannel(remoteNodeId, 200_000 sat, None, None, None, None, None, None)) + probe.send(peer, Peer.OpenChannel(remoteNodeId, 200_000 sat, None, None, None, None, None, None, None)) val init = channel.expectMsgType[INPUT_INIT_CHANNEL_INITIATOR] assert(init.localParams.maxHtlcValueInFlightMsat == 50_000_000.msat) // max-htlc-value-in-flight-percent } { - probe.send(peer, Peer.OpenChannel(remoteNodeId, 500_000 sat, None, None, None, None, None, None)) + probe.send(peer, Peer.OpenChannel(remoteNodeId, 500_000 sat, None, None, None, None, None, None, None)) val init = channel.expectMsgType[INPUT_INIT_CHANNEL_INITIATOR] assert(init.localParams.maxHtlcValueInFlightMsat == 100_000_000.msat) // max-htlc-value-in-flight-msat } @@ -548,7 +548,7 @@ class PeerSpec extends FixtureSpec { import f._ intercept[IllegalArgumentException] { - Peer.OpenChannel(remoteNodeId, 24000 sat, Some(ChannelTypes.AnchorOutputsZeroFeeHtlcTx(scidAlias = true, zeroConf = true)), None, None, None, Some(ChannelFlags(announceChannel = true)), None) + Peer.OpenChannel(remoteNodeId, 24000 sat, None, Some(ChannelTypes.AnchorOutputsZeroFeeHtlcTx(scidAlias = true, zeroConf = true)), None, None, None, Some(ChannelFlags(announceChannel = true)), None) } } @@ -557,7 +557,7 @@ class PeerSpec extends FixtureSpec { val probe = TestProbe() connect(remoteNodeId, peer, peerConnection, switchboard) - probe.send(peer, Peer.OpenChannel(remoteNodeId, 15000 sat, None, Some(100 msat), None, None, None, None)) + probe.send(peer, Peer.OpenChannel(remoteNodeId, 15000 sat, None, None, Some(100 msat), None, None, None, None)) val init = channel.expectMsgType[INPUT_INIT_CHANNEL_INITIATOR] assert(init.replyTo == probe.ref.toTyped[OpenChannelResponse]) } diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Channel.scala b/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Channel.scala index fd5dd468b5..ad23d5c459 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Channel.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/api/handlers/Channel.scala @@ -52,8 +52,8 @@ trait Channel { ).map(ct => ct.toString -> ct).toMap // we use the toString method as name in the api val open: Route = postRequest("open") { implicit t => - formFields(nodeIdFormParam, "fundingSatoshis".as[Satoshi], "pushMsat".as[MilliSatoshi].?, "channelType".?, "fundingFeerateSatByte".as[FeeratePerByte].?, "fundingFeeBudgetSatoshis".as[Satoshi].?, "announceChannel".as[Boolean].?, "openTimeoutSeconds".as[Timeout].?) { - (nodeId, fundingSatoshis, pushMsat, channelTypeName_opt, fundingFeerateSatByte, fundingFeeBudget_opt, announceChannel_opt, openTimeout_opt) => + formFields(nodeIdFormParam, "fundingSatoshis".as[Satoshi], "maxExcess".as[Satoshi].?, "pushMsat".as[MilliSatoshi].?, "channelType".?, "fundingFeerateSatByte".as[FeeratePerByte].?, "fundingFeeBudgetSatoshis".as[Satoshi].?, "announceChannel".as[Boolean].?, "openTimeoutSeconds".as[Timeout].?) { + (nodeId, fundingSatoshis, maxExcess, pushMsat, channelTypeName_opt, fundingFeerateSatByte, fundingFeeBudget_opt, announceChannel_opt, openTimeout_opt) => val (channelTypeOk, channelType_opt) = channelTypeName_opt match { case Some(channelTypeName) => supportedChannelTypes.get(channelTypeName) match { case Some(channelType) => (true, Some(channelType)) @@ -64,7 +64,7 @@ trait Channel { if (!channelTypeOk) { reject(MalformedFormFieldRejection("channelType", s"Channel type not supported: must be one of ${supportedChannelTypes.keys.mkString(",")}")) } else { - complete(eclairApi.open(nodeId, fundingSatoshis, pushMsat, channelType_opt, fundingFeerateSatByte, fundingFeeBudget_opt, announceChannel_opt, openTimeout_opt)) + complete(eclairApi.open(nodeId, fundingSatoshis, maxExcess, pushMsat, channelType_opt, fundingFeerateSatByte, fundingFeeBudget_opt, announceChannel_opt, openTimeout_opt)) } } } diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala index aabcd8eb09..593585df2b 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala @@ -277,7 +277,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM val fundingTxId = TxId.fromValidHex("a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4") val eclair = mock[Eclair] - eclair.open(any, any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(OpenChannelResponse.Created(channelId, fundingTxId, 100 sat)) + eclair.open(any, any, any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(OpenChannelResponse.Created(channelId, fundingTxId, 100 sat)) val mockService = new MockService(eclair) Post("/open", FormData("nodeId" -> nodeId.toString(), "fundingSatoshis" -> "100002").toEntity) ~> @@ -288,7 +288,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM assert(handled) assert(status == OK) assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e with fundingTxId=a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4 and fees=100 sat\"") - eclair.open(nodeId, 100002 sat, None, None, None, None, None, None)(any[Timeout]).wasCalled(once) + eclair.open(nodeId, 100002 sat, None, None, None, None, None, None, None)(any[Timeout]).wasCalled(once) } } @@ -314,7 +314,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM val fundingTxId = TxId.fromValidHex("a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4") val eclair = mock[Eclair] - eclair.open(any, any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(OpenChannelResponse.Created(channelId, fundingTxId, 0 sat)) + eclair.open(any, any, any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(OpenChannelResponse.Created(channelId, fundingTxId, 0 sat)) val mockService = new MockService(eclair) Post("/open", FormData("nodeId" -> nodeId.toString(), "fundingSatoshis" -> "25000", "channelType" -> "standard").toEntity) ~> @@ -325,7 +325,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM assert(handled) assert(status == OK) assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e with fundingTxId=a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4 and fees=0 sat\"") - eclair.open(nodeId, 25000 sat, None, Some(ChannelTypes.Standard()), None, None, None, None)(any[Timeout]).wasCalled(once) + eclair.open(nodeId, 25000 sat, None, None, Some(ChannelTypes.Standard()), None, None, None, None)(any[Timeout]).wasCalled(once) } } @@ -335,7 +335,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM val fundingTxId = TxId.fromValidHex("a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4") val eclair = mock[Eclair] - eclair.open(any, any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(OpenChannelResponse.Created(channelId, fundingTxId, 1 sat)) + eclair.open(any, any, any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(OpenChannelResponse.Created(channelId, fundingTxId, 1 sat)) val mockService = new MockService(eclair) Post("/open", FormData("nodeId" -> nodeId.toString(), "fundingSatoshis" -> "25000", "channelType" -> "static_remotekey").toEntity) ~> @@ -346,7 +346,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM assert(handled) assert(status == OK) assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e with fundingTxId=a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4 and fees=1 sat\"") - eclair.open(nodeId, 25000 sat, None, Some(ChannelTypes.StaticRemoteKey()), None, None, None, None)(any[Timeout]).wasCalled(once) + eclair.open(nodeId, 25000 sat, None, None, Some(ChannelTypes.StaticRemoteKey()), None, None, None, None)(any[Timeout]).wasCalled(once) } } @@ -356,7 +356,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM val fundingTxId = TxId.fromValidHex("a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4") val eclair = mock[Eclair] - eclair.open(any, any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(OpenChannelResponse.Created(channelId, fundingTxId, 500 sat)) + eclair.open(any, any, any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(OpenChannelResponse.Created(channelId, fundingTxId, 500 sat)) val mockService = new MockService(eclair) Post("/open", FormData("nodeId" -> nodeId.toString(), "fundingSatoshis" -> "25000", "channelType" -> "anchor_outputs").toEntity) ~> @@ -367,7 +367,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM assert(handled) assert(status == OK) assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e with fundingTxId=a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4 and fees=500 sat\"") - eclair.open(nodeId, 25000 sat, None, Some(ChannelTypes.AnchorOutputs()), None, None, None, None)(any[Timeout]).wasCalled(once) + eclair.open(nodeId, 25000 sat, None, None, Some(ChannelTypes.AnchorOutputs()), None, None, None, None)(any[Timeout]).wasCalled(once) } Post("/open", FormData("nodeId" -> nodeId.toString(), "fundingSatoshis" -> "25000", "channelType" -> "anchor_outputs_zero_fee_htlc_tx").toEntity) ~> @@ -378,7 +378,7 @@ class ApiServiceSpec extends AnyFunSuite with ScalatestRouteTest with IdiomaticM assert(handled) assert(status == OK) assert(entityAs[String] == "\"created channel 56d7d6eda04d80138270c49709f1eadb5ab4939e5061309ccdacdb98ce637d0e with fundingTxId=a86b3f93c1b2ea3f221159869d6f556cae1ba2622cc8c7eb71c7f4f64e0fbca4 and fees=500 sat\"") - eclair.open(nodeId, 25000 sat, None, Some(ChannelTypes.AnchorOutputsZeroFeeHtlcTx()), None, None, None, None)(any[Timeout]).wasCalled(once) + eclair.open(nodeId, 25000 sat, None, None, Some(ChannelTypes.AnchorOutputsZeroFeeHtlcTx()), None, None, None, None)(any[Timeout]).wasCalled(once) } } From 4a03134b8287c965e2a0010cdf8bf285111c9afd Mon Sep 17 00:00:00 2001 From: Richard Myers Date: Mon, 22 Jul 2024 14:57:47 +0200 Subject: [PATCH 6/8] Add maxExcess and addExcessToRecipientPosition params to fundTransaction calls These parameters are only supported in a testing branch of bitcoind for now. --- .../eclair/blockchain/OnChainWallet.scala | 4 ++-- .../bitcoind/rpc/BitcoinCoreClient.scala | 18 ++++++++++-------- .../channel/fsm/ChannelOpenSingleFunded.scala | 2 +- .../channel/fund/InteractiveTxFunder.scala | 2 +- .../channel/publish/ReplaceableTxFunder.scala | 2 +- .../eclair/blockchain/DummyOnChainWallet.scala | 14 +++++++------- 6 files changed, 22 insertions(+), 20 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/OnChainWallet.scala b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/OnChainWallet.scala index 3544636678..12606ad9f7 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/OnChainWallet.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/OnChainWallet.scala @@ -37,7 +37,7 @@ trait OnChainChannelFunder { * Fund the provided transaction by adding inputs (and a change output if necessary). * Callers must verify that the resulting transaction isn't sending funds to unexpected addresses (malicious bitcoin node). */ - def fundTransaction(tx: Transaction, feeRate: FeeratePerKw, replaceable: Boolean, externalInputsWeight: Map[OutPoint, Long] = Map.empty, feeBudget_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[FundTransactionResponse] + def fundTransaction(tx: Transaction, feeRate: FeeratePerKw, replaceable: Boolean, externalInputsWeight: Map[OutPoint, Long] = Map.empty, feeBudget_opt: Option[Satoshi], addExcessToRecipientPosition_opt: Option[Int], maxExcess_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[FundTransactionResponse] /** * Sign a PSBT. Result may be partially signed: only inputs known to our bitcoin wallet will be signed. * @@ -55,7 +55,7 @@ trait OnChainChannelFunder { def publishTransaction(tx: Transaction)(implicit ec: ExecutionContext): Future[TxId] /** Create a fully signed channel funding transaction with the provided pubkeyScript. */ - def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRate: FeeratePerKw, feeBudget_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] + def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRate: FeeratePerKw, feeBudget_opt: Option[Satoshi], maxExcess_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] /** * Committing *must* include publishing the transaction on the network. diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/bitcoind/rpc/BitcoinCoreClient.scala b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/bitcoind/rpc/BitcoinCoreClient.scala index f6a8daa965..3bb199f9ac 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/bitcoind/rpc/BitcoinCoreClient.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/bitcoind/rpc/BitcoinCoreClient.scala @@ -262,8 +262,8 @@ class BitcoinCoreClient(val rpcClient: BitcoinJsonRPCClient, val onChainKeyManag }) } - def fundTransaction(tx: Transaction, feeRate: FeeratePerKw, replaceable: Boolean, externalInputsWeight: Map[OutPoint, Long] = Map.empty, feeBudget_opt: Option[Satoshi] = None)(implicit ec: ExecutionContext): Future[FundTransactionResponse] = { - fundTransaction(tx, FundTransactionOptions(feeRate, replaceable, inputWeights = externalInputsWeight.map { case (outpoint, weight) => InputWeight(outpoint, weight) }.toSeq), feeBudget_opt = feeBudget_opt) + def fundTransaction(tx: Transaction, feeRate: FeeratePerKw, replaceable: Boolean, externalInputsWeight: Map[OutPoint, Long] = Map.empty, feeBudget_opt: Option[Satoshi] = None, addExcessToRecipientPosition_opt: Option[Int] = None, maxExcess_opt: Option[Satoshi] = None)(implicit ec: ExecutionContext): Future[FundTransactionResponse] = { + fundTransaction(tx, FundTransactionOptions(feeRate, replaceable, inputWeights = externalInputsWeight.map { case (outpoint, weight) => InputWeight(outpoint, weight) }.toSeq, add_excess_to_recipient_position = addExcessToRecipientPosition_opt, max_excess = maxExcess_opt), feeBudget_opt = feeBudget_opt) } private def processPsbt(psbt: Psbt, sign: Boolean = true, sighashType: Option[Int] = None)(implicit ec: ExecutionContext): Future[ProcessPsbtResponse] = { @@ -308,7 +308,7 @@ class BitcoinCoreClient(val rpcClient: BitcoinJsonRPCClient, val onChainKeyManag } } - def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, targetFeerate: FeeratePerKw, feeBudget_opt: Option[Satoshi] = None)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { + def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, targetFeerate: FeeratePerKw, feeBudget_opt: Option[Satoshi] = None, maxExcess_opt: Option[Satoshi] = None)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { def verifyAndSign(tx: Transaction, fees: Satoshi, requestedFeeRate: FeeratePerKw): Future[MakeFundingTxResponse] = { import KotlinUtils._ @@ -344,7 +344,7 @@ class BitcoinCoreClient(val rpcClient: BitcoinJsonRPCClient, val onChainKeyManag // TODO: we should check that mempoolMinFee is not dangerously high feerate <- mempoolMinFee().map(minFee => FeeratePerKw(minFee).max(targetFeerate)) // we ask bitcoin core to add inputs to the funding tx, and use the specified change address - FundTransactionResponse(tx, fee, _) <- fundTransaction(partialFundingTx, FundTransactionOptions(feerate), feeBudget_opt = feeBudget_opt) + FundTransactionResponse(tx, fee, _) <- fundTransaction(partialFundingTx, FundTransactionOptions(feerate, add_excess_to_recipient_position = None, max_excess = maxExcess_opt), feeBudget_opt = feeBudget_opt) lockedUtxos = tx.txIn.map(_.outPoint) signedTx <- unlockIfFails(lockedUtxos)(verifyAndSign(tx, fee, feerate)) } yield signedTx @@ -585,7 +585,7 @@ class BitcoinCoreClient(val rpcClient: BitcoinJsonRPCClient, val onChainKeyManag val theirOutput = TxOut(amount, pubkeyScript) val tx = Transaction(version = 2, txIn = Nil, txOut = theirOutput :: Nil, lockTime = 0) for { - fundedTx <- fundTransaction(tx, feeratePerKw, replaceable = true) + fundedTx <- fundTransaction(tx, feeratePerKw, replaceable = true, addExcessToRecipientPosition_opt = None, maxExcess_opt = None) lockedOutputs = fundedTx.tx.txIn.map(_.outPoint) theirOutputPos = fundedTx.tx.txOut.indexOf(theirOutput) signedPsbt <- unlockIfFails(lockedOutputs)(signPsbt(new Psbt(fundedTx.tx), fundedTx.tx.txIn.indices, fundedTx.tx.txOut.indices.filterNot(_ == theirOutputPos))) @@ -704,10 +704,10 @@ object BitcoinCoreClient { def apply(outPoint: OutPoint, weight: Long): InputWeight = InputWeight(outPoint.txid.value.toHex, outPoint.index, weight) } - case class FundTransactionOptions(feeRate: BigDecimal, replaceable: Boolean, lockUnspents: Boolean, changePosition: Option[Int], input_weights: Option[Seq[InputWeight]]) + case class FundTransactionOptions(feeRate: BigDecimal, replaceable: Boolean, lockUnspents: Boolean, changePosition: Option[Int], input_weights: Option[Seq[InputWeight]], add_excess_to_recipient_position: Option[Int], max_excess: Option[Satoshi]) object FundTransactionOptions { - def apply(feerate: FeeratePerKw, replaceable: Boolean = true, changePosition: Option[Int] = None, inputWeights: Seq[InputWeight] = Nil): FundTransactionOptions = { + def apply(feerate: FeeratePerKw, replaceable: Boolean = true, changePosition: Option[Int] = None, inputWeights: Seq[InputWeight] = Nil, add_excess_to_recipient_position: Option[Int] = None, max_excess: Option[Satoshi] = None): FundTransactionOptions = { FundTransactionOptions( BigDecimal(FeeratePerKB(feerate).toLong).bigDecimal.scaleByPowerOfTen(-8), replaceable, @@ -721,7 +721,9 @@ object BitcoinCoreClient { // potentially be double-spent. lockUnspents = true, changePosition, - if (inputWeights.isEmpty) None else Some(inputWeights) + if (inputWeights.isEmpty) None else Some(inputWeights), + add_excess_to_recipient_position, + max_excess ) } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala index 4eaf218f91..673982afa9 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala @@ -186,7 +186,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers { log.info("remote will use fundingMinDepth={}", accept.minimumDepth) val localFundingPubkey = keyManager.fundingPublicKey(init.localParams.fundingKeyPath, fundingTxIndex = 0) val fundingPubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingPubkey.publicKey, accept.fundingPubkey))) - wallet.makeFundingTx(fundingPubkeyScript, init.fundingAmount, init.fundingTxFeerate, init.fundingTxFeeBudget_opt).pipeTo(self) + wallet.makeFundingTx(fundingPubkeyScript, init.fundingAmount, init.fundingTxFeerate, init.fundingTxFeeBudget_opt, maxExcess_opt = None).pipeTo(self) val params = ChannelParams(init.temporaryChannelId, init.channelConfig, channelFeatures, init.localParams, remoteParams, open.channelFlags) goto(WAIT_FOR_FUNDING_INTERNAL) using DATA_WAIT_FOR_FUNDING_INTERNAL(params, init.fundingAmount, init.pushAmount_opt.getOrElse(0 msat), init.commitTxFeerate, accept.fundingPubkey, accept.firstPerCommitmentPoint, d.initFunder.replyTo) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala index be5d540f1c..26341b618b 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala @@ -212,7 +212,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response case p: SpliceTxRbf => p.feeBudget_opt case _ => None } - context.pipeToSelf(wallet.fundTransaction(txNotFunded, fundingParams.targetFeerate, replaceable = true, externalInputsWeight = sharedInputWeight, feeBudget_opt = feeBudget_opt)) { + context.pipeToSelf(wallet.fundTransaction(txNotFunded, fundingParams.targetFeerate, replaceable = true, externalInputsWeight = sharedInputWeight, feeBudget_opt = feeBudget_opt, addExcessToRecipientPosition_opt = None, maxExcess_opt = None)) { case Failure(t) => WalletFailure(t) case Success(result) => FundTransactionResult(result.tx, result.changePosition) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxFunder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxFunder.scala index 4c3d1777aa..cc88d44e56 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxFunder.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxFunder.scala @@ -434,7 +434,7 @@ private class ReplaceableTxFunder(nodeParams: NodeParams, // start with a dummy output and later merge that dummy output with the optional change output added by bitcoind. val txNotFunded = anchorTx.txInfo.tx.copy(txOut = TxOut(dustLimit, Script.pay2wpkh(PlaceHolderPubKey)) :: Nil) val anchorWeight = Seq(InputWeight(anchorTx.txInfo.input.outPoint, anchorInputWeight)) - bitcoinClient.fundTransaction(txNotFunded, FundTransactionOptions(targetFeerate, inputWeights = anchorWeight), feeBudget_opt = None).flatMap { fundTxResponse => + bitcoinClient.fundTransaction(txNotFunded, FundTransactionOptions(targetFeerate, inputWeights = anchorWeight, add_excess_to_recipient_position = None, max_excess = None), feeBudget_opt = None).flatMap { fundTxResponse => // Bitcoin Core may not preserve the order of inputs, we need to make sure the anchor is the first input. val txIn = anchorTx.txInfo.tx.txIn ++ fundTxResponse.tx.txIn.filterNot(_.outPoint == anchorTx.txInfo.input.outPoint) // We merge our dummy change output with the one added by Bitcoin Core, if any. diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala index a3c6c04863..672747aea4 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala @@ -50,7 +50,7 @@ class DummyOnChainWallet extends OnChainWallet with OnchainPubkeyCache { override def getP2wpkhPubkey()(implicit ec: ExecutionContext): Future[Crypto.PublicKey] = Future.successful(dummyReceivePubkey) - override def fundTransaction(tx: Transaction, feeRate: FeeratePerKw, replaceable: Boolean, externalInputsWeight: Map[OutPoint, Long], feeBudget_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[FundTransactionResponse] = { + override def fundTransaction(tx: Transaction, feeRate: FeeratePerKw, replaceable: Boolean, externalInputsWeight: Map[OutPoint, Long], feeBudget_opt: Option[Satoshi], addExcessToRecipientPosition_opt: Option[Int], maxExcess_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[FundTransactionResponse] = { funded += (tx.txid -> tx) Future.successful(FundTransactionResponse(tx, 0 sat, None)) } @@ -62,7 +62,7 @@ class DummyOnChainWallet extends OnChainWallet with OnchainPubkeyCache { Future.successful(tx.txid) } - override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: FeeratePerKw, feeBudget_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { + override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: FeeratePerKw, feeBudget_opt: Option[Satoshi], maxExcess_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { val tx = DummyOnChainWallet.makeDummyFundingTx(pubkeyScript, amount) funded += (tx.fundingTx.txid -> tx.fundingTx) Future.successful(tx) @@ -105,13 +105,13 @@ class NoOpOnChainWallet extends OnChainWallet with OnchainPubkeyCache { override def getP2wpkhPubkey()(implicit ec: ExecutionContext): Future[Crypto.PublicKey] = Future.successful(dummyReceivePubkey) - override def fundTransaction(tx: Transaction, feeRate: FeeratePerKw, replaceable: Boolean, externalInputsWeight: Map[OutPoint, Long], feeBudget_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[FundTransactionResponse] = Promise().future // will never be completed + override def fundTransaction(tx: Transaction, feeRate: FeeratePerKw, replaceable: Boolean, externalInputsWeight: Map[OutPoint, Long], feeBudget_opt: Option[Satoshi], addExcessToRecipientPosition_opt: Option[Int], maxExcess_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[FundTransactionResponse] = Promise().future // will never be completed override def signPsbt(psbt: Psbt, ourInputs: Seq[Int], ourOutputs: Seq[Int])(implicit ec: ExecutionContext): Future[ProcessPsbtResponse] = Promise().future // will never be completed override def publishTransaction(tx: Transaction)(implicit ec: ExecutionContext): Future[TxId] = Future.successful(tx.txid) - override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: FeeratePerKw, feeBudget_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = Promise().future // will never be completed + override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: FeeratePerKw, feeBudget_opt: Option[Satoshi], maxExcess_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = Promise().future // will never be completed override def commit(tx: Transaction)(implicit ec: ExecutionContext): Future[Boolean] = Future.successful(true) @@ -152,7 +152,7 @@ class SingleKeyOnChainWallet extends OnChainWallet with OnchainPubkeyCache { override def getP2wpkhPubkey()(implicit ec: ExecutionContext): Future[Crypto.PublicKey] = Future.successful(pubkey) - override def fundTransaction(tx: Transaction, feeRate: FeeratePerKw, replaceable: Boolean, externalInputsWeight: Map[OutPoint, Long], feeBudget_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[FundTransactionResponse] = synchronized { + override def fundTransaction(tx: Transaction, feeRate: FeeratePerKw, replaceable: Boolean, externalInputsWeight: Map[OutPoint, Long], feeBudget_opt: Option[Satoshi], addExcessToRecipientPosition_opt: Option[Int], maxExcess_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[FundTransactionResponse] = synchronized { val currentAmountIn = tx.txIn.flatMap(txIn => inputs.find(_.txid == txIn.outPoint.txid).flatMap(_.txOut.lift(txIn.outPoint.index.toInt))).map(_.amount).sum val amountOut = tx.txOut.map(_.amount).sum // We add a single input to reach the desired feerate. @@ -213,10 +213,10 @@ class SingleKeyOnChainWallet extends OnChainWallet with OnchainPubkeyCache { Future.successful(ProcessPsbtResponse(signedPsbt, complete)) } - override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: FeeratePerKw, feeBudget_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { + override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: FeeratePerKw, feeBudget_opt: Option[Satoshi], maxExcess_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { val tx = Transaction(2, Nil, Seq(TxOut(amount, pubkeyScript)), 0) for { - fundedTx <- fundTransaction(tx, feeRatePerKw, replaceable = true, feeBudget_opt = feeBudget_opt) + fundedTx <- fundTransaction(tx, feeRatePerKw, replaceable = true, feeBudget_opt = feeBudget_opt, addExcessToRecipientPosition_opt = Some(0), maxExcess_opt = maxExcess_opt) signedTx <- signTransaction(fundedTx.tx) } yield MakeFundingTxResponse(signedTx.tx, 0, fundedTx.fee) } From 8cc2e4afda3933d0123c04d388222c100c7e4630 Mon Sep 17 00:00:00 2001 From: Richard Myers Date: Tue, 27 Aug 2024 14:50:03 +0200 Subject: [PATCH 7/8] Add maxExcess_opt to ChannelParams in NodeParams This will be used by non-initiators when funding dual funded channels and splicing --- eclair-core/src/main/resources/reference.conf | 4 ++++ eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala | 5 ++++- .../main/scala/fr/acinq/eclair/channel/fsm/Channel.scala | 3 ++- .../src/test/scala/fr/acinq/eclair/TestConstants.scala | 6 ++++-- 4 files changed, 14 insertions(+), 4 deletions(-) diff --git a/eclair-core/src/main/resources/reference.conf b/eclair-core/src/main/resources/reference.conf index ca283b18b6..be0e6b722b 100644 --- a/eclair-core/src/main/resources/reference.conf +++ b/eclair-core/src/main/resources/reference.conf @@ -197,6 +197,10 @@ eclair { min-time-between-updates = 1 hour // minimum time between channel updates because the balance changed } + + // if > 0, excess channel funding up to this amount will be added to the amount contributed when funding + // dual-funded channels or splices. This is useful to avoid paying fees to add a small change output. + max-excess = 0 } balance-check-interval = 1 hour diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala b/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala index 1c66625b53..68ef887768 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala @@ -476,6 +476,8 @@ object NodeParams extends Logging { val maxNoChannels = config.getInt("peer-connection.max-no-channels") require(maxNoChannels > 0, "peer-connection.max-no-channels must be > 0") + val maxExcess_opt = if (config.getInt("channel.max-excess") > 0) Some(Satoshi(config.getInt("channel.max-excess"))) else None + NodeParams( nodeKeyManager = nodeKeyManager, channelKeyManager = channelKeyManager, @@ -524,7 +526,8 @@ object NodeParams extends Logging { quiescenceTimeout = FiniteDuration(config.getDuration("channel.quiescence-timeout").getSeconds, TimeUnit.SECONDS), balanceThresholds = config.getConfigList("channel.channel-update.balance-thresholds").asScala.map(conf => BalanceThreshold(Satoshi(conf.getLong("available-sat")), Satoshi(conf.getLong("max-htlc-sat")))).toSeq, minTimeBetweenUpdates = FiniteDuration(config.getDuration("channel.channel-update.min-time-between-updates").getSeconds, TimeUnit.SECONDS), - acceptIncomingStaticRemoteKeyChannels = config.getBoolean("channel.accept-incoming-static-remote-key-channels") + acceptIncomingStaticRemoteKeyChannels = config.getBoolean("channel.accept-incoming-static-remote-key-channels"), + maxExcess_opt = maxExcess_opt, ), onChainFeeConf = OnChainFeeConf( feeTargets = feeTargets, diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index 757c804031..356415856b 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -104,7 +104,8 @@ object Channel { quiescenceTimeout: FiniteDuration, balanceThresholds: Seq[BalanceThreshold], minTimeBetweenUpdates: FiniteDuration, - acceptIncomingStaticRemoteKeyChannels: Boolean) { + acceptIncomingStaticRemoteKeyChannels: Boolean, + maxExcess_opt: Option[Satoshi]) { require(0 <= maxHtlcValueInFlightPercent && maxHtlcValueInFlightPercent <= 100, "max-htlc-value-in-flight-percent must be between 0 and 100") require(balanceThresholds.sortBy(_.available) == balanceThresholds, "channel-update.balance-thresholds must be sorted by available-sat") diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala index 6f520c1fb6..dde428f09c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala @@ -144,7 +144,8 @@ object TestConstants { quiescenceTimeout = 2 minutes, balanceThresholds = Nil, minTimeBetweenUpdates = 0 hours, - acceptIncomingStaticRemoteKeyChannels = false + acceptIncomingStaticRemoteKeyChannels = false, + maxExcess_opt = None ), onChainFeeConf = OnChainFeeConf( feeTargets = FeeTargets(funding = ConfirmationPriority.Medium, closing = ConfirmationPriority.Medium), @@ -316,7 +317,8 @@ object TestConstants { quiescenceTimeout = 2 minutes, balanceThresholds = Nil, minTimeBetweenUpdates = 0 hour, - acceptIncomingStaticRemoteKeyChannels = false + acceptIncomingStaticRemoteKeyChannels = false, + maxExcess_opt = None ), onChainFeeConf = OnChainFeeConf( feeTargets = FeeTargets(funding = ConfirmationPriority.Medium, closing = ConfirmationPriority.Medium), From 38ae09a1c8a4c03e4d11e0d00f44509022a826eb Mon Sep 17 00:00:00 2001 From: Richard Myers Date: Tue, 27 Aug 2024 14:51:05 +0200 Subject: [PATCH 8/8] Update to select funding inputs before sending open_channel2 and splice_init - If `addExcessToRecipientPosition_opt` is set, excess from funding (if any) will be added to the proposed `fundingAmount`/`fundingContribution` before sending `open_channel2`/`splice_init` respectively. - We assume our peer requires confirmed inputs. In the future we could add a heuristic for this, but it's safer to assume they want confirmed inputs. --- .../fr/acinq/eclair/channel/ChannelData.scala | 11 +- .../fr/acinq/eclair/channel/fsm/Channel.scala | 51 ++++++- .../channel/fsm/ChannelOpenDualFunded.scala | 125 +++++++++++++----- .../channel/fsm/DualFundingHandlers.scala | 9 +- .../channel/fund/InteractiveTxBuilder.scala | 81 +++++++----- .../channel/fund/InteractiveTxFunder.scala | 37 +++--- .../acinq/eclair/io/PeerReadyNotifier.scala | 1 + .../blockchain/DummyOnChainWallet.scala | 10 ++ .../channel/InteractiveTxBuilderSpec.scala | 8 ++ .../ChannelStateTestsHelperMethods.scala | 4 +- ...ngInternalDualFundedChannelStateSpec.scala | 107 +++++++++++++++ .../WaitForDualFundingCreatedStateSpec.scala | 6 +- .../b/WaitForDualFundingSignedStateSpec.scala | 6 +- ...WaitForDualFundingConfirmedStateSpec.scala | 6 +- .../states/e/NormalSplicesStateSpec.scala | 13 ++ .../basic/fixtures/MinimalNodeFixture.scala | 6 +- 16 files changed, 377 insertions(+), 104 deletions(-) create mode 100644 eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForFundingInternalDualFundedChannelStateSpec.scala diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala index ff3809ff0d..097837c707 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala @@ -22,7 +22,7 @@ import fr.acinq.bitcoin.scalacompat.{ByteVector32, DeterministicWallet, OutPoint import fr.acinq.eclair.blockchain.fee.{ConfirmationTarget, FeeratePerKw} import fr.acinq.eclair.channel.LocalFundingStatus.DualFundedUnconfirmedFundingTx import fr.acinq.eclair.channel.fund.InteractiveTxBuilder._ -import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession} +import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxFunder, InteractiveTxSigningSession} import fr.acinq.eclair.io.Peer import fr.acinq.eclair.transactions.CommitmentSpec import fr.acinq.eclair.transactions.Transactions._ @@ -62,6 +62,7 @@ case object WAIT_FOR_FUNDING_CONFIRMED extends ChannelState case object WAIT_FOR_CHANNEL_READY extends ChannelState // Dual-funded channel opening: case object WAIT_FOR_INIT_DUAL_FUNDED_CHANNEL extends ChannelState +case object WAIT_FOR_DUAL_FUNDING_INTERNAL extends ChannelState case object WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL extends ChannelState case object WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL extends ChannelState case object WAIT_FOR_DUAL_FUNDING_CREATED extends ChannelState @@ -506,7 +507,7 @@ object 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 SpliceStatus + case class SpliceRequested(cmd: CMD_SPLICE, init: SpliceInit, fundingContributions_opt: Option[InteractiveTxFunder.FundingContributions]) 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. */ @@ -583,10 +584,14 @@ final case class DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments: Commitments, } final case class DATA_WAIT_FOR_CHANNEL_READY(commitments: Commitments, shortIds: ShortIds) extends ChannelDataWithCommitments +final case class DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL(input: INPUT_INIT_CHANNEL_INITIATOR) extends TransientChannelData { + val channelId: ByteVector32 = input.temporaryChannelId +} + final case class DATA_WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL(init: INPUT_INIT_CHANNEL_NON_INITIATOR) extends TransientChannelData { val channelId: ByteVector32 = init.temporaryChannelId } -final case class DATA_WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL(init: INPUT_INIT_CHANNEL_INITIATOR, lastSent: OpenDualFundedChannel) extends TransientChannelData { +final case class DATA_WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL(init: INPUT_INIT_CHANNEL_INITIATOR, lastSent: OpenDualFundedChannel, fundingContributions: InteractiveTxFunder.FundingContributions) extends TransientChannelData { val channelId: ByteVector32 = lastSent.temporaryChannelId } final case class DATA_WAIT_FOR_DUAL_FUNDING_CREATED(channelId: ByteVector32, diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index 356415856b..6378b4cb8e 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -21,7 +21,7 @@ import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapte import akka.actor.{Actor, ActorContext, ActorRef, FSM, OneForOneStrategy, PossiblyHarmful, Props, SupervisorStrategy, typed} import akka.event.Logging.MDC import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} -import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, SatoshiLong, Transaction, TxId} +import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, SatoshiLong, Script, Transaction, TxId} import fr.acinq.eclair.Logs.LogCategory import fr.acinq.eclair._ import fr.acinq.eclair.blockchain.OnChainWallet.MakeFundingTxResponse @@ -944,7 +944,28 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with context.system.scheduler.scheduleOnce(2 second, peer, Peer.Disconnect(remoteNodeId)) stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending Warning(d.channelId, f.getMessage) case Right(spliceInit) => - stay() using d.copy(spliceStatus = SpliceStatus.SpliceRequested(cmd, spliceInit)) sending spliceInit + // use our fundingPubKey as a placeholder for the remote funder's pubkey + val fundingPubKey = spliceInit.fundingPubKey + val parentCommitment = d.commitments.latest.commitment + // assume our peer requires confirmed inputs when we initiate a splice + val requireConfirmedInputs = RequireConfirmedInputs(forLocal = true, forRemote = nodeParams.channelConf.requireConfirmedInputsForDualFunding) + val fundingParams = InteractiveTxParams( + channelId = spliceInit.channelId, + isInitiator = true, + localContribution = spliceInit.fundingContribution, + remoteContribution = 0 sat, + sharedInput_opt = Some(Multisig2of2Input(parentCommitment)), + remoteFundingPubKey = fundingPubKey, + localOutputs = cmd.spliceOutputs, + lockTime = nodeParams.currentBlockHeight.toLong, + dustLimit = d.commitments.params.localParams.dustLimit, + targetFeerate = spliceInit.feerate, + requireConfirmedInputs = requireConfirmedInputs + ) + val dummyFundingPubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(fundingPubKey, fundingPubKey))) + val txFunder = context.spawnAnonymous(InteractiveTxFunder(remoteNodeId, fundingParams, dummyFundingPubkeyScript, purpose = InteractiveTxBuilder.SpliceTx(parentCommitment), wallet, nodeParams.channelConf.maxExcess_opt)) + txFunder ! InteractiveTxFunder.FundTransaction(self) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceRequested(cmd, spliceInit, None)) } case cmd: CMD_BUMP_FUNDING_FEE => initiateSpliceRbf(cmd, d) match { case Left(f) => @@ -1018,6 +1039,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with channelParams = d.commitments.params, purpose = InteractiveTxBuilder.SpliceTx(parentCommitment), localPushAmount = spliceAck.pushAmount, remotePushAmount = msg.pushAmount, + None, wallet )) txBuilder ! InteractiveTxBuilder.Start(self) @@ -1036,7 +1058,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with case Event(msg: SpliceAck, d: DATA_NORMAL) => d.spliceStatus match { - case SpliceStatus.SpliceRequested(cmd, spliceInit) => + case SpliceStatus.SpliceRequested(cmd, spliceInit, fundingContributions_opt) => log.info("our peer accepted our splice request and will contribute {} to the funding transaction", msg.fundingContribution) val parentCommitment = d.commitments.latest.commitment val fundingParams = InteractiveTxParams( @@ -1059,6 +1081,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with channelParams = d.commitments.params, purpose = InteractiveTxBuilder.SpliceTx(parentCommitment), localPushAmount = cmd.pushAmount, remotePushAmount = msg.pushAmount, + fundingContributions_opt = fundingContributions_opt, wallet )) txBuilder ! InteractiveTxBuilder.Start(self) @@ -1068,6 +1091,22 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with stay() } + case Event(msg: InteractiveTxFunder.Response, d: DATA_NORMAL) => + d.spliceStatus match { + case SpliceStatus.SpliceRequested(cmd, spliceInit, _) => + msg match { + case InteractiveTxFunder.FundingFailed => + cmd.replyTo ! RES_FAILURE(cmd, ChannelFundingError(d.channelId)) + stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) calling endQuiescence(d) + case fundingContributions: InteractiveTxFunder.FundingContributions => + val spliceInit1 = spliceInit.copy(fundingContribution = spliceInit.fundingContribution + fundingContributions.addExcess_opt.getOrElse(0 sat)) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceRequested(cmd, spliceInit1, Some(fundingContributions))) sending spliceInit1 + } + case _ => + log.warning("received unexpected response from txFunder: {}, current splice status is {}", msg, d.spliceStatus) + stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) calling endQuiescence(d) + } + case Event(msg: InteractiveTxConstructionMessage, d: DATA_NORMAL) => d.spliceStatus match { case SpliceStatus.SpliceInProgress(_, _, txBuilder, _) => @@ -1116,6 +1155,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with channelParams = d.commitments.params, purpose = rbf, localPushAmount = 0 msat, remotePushAmount = 0 msat, + None, wallet )) txBuilder ! InteractiveTxBuilder.Start(self) @@ -1160,6 +1200,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with channelParams = d.commitments.params, purpose = rbf, localPushAmount = 0 msat, remotePushAmount = 0 msat, + None, wallet )) txBuilder ! InteractiveTxBuilder.Start(self) @@ -1183,7 +1224,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with log.info("our peer aborted the splice attempt: ascii='{}' bin={}", msg.toAscii, msg.data) rollbackFundingAttempt(signingSession.fundingTx.tx, previousTxs = Seq.empty) // no splice rbf yet stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage) calling endQuiescence(d) - case SpliceStatus.SpliceRequested(cmd, _) => + case SpliceStatus.SpliceRequested(cmd, _, _) => 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) @@ -2996,7 +3037,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with private def reportSpliceFailure(spliceStatus: SpliceStatus, f: Throwable): Unit = { val cmd_opt = spliceStatus match { case SpliceStatus.NegotiatingQuiescence(cmd_opt, _) => cmd_opt - case SpliceStatus.SpliceRequested(cmd, _) => Some(cmd) + case SpliceStatus.SpliceRequested(cmd, _, _) => Some(cmd) case SpliceStatus.RbfRequested(cmd, _) => Some(cmd) case SpliceStatus.SpliceInProgress(cmd_opt, _, txBuilder, _) => txBuilder ! InteractiveTxBuilder.Abort diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala index 35f7cf6297..5d0b4cbbe7 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala @@ -16,17 +16,19 @@ package fr.acinq.eclair.channel.fsm +import akka.actor.Status import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapter} import com.softwaremill.quicklens.{ModifyPimp, QuicklensAt} -import fr.acinq.bitcoin.scalacompat.SatoshiLong +import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Script} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ -import fr.acinq.eclair.channel._ +import fr.acinq.eclair.channel.{WAIT_FOR_DUAL_FUNDING_INTERNAL, WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL, _} import fr.acinq.eclair.channel.fsm.Channel._ -import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTransaction, InteractiveTxParams, PartiallySignedSharedTransaction, RequireConfirmedInputs} -import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession} +import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTransaction, InteractiveTxParams, LocalFailure, Output, PartiallySignedSharedTransaction, RequireConfirmedInputs} +import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxFunder, InteractiveTxSigningSession} import fr.acinq.eclair.channel.publish.TxPublisher.SetChannelId import fr.acinq.eclair.crypto.ShaChain import fr.acinq.eclair.io.Peer.OpenChannelResponse +import fr.acinq.eclair.transactions.Scripts import fr.acinq.eclair.wire.protocol._ import fr.acinq.eclair.{MilliSatoshiLong, RealShortChannelId, ToMilliSatoshiConversion, UInt64, randomBytes32} @@ -104,37 +106,89 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { when(WAIT_FOR_INIT_DUAL_FUNDED_CHANNEL)(handleExceptions { case Event(input: INPUT_INIT_CHANNEL_INITIATOR, _) => + // use our fundingPubKey as a placeholder for the remote funder's pubkey val fundingPubKey = keyManager.fundingPublicKey(input.localParams.fundingKeyPath, fundingTxIndex = 0).publicKey - val channelKeyPath = keyManager.keyPath(input.localParams, input.channelConfig) - val upfrontShutdownScript_opt = input.localParams.upfrontShutdownScript_opt.map(scriptPubKey => ChannelTlv.UpfrontShutdownScriptTlv(scriptPubKey)) - val tlvs: Set[OpenDualFundedChannelTlv] = Set( - upfrontShutdownScript_opt, - Some(ChannelTlv.ChannelTypeTlv(input.channelType)), - input.pushAmount_opt.map(amount => ChannelTlv.PushAmountTlv(amount)), - if (input.requireConfirmedInputs) Some(ChannelTlv.RequireConfirmedInputsTlv()) else None, - ).flatten - val open = OpenDualFundedChannel( - chainHash = nodeParams.chainHash, - temporaryChannelId = input.temporaryChannelId, - fundingFeerate = input.fundingTxFeerate, - commitmentFeerate = input.commitTxFeerate, - fundingAmount = input.fundingAmount, - dustLimit = input.localParams.dustLimit, - maxHtlcValueInFlightMsat = UInt64(input.localParams.maxHtlcValueInFlightMsat.toLong), - htlcMinimum = input.localParams.htlcMinimum, - toSelfDelay = input.localParams.toSelfDelay, - maxAcceptedHtlcs = input.localParams.maxAcceptedHtlcs, + // assume our peer requires confirmed inputs when we initiate a dual funded channel open + val requireConfirmedInputs = RequireConfirmedInputs(forLocal = true, forRemote = nodeParams.channelConf.requireConfirmedInputsForDualFunding) + val fundingParams = InteractiveTxParams( + channelId = input.temporaryChannelId, + isInitiator = true, + localContribution = input.fundingAmount, + remoteContribution = 0 sat, + sharedInput_opt = None, + remoteFundingPubKey = fundingPubKey, + localOutputs = Nil, lockTime = nodeParams.currentBlockHeight.toLong, - fundingPubkey = fundingPubKey, - revocationBasepoint = keyManager.revocationPoint(channelKeyPath).publicKey, - paymentBasepoint = input.localParams.walletStaticPaymentBasepoint.getOrElse(keyManager.paymentPoint(channelKeyPath).publicKey), - delayedPaymentBasepoint = keyManager.delayedPaymentPoint(channelKeyPath).publicKey, - htlcBasepoint = keyManager.htlcPoint(channelKeyPath).publicKey, - firstPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 0), - secondPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1), - channelFlags = input.channelFlags, - tlvStream = TlvStream(tlvs)) - goto(WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL) using DATA_WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL(input, open) sending open + dustLimit = input.localParams.dustLimit, + targetFeerate = input.fundingTxFeerate, + requireConfirmedInputs = requireConfirmedInputs + ) + val dummyPurpose = InteractiveTxBuilder.DummyFundingTx(feeBudget_opt = input.fundingTxFeeBudget_opt) + val dummyFundingPubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(fundingPubKey, fundingPubKey))) + val txFunder = context.spawnAnonymous(InteractiveTxFunder(remoteNodeId, fundingParams, dummyFundingPubkeyScript, dummyPurpose, wallet)) + txFunder ! InteractiveTxFunder.FundTransaction(self) + goto(WAIT_FOR_DUAL_FUNDING_INTERNAL) using DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL(input) + }) + + when(WAIT_FOR_DUAL_FUNDING_INTERNAL)(handleExceptions { + case Event(msg: InteractiveTxFunder.Response, d: DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL) => msg match { + case InteractiveTxFunder.FundingFailed => + d.input.replyTo ! OpenChannelResponse.Rejected(LocalFailure(ChannelFundingError(d.channelId)).cause.getMessage) + goto(CLOSED) + case fundingContributions: InteractiveTxFunder.FundingContributions => + val fundingPubKey = keyManager.fundingPublicKey(d.input.localParams.fundingKeyPath, fundingTxIndex = 0).publicKey + val channelKeyPath = keyManager.keyPath(d.input.localParams, d.input.channelConfig) + val upfrontShutdownScript_opt = d.input.localParams.upfrontShutdownScript_opt.map(scriptPubKey => ChannelTlv.UpfrontShutdownScriptTlv(scriptPubKey)) + val tlvs: Set[OpenDualFundedChannelTlv] = Set( + upfrontShutdownScript_opt, + Some(ChannelTlv.ChannelTypeTlv(d.input.channelType)), + d.input.pushAmount_opt.map(amount => ChannelTlv.PushAmountTlv(amount)), + if (d.input.requireConfirmedInputs) Some(ChannelTlv.RequireConfirmedInputsTlv()) else None, + ).flatten + val fundingAmount1 = d.input.fundingAmount + fundingContributions.addExcess_opt.getOrElse(0 sat) + val open = OpenDualFundedChannel( + chainHash = nodeParams.chainHash, + temporaryChannelId = d.input.temporaryChannelId, + fundingFeerate = d.input.fundingTxFeerate, + commitmentFeerate = d.input.commitTxFeerate, + fundingAmount = fundingAmount1, + dustLimit = d.input.localParams.dustLimit, + maxHtlcValueInFlightMsat = UInt64(d.input.localParams.maxHtlcValueInFlightMsat.toLong), + htlcMinimum = d.input.localParams.htlcMinimum, + toSelfDelay = d.input.localParams.toSelfDelay, + maxAcceptedHtlcs = d.input.localParams.maxAcceptedHtlcs, + lockTime = nodeParams.currentBlockHeight.toLong, + fundingPubkey = fundingPubKey, + revocationBasepoint = keyManager.revocationPoint(channelKeyPath).publicKey, + paymentBasepoint = d.input.localParams.walletStaticPaymentBasepoint.getOrElse(keyManager.paymentPoint(channelKeyPath).publicKey), + delayedPaymentBasepoint = keyManager.delayedPaymentPoint(channelKeyPath).publicKey, + htlcBasepoint = keyManager.htlcPoint(channelKeyPath).publicKey, + firstPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 0), + secondPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1), + channelFlags = d.input.channelFlags, + tlvStream = TlvStream(tlvs)) + goto(WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL) using DATA_WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL(d.input.copy(fundingAmount = fundingAmount1), open, fundingContributions) sending open + } + case Event(Status.Failure(t), d: DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL) => + log.error(t, s"wallet returned error: ") + d.input.replyTo ! OpenChannelResponse.Rejected(s"wallet error: ${t.getMessage}") + goto(CLOSED) + + case Event(c: CloseCommand, d: DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL) => + d.input.replyTo ! OpenChannelResponse.Cancelled + handleFastClose(c, d.channelId) + + case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL) => + d.input.replyTo ! OpenChannelResponse.RemoteError(e.toAscii) + handleRemoteError(e, d) + + case Event(INPUT_DISCONNECTED, d: DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL) => + d.input.replyTo ! OpenChannelResponse.Disconnected + goto(CLOSED) + + case Event(TickChannelOpenTimeout, d: DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL) => + d.input.replyTo ! OpenChannelResponse.TimedOut + goto(CLOSED) }) when(WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL)(handleExceptions { @@ -218,6 +272,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { nodeParams, fundingParams, channelParams, purpose, localPushAmount = accept.pushAmount, remotePushAmount = open.pushAmount, + fundingContributions_opt = None, wallet)) txBuilder ! InteractiveTxBuilder.Start(self) goto(WAIT_FOR_DUAL_FUNDING_CREATED) using DATA_WAIT_FOR_DUAL_FUNDING_CREATED(channelId, channelParams, open.secondPerCommitmentPoint, accept.pushAmount, open.pushAmount, txBuilder, deferred = None, replyTo_opt = None) sending accept @@ -235,6 +290,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { import d.init.{localParams, remoteInit} Helpers.validateParamsDualFundedInitiator(nodeParams, d.init.channelType, localParams.initFeatures, remoteInit.features, d.lastSent, accept) match { case Left(t) => + rollbackOpenAttempt(d.fundingContributions) d.init.replyTo ! OpenChannelResponse.Rejected(t.getMessage) handleLocalError(t, d, Some(accept)) case Right((channelFeatures, remoteShutdownScript)) => @@ -281,6 +337,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { nodeParams, fundingParams, channelParams, purpose, localPushAmount = d.lastSent.pushAmount, remotePushAmount = accept.pushAmount, + fundingContributions_opt = Some(d.fundingContributions), wallet)) txBuilder ! InteractiveTxBuilder.Start(self) goto(WAIT_FOR_DUAL_FUNDING_CREATED) using DATA_WAIT_FOR_DUAL_FUNDING_CREATED(channelId, channelParams, accept.secondPerCommitmentPoint, d.lastSent.pushAmount, accept.pushAmount, txBuilder, deferred = None, replyTo_opt = Some(d.init.replyTo)) @@ -551,6 +608,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { channelParams = d.commitments.params, purpose = InteractiveTxBuilder.FundingTxRbf(d.commitments.active.head, previousTransactions = d.allFundingTxs.map(_.sharedTx), feeBudget_opt = None), localPushAmount = d.localPushAmount, remotePushAmount = d.remotePushAmount, + fundingContributions_opt = None, wallet)) txBuilder ! InteractiveTxBuilder.Start(self) val toSend = Seq( @@ -589,6 +647,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { channelParams = d.commitments.params, purpose = InteractiveTxBuilder.FundingTxRbf(d.commitments.active.head, previousTransactions = d.allFundingTxs.map(_.sharedTx), feeBudget_opt = Some(cmd.fundingFeeBudget)), localPushAmount = d.localPushAmount, remotePushAmount = d.remotePushAmount, + fundingContributions_opt = None, wallet)) txBuilder ! InteractiveTxBuilder.Start(self) stay() using d.copy(status = DualFundingStatus.RbfInProgress(cmd_opt = Some(cmd), txBuilder, remoteCommitSig = None)) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala index eb26fa4df9..c56ddcba01 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala @@ -25,7 +25,7 @@ import fr.acinq.eclair.channel.LocalFundingStatus.DualFundedUnconfirmedFundingTx import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel.BITCOIN_FUNDING_DOUBLE_SPENT import fr.acinq.eclair.channel.fund.InteractiveTxBuilder._ -import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession} +import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxFunder, InteractiveTxSigningSession} import fr.acinq.eclair.wire.protocol.{ChannelReady, Error} import scala.concurrent.Future @@ -116,6 +116,13 @@ trait DualFundingHandlers extends CommonFundingHandlers { * bitcoind when transactions are published. But if we couldn't publish those transactions (e.g. because our peer * never sent us their signatures, or the transaction wasn't accepted in our mempool), their inputs may still be locked. */ + def rollbackOpenAttempt(fundingContributions: InteractiveTxFunder.FundingContributions): Unit = { + val inputs = fundingContributions.inputs.map(i => TxIn(i.outPoint, Nil, 0)) + if (inputs.nonEmpty) { + wallet.rollback(Transaction(2, inputs, Nil, 0)) + } + } + def rollbackDualFundingTxs(txs: Seq[SignedSharedTransaction]): Unit = { val inputs = txs.flatMap(sharedTx => sharedTx.tx.localInputs ++ sharedTx.tx.sharedInput_opt.toSeq).distinctBy(_.serialId).map(i => TxIn(i.outPoint, Nil, 0)) if (inputs.nonEmpty) { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala index 7b1de61349..c9cce516a2 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala @@ -157,17 +157,26 @@ object InteractiveTxBuilder { } // @formatter:off - sealed trait Purpose { + sealed trait FundingInfo { def previousLocalBalance: MilliSatoshi def previousRemoteBalance: MilliSatoshi def previousFundingAmount: Satoshi + def localHtlcs: Set[DirectedHtlc] + def htlcBalance: MilliSatoshi = localHtlcs.toSeq.map(_.add.amountMsat).sum + } + sealed trait CommitmentInfo { def localCommitIndex: Long def remoteCommitIndex: Long def remotePerCommitmentPoint: PublicKey def commitTxFeerate: FeeratePerKw def fundingTxIndex: Long - def localHtlcs: Set[DirectedHtlc] - def htlcBalance: MilliSatoshi = localHtlcs.toSeq.map(_.add.amountMsat).sum + } + sealed trait Purpose extends FundingInfo with CommitmentInfo + case class DummyFundingTx(feeBudget_opt: Option[Satoshi]) extends FundingInfo { + override val previousLocalBalance: MilliSatoshi = 0 msat + override val previousRemoteBalance: MilliSatoshi = 0 msat + override val previousFundingAmount: Satoshi = 0 sat + override val localHtlcs: Set[DirectedHtlc] = Set.empty } case class FundingTx(commitTxFeerate: FeeratePerKw, remotePerCommitmentPoint: PublicKey, feeBudget_opt: Option[Satoshi]) extends Purpose { override val previousLocalBalance: MilliSatoshi = 0 msat @@ -364,6 +373,7 @@ object InteractiveTxBuilder { purpose: Purpose, localPushAmount: MilliSatoshi, remotePushAmount: MilliSatoshi, + fundingContributions_opt: Option[InteractiveTxFunder.FundingContributions], wallet: OnChainChannelFunder)(implicit ec: ExecutionContext): Behavior[Command] = { Behaviors.setup { context => // The stash is used to buffer messages that arrive while we're funding the transaction. @@ -384,7 +394,7 @@ object InteractiveTxBuilder { Behaviors.stopped } else { val actor = new InteractiveTxBuilder(replyTo, sessionId, nodeParams, channelParams, fundingParams, purpose, localPushAmount, remotePushAmount, wallet, stash, context) - actor.start() + actor.start(fundingContributions_opt) } case Abort => Behaviors.stopped } @@ -423,34 +433,42 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon case _ => Nil } - def start(): Behavior[Command] = { - val txFunder = context.spawnAnonymous(InteractiveTxFunder(remoteNodeId, fundingParams, fundingPubkeyScript, purpose, wallet)) - txFunder ! InteractiveTxFunder.FundTransaction(context.messageAdapter[InteractiveTxFunder.Response](r => FundTransactionResult(r))) - Behaviors.receiveMessagePartial { - case FundTransactionResult(result) => result match { - case InteractiveTxFunder.FundingFailed => - if (previousTransactions.nonEmpty && !fundingParams.isInitiator) { - // We don't have enough funds to reach the desired feerate, but this is an RBF attempt that we did not initiate. - // It still makes sense for us to contribute whatever we're able to (by using our previous set of inputs and - // outputs): the final feerate will be less than what the initiator intended, but it's still better than being - // stuck with a low feerate transaction that won't confirm. - log.warn("could not fund interactive tx at {}, re-using previous inputs and outputs", fundingParams.targetFeerate) - val previousTx = previousTransactions.head.tx - stash.unstashAll(buildTx(InteractiveTxFunder.FundingContributions(previousTx.localInputs, previousTx.localOutputs))) - } else { - // We use a generic exception and don't send the internal error to the peer. - replyTo ! LocalFailure(ChannelFundingError(fundingParams.channelId)) - Behaviors.stopped - } - case fundingContributions: InteractiveTxFunder.FundingContributions => - stash.unstashAll(buildTx(fundingContributions)) + def start(fundingContributions_opt: Option[InteractiveTxFunder.FundingContributions]): Behavior[Command] = { + if (fundingContributions_opt.isDefined) { + val fundingContributions1 = fundingContributions_opt.get.copy(outputs = fundingContributions_opt.get.outputs.map { + case o: InteractiveTxBuilder.Output.Shared => Output.Shared(o.serialId, fundingPubkeyScript, purpose.previousLocalBalance + fundingParams.localContribution, purpose.previousRemoteBalance + fundingParams.remoteContribution, purpose.htlcBalance) + case o => o + }) + stash.unstashAll(buildTx(fundingContributions1)) + } else { + val txFunder = context.spawnAnonymous(InteractiveTxFunder(remoteNodeId, fundingParams, fundingPubkeyScript, purpose, wallet)) + txFunder ! InteractiveTxFunder.FundTransaction(context.messageAdapter[InteractiveTxFunder.Response](r => FundTransactionResult(r))) + Behaviors.receiveMessagePartial { + case FundTransactionResult(result) => result match { + case InteractiveTxFunder.FundingFailed => + if (previousTransactions.nonEmpty && !fundingParams.isInitiator) { + // We don't have enough funds to reach the desired feerate, but this is an RBF attempt that we did not initiate. + // It still makes sense for us to contribute whatever we're able to (by using our previous set of inputs and + // outputs): the final feerate will be less than what the initiator intended, but it's still better than being + // stuck with a low feerate transaction that won't confirm. + log.warn("could not fund interactive tx at {}, re-using previous inputs and outputs", fundingParams.targetFeerate) + val previousTx = previousTransactions.head.tx + stash.unstashAll(buildTx(InteractiveTxFunder.FundingContributions(previousTx.localInputs, previousTx.localOutputs, addExcess_opt = None))) + } else { + // We use a generic exception and don't send the internal error to the peer. + replyTo ! LocalFailure(ChannelFundingError(fundingParams.channelId)) + Behaviors.stopped + } + case fundingContributions: InteractiveTxFunder.FundingContributions => + stash.unstashAll(buildTx(fundingContributions)) + } + case msg: ReceiveMessage => + stash.stash(msg) + Behaviors.same + case Abort => + stash.stash(Abort) + Behaviors.same } - case msg: ReceiveMessage => - stash.stash(msg) - Behaviors.same - case Abort => - stash.stash(Abort) - Behaviors.same } } @@ -769,6 +787,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon private def signCommitTx(completeTx: SharedTransaction): Behavior[Command] = { val fundingTx = completeTx.buildUnsignedTx() val fundingOutputIndex = fundingTx.txOut.indexWhere(_.publicKeyScript == fundingPubkeyScript) + require(fundingOutputIndex >= 0, "shared output not found in funding tx!") Funding.makeCommitTxs(keyManager, channelParams, fundingAmount = fundingParams.fundingAmount, toLocal = completeTx.sharedOutput.localAmount - localPushAmount + remotePushAmount, diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala index 26341b618b..059dfee933 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala @@ -45,22 +45,22 @@ object InteractiveTxFunder { // @formatter:off sealed trait Command case class FundTransaction(replyTo: ActorRef[Response]) extends Command - private case class FundTransactionResult(tx: Transaction, changePosition: Option[Int]) extends Command + private case class FundTransactionResult(tx: Transaction, changePosition: Option[Int], addExcess_opt: Option[Satoshi] ) extends Command private case class InputDetails(usableInputs: Seq[OutgoingInput], unusableInputs: Set[UnusableInput]) extends Command private case class WalletFailure(t: Throwable) extends Command private case object UtxosUnlocked extends Command sealed trait Response - case class FundingContributions(inputs: Seq[OutgoingInput], outputs: Seq[OutgoingOutput]) extends Response + case class FundingContributions(inputs: Seq[OutgoingInput], outputs: Seq[OutgoingOutput], addExcess_opt: Option[Satoshi]) extends Response case object FundingFailed extends Response // @formatter:on - def apply(remoteNodeId: PublicKey, fundingParams: InteractiveTxParams, fundingPubkeyScript: ByteVector, purpose: InteractiveTxBuilder.Purpose, wallet: OnChainChannelFunder)(implicit ec: ExecutionContext): Behavior[Command] = { + def apply(remoteNodeId: PublicKey, fundingParams: InteractiveTxParams, fundingPubkeyScript: ByteVector, purpose: InteractiveTxBuilder.FundingInfo, wallet: OnChainChannelFunder, maxExcess_opt: Option[Satoshi] = None)(implicit ec: ExecutionContext): Behavior[Command] = { Behaviors.setup { context => Behaviors.withMdc(Logs.mdc(remoteNodeId_opt = Some(remoteNodeId), channelId_opt = Some(fundingParams.channelId))) { Behaviors.receiveMessagePartial { case FundTransaction(replyTo) => - val actor = new InteractiveTxFunder(replyTo, fundingParams, fundingPubkeyScript, purpose, wallet, context) + val actor = new InteractiveTxFunder(replyTo, fundingParams, fundingPubkeyScript, purpose, wallet, maxExcess_opt, context) actor.start() } } @@ -121,7 +121,7 @@ object InteractiveTxFunder { previousTxSizeOk && isNativeSegwit && confirmationsOk } - private def sortFundingContributions(fundingParams: InteractiveTxParams, inputs: Seq[OutgoingInput], outputs: Seq[OutgoingOutput]): FundingContributions = { + private def sortFundingContributions(fundingParams: InteractiveTxParams, inputs: Seq[OutgoingInput], outputs: Seq[OutgoingOutput], addExcess_opt: Option[Satoshi]): FundingContributions = { // We always randomize the order of inputs and outputs. val sortedInputs = Random.shuffle(inputs).zipWithIndex.map { case (input, i) => val serialId = UInt64(2 * i + fundingParams.serialIdParity) @@ -138,7 +138,7 @@ object InteractiveTxFunder { case output: Output.Shared => output.copy(serialId = serialId) } } - FundingContributions(sortedInputs, sortedOutputs) + FundingContributions(sortedInputs, sortedOutputs, addExcess_opt) } } @@ -146,8 +146,9 @@ object InteractiveTxFunder { private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response], fundingParams: InteractiveTxParams, fundingPubkeyScript: ByteVector, - purpose: InteractiveTxBuilder.Purpose, + purpose: InteractiveTxBuilder.FundingInfo, wallet: OnChainChannelFunder, + maxExcess_opt: Option[Satoshi], context: ActorContext[InteractiveTxFunder.Command])(implicit ec: ExecutionContext) { import InteractiveTxFunder._ @@ -158,6 +159,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response case rbf: InteractiveTxBuilder.SpliceTxRbf => rbf.previousTransactions case _ => Nil } + private val addExcessToRecipientPosition_opt = maxExcess_opt.map(_ => 0) def start(): Behavior[Command] = { // We always double-spend all our previous inputs. It's technically overkill because we only really need to double @@ -177,12 +179,12 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response val sharedInput = fundingParams.sharedInput_opt.toSeq.map(sharedInput => Input.Shared(UInt64(0), sharedInput.info.outPoint, sharedInput.info.txOut.publicKeyScript, 0xfffffffdL, purpose.previousLocalBalance, purpose.previousRemoteBalance, purpose.htlcBalance)) val sharedOutput = Output.Shared(UInt64(0), fundingPubkeyScript, purpose.previousLocalBalance + fundingParams.localContribution, purpose.previousRemoteBalance + fundingParams.remoteContribution, purpose.htlcBalance) val nonChangeOutputs = fundingParams.localOutputs.map(txOut => Output.Local.NonChange(UInt64(0), txOut.amount, txOut.publicKeyScript)) - val fundingContributions = sortFundingContributions(fundingParams, sharedInput ++ previousWalletInputs, sharedOutput +: nonChangeOutputs) + val fundingContributions = sortFundingContributions(fundingParams, sharedInput ++ previousWalletInputs, sharedOutput +: nonChangeOutputs, addExcess_opt = Some(0 sat)) replyTo ! fundingContributions Behaviors.stopped } else { val nonChangeOutputs = fundingParams.localOutputs.map(txOut => Output.Local.NonChange(UInt64(0), txOut.amount, txOut.publicKeyScript)) - val fundingContributions = sortFundingContributions(fundingParams, previousWalletInputs, nonChangeOutputs) + val fundingContributions = sortFundingContributions(fundingParams, previousWalletInputs, nonChangeOutputs, addExcess_opt = Some(0 sat)) replyTo ! fundingContributions Behaviors.stopped } @@ -207,17 +209,18 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response private def fund(txNotFunded: Transaction, currentInputs: Seq[OutgoingInput], unusableInputs: Set[UnusableInput]): Behavior[Command] = { val sharedInputWeight = fundingParams.sharedInput_opt.toSeq.map(i => i.info.outPoint -> i.weight.toLong).toMap val feeBudget_opt = purpose match { + case p: DummyFundingTx => p.feeBudget_opt case p: FundingTx => 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, addExcessToRecipientPosition_opt = None, maxExcess_opt = None)) { + context.pipeToSelf(wallet.fundTransaction(txNotFunded, fundingParams.targetFeerate, replaceable = true, externalInputsWeight = sharedInputWeight, feeBudget_opt = feeBudget_opt, addExcessToRecipientPosition_opt = addExcessToRecipientPosition_opt, maxExcess_opt = maxExcess_opt)) { case Failure(t) => WalletFailure(t) - case Success(result) => FundTransactionResult(result.tx, result.changePosition) + case Success(result) => FundTransactionResult(result.tx, result.changePosition, addExcessToRecipientPosition_opt.map(pos => result.tx.txOut(pos).amount - txNotFunded.txOut(pos).amount)) } Behaviors.receiveMessagePartial { - case FundTransactionResult(fundedTx, changePosition) => + case FundTransactionResult(fundedTx, changePosition, addExcess_opt) => // Those inputs were already selected by bitcoind and considered unsuitable for interactive tx. val lockedUnusableInputs = fundedTx.txIn.map(_.outPoint).filter(o => unusableInputs.map(_.outpoint).contains(o)) if (lockedUnusableInputs.nonEmpty) { @@ -226,7 +229,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response log.error("could not fund interactive tx: bitcoind included already known unusable inputs that should have been locked: {}", lockedUnusableInputs.mkString(",")) sendResultAndStop(FundingFailed, currentInputs.map(_.outPoint).toSet ++ fundedTx.txIn.map(_.outPoint) ++ unusableInputs.map(_.outpoint)) } else { - filterInputs(fundedTx, changePosition, currentInputs, unusableInputs) + filterInputs(fundedTx, changePosition, currentInputs, unusableInputs, addExcess_opt) } case WalletFailure(t) => log.error("could not fund interactive tx: ", t) @@ -235,7 +238,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response } /** Not all inputs are suitable for interactive tx construction. */ - private def filterInputs(fundedTx: Transaction, changePosition: Option[Int], currentInputs: Seq[OutgoingInput], unusableInputs: Set[UnusableInput]): Behavior[Command] = { + private def filterInputs(fundedTx: Transaction, changePosition: Option[Int], currentInputs: Seq[OutgoingInput], unusableInputs: Set[UnusableInput], addExcess_opt: Option[Satoshi]): Behavior[Command] = { context.pipeToSelf(Future.sequence(fundedTx.txIn.map(txIn => getInputDetails(txIn, currentInputs)))) { case Failure(t) => WalletFailure(t) case Success(results) => InputDetails(results.collect { case Right(i) => i }, results.collect { case Left(i) => i }.toSet) @@ -256,9 +259,9 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response val fundingContributions = if (fundingParams.isInitiator) { // The initiator is responsible for adding the shared output and the shared input. val inputs = inputDetails.usableInputs - val fundingOutput = Output.Shared(UInt64(0), fundingPubkeyScript, purpose.previousLocalBalance + fundingParams.localContribution, purpose.previousRemoteBalance + fundingParams.remoteContribution, purpose.htlcBalance) + val fundingOutput = Output.Shared(UInt64(0), fundingPubkeyScript, purpose.previousLocalBalance + fundingParams.localContribution + addExcess_opt.getOrElse(0 sat), purpose.previousRemoteBalance + fundingParams.remoteContribution, purpose.htlcBalance) val outputs = Seq(fundingOutput) ++ nonChangeOutputs ++ changeOutput_opt.toSeq - sortFundingContributions(fundingParams, inputs, outputs) + sortFundingContributions(fundingParams, inputs, outputs, addExcess_opt) } else { // The non-initiator must not include the shared input or the shared output. val inputs = inputDetails.usableInputs.filterNot(_.isInstanceOf[Input.Shared]) @@ -279,7 +282,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response nonChangeOutputs :+ changeOutput.copy(amount = changeOutput.amount + overpaidFees) case None => nonChangeOutputs } - sortFundingContributions(fundingParams, inputs, outputs) + sortFundingContributions(fundingParams, inputs, outputs, addExcess_opt) } log.debug("added {} inputs and {} outputs to interactive tx", fundingContributions.inputs.length, fundingContributions.outputs.length) // We unlock the unusable inputs (if any) as they can be used outside of interactive-tx sessions. diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/io/PeerReadyNotifier.scala b/eclair-core/src/main/scala/fr/acinq/eclair/io/PeerReadyNotifier.scala index 81d6c71b5c..57d168a8ab 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/io/PeerReadyNotifier.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/io/PeerReadyNotifier.scala @@ -174,6 +174,7 @@ object PeerReadyNotifier { case channel.WAIT_FOR_INIT_INTERNAL => false case channel.WAIT_FOR_INIT_SINGLE_FUNDED_CHANNEL => false case channel.WAIT_FOR_INIT_DUAL_FUNDED_CHANNEL => false + case channel.WAIT_FOR_DUAL_FUNDING_INTERNAL => false case channel.OFFLINE => false case channel.SYNCING => false case channel.WAIT_FOR_OPEN_CHANNEL => true diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala index 672747aea4..8e33d2257f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala @@ -22,6 +22,7 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.bitcoin.scalacompat.{Crypto, OutPoint, Satoshi, SatoshiLong, Script, Transaction, TxId, TxIn, TxOut} import fr.acinq.bitcoin.{Bech32, SigHash, SigVersion} import fr.acinq.eclair.TestUtils.randomTxId +import fr.acinq.eclair.blockchain.DummyOnChainWallet.SingleKeyOnChainWallet.invalidFundingAmount import fr.acinq.eclair.blockchain.OnChainWallet.{FundTransactionResponse, MakeFundingTxResponse, OnChainBalance, ProcessPsbtResponse} import fr.acinq.eclair.blockchain.bitcoind.BitcoindService.SignTransactionResponse import fr.acinq.eclair.blockchain.fee.FeeratePerKw @@ -155,6 +156,7 @@ class SingleKeyOnChainWallet extends OnChainWallet with OnchainPubkeyCache { override def fundTransaction(tx: Transaction, feeRate: FeeratePerKw, replaceable: Boolean, externalInputsWeight: Map[OutPoint, Long], feeBudget_opt: Option[Satoshi], addExcessToRecipientPosition_opt: Option[Int], maxExcess_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[FundTransactionResponse] = synchronized { val currentAmountIn = tx.txIn.flatMap(txIn => inputs.find(_.txid == txIn.outPoint.txid).flatMap(_.txOut.lift(txIn.outPoint.index.toInt))).map(_.amount).sum val amountOut = tx.txOut.map(_.amount).sum + if (amountOut >= invalidFundingAmount) return Future.failed(new RuntimeException(s"invalid funding amount")) // We add a single input to reach the desired feerate. val inputAmount = amountOut + 100_000.sat val inputTx = Transaction(2, Seq(TxIn(OutPoint(randomTxId(), 1), Nil, 0)), Seq(TxOut(inputAmount, Script.pay2wpkh(pubkey))), 0) @@ -249,6 +251,10 @@ class SingleKeyOnChainWallet extends OnChainWallet with OnchainPubkeyCache { override def getP2wpkhPubkey(renew: Boolean): PublicKey = pubkey } +class SingleKeyOnChainWalletWithConfirmedInputs extends SingleKeyOnChainWallet { + override def getTxConfirmations(txid: TxId)(implicit ec: ExecutionContext): Future[Option[Int]] = Future.successful(Some(6)) +} + object DummyOnChainWallet { val dummyReceiveAddress: String = "bcrt1qwcv8naajwn8fjhu8z59q9e6ucrqr068rlcenux" @@ -264,4 +270,8 @@ object DummyOnChainWallet { MakeFundingTxResponse(fundingTx, 0, 420 sat) } + object SingleKeyOnChainWallet { + val invalidFundingAmount: Satoshi = 2_100_000_000 sat + } + } \ No newline at end of file diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala index 54a8f1c29e..2de129ad0f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala @@ -127,6 +127,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsA, fundingParams, channelParamsA, FundingTx(commitFeerate, firstPerCommitmentPointB, feeBudget_opt = None), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def spawnTxBuilderRbfAlice(fundingParams: InteractiveTxParams, commitment: Commitment, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( @@ -134,6 +135,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsA, fundingParams, channelParamsA, FundingTxRbf(commitment, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def spawnTxBuilderSpliceAlice(fundingParams: InteractiveTxParams, commitment: Commitment, wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( @@ -141,6 +143,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsA, fundingParams, channelParamsA, SpliceTx(commitment), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def spawnTxBuilderSpliceRbfAlice(fundingParams: InteractiveTxParams, parentCommitment: Commitment, latestFundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( @@ -148,6 +151,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsA, fundingParams, channelParamsA, SpliceTxRbf(parentCommitment, latestFundingTx, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def spawnTxBuilderBob(wallet: OnChainWallet, fundingParams: InteractiveTxParams = fundingParamsB): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( @@ -155,6 +159,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsB, fundingParams, channelParamsB, FundingTx(commitFeerate, firstPerCommitmentPointA, feeBudget_opt = None), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def spawnTxBuilderRbfBob(fundingParams: InteractiveTxParams, commitment: Commitment, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( @@ -162,6 +167,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsB, fundingParams, channelParamsB, FundingTxRbf(commitment, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def spawnTxBuilderSpliceBob(fundingParams: InteractiveTxParams, commitment: Commitment, wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( @@ -169,6 +175,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsB, fundingParams, channelParamsB, SpliceTx(commitment), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def spawnTxBuilderSpliceRbfBob(fundingParams: InteractiveTxParams, parentCommitment: Commitment, latestFundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( @@ -176,6 +183,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsB, fundingParams, channelParamsB, SpliceTxRbf(parentCommitment, latestFundingTx, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def exchangeSigsAliceFirst(fundingParams: InteractiveTxParams, successA: InteractiveTxBuilder.Succeeded, successB: InteractiveTxBuilder.Succeeded): (FullySignedSharedTransaction, Commitment, FullySignedSharedTransaction, Commitment) = { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala index d6142fda40..76f969d39e 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala @@ -27,7 +27,7 @@ import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair._ import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ import fr.acinq.eclair.blockchain.fee.{FeeratePerKw, FeeratesPerKw} -import fr.acinq.eclair.blockchain.{DummyOnChainWallet, OnChainWallet, OnchainPubkeyCache, SingleKeyOnChainWallet} +import fr.acinq.eclair.blockchain.{DummyOnChainWallet, OnChainWallet, OnchainPubkeyCache, SingleKeyOnChainWalletWithConfirmedInputs} import fr.acinq.eclair.channel.fsm.Channel import fr.acinq.eclair.channel.publish.TxPublisher import fr.acinq.eclair.channel.publish.TxPublisher.PublishReplaceableTx @@ -159,7 +159,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually { .modify(_.channelConf.balanceThresholds).setToIf(tags.contains(ChannelStateTestsTags.AdaptMaxHtlcAmount))(Seq(Channel.BalanceThreshold(1_000 sat, 0 sat), Channel.BalanceThreshold(5_000 sat, 1_000 sat), Channel.BalanceThreshold(10_000 sat, 5_000 sat))) val wallet = wallet_opt match { case Some(wallet) => wallet - case None => if (tags.contains(ChannelStateTestsTags.DualFunding)) new SingleKeyOnChainWallet() else new DummyOnChainWallet() + case None => if (tags.contains(ChannelStateTestsTags.DualFunding)) new SingleKeyOnChainWalletWithConfirmedInputs() else new DummyOnChainWallet() } val alice: TestFSMRef[ChannelState, ChannelData, Channel] = { implicit val system: ActorSystem = systemA diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForFundingInternalDualFundedChannelStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForFundingInternalDualFundedChannelStateSpec.scala new file mode 100644 index 0000000000..30c36d470a --- /dev/null +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForFundingInternalDualFundedChannelStateSpec.scala @@ -0,0 +1,107 @@ +/* + * Copyright 2024 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.channel.states.a + +import akka.actor.Status +import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps +import akka.testkit.{TestFSMRef, TestProbe} +import fr.acinq.bitcoin.scalacompat.ByteVector32 +import fr.acinq.eclair.blockchain.NoOpOnChainWallet +import fr.acinq.eclair.channel._ +import fr.acinq.eclair.channel.fsm.Channel +import fr.acinq.eclair.channel.fsm.Channel.TickChannelOpenTimeout +import fr.acinq.eclair.channel.fund.InteractiveTxFunder +import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags} +import fr.acinq.eclair.io.Peer.OpenChannelResponse +import fr.acinq.eclair.wire.protocol._ +import fr.acinq.eclair.{TestConstants, TestKitBaseClass} +import org.scalatest.Outcome +import org.scalatest.funsuite.FixtureAnyFunSuiteLike + +import scala.concurrent.duration._ + +class WaitForFundingInternalDualFundedChannelStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ChannelStateTestsBase { + + case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], aliceOpenReplyTo: TestProbe, alice2bob: TestProbe, listener: TestProbe) + + override def withFixture(test: OneArgTest): Outcome = { + val setup = init(wallet_opt = Some(new NoOpOnChainWallet()), tags = test.tags + ChannelStateTestsTags.DualFunding) + import setup._ + val channelConfig = ChannelConfig.standard + val channelFlags = ChannelFlags(announceChannel = false) + val (aliceParams, bobParams, channelType) = computeFeatures(setup, test.tags, channelFlags) + val bobInit = Init(bobParams.initFeatures) + val listener = TestProbe() + within(30 seconds) { + alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted]) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, maxExcess_opt = None, dualFunded = true, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = true, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + withFixture(test.toNoArgTest(FixtureParam(alice, aliceOpenReplyTo, alice2bob, listener))) + } + } + + test("recv Status.Failure (wallet error)") { f => + import f._ + awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_INTERNAL) + alice ! Status.Failure(new RuntimeException("insufficient funds")) + listener.expectMsgType[ChannelAborted] + awaitCond(alice.stateName == CLOSED) + aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected] + } + + test("recv Error") { f => + import f._ + alice ! Error(ByteVector32.Zeroes, "oops") + listener.expectMsgType[ChannelAborted] + awaitCond(alice.stateName == CLOSED) + aliceOpenReplyTo.expectMsgType[OpenChannelResponse.RemoteError] + } + + test("recv CMD_CLOSE") { f => + import f._ + val sender = TestProbe() + val c = CMD_CLOSE(sender.ref, None, None) + alice ! c + sender.expectMsg(RES_SUCCESS(c, ByteVector32.Zeroes)) + listener.expectMsgType[ChannelAborted] + awaitCond(alice.stateName == CLOSED) + aliceOpenReplyTo.expectMsg(OpenChannelResponse.Cancelled) + } + + test("recv INPUT_DISCONNECTED") { f => + import f._ + alice ! INPUT_DISCONNECTED + listener.expectMsgType[ChannelAborted] + awaitCond(alice.stateName == CLOSED) + aliceOpenReplyTo.expectMsg(OpenChannelResponse.Disconnected) + } + + test("recv TickChannelOpenTimeout") { f => + import f._ + alice ! TickChannelOpenTimeout + listener.expectMsgType[ChannelAborted] + awaitCond(alice.stateName == CLOSED) + aliceOpenReplyTo.expectMsg(OpenChannelResponse.TimedOut) + } + + test("recv funding success") { f => + import f._ + alice ! InteractiveTxFunder.FundingContributions(Seq(), Seq(), None) + alice2bob.expectMsgType[OpenDualFundedChannel] + awaitCond(alice.stateName == WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL) + } + +} diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala index 239dea6d4c..a69946e944 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala @@ -19,7 +19,7 @@ package fr.acinq.eclair.channel.states.b import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps import akka.testkit.{TestFSMRef, TestProbe} import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong, Script} -import fr.acinq.eclair.blockchain.SingleKeyOnChainWallet +import fr.acinq.eclair.blockchain.SingleKeyOnChainWalletWithConfirmedInputs import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel @@ -37,10 +37,10 @@ import scala.concurrent.duration.DurationInt class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ChannelStateTestsBase { - case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], aliceOpenReplyTo: TestProbe, alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, bob2blockchain: TestProbe, wallet: SingleKeyOnChainWallet, aliceListener: TestProbe, bobListener: TestProbe) + case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], aliceOpenReplyTo: TestProbe, alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, bob2blockchain: TestProbe, wallet: SingleKeyOnChainWalletWithConfirmedInputs, aliceListener: TestProbe, bobListener: TestProbe) override def withFixture(test: OneArgTest): Outcome = { - val wallet = new SingleKeyOnChainWallet() + val wallet = new SingleKeyOnChainWalletWithConfirmedInputs() val setup = init(wallet_opt = Some(wallet), tags = test.tags) import setup._ val channelConfig = ChannelConfig.standard diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingSignedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingSignedStateSpec.scala index 72b18f3a07..74a63c2d39 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingSignedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingSignedStateSpec.scala @@ -20,7 +20,7 @@ import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps import akka.testkit.{TestFSMRef, TestProbe} import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, SatoshiLong, TxId} import fr.acinq.eclair.TestUtils.randomTxId -import fr.acinq.eclair.blockchain.SingleKeyOnChainWallet +import fr.acinq.eclair.blockchain.SingleKeyOnChainWalletWithConfirmedInputs import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchPublished} import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.channel._ @@ -39,10 +39,10 @@ import scala.concurrent.duration.DurationInt class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ChannelStateTestsBase { - case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, bob2blockchain: TestProbe, wallet: SingleKeyOnChainWallet, aliceListener: TestProbe, bobListener: TestProbe) + case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, bob2blockchain: TestProbe, wallet: SingleKeyOnChainWalletWithConfirmedInputs, aliceListener: TestProbe, bobListener: TestProbe) override def withFixture(test: OneArgTest): Outcome = { - val wallet = new SingleKeyOnChainWallet() + val wallet = new SingleKeyOnChainWalletWithConfirmedInputs() val setup = init(wallet_opt = Some(wallet), tags = test.tags) import setup._ val channelConfig = ChannelConfig.standard diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala index ca772d62aa..460736b2d3 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala @@ -21,7 +21,7 @@ import akka.testkit.{TestFSMRef, TestProbe} import com.softwaremill.quicklens.{ModifyPimp, QuicklensAt} import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong, Transaction} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ -import fr.acinq.eclair.blockchain.{CurrentBlockHeight, SingleKeyOnChainWallet} +import fr.acinq.eclair.blockchain.{CurrentBlockHeight, SingleKeyOnChainWalletWithConfirmedInputs} import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel import fr.acinq.eclair.channel.fsm.Channel.ProcessCurrentBlockHeight @@ -40,10 +40,10 @@ import scala.concurrent.duration.DurationInt class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ChannelStateTestsBase { - 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) + 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: SingleKeyOnChainWalletWithConfirmedInputs) override def withFixture(test: OneArgTest): Outcome = { - val wallet = new SingleKeyOnChainWallet() + val wallet = new SingleKeyOnChainWalletWithConfirmedInputs() val setup = init(wallet_opt = Some(wallet), tags = test.tags) import setup._ diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala index c8abe3e2fe..a2c2b0ef53 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala @@ -23,6 +23,7 @@ import fr.acinq.bitcoin.ScriptFlags import fr.acinq.bitcoin.scalacompat.NumericSatoshi.abs import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, SatoshiLong, Transaction, TxIn} import fr.acinq.eclair._ +import fr.acinq.eclair.blockchain.DummyOnChainWallet.SingleKeyOnChainWallet.invalidFundingAmount import fr.acinq.eclair.blockchain.SingleKeyOnChainWallet import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ import fr.acinq.eclair.blockchain.fee.FeeratePerKw @@ -1060,6 +1061,18 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.forall(_.localCommit.spec.htlcs.size == 1)) } + test("Funding failed before a splice is requested from our peer") { f => + import f._ + val sender = TestProbe() + val cmd = CMD_SPLICE(sender.ref, spliceIn_opt = Some(SpliceIn(invalidFundingAmount, pushAmount = 0 msat)), spliceOut_opt = None) + alice ! cmd + exchangeStfu(f) + awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus.isInstanceOf[SpliceStatus.SpliceRequested]) + sender.expectMsg(RES_FAILURE(cmd, ChannelFundingError(channelId(alice)))) + awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus == SpliceStatus.NoSplice) + alice2bob.expectNoMessage(100 millis) + } + test("recv CMD_ADD_HTLC while a splice is requested") { f => import f._ val sender = TestProbe() diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala index 27a4cd58de..ce2636ce23 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala @@ -10,7 +10,7 @@ import com.typesafe.config.ConfigFactory import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, Satoshi, SatoshiLong, Transaction, TxId} import fr.acinq.eclair.ShortChannelId.txIndex -import fr.acinq.eclair.blockchain.SingleKeyOnChainWallet +import fr.acinq.eclair.blockchain.SingleKeyOnChainWalletWithConfirmedInputs import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchFundingConfirmedTriggered, WatchFundingDeeplyBuried, WatchFundingDeeplyBuriedTriggered} import fr.acinq.eclair.blockchain.fee.{FeeratePerKw, FeeratesPerKw} @@ -56,7 +56,7 @@ case class MinimalNodeFixture private(nodeParams: NodeParams, offerManager: typed.ActorRef[OfferManager.Command], postman: typed.ActorRef[Postman.Command], watcher: TestProbe, - wallet: SingleKeyOnChainWallet, + wallet: SingleKeyOnChainWalletWithConfirmedInputs, bitcoinClient: TestBitcoinCoreClient) { val nodeId = nodeParams.nodeId val routeParams = nodeParams.routerConf.pathFindingExperimentConf.experiments.values.head.getDefaultRouteParams @@ -88,7 +88,7 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat val readyListener = TestProbe("ready-listener") system.eventStream.subscribe(readyListener.ref, classOf[SubscriptionsComplete]) val bitcoinClient = new TestBitcoinCoreClient() - val wallet = new SingleKeyOnChainWallet() + val wallet = new SingleKeyOnChainWalletWithConfirmedInputs() val watcher = TestProbe("watcher") val triggerer = TestProbe("payment-triggerer") val watcherTyped = watcher.ref.toTyped[ZmqWatcher.Command]