From 6d26f6d4030d073a87d284838b2e8e9811a3c477 Mon Sep 17 00:00:00 2001 From: t-bast Date: Wed, 11 Jun 2025 16:37:20 +0200 Subject: [PATCH 1/4] 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. We do the same thing for the `commit_sig` TLV. For peers who support the official splicing version, we insert the `start_batch` message before the batch of `commit_sig` messages. This guarantees backwards-compatibility with peers who only support the experimental feature. --- docs/release-notes/eclair-vnext.md | 33 +++ eclair-core/src/main/resources/reference.conf | 3 +- .../main/scala/fr/acinq/eclair/Features.scala | 17 +- .../fr/acinq/eclair/channel/Commitments.scala | 12 +- .../fr/acinq/eclair/channel/fsm/Channel.scala | 8 +- .../channel/fund/InteractiveTxBuilder.scala | 2 +- .../fr/acinq/eclair/io/PeerConnection.scala | 72 ++++++- .../eclair/wire/protocol/ChannelTlv.scala | 18 ++ .../acinq/eclair/wire/protocol/HtlcTlv.scala | 23 +- .../wire/protocol/InteractiveTxTlv.scala | 12 +- .../protocol/LightningMessageCodecs.scala | 34 ++- .../wire/protocol/LightningMessageTypes.scala | 72 +++++-- .../fundee/data.json | 6 +- .../funder/data.json | 6 +- .../funder/data.json | 6 +- .../fundee/data.json | 6 +- .../funder/data.json | 6 +- .../fundee/data.json | 6 +- .../funder/data.json | 6 +- .../funder/data.json | 6 +- .../announced-splice/data.json | 6 +- .../050006-DATA_NORMAL/announced/data.json | 6 +- .../050006-DATA_NORMAL/fundee/data.json | 6 +- .../050006-DATA_NORMAL/funder/data.json | 6 +- .../splice-commitment-upgrade/data.json | 6 +- .../anchor-outputs/data.json | 6 +- .../050007-DATA_SHUTDOWN/taproot/data.json | 6 +- .../050008-DATA_NEGOTIATING/fundee/data.json | 6 +- .../anchor-outputs/data.json | 6 +- .../taproot/data.json | 6 +- .../05000a-DATA_CLOSING/local/data.json | 6 +- .../05000a-DATA_CLOSING/next-remote/data.json | 6 +- .../05000a-DATA_CLOSING/remote/data.json | 6 +- .../05000a-DATA_CLOSING/revoked/data.json | 6 +- .../scala/fr/acinq/eclair/TestConstants.scala | 4 +- .../channel/InteractiveTxBuilderSpec.scala | 2 +- .../ChannelStateTestsHelperMethods.scala | 2 +- .../states/e/NormalSplicesStateSpec.scala | 32 ++- .../basic/channel/GossipIntegrationSpec.scala | 2 - .../io/OpenChannelInterceptorSpec.scala | 4 +- .../acinq/eclair/io/PeerConnectionSpec.scala | 204 ++++++++++++++++-- .../payment/relay/OnTheFlyFundingSpec.scala | 6 +- .../protocol/LightningMessageCodecsSpec.scala | 60 ++++-- 43 files changed, 565 insertions(+), 189 deletions(-) diff --git a/docs/release-notes/eclair-vnext.md b/docs/release-notes/eclair-vnext.md index 9665470690..0dd1193041 100644 --- a/docs/release-notes/eclair-vnext.md +++ b/docs/release-notes/eclair-vnext.md @@ -4,6 +4,39 @@ ## 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 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. + ### Remove support for non-anchor channels We remove the code used to support legacy channels that don't use anchor outputs or taproot. diff --git a/eclair-core/src/main/resources/reference.conf b/eclair-core/src/main/resources/reference.conf index a6b71251d3..97719fb39e 100644 --- a/eclair-core/src/main/resources/reference.conf +++ b/eclair-core/src/main/resources/reference.conf @@ -88,6 +88,7 @@ eclair { option_zeroconf = disabled keysend = disabled option_simple_close = optional + option_splice = optional trampoline_payment_prototype = disabled async_payment_prototype = disabled on_the_fly_funding = disabled @@ -109,7 +110,7 @@ eclair { funding { // Each RBF attempt adds more data that we need to store and process, so we want to limit our peers to a reasonable use of RBF. remote-rbf-limits { - max-attempts = 5 // maximum number of RBF attempts our peer is allowed to make + max-attempts = 10 // maximum number of RBF attempts our peer is allowed to make attempt-delta-blocks = 6 // minimum number of blocks between RBF attempts } // Duration after which we abort a channel creation. If our peer seems unresponsive and doesn't complete the 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 12e5127208..dfe3fd5681 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Features.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Features.scala @@ -261,8 +261,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 } @@ -312,6 +311,11 @@ object Features { val mandatory = 60 } + case object Splicing extends Feature with InitFeature with NodeFeature { + val rfcName = "option_splice" + val mandatory = 62 + } + case object PhoenixZeroReserve extends Feature with InitFeature with ChannelTypeFeature with PermanentChannelFeature { val rfcName = "phoenix_zero_reserve" val mandatory = 128 @@ -340,12 +344,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 - } - case object SimpleTaprootChannelsPhoenix extends Feature with InitFeature with NodeFeature with ChannelTypeFeature { val rfcName = "option_simple_taproot_phoenix" val mandatory = 564 @@ -399,12 +397,12 @@ object Features { ZeroConf, KeySend, SimpleClose, + Splicing, SimpleTaprootChannelsPhoenix, SimpleTaprootChannelsStaging, WakeUpNotificationClient, TrampolinePaymentPrototype, AsyncPaymentPrototype, - SplicePrototype, OnTheFlyFunding, FundingFeeCredit, PhoenixZeroReserve @@ -423,7 +421,6 @@ object Features { SimpleClose -> (ShutdownAnySegwit :: Nil), SimpleTaprootChannelsPhoenix -> (ChannelType :: SimpleClose :: Nil), AsyncPaymentPrototype -> (TrampolinePaymentPrototype :: Nil), - OnTheFlyFunding -> (SplicePrototype :: Nil), FundingFeeCredit -> (OnTheFlyFunding :: Nil) ) 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 d2b4065fda..b4edb23c36 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 @@ -209,14 +209,14 @@ case class RemoteCommit(index: Long, spec: CommitmentSpec, txId: TxId, remotePer commitmentFormat match { case _: SegwitV0CommitmentFormat => val sig = remoteCommitTx.sign(fundingKey, remoteFundingPubKey) - Right(CommitSig(channelParams.channelId, sig, htlcSigs.toList, batchSize)) + Right(CommitSig(channelParams.channelId, commitInput.outPoint.txid, sig, htlcSigs.toList, batchSize)) case _: SimpleTaprootChannelCommitmentFormat => remoteNonce_opt match { case Some(remoteNonce) => val localNonce = NonceGenerator.signingNonce(fundingKey.publicKey, remoteFundingPubKey, commitInput.outPoint.txid) remoteCommitTx.partialSign(fundingKey, remoteFundingPubKey, localNonce, Seq(localNonce.publicNonce, remoteNonce)) match { case Left(_) => Left(InvalidCommitNonce(channelParams.channelId, commitInput.outPoint.txid, index)) - case Right(psig) => Right(CommitSig(channelParams.channelId, psig, htlcSigs.toList, batchSize)) + case Right(psig) => Right(CommitSig(channelParams.channelId, commitInput.outPoint.txid, psig, htlcSigs.toList, batchSize)) } case None => Left(MissingCommitNonce(channelParams.channelId, commitInput.outPoint.txid, index)) } @@ -651,7 +651,7 @@ case class Commitment(fundingTxIndex: Long, case None => return Left(MissingCommitNonce(params.channelId, fundingTxId, remoteCommit.index + 1)) } } - val commitSig = CommitSig(params.channelId, sig, htlcSigs.toList, batchSize) + val commitSig = CommitSig(params.channelId, fundingTxId, sig, htlcSigs.toList, batchSize) val nextRemoteCommit = RemoteCommit(remoteCommit.index + 1, spec, remoteCommitTx.tx.txid, remoteNextPerCommitmentPoint) Right((copy(nextRemoteCommit_opt = Some(nextRemoteCommit)), commitSig)) } @@ -1092,9 +1092,11 @@ case class Commitments(channelParams: ChannelParams, case _: CommitSig if active.size > 1 => return Left(CommitSigCountMismatch(channelId, active.size, 1)) case commitSig: CommitSig => Seq(commitSig) } - // Signatures are sent in order (most recent first), calling `zip` will drop trailing sigs that are for deactivated/pruned commitments. val commitKeys = LocalCommitmentKeys(channelParams, channelKeys, localCommitIndex + 1) - val active1 = active.zip(sigs).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 = sigs.find(_.fundingTxId_opt.contains(commitment.fundingTxId)).getOrElse(sigs(idx)) commitment.receiveCommit(channelParams, channelKeys, commitKeys, changes, 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 84ac61dce2..da3bd15455 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 @@ -952,7 +952,7 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall } case Event(cmd: CMD_SPLICE, d: DATA_NORMAL) => - if (!d.commitments.remoteChannelParams.initFeatures.hasFeature(Features.SplicePrototype)) { + if (!d.commitments.remoteChannelParams.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() @@ -2467,7 +2467,8 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall } case _ => Set.empty } - val lastFundingLockedTlvs: Set[ChannelReestablishTlv] = if (d.commitments.remoteChannelParams.initFeatures.hasFeature(Features.SplicePrototype)) { + val remoteFeatures = d.commitments.remoteChannelParams.initFeatures + val lastFundingLockedTlvs: Set[ChannelReestablishTlv] = if (remoteFeatures.hasFeature(Features.Splicing) || remoteFeatures.unknown.contains(UnknownFeature(154)) || remoteFeatures.unknown.contains(UnknownFeature(155))) { d.commitments.lastLocalLocked_opt.map(c => ChannelReestablishTlv.MyCurrentFundingLockedTlv(c.fundingTxId)).toSet ++ d.commitments.lastRemoteLocked_opt.map(c => ChannelReestablishTlv.YourLastFundingLockedTlv(c.fundingTxId)).toSet } else Set.empty @@ -3390,7 +3391,8 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall // We only send channel_ready for initial funding transactions. case Some(c) if c.fundingTxIndex != 0 => (None, None) case Some(c) => - val remoteSpliceSupport = d.commitments.remoteChannelParams.initFeatures.hasFeature(Features.SplicePrototype) + val remoteFeatures = d.commitments.remoteChannelParams.initFeatures + val remoteSpliceSupport = remoteFeatures.hasFeature(Features.Splicing) || remoteFeatures.unknown.contains(UnknownFeature(154)) || remoteFeatures.unknown.contains(UnknownFeature(155)) // If our peer has not received our channel_ready, we retransmit it. val notReceivedByRemote = remoteSpliceSupport && channelReestablish.yourLastFundingLocked_opt.isEmpty // If next_local_commitment_number is 1 in both the channel_reestablish it sent and received, then the node 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 8eb18a7a2c..dee0675f8a 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 @@ -932,7 +932,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon unlockAndStop(completeTx) case Right(localSigOfRemoteTx) => val htlcSignatures = sortedHtlcTxs.map(_.localSig(remoteCommitmentKeys)).toList - val localCommitSig = CommitSig(fundingParams.channelId, localSigOfRemoteTx, htlcSignatures, batchSize = 1) + val localCommitSig = CommitSig(fundingParams.channelId, fundingTx.txid, localSigOfRemoteTx, htlcSignatures, batchSize = 1) val localCommit = UnsignedLocalCommit(purpose.localCommitIndex, localSpec, localCommitTx.tx.txid) val remoteCommit = RemoteCommit(purpose.remoteCommitIndex, remoteSpec, remoteCommitTx.tx.txid, purpose.remotePerCommitmentPoint) signFundingTx(completeTx, remoteFundingNonce_opt, remoteCommitNonces_opt.map(_.nextCommitNonce), localCommitSig, localCommit, remoteCommit) 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 8f2fe0254c..967a939e94 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 @@ -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, Features, InitFeature, Logs, TimestampMilli, TimestampSecond} +import fr.acinq.eclair.{FSMDiagnosticActorLogging, Features, InitFeature, Logs, TimestampMilli, TimestampSecond, UnknownFeature} import scodec.Attempt import scodec.bits.ByteVector @@ -206,9 +206,19 @@ 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 + val useExperimentalSplice = d.remoteInit.features.unknown.contains(UnknownFeature(154)) || d.remoteInit.features.unknown.contains(UnknownFeature(155)) msg match { - case batch: CommitSigBatch => batch.messages.foreach(msg => d.transport forward msg) - case msg => d.transport forward msg + // 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: TxSignatures if useExperimentalSplice => d.transport forward msg.copy(tlvStream = TlvStream(msg.tlvStream.records.filterNot(_.isInstanceOf[TxSignaturesTlv.PreviousFundingTxSig]))) + case batch: CommitSigBatch if useExperimentalSplice => batch.messages.foreach(msg => d.transport forward msg.copy(tlvStream = TlvStream(msg.tlvStream.records.filterNot(_.isInstanceOf[CommitSigTlv.FundingTx])))) + case batch: CommitSigBatch => + d.transport forward StartBatch.commitSigBatch(batch.channelId, batch.batchSize) + batch.messages.foreach(msg => d.transport forward msg.copy(tlvStream = TlvStream(msg.tlvStream.records.filterNot(_.isInstanceOf[CommitSigTlv.ExperimentalBatchTlv])))) + case _ => d.transport forward msg } msg match { // If we send any channel management message to this peer, the connection should be persistent. @@ -348,8 +358,51 @@ class PeerConnection(keyPair: KeyPair, conf: PeerConnection.Conf, switchboard: A // We immediately forward messages to the peer, unless they are part of a batch, in which case we wait to // receive the whole batch before forwarding. msg match { + case msg: StartBatch => + if (!msg.messageType_opt.contains(132)) { + log.debug("ignoring start_batch: we only support batching commit_sig messages") + d.transport ! Warning(msg.channelId, "invalid start_batch message: we only support batching commit_sig messages") + stay() + } else if (msg.batchSize > 20) { + log.debug("ignoring start_batch with batch_size = {} > 20", msg.batchSize) + d.transport ! Warning(msg.channelId, "invalid start_batch message: batch_size must not be greater than 20") + stay() + } else { + log.debug("starting commit_sig batch of size {} for channel_id={}", msg.batchSize, msg.channelId) + d.commitSigBatch_opt match { + case Some(pending) if pending.received.nonEmpty => + log.warning("starting batch with incomplete previous batch ({}/{} received)", pending.received.size, pending.batchSize) + // This is a spec violation from our peer: this will likely lead to a force-close. + d.transport ! Warning(msg.channelId, "invalid start_batch message: the previous batch is not done yet") + d.peer ! CommitSigBatch(pending.received) + case _ => () + } + stay() using d.copy(commitSigBatch_opt = Some(PendingCommitSigBatch(msg.channelId, msg.batchSize, Nil))) + } + case msg: HasChannelId if d.commitSigBatch_opt.nonEmpty => + // We only support batches of commit_sig messages: other messages will simply be relayed individually. + val pending = d.commitSigBatch_opt.get + msg match { + case msg: CommitSig if msg.channelId == pending.channelId => + val received1 = pending.received :+ msg + if (received1.size == pending.batchSize) { + log.debug("received last commit_sig in batch for channel_id={}", msg.channelId) + d.peer ! CommitSigBatch(received1) + stay() using d.copy(commitSigBatch_opt = None) + } else { + log.debug("received commit_sig {}/{} in batch for channel_id={}", received1.size, pending.batchSize, msg.channelId) + stay() using d.copy(commitSigBatch_opt = Some(pending.copy(received = received1))) + } + case _ => + log.warning("received {} as part of a batch: we don't support batching that kind of messages", msg.getClass.getSimpleName) + if (pending.received.nonEmpty) d.peer ! CommitSigBatch(pending.received) + d.peer ! msg + stay() using d.copy(commitSigBatch_opt = None) + } case msg: CommitSig => - msg.tlvStream.get[CommitSigTlv.BatchTlv].map(_.size) match { + // We keep supporting the experimental version of splicing that older Phoenix wallets use. + // Once we're confident that enough Phoenix users have upgraded, we should remove this branch. + msg.tlvStream.get[CommitSigTlv.ExperimentalBatchTlv].map(_.size) match { case Some(batchSize) if batchSize > 25 => log.warning("received legacy batch of commit_sig exceeding our threshold ({} > 25), processing messages individually", batchSize) // We don't want peers to be able to exhaust our memory by sending batches of dummy messages that we keep in RAM. @@ -381,6 +434,16 @@ class PeerConnection(keyPair: KeyPair, conf: PeerConnection.Conf, switchboard: A d.peer ! msg stay() } + // If our peer is using the experimental splice version, we convert splice messages. + case msg: ExperimentalSpliceInit => + d.peer ! msg.toSpliceInit() + stay() + case msg: ExperimentalSpliceAck => + d.peer ! msg.toSpliceAck() + stay() + case msg: ExperimentalSpliceLocked => + d.peer ! msg.toSpliceLocked() + stay() case _ => d.peer ! msg stay() @@ -613,6 +676,7 @@ object PeerConnection { gossipTimestampFilter: Option[GossipTimestampFilter] = None, behavior: Behavior = Behavior(), expectedPong_opt: Option[ExpectedPong] = None, + commitSigBatch_opt: Option[PendingCommitSigBatch] = None, legacyCommitSigBatch_opt: Option[PendingCommitSigBatch] = None, isPersistent: Boolean) extends Data with HasTransport diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/ChannelTlv.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/ChannelTlv.scala index ac0f921e50..9aad296841 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/ChannelTlv.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/ChannelTlv.scala @@ -255,8 +255,16 @@ sealed trait ChannelReestablishTlv extends Tlv object ChannelReestablishTlv { + /** + * When disconnected in the middle of an interactive-tx session, this field is used to request a retransmission of + * [[TxSignatures]] for the given [[txId]]. + */ case class NextFundingTlv(txId: TxId) extends ChannelReestablishTlv + + /** The txid of the last [[ChannelReady]] or [[SpliceLocked]] message received before disconnecting, if any. */ case class YourLastFundingLockedTlv(txId: TxId) extends ChannelReestablishTlv + + /** The txid of our latest outgoing [[ChannelReady]] or [[SpliceLocked]] for this channel. */ case class MyCurrentFundingLockedTlv(txId: TxId) extends ChannelReestablishTlv /** @@ -395,3 +403,13 @@ object ClosingSigTlv { ) } +sealed trait StartBatchTlv extends Tlv + +object StartBatchTlv { + /** Type of [[LightningMessage]] that is included in the batch, when batching a single message type. */ + case class MessageType(tag: Int) extends StartBatchTlv + + val startBatchTlvCodec: Codec[TlvStream[StartBatchTlv]] = tlvStream(discriminated[StartBatchTlv].by(varint) + .typecase(UInt64(1), tlvField(uint16.as[MessageType])) + ) +} 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 2a62118e40..2c711f5a92 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 @@ -89,13 +89,23 @@ 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: this field maps it to the corresponding funding transaction. + * + * @param txId the funding transaction spent by this commitment. + */ + case class FundingTx(txId: TxId) extends CommitSigTlv - /** @param size the number of [[CommitSig]] messages in the batch */ - case class BatchTlv(size: Int) extends CommitSigTlv + private val fundingTxTlv: Codec[FundingTx] = tlvField(txIdAsHash) - object BatchTlv { - val codec: Codec[BatchTlv] = tlvField(tu16) - } + /** + * The experimental version of splicing included the number of [[CommitSig]] messages in the batch. + * This TLV can be removed once Phoenix users have upgraded to the official version of splicing. + */ + case class ExperimentalBatchTlv(size: Int) extends CommitSigTlv + + private val experimentalBatchTlv: Codec[ExperimentalBatchTlv] = tlvField(tu16) /** Partial signature signature for the current commitment transaction, along with the signing nonce used (when using taproot channels). */ case class PartialSignatureWithNonceTlv(partialSigWithNonce: PartialSignatureWithNonce) extends CommitSigTlv @@ -105,8 +115,9 @@ object CommitSigTlv { } val commitSigTlvCodec: Codec[TlvStream[CommitSigTlv]] = tlvStream(discriminated[CommitSigTlv].by(varint) + .typecase(UInt64(1), fundingTxTlv) .typecase(UInt64(2), PartialSignatureWithNonceTlv.codec) - .typecase(UInt64(0x47010005), BatchTlv.codec) + .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 f8ae3d15a8..3eb0c34b98 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 @@ -36,6 +36,9 @@ 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 + /** * When creating an interactive-tx where both participants sign a taproot input, we don't need to provide the entire * previous transaction in [[TxAddInput]]: signatures will commit to the txOut of *all* of the transaction's inputs, @@ -49,7 +52,8 @@ object 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])) .typecase(UInt64(1111), PrevTxOut.codec) ) } @@ -102,9 +106,13 @@ object TxSignaturesTlv { /** When doing a splice for a taproot channel, each peer must provide their partial signature for the previous musig2 funding output. */ case class PreviousFundingTxPartialSig(partialSigWithNonce: PartialSignatureWithNonce) 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(0), tlvField(bytes64.as[PreviousFundingTxSig])) .typecase(UInt64(2), tlvField(partialSignatureWithNonce.as[PreviousFundingTxPartialSig])) - .typecase(UInt64(601), 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 f506a3cc57..9a8977308e 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 @@ -244,6 +244,11 @@ object LightningMessageCodecs { ("lockTime" | uint32) :: ("tlvStream" | ClosingSigTlv.closingSigTlvCodec)).as[ClosingSig] + val startBatchCodec: Codec[StartBatch] = ( + ("channelId" | bytes32) :: + ("batchSize" | uint16) :: + ("tlvStream" | StartBatchTlv.startBatchTlvCodec)).as[StartBatch] + val updateAddHtlcCodec: Codec[UpdateAddHtlc] = ( ("channelId" | bytes32) :: ("id" | uint64overflow) :: @@ -435,17 +440,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] @@ -528,6 +552,10 @@ object LightningMessageCodecs { .typecase(72, txInitRbfCodec) .typecase(73, txAckRbfCodec) .typecase(74, txAbortCodec) + .typecase(77, spliceLockedCodec) + .typecase(80, spliceInitCodec) + .typecase(81, spliceAckCodec) + .typecase(127, startBatchCodec) .typecase(128, updateAddHtlcCodec) .typecase(130, updateFulfillHtlcCodec) .typecase(131, updateFailHtlcCodec) @@ -559,9 +587,9 @@ object LightningMessageCodecs { .typecase(41045, addFeeCreditCodec) .typecase(41046, currentFeeCreditCodec) // - .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 ba4d08fe57..35e2950809 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 @@ -97,12 +97,16 @@ case class TxAddInput(channelId: ByteVector32, tlvStream: TlvStream[TxAddInputTlv] = TlvStream.empty) extends InteractiveTxConstructionMessage with HasChannelId with HasSerialId { /** This field may replace [[previousTx_opt]] when using taproot. */ val previousTxOut_opt: Option[InputInfo] = tlvStream.get[TxAddInputTlv.PrevTxOut].map(tlv => InputInfo(OutPoint(tlv.txId, previousTxOutput), TxOut(tlv.amount, tlv.publicKeyScript))) - val sharedInput_opt: Option[OutPoint] = tlvStream.get[TxAddInputTlv.SharedInputTxId].map(i => OutPoint(i.txId, previousTxOutput)) + 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)) } } @@ -140,18 +144,17 @@ case class TxSignatures(channelId: ByteVector32, txId: TxId, witnesses: Seq[ScriptWitness], tlvStream: TlvStream[TxSignaturesTlv] = TlvStream.empty) extends InteractiveTxMessage with HasChannelId { - val previousFundingTxSig_opt: Option[ByteVector64] = tlvStream.get[TxSignaturesTlv.PreviousFundingTxSig].map(_.sig) + val previousFundingTxSig_opt: Option[ByteVector64] = tlvStream.get[TxSignaturesTlv.PreviousFundingTxSig].map(_.sig).orElse(tlvStream.get[TxSignaturesTlv.ExperimentalPreviousFundingTxSig].map(_.sig)) val previousFundingTxPartialSig_opt: Option[PartialSignatureWithNonce] = tlvStream.get[TxSignaturesTlv.PreviousFundingTxPartialSig].map(_.partialSigWithNonce) } object TxSignatures { def apply(channelId: ByteVector32, tx: Transaction, witnesses: Seq[ScriptWitness], previousFundingSig_opt: Option[ChannelSpendSignature]): TxSignatures = { - val tlvs: Set[TxSignaturesTlv] = Set( - previousFundingSig_opt.map { - case IndividualSignature(sig) => TxSignaturesTlv.PreviousFundingTxSig(sig) - case partialSig: PartialSignatureWithNonce => TxSignaturesTlv.PreviousFundingTxPartialSig(partialSig) - } - ).flatten + val tlvs: Set[TxSignaturesTlv] = previousFundingSig_opt match { + case Some(IndividualSignature(sig)) => Set(TxSignaturesTlv.PreviousFundingTxSig(sig), TxSignaturesTlv.ExperimentalPreviousFundingTxSig(sig)) + case Some(partialSig: PartialSignatureWithNonce) => Set(TxSignaturesTlv.PreviousFundingTxPartialSig(partialSig)) + case None => Set.empty + } TxSignatures(channelId, tx.txid, witnesses, TlvStream(tlvs)) } } @@ -409,6 +412,19 @@ object SpliceInit { apply(channelId, fundingContribution, lockTime, feerate, fundingPubKey, pushAmount, requireConfirmedInputs, requestFunding_opt, None) } +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, @@ -435,11 +451,32 @@ object SpliceAck { apply(channelId, fundingContribution, fundingPubKey, pushAmount, requireConfirmedInputs, willFund_opt, feeCreditUsed_opt, None) } +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 ForbiddenMessageWhenQuiescent { @@ -476,6 +513,15 @@ case class ClosingSig(channelId: ByteVector32, closerScriptPubKey: ByteVector, c val nextCloseeNonce_opt: Option[IndividualNonce] = tlvStream.get[ClosingSigTlv.NextCloseeNonce].map(_.nonce) } +/** This message is used to indicate that the next [[batchSize]] messages form a single logical message. */ +case class StartBatch(channelId: ByteVector32, batchSize: Int, tlvStream: TlvStream[StartBatchTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId { + val messageType_opt: Option[Long] = tlvStream.get[StartBatchTlv.MessageType].map(_.tag) +} + +object StartBatch { + def commitSigBatch(channelId: ByteVector32, batchSize: Int): StartBatch = StartBatch(channelId, batchSize, TlvStream(StartBatchTlv.MessageType(132))) +} + case class UpdateAddHtlc(channelId: ByteVector32, id: Long, amountMsat: MilliSatoshi, @@ -545,19 +591,21 @@ case class CommitSig(channelId: ByteVector32, signature: IndividualSignature, htlcSignatures: List[ByteVector64], tlvStream: TlvStream[CommitSigTlv] = TlvStream.empty) extends CommitSigs { + val fundingTxId_opt: Option[TxId] = tlvStream.get[CommitSigTlv.FundingTx].map(_.txId) val partialSignature_opt: Option[PartialSignatureWithNonce] = tlvStream.get[CommitSigTlv.PartialSignatureWithNonceTlv].map(_.partialSigWithNonce) val sigOrPartialSig: ChannelSpendSignature = partialSignature_opt.getOrElse(signature) } object CommitSig { - def apply(channelId: ByteVector32, signature: ChannelSpendSignature, htlcSignatures: List[ByteVector64], batchSize: Int): CommitSig = { + def apply(channelId: ByteVector32, fundingTxId: TxId, signature: ChannelSpendSignature, htlcSignatures: List[ByteVector64], batchSize: Int): CommitSig = { val (individualSig, partialSig_opt) = signature match { case sig: IndividualSignature => (sig, None) case psig: PartialSignatureWithNonce => (IndividualSignature(ByteVector64.Zeroes), Some(psig)) } val tlvs = Set( - if (batchSize > 1) Some(CommitSigTlv.BatchTlv(batchSize)) else None, - partialSig_opt.map(CommitSigTlv.PartialSignatureWithNonceTlv(_)) + Some(CommitSigTlv.FundingTx(fundingTxId)), + partialSig_opt.map(CommitSigTlv.PartialSignatureWithNonceTlv(_)), + if (batchSize > 1) Some(CommitSigTlv.ExperimentalBatchTlv(batchSize)) else None, ).flatten[CommitSigTlv] CommitSig(channelId, individualSig, htlcSignatures, TlvStream(tlvs)) } diff --git a/eclair-core/src/test/resources/nonreg/codecs/050001-DATA_WAIT_FOR_FUNDING_CONFIRMED/fundee/data.json b/eclair-core/src/test/resources/nonreg/codecs/050001-DATA_WAIT_FOR_FUNDING_CONFIRMED/fundee/data.json index 5aa1929ddc..e81bb1f2ff 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050001-DATA_WAIT_FOR_FUNDING_CONFIRMED/fundee/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050001-DATA_WAIT_FOR_FUNDING_CONFIRMED/fundee/data.json @@ -14,7 +14,6 @@ "initFeatures" : { "activated" : { "option_route_blinding" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -28,7 +27,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "activated" : { "option_route_blinding" : "optional", "option_provide_storage" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050001-DATA_WAIT_FOR_FUNDING_CONFIRMED/funder/data.json b/eclair-core/src/test/resources/nonreg/codecs/050001-DATA_WAIT_FOR_FUNDING_CONFIRMED/funder/data.json index 6d01ae414d..e2dc821fad 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050001-DATA_WAIT_FOR_FUNDING_CONFIRMED/funder/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050001-DATA_WAIT_FOR_FUNDING_CONFIRMED/funder/data.json @@ -15,7 +15,6 @@ "activated" : { "option_route_blinding" : "optional", "option_provide_storage" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "initFeatures" : { "activated" : { "option_route_blinding" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050002-DATA_WAIT_FOR_CHANNEL_READY/funder/data.json b/eclair-core/src/test/resources/nonreg/codecs/050002-DATA_WAIT_FOR_CHANNEL_READY/funder/data.json index 8e4b03ca79..7f499fd908 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050002-DATA_WAIT_FOR_CHANNEL_READY/funder/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050002-DATA_WAIT_FOR_CHANNEL_READY/funder/data.json @@ -15,7 +15,6 @@ "activated" : { "option_route_blinding" : "optional", "option_provide_storage" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "initFeatures" : { "activated" : { "option_route_blinding" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050003-DATA_WAIT_FOR_DUAL_FUNDING_SIGNED/fundee/data.json b/eclair-core/src/test/resources/nonreg/codecs/050003-DATA_WAIT_FOR_DUAL_FUNDING_SIGNED/fundee/data.json index 5e71bd01d4..59738810d5 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050003-DATA_WAIT_FOR_DUAL_FUNDING_SIGNED/fundee/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050003-DATA_WAIT_FOR_DUAL_FUNDING_SIGNED/fundee/data.json @@ -13,7 +13,6 @@ "activated" : { "option_route_blinding" : "optional", "option_dual_fund" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -27,7 +26,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "remoteParams" : { @@ -41,7 +40,6 @@ "option_route_blinding" : "optional", "option_provide_storage" : "optional", "option_dual_fund" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -55,7 +53,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050003-DATA_WAIT_FOR_DUAL_FUNDING_SIGNED/funder/data.json b/eclair-core/src/test/resources/nonreg/codecs/050003-DATA_WAIT_FOR_DUAL_FUNDING_SIGNED/funder/data.json index 7d6286ed80..f6a1688f94 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050003-DATA_WAIT_FOR_DUAL_FUNDING_SIGNED/funder/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050003-DATA_WAIT_FOR_DUAL_FUNDING_SIGNED/funder/data.json @@ -14,7 +14,6 @@ "option_route_blinding" : "optional", "option_provide_storage" : "optional", "option_dual_fund" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -28,7 +27,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "remoteParams" : { @@ -41,7 +40,6 @@ "activated" : { "option_route_blinding" : "optional", "option_dual_fund" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -55,7 +53,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050004-DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED/fundee/data.json b/eclair-core/src/test/resources/nonreg/codecs/050004-DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED/fundee/data.json index ebbbb2e890..d4816302a9 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050004-DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED/fundee/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050004-DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED/fundee/data.json @@ -14,7 +14,6 @@ "activated" : { "option_route_blinding" : "optional", "option_dual_fund" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -28,7 +27,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "option_route_blinding" : "optional", "option_provide_storage" : "optional", "option_dual_fund" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050004-DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED/funder/data.json b/eclair-core/src/test/resources/nonreg/codecs/050004-DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED/funder/data.json index 52f5323644..dc1be640d2 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050004-DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED/funder/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050004-DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED/funder/data.json @@ -15,7 +15,6 @@ "option_route_blinding" : "optional", "option_provide_storage" : "optional", "option_dual_fund" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "activated" : { "option_route_blinding" : "optional", "option_dual_fund" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050005-DATA_WAIT_FOR_DUAL_FUNDING_READY/funder/data.json b/eclair-core/src/test/resources/nonreg/codecs/050005-DATA_WAIT_FOR_DUAL_FUNDING_READY/funder/data.json index 6e0084590b..71ccf5885e 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050005-DATA_WAIT_FOR_DUAL_FUNDING_READY/funder/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050005-DATA_WAIT_FOR_DUAL_FUNDING_READY/funder/data.json @@ -15,7 +15,6 @@ "option_route_blinding" : "optional", "option_provide_storage" : "optional", "option_dual_fund" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "activated" : { "option_route_blinding" : "optional", "option_dual_fund" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/announced-splice/data.json b/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/announced-splice/data.json index 069f8d6a60..a27f23bfed 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/announced-splice/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/announced-splice/data.json @@ -15,7 +15,6 @@ "option_route_blinding" : "optional", "option_provide_storage" : "optional", "option_dual_fund" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "activated" : { "option_route_blinding" : "optional", "option_dual_fund" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/announced/data.json b/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/announced/data.json index 7a3a08b754..068bb4a46e 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/announced/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/announced/data.json @@ -15,7 +15,6 @@ "activated" : { "option_route_blinding" : "optional", "option_provide_storage" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "initFeatures" : { "activated" : { "option_route_blinding" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/fundee/data.json b/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/fundee/data.json index 0117c91e38..eb527a93e5 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/fundee/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/fundee/data.json @@ -14,7 +14,6 @@ "initFeatures" : { "activated" : { "option_route_blinding" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -28,7 +27,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "activated" : { "option_route_blinding" : "optional", "option_provide_storage" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/funder/data.json b/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/funder/data.json index bca7cfcc95..22979e2a55 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/funder/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/funder/data.json @@ -15,7 +15,6 @@ "activated" : { "option_route_blinding" : "optional", "option_provide_storage" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "initFeatures" : { "activated" : { "option_route_blinding" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/splice-commitment-upgrade/data.json b/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/splice-commitment-upgrade/data.json index 7abd5d5000..04208533ef 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/splice-commitment-upgrade/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050006-DATA_NORMAL/splice-commitment-upgrade/data.json @@ -15,7 +15,6 @@ "option_route_blinding" : "optional", "option_provide_storage" : "optional", "option_dual_fund" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "activated" : { "option_route_blinding" : "optional", "option_dual_fund" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050007-DATA_SHUTDOWN/anchor-outputs/data.json b/eclair-core/src/test/resources/nonreg/codecs/050007-DATA_SHUTDOWN/anchor-outputs/data.json index 607511d322..9805e4f754 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050007-DATA_SHUTDOWN/anchor-outputs/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050007-DATA_SHUTDOWN/anchor-outputs/data.json @@ -15,7 +15,6 @@ "activated" : { "option_route_blinding" : "optional", "option_provide_storage" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "initFeatures" : { "activated" : { "option_route_blinding" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050007-DATA_SHUTDOWN/taproot/data.json b/eclair-core/src/test/resources/nonreg/codecs/050007-DATA_SHUTDOWN/taproot/data.json index 0bd1a05109..5c830e3dc2 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050007-DATA_SHUTDOWN/taproot/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050007-DATA_SHUTDOWN/taproot/data.json @@ -16,7 +16,6 @@ "option_simple_close" : "optional", "option_simple_taproot_phoenix" : "optional", "option_route_blinding" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_quiesce" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "remoteParams" : { @@ -46,7 +45,6 @@ "option_simple_taproot_phoenix" : "optional", "option_route_blinding" : "optional", "option_provide_storage" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_quiesce" : "optional", @@ -57,7 +55,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050008-DATA_NEGOTIATING/fundee/data.json b/eclair-core/src/test/resources/nonreg/codecs/050008-DATA_NEGOTIATING/fundee/data.json index 869b350741..b64698af7e 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050008-DATA_NEGOTIATING/fundee/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050008-DATA_NEGOTIATING/fundee/data.json @@ -14,7 +14,6 @@ "initFeatures" : { "activated" : { "option_route_blinding" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -28,7 +27,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "activated" : { "option_route_blinding" : "optional", "option_provide_storage" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050009-DATA_NEGOTIATING_SIMPLE/anchor-outputs/data.json b/eclair-core/src/test/resources/nonreg/codecs/050009-DATA_NEGOTIATING_SIMPLE/anchor-outputs/data.json index 5ced7c1191..030e7a1fae 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050009-DATA_NEGOTIATING_SIMPLE/anchor-outputs/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050009-DATA_NEGOTIATING_SIMPLE/anchor-outputs/data.json @@ -15,7 +15,6 @@ "activated" : { "option_simple_close" : "optional", "option_route_blinding" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "remoteParams" : { @@ -44,7 +43,6 @@ "option_simple_close" : "optional", "option_route_blinding" : "optional", "option_provide_storage" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -58,7 +56,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/050009-DATA_NEGOTIATING_SIMPLE/taproot/data.json b/eclair-core/src/test/resources/nonreg/codecs/050009-DATA_NEGOTIATING_SIMPLE/taproot/data.json index 7feae98792..ad763be66f 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/050009-DATA_NEGOTIATING_SIMPLE/taproot/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/050009-DATA_NEGOTIATING_SIMPLE/taproot/data.json @@ -18,7 +18,6 @@ "option_simple_taproot_phoenix" : "optional", "option_route_blinding" : "optional", "option_provide_storage" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_quiesce" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "remoteParams" : { @@ -44,7 +43,6 @@ "option_simple_close" : "optional", "option_simple_taproot_phoenix" : "optional", "option_route_blinding" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_quiesce" : "optional", @@ -57,7 +55,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/local/data.json b/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/local/data.json index 96ab01a0f0..4163c6d7bd 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/local/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/local/data.json @@ -15,7 +15,6 @@ "activated" : { "option_route_blinding" : "optional", "option_provide_storage" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "initFeatures" : { "activated" : { "option_route_blinding" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/next-remote/data.json b/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/next-remote/data.json index 8a2b872db9..90da17f705 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/next-remote/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/next-remote/data.json @@ -15,7 +15,6 @@ "activated" : { "option_route_blinding" : "optional", "option_provide_storage" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "initFeatures" : { "activated" : { "option_route_blinding" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/remote/data.json b/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/remote/data.json index 603860aefc..9a26936c2c 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/remote/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/remote/data.json @@ -15,7 +15,6 @@ "activated" : { "option_route_blinding" : "optional", "option_provide_storage" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "initFeatures" : { "activated" : { "option_route_blinding" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "channelFlags" : { diff --git a/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/revoked/data.json b/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/revoked/data.json index 6243f0f317..ec3a3fa111 100644 --- a/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/revoked/data.json +++ b/eclair-core/src/test/resources/nonreg/codecs/05000a-DATA_CLOSING/revoked/data.json @@ -15,7 +15,6 @@ "activated" : { "option_route_blinding" : "optional", "option_provide_storage" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -29,7 +28,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ 50001 ] + "unknown" : [ 155, 50001 ] } }, "remoteParams" : { @@ -42,7 +41,6 @@ "initFeatures" : { "activated" : { "option_route_blinding" : "optional", - "splice_prototype" : "optional", "payment_secret" : "mandatory", "gossip_queries_ex" : "optional", "option_anchor_outputs" : "optional", @@ -56,7 +54,7 @@ "basic_mpp" : "optional", "gossip_queries" : "optional" }, - "unknown" : [ ] + "unknown" : [ 155 ] } }, "channelFlags" : { 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 97b8a51464..973f4ebd13 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala @@ -115,7 +115,7 @@ object TestConstants { Features.StaticRemoteKey -> FeatureSupport.Mandatory, Features.AnchorOutputsZeroFeeHtlcTx -> FeatureSupport.Optional, Features.Quiescence -> FeatureSupport.Optional, - Features.SplicePrototype -> FeatureSupport.Optional, + Features.Splicing -> FeatureSupport.Optional, Features.ProvideStorage -> FeatureSupport.Optional, Features.ChannelType -> FeatureSupport.Mandatory ), @@ -309,7 +309,7 @@ object TestConstants { Features.StaticRemoteKey -> FeatureSupport.Mandatory, Features.AnchorOutputsZeroFeeHtlcTx -> FeatureSupport.Optional, Features.Quiescence -> FeatureSupport.Optional, - Features.SplicePrototype -> FeatureSupport.Optional, + Features.Splicing -> FeatureSupport.Optional, Features.ChannelType -> FeatureSupport.Mandatory ), pluginParams = Nil, 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 bcddf81d43..bb64e14fbd 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 @@ -2954,7 +2954,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit bob ! ReceiveMessage(alice2bob.expectMsgType[SendMessage].msg.asInstanceOf[TxComplete]) // Alice <-- commit_sig --- Bob val successA1 = alice2bob.expectMsgType[Succeeded] - val invalidCommitSig = CommitSig(params.channelId, PartialSignatureWithNonce(randomBytes32(), txCompleteBob.commitNonces_opt.get.commitNonce), Nil, batchSize = 1) + val invalidCommitSig = CommitSig(params.channelId, successA1.signingSession.fundingTxId, PartialSignatureWithNonce(randomBytes32(), txCompleteBob.commitNonces_opt.get.commitNonce), Nil, batchSize = 1) val Left(error) = successA1.signingSession.receiveCommitSig(params.channelParamsA, params.channelKeysA, invalidCommitSig, params.nodeParamsA.currentBlockHeight)(akka.event.NoLogging) assert(error.isInstanceOf[InvalidCommitmentSignature]) } 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 a2aa6ae263..4ece0af547 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 @@ -273,7 +273,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually { .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.SimpleClose))(_.updated(Features.SimpleClose, FeatureSupport.Optional)) - .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.DisableSplice))(_.removed(Features.SplicePrototype)) + .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.DisableSplice))(_.removed(Features.Splicing)) .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.AnchorOutputsPhoenix))(_.removed(Features.AnchorOutputsZeroFeeHtlcTx).updated(Features.AnchorOutputs, FeatureSupport.Optional)) .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.OptionSimpleTaprootPhoenix))(_.removed(Features.SimpleTaprootChannelsStaging).updated(Features.SimpleTaprootChannelsPhoenix, FeatureSupport.Optional).updated(Features.PhoenixZeroReserve, FeatureSupport.Optional)) .modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.OptionSimpleTaproot))(_.updated(Features.SimpleTaprootChannelsStaging, FeatureSupport.Optional)) 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 9f997cb960..0d5bb006a7 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 @@ -1562,16 +1562,18 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice ! CMD_SIGN() val sigsA = alice2bob.expectMsgType[CommitSigBatch] assert(sigsA.batchSize == 2) + assert(sigsA.messages.flatMap(_.fundingTxId_opt).toSet == alice.commitments.active.map(_.fundingTxId).toSet) alice2bob.forward(bob, sigsA) bob2alice.expectMsgType[RevokeAndAck] bob2alice.forward(alice) val sigsB = bob2alice.expectMsgType[CommitSigBatch] assert(sigsB.batchSize == 2) + assert(sigsB.messages.flatMap(_.fundingTxId_opt).toSet == alice.commitments.active.map(_.fundingTxId).toSet) bob2alice.forward(alice, sigsB) alice2bob.expectMsgType[RevokeAndAck] alice2bob.forward(bob) - awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.forall(_.localCommit.spec.htlcs.size == 1)) - awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.forall(_.localCommit.spec.htlcs.size == 1)) + awaitCond(alice.commitments.active.forall(_.localCommit.spec.htlcs.size == 1)) + awaitCond(bob.commitments.active.forall(_.localCommit.spec.htlcs.size == 1)) } test("recv CMD_ADD_HTLC with multiple commitments (missing nonces)", Tag(ChannelStateTestsTags.OptionSimpleTaproot)) { f => @@ -1619,11 +1621,13 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice2bob.forward(bob) val sigsA = alice2bob.expectMsgType[CommitSigBatch] assert(sigsA.batchSize == 2) + assert(sigsA.messages.flatMap(_.fundingTxId_opt).toSet == alice.commitments.active.map(_.fundingTxId).toSet) alice2bob.forward(bob, sigsA) assert(bob2alice.expectMsgType[RevokeAndAck].nextCommitNonces.size == 2) bob2alice.forward(alice) val sigsB = bob2alice.expectMsgType[CommitSigBatch] assert(sigsB.batchSize == 2) + assert(sigsB.messages.flatMap(_.fundingTxId_opt).toSet == alice.commitments.active.map(_.fundingTxId).toSet) bob2alice.forward(alice, sigsB) assert(alice2bob.expectMsgType[RevokeAndAck].nextCommitNonces.size == 2) alice2bob.forward(bob) @@ -1752,7 +1756,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice2bob.forward(bob, commitSigsAlice) bob2alice.expectMsgType[RevokeAndAck] bob2alice.forward(alice) - bob2alice.expectMsgType[CommitSig] + assert(bob2alice.expectMsgType[CommitSig].fundingTxId_opt.contains(spliceTx2.txid)) bob2alice.forward(alice) alice2bob.expectMsgType[RevokeAndAck] alice2bob.forward(bob) @@ -1856,10 +1860,12 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val bobCommitIndex = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommitIndex val sender = initiateSpliceWithoutSigs(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey))) - alice2bob.expectMsgType[CommitSig] // Bob doesn't receive Alice's commit_sig - bob2alice.expectMsgType[CommitSig] // Alice doesn't receive Bob's commit_sig + val commitSigA = alice2bob.expectMsgType[CommitSig] // Bob doesn't receive Alice's commit_sig + val commitSigB = bob2alice.expectMsgType[CommitSig] // Alice doesn't receive Bob's commit_sig awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus.isInstanceOf[SpliceStatus.SpliceWaitingForSigs]) val spliceStatus = alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus.asInstanceOf[SpliceStatus.SpliceWaitingForSigs] + assert(commitSigA.fundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId)) + assert(commitSigB.fundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId)) disconnect(f) @@ -1872,7 +1878,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice2bob.forward(bob, channelReestablishAlice1) bob2alice.forward(alice, channelReestablishBob1.copy(nextLocalCommitmentNumber = bobCommitIndex + 1)) // In that case Alice won't retransmit commit_sig and the splice won't complete since they haven't exchanged tx_signatures. - bob2alice.expectMsgType[CommitSig] + assert(bob2alice.expectMsgType[CommitSig].fundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId)) bob2alice.forward(alice) alice2bob.expectNoMessage(100 millis) assert(alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus.isInstanceOf[SpliceStatus.SpliceWaitingForSigs]) @@ -1893,7 +1899,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik assert(channelReestablishBob2.nextLocalCommitmentNumber == bobCommitIndex) // Alice retransmits commit_sig and both retransmit tx_signatures. - alice2bob.expectMsgType[CommitSig] + assert(alice2bob.expectMsgType[CommitSig].fundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId)) alice2bob.forward(bob) bob2alice.expectMsgType[TxSignatures] bob2alice.forward(alice) @@ -1902,6 +1908,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik sender.expectMsgType[RES_SPLICE] val spliceTx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get + assert(spliceTx.txid == spliceStatus.signingSession.fundingTx.txId) alice2blockchain.expectWatchFundingConfirmed(spliceTx.txid) bob2blockchain.expectWatchFundingConfirmed(spliceTx.txid) alice ! WatchFundingConfirmedTriggered(BlockHeight(42), 0, spliceTx) @@ -2761,11 +2768,14 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice2bob.expectMsgType[TxSignatures] // Bob doesn't receive Alice's tx_signatures awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus == SpliceStatus.NoSplice) + val fundingTxIds = alice.commitments.active.map(_.fundingTxId).toSet + assert(fundingTxIds.size == 2) val (_, cmd) = makeCmdAdd(25_000_000 msat, bob.nodeParams.nodeId, bob.nodeParams.currentBlockHeight) alice ! cmd.copy(commit = true) alice2bob.expectMsgType[UpdateAddHtlc] // Bob doesn't receive Alice's update_add_htlc inside(alice2bob.expectMsgType[CommitSigBatch]) { batch => // Bob doesn't receive Alice's commit_sigs assert(batch.batchSize == 2) + assert(batch.messages.flatMap(_.fundingTxId_opt).toSet == fundingTxIds) } alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) @@ -2781,12 +2791,14 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice2bob.forward(bob) inside(alice2bob.expectMsgType[CommitSigBatch]) { batch => assert(batch.batchSize == 2) + assert(batch.messages.flatMap(_.fundingTxId_opt).toSet == fundingTxIds) alice2bob.forward(bob) } bob2alice.expectMsgType[RevokeAndAck] bob2alice.forward(alice) inside(bob2alice.expectMsgType[CommitSigBatch]) { batch => assert(batch.batchSize == 2) + assert(batch.messages.flatMap(_.fundingTxId_opt).toSet == fundingTxIds) bob2alice.forward(alice) } alice2bob.expectMsgType[RevokeAndAck] @@ -2839,12 +2851,12 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val sender = TestProbe() alice ! CMD_SIGN(Some(sender.ref)) sender.expectMsgType[RES_SUCCESS[CMD_SIGN]] - alice2bob.expectMsgType[CommitSig] + assert(alice2bob.expectMsgType[CommitSig].fundingTxId_opt.contains(fundingTx.txid)) alice2bob.forward(bob) bob2alice.forward(alice, bobSpliceLocked) bob2alice.expectMsgType[RevokeAndAck] bob2alice.forward(alice) - bob2alice.expectMsgType[CommitSig] + assert(bob2alice.expectMsgType[CommitSig].fundingTxId_opt.contains(fundingTx.txid)) bob2alice.forward(alice) alice2bob.expectMsgType[RevokeAndAck] alice2bob.forward(bob) @@ -2899,7 +2911,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik assert(bob.commitments.active.size == 1) alice2bob.expectMsgType[UpdateAddHtlc] alice2bob.forward(bob) - assert(alice2bob.expectMsgType[CommitSig].tlvStream.get[CommitSigTlv.BatchTlv].isEmpty) + assert(alice2bob.expectMsgType[CommitSig].tlvStream.get[CommitSigTlv.ExperimentalBatchTlv].isEmpty) alice2bob.forward(bob) bob2alice.expectMsgType[RevokeAndAck] bob2alice.forward(alice) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/channel/GossipIntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/channel/GossipIntegrationSpec.scala index 24bcd32ebe..1eb6eab12c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/channel/GossipIntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/channel/GossipIntegrationSpec.scala @@ -25,9 +25,7 @@ class GossipIntegrationSpec extends FixtureSpec with IntegrationPatience { override def createFixture(testData: TestData): FixtureParam = { // seeds have been chosen so that node ids start with 02aaaa for alice, 02bbbb for bob, etc. val aliceParams = nodeParamsFor("alice", ByteVector32(hex"b4acd47335b25ab7b84b8c020997b12018592bb4631b868762154d77fa8b93a3")) - .modify(_.features).using(_.add(Features.SplicePrototype, FeatureSupport.Optional)) val bobParams = nodeParamsFor("bob", ByteVector32(hex"7620226fec887b0b2ebe76492e5a3fd3eb0e47cd3773263f6a81b59a704dc492")) - .modify(_.features).using(_.add(Features.SplicePrototype, FeatureSupport.Optional)) val carolParams = nodeParamsFor("carol", ByteVector32(hex"ebd5a5d3abfb3ef73731eb3418d918f247445183180522674666db98a66411cc")) ThreeNodesFixture(aliceParams, bobParams, carolParams, testData.name) } 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 6b582ec107..6f6f47689c 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 @@ -118,7 +118,7 @@ class OpenChannelInterceptorSpec extends ScalaTestWithActorTestKit(ConfigFactory test("add liquidity if on-the-fly funding is used", Tag(noPlugin)) { f => import f._ - val features = defaultFeatures.add(Features.SplicePrototype, FeatureSupport.Optional).add(Features.OnTheFlyFunding, FeatureSupport.Optional) + val features = defaultFeatures.add(Features.Splicing, FeatureSupport.Optional).add(Features.OnTheFlyFunding, FeatureSupport.Optional) val requestFunding = LiquidityAds.RequestFunding(250_000 sat, TestConstants.defaultLiquidityRates.fundingRates.head, LiquidityAds.PaymentDetails.FromChannelBalanceForFutureHtlc(randomBytes32() :: Nil)) val open = createOpenDualFundedChannelMessage(ChannelTypes.AnchorOutputsZeroFeeHtlcTx(), Some(requestFunding)).copy( channelFlags = ChannelFlags(nonInitiatorPaysCommitFees = true, announceChannel = false), @@ -210,7 +210,7 @@ class OpenChannelInterceptorSpec extends ScalaTestWithActorTestKit(ConfigFactory test("reject on-the-fly channel if another channel exists", Tag(noPlugin)) { f => import f._ - val features = defaultFeatures.add(Features.SplicePrototype, FeatureSupport.Optional).add(Features.OnTheFlyFunding, FeatureSupport.Optional) + val features = defaultFeatures.add(Features.Splicing, FeatureSupport.Optional).add(Features.OnTheFlyFunding, FeatureSupport.Optional) val requestFunding = LiquidityAds.RequestFunding(250_000 sat, TestConstants.defaultLiquidityRates.fundingRates.head, LiquidityAds.PaymentDetails.FromChannelBalanceForFutureHtlc(randomBytes32() :: Nil)) val open = createOpenDualFundedChannelMessage(ChannelTypes.AnchorOutputsZeroFeeHtlcTx(), Some(requestFunding)).copy( channelFlags = ChannelFlags(nonInitiatorPaysCommitFees = true, announceChannel = false), 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 33801cdb8a..6153e26d8a 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,11 +19,12 @@ 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._ import fr.acinq.eclair.TestConstants._ import fr.acinq.eclair.TestUtils.randomTxId +import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.channel.ChannelSpendSignature.IndividualSignature import fr.acinq.eclair.crypto.TransportHandler import fr.acinq.eclair.io.Peer.ConnectionDown @@ -346,6 +347,8 @@ class PeerConnectionSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wi CommitSig(channelId, IndividualSignature(randomBytes64()), Nil), ) probe.send(peerConnection, CommitSigBatch(commitSigs)) + // We insert a start_batch message. + transport.expectMsg(StartBatch(channelId, batchSize = 3, TlvStream(StartBatchTlv.MessageType(132)))) commitSigs.foreach(commitSig => transport.expectMsg(commitSig)) transport.expectNoMessage(100 millis) } @@ -357,8 +360,8 @@ class PeerConnectionSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wi // We receive a batch of commit_sig messages from a first channel. val channelId1 = randomBytes32() val commitSigs1 = Seq( - CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, batchSize = 2), - CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, batchSize = 2), + CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.ExperimentalBatchTlv(2))), + CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.ExperimentalBatchTlv(2))), ) transport.send(peerConnection, commitSigs1.head) transport.expectMsg(TransportHandler.ReadAck(commitSigs1.head)) @@ -370,9 +373,9 @@ class PeerConnectionSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wi // We receive a batch of commit_sig messages from a second channel. val channelId2 = randomBytes32() val commitSigs2 = Seq( - CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, batchSize = 3), - CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, batchSize = 3), - CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, batchSize = 3), + CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.ExperimentalBatchTlv(3))), + CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.ExperimentalBatchTlv(3))), + CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.ExperimentalBatchTlv(3))), ) commitSigs2.dropRight(1).foreach(commitSig => { transport.send(peerConnection, commitSig) @@ -385,8 +388,8 @@ class PeerConnectionSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wi // We receive another batch of commit_sig messages from the first channel, with unrelated messages in the batch. val commitSigs3 = Seq( - CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, batchSize = 2), - CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, batchSize = 2), + CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.ExperimentalBatchTlv(2))), + CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.ExperimentalBatchTlv(2))), ) transport.send(peerConnection, commitSigs3.head) transport.expectMsg(TransportHandler.ReadAck(commitSigs3.head)) @@ -406,9 +409,9 @@ class PeerConnectionSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wi // We start receiving a batch of commit_sig messages from the first channel, interleaved with a batch from the second // channel, which is not supported. val commitSigs4 = Seq( - CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, batchSize = 2), - CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, batchSize = 2), - CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, batchSize = 2), + CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.ExperimentalBatchTlv(2))), + CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.ExperimentalBatchTlv(2))), + CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.ExperimentalBatchTlv(2))), ) transport.send(peerConnection, commitSigs4.head) transport.expectMsg(TransportHandler.ReadAck(commitSigs4.head)) @@ -421,7 +424,7 @@ class PeerConnectionSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wi peer.expectMsg(CommitSigBatch(commitSigs4.tail)) // We receive a batch that exceeds our threshold: we process them individually. - val invalidCommitSigs = (0 until 30).map(_ => CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, batchSize = 30)) + val invalidCommitSigs = (0 until 30).map(_ => CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.ExperimentalBatchTlv(30)))) invalidCommitSigs.foreach(commitSig => { transport.send(peerConnection, commitSig) transport.expectMsg(TransportHandler.ReadAck(commitSig)) @@ -429,6 +432,135 @@ class PeerConnectionSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wi }) } + test("receive batch of commit_sig messages") { f => + import f._ + connect(nodeParams, remoteNodeId, switchboard, router, connection, transport, peerConnection, peer) + + // A first channel has a pending splice. + val channelId1 = randomBytes32() + val startBatch1 = StartBatch(channelId1, batchSize = 2, TlvStream(StartBatchTlv.MessageType(132))) + val commitSigs1 = Seq( + CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(randomTxId()))), + CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(randomTxId()))), + ) + transport.send(peerConnection, startBatch1) + transport.expectMsg(TransportHandler.ReadAck(startBatch1)) + transport.send(peerConnection, commitSigs1(0)) + transport.expectMsg(TransportHandler.ReadAck(commitSigs1(0))) + peer.expectNoMessage(100 millis) + transport.send(peerConnection, commitSigs1(1)) + transport.expectMsg(TransportHandler.ReadAck(commitSigs1(1))) + peer.expectMsg(CommitSigBatch(commitSigs1)) + + // Another channel has 2 pending splices. + val channelId2 = randomBytes32() + val startBatch2 = StartBatch(channelId2, batchSize = 3, TlvStream(StartBatchTlv.MessageType(132))) + val commitSigs2 = Seq( + CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(randomTxId()))), + CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(randomTxId()))), + CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(randomTxId()))), + ) + transport.send(peerConnection, startBatch2) + transport.expectMsg(TransportHandler.ReadAck(startBatch2)) + transport.send(peerConnection, commitSigs2(0)) + transport.expectMsg(TransportHandler.ReadAck(commitSigs2(0))) + transport.send(peerConnection, commitSigs2(1)) + transport.expectMsg(TransportHandler.ReadAck(commitSigs2(1))) + peer.expectNoMessage(100 millis) + transport.send(peerConnection, commitSigs2(2)) + transport.expectMsg(TransportHandler.ReadAck(commitSigs2(2))) + peer.expectMsg(CommitSigBatch(commitSigs2)) + + // We receive another batch of commit_sig messages for the first channel. + val startBatch3 = StartBatch(channelId1, batchSize = 3, TlvStream(StartBatchTlv.MessageType(132))) + val commitSigs3 = Seq( + CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(randomTxId()))), + CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(randomTxId()))), + CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(randomTxId()))), + ) + transport.send(peerConnection, startBatch3) + transport.expectMsg(TransportHandler.ReadAck(startBatch3)) + transport.send(peerConnection, commitSigs3(0)) + transport.expectMsg(TransportHandler.ReadAck(commitSigs3(0))) + transport.send(peerConnection, commitSigs3(1)) + transport.expectMsg(TransportHandler.ReadAck(commitSigs3(1))) + peer.expectNoMessage(100 millis) + transport.send(peerConnection, commitSigs3(2)) + transport.expectMsg(TransportHandler.ReadAck(commitSigs3(2))) + peer.expectMsg(CommitSigBatch(commitSigs3)) + + // We reject batches that mix unrelated channels. + val startBatch4 = StartBatch(channelId1, batchSize = 3, TlvStream(StartBatchTlv.MessageType(132))) + val commitSigs4 = Seq( + CommitSig(channelId1, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(randomTxId()))), + CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(randomTxId()))), + CommitSig(channelId2, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(randomTxId()))), + ) + transport.send(peerConnection, startBatch4) + transport.expectMsg(TransportHandler.ReadAck(startBatch4)) + transport.send(peerConnection, commitSigs4(0)) + transport.expectMsg(TransportHandler.ReadAck(commitSigs4(0))) + peer.expectNoMessage(100 millis) + transport.send(peerConnection, commitSigs4(1)) + transport.expectMsg(TransportHandler.ReadAck(commitSigs4(1))) + peer.expectMsg(CommitSigBatch(commitSigs4.take(1))) + peer.expectMsg(commitSigs4(1)) + peer.expectNoMessage(100 millis) + transport.send(peerConnection, commitSigs4(2)) + transport.expectMsg(TransportHandler.ReadAck(commitSigs4(2))) + peer.expectMsg(commitSigs4(2)) + peer.expectNoMessage(100 millis) + } + + test("receive unsupported batch of channel messages") { f => + import f._ + connect(nodeParams, remoteNodeId, switchboard, router, connection, transport, peerConnection, peer) + + // We receive a batch of splice_locked messages: we forward them individually. + val channelId = randomBytes32() + val startBatch1 = StartBatch(channelId, batchSize = 2) // note that start_batch doesn't indicate the message type + val spliceLocked1 = SpliceLocked(channelId, randomTxId()) + val spliceLocked2 = SpliceLocked(channelId, randomTxId()) + transport.send(peerConnection, startBatch1) + transport.expectMsg(TransportHandler.ReadAck(startBatch1)) + transport.expectMsgType[Warning] + transport.send(peerConnection, spliceLocked1) + transport.expectMsg(TransportHandler.ReadAck(spliceLocked1)) + peer.expectMsg(spliceLocked1) + transport.send(peerConnection, spliceLocked2) + transport.expectMsg(TransportHandler.ReadAck(spliceLocked2)) + peer.expectMsg(spliceLocked2) + + // We receive a batch containing commit_sig and an unrelated message. + val startBatch2 = StartBatch(channelId, batchSize = 3, TlvStream(StartBatchTlv.MessageType(132))) + val commitSig1 = CommitSig(channelId, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(randomTxId()))) + val commitSig2 = CommitSig(channelId, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(randomTxId()))) + val spliceLocked3 = SpliceLocked(channelId, randomTxId()) + transport.send(peerConnection, startBatch2) + transport.expectMsg(TransportHandler.ReadAck(startBatch2)) + transport.send(peerConnection, commitSig1) + transport.expectMsg(TransportHandler.ReadAck(commitSig1)) + transport.send(peerConnection, commitSig2) + transport.expectMsg(TransportHandler.ReadAck(commitSig2)) + peer.expectNoMessage(100 millis) + transport.send(peerConnection, spliceLocked3) + transport.expectMsg(TransportHandler.ReadAck(spliceLocked3)) + peer.expectMsg(CommitSigBatch(commitSig1 :: commitSig2 :: Nil)) + peer.expectMsg(spliceLocked3) + peer.expectNoMessage(100 millis) + + // We receive a batch exceeding 20 elements: we relay messages individually. + val startBatch3 = StartBatch(channelId, batchSize = 21, TlvStream(StartBatchTlv.MessageType(132))) + val commitSig3 = CommitSig(channelId, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(randomTxId()))) + transport.send(peerConnection, startBatch3) + transport.expectMsg(TransportHandler.ReadAck(startBatch3)) + transport.expectMsgType[Warning] + transport.send(peerConnection, commitSig3) + transport.expectMsg(TransportHandler.ReadAck(commitSig3)) + peer.expectMsg(commitSig3) + peer.expectNoMessage(100 millis) + } + test("react to peer's bad behavior") { f => import f._ val probe = TestProbe() @@ -592,5 +724,53 @@ 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(IndividualSignature(randomBytes64()))) + assert(txSignatures.tlvStream.get[TxSignaturesTlv.PreviousFundingTxSig].nonEmpty) + peer.send(peerConnection, txSignatures) + assert(transport.expectMsgType[TxSignatures].tlvStream.get[TxSignaturesTlv.PreviousFundingTxSig].isEmpty) + val channelId = randomBytes32() + val commitSigBatch = CommitSigBatch(Seq( + CommitSig(channelId, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(TxId(randomBytes32())), CommitSigTlv.ExperimentalBatchTlv(2))), + CommitSig(channelId, IndividualSignature(randomBytes64()), Nil, TlvStream(CommitSigTlv.FundingTx(TxId(randomBytes32())), CommitSigTlv.ExperimentalBatchTlv(2))), + )) + peer.send(peerConnection, commitSigBatch) + assert(transport.expectMsgType[CommitSig].tlvStream.get[CommitSigTlv.FundingTx].isEmpty) + assert(transport.expectMsgType[CommitSig].tlvStream.get[CommitSigTlv.FundingTx].isEmpty) + transport.expectNoMessage(100 millis) + } + } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/OnTheFlyFundingSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/OnTheFlyFundingSpec.scala index da6f822b49..95642f4731 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/OnTheFlyFundingSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/OnTheFlyFundingSpec.scala @@ -51,13 +51,13 @@ class OnTheFlyFundingSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike { Features.StaticRemoteKey -> FeatureSupport.Optional, Features.AnchorOutputsZeroFeeHtlcTx -> FeatureSupport.Optional, Features.DualFunding -> FeatureSupport.Optional, - Features.SplicePrototype -> FeatureSupport.Optional, + Features.Splicing -> FeatureSupport.Optional, Features.OnTheFlyFunding -> FeatureSupport.Optional, ) val remoteFeaturesWithFeeCredit = Features( Features.DualFunding -> FeatureSupport.Optional, - Features.SplicePrototype -> FeatureSupport.Optional, + Features.Splicing -> FeatureSupport.Optional, Features.OnTheFlyFunding -> FeatureSupport.Optional, Features.FundingFeeCredit -> FeatureSupport.Optional, ) @@ -183,7 +183,7 @@ class OnTheFlyFundingSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike { val nodeParams = TestConstants.Alice.nodeParams .modify(_.features.activated).using(_ + (Features.AnchorOutputsZeroFeeHtlcTx -> FeatureSupport.Optional)) .modify(_.features.activated).using(_ + (Features.DualFunding -> FeatureSupport.Optional)) - .modify(_.features.activated).using(_ + (Features.SplicePrototype -> FeatureSupport.Optional)) + .modify(_.features.activated).using(_ + (Features.Splicing -> FeatureSupport.Optional)) .modify(_.features.activated).using(_ + (Features.OnTheFlyFunding -> FeatureSupport.Optional)) .modify(_.features.activated).usingIf(test.tags.contains(withFeeCredit))(_ + (Features.FundingFeeCredit -> FeatureSupport.Optional)) val remoteNodeId = randomKey().publicKey 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 589ca88ece..9d502b9933 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 @@ -169,7 +169,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, IndividualSignature(signature), Nil), - hex"0084" ++ channelId ++ ByteVector64.Zeroes ++ hex"0000" ++ hex"02 62" ++ partialSig ++ nonce.data -> CommitSig(channelId, PartialSignatureWithNonce(partialSig, nonce), Nil, batchSize = 1), + hex"0084" ++ channelId ++ ByteVector64.Zeroes ++ hex"0000" ++ hex"01 20" ++ txId.value.reverse ++ hex"02 62" ++ partialSig ++ nonce.data -> CommitSig(channelId, txId, PartialSignatureWithNonce(partialSig, nonce), Nil, batchSize = 1), hex"0084" ++ channelId ++ signature ++ hex"0000 fe47010000 00" -> CommitSig(channelId, IndividualSignature(signature), Nil, TlvStream[CommitSigTlv](Set.empty[CommitSigTlv], Set(GenericTlv(tlvTag, ByteVector.empty)))), hex"0084" ++ channelId ++ signature ++ hex"0000 fe47010000 07 cccccccccccccc" -> CommitSig(channelId, IndividualSignature(signature), Nil, TlvStream[CommitSigTlv](Set.empty[CommitSigTlv], Set(GenericTlv(tlvTag, hex"cccccccccccccc")))), @@ -223,7 +223,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", TxAddInput(channelId1, UInt64(561), None, 1, 0xfffffffdL, TlvStream(TxAddInputTlv.PrevTxOut(tx2.txid, 22_549_834 sat, hex"00148d2e0b57adcb8869e603fd35b5179caf05336125"))) -> hex"0042 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000000231 0000 00000001 fffffffd fd04573efc7aa8845f192959202c1b7ff704e7cbddded463c05e844676a94ccb4bed69f1000000000158154a00148d2e0b57adcb8869e603fd35b5179caf05336125", 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", @@ -235,7 +235,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(IndividualSignature(signature))) -> hex"0047 bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb 1f2ec025a33e39ef8e177afcdc1adc855bf128dc906182255aeb64efa825f106 0000 fd0259 40 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb", + TxSignatures(channelId2, tx1, Nil, Some(IndividualSignature(signature))) -> hex"0047 bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb 1f2ec025a33e39ef8e177afcdc1adc855bf128dc906182255aeb64efa825f106 0000 0040aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb fd025940aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb", TxSignatures(channelId2, tx1, Nil, Some(PartialSignatureWithNonce(partialSig, fundingNonce))) -> hex"0047 bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb 1f2ec025a33e39ef8e177afcdc1adc855bf128dc906182255aeb64efa825f106 0000 02 62 bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb a49ff67b08c720b993c946556cde1be1c3b664bc847c4792135dfd6ef0986e00e9871808c6620b0420567dad525b27431453d4434fd326f8ac56496639b72326eb5d", TxInitRbf(channelId1, 8388607, FeeratePerKw(4000 sat)) -> hex"0048 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 007fffff 00000fa0", TxInitRbf(channelId1, 0, FeeratePerKw(4000 sat), 1_500_000 sat, requireConfirmedInputs = true, None) -> hex"0048 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000 00000fa0 0008000000000016e360 0200", @@ -415,29 +415,47 @@ class LightningMessageCodecsSpec extends AnyFunSuite { val channelId = ByteVector32(hex"aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa") val fundingTxId = TxId(TxHash(ByteVector32(hex"24e1b2c94c4e734dd5b9c5f3c910fbb6b3b436ced6382c7186056a5a23f14566"))) val fundingPubkey = PublicKey(hex"0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798") - val fundingRate = LiquidityAds.FundingRate(100_000.sat, 100_000.sat, 400, 150, 0.sat, 0.sat) + val fundingRate = LiquidityAds.FundingRate(100_000 sat, 100_000 sat, 400, 150, 0 sat, 0 sat) val testCases = Seq( // @formatter:off - SpliceInit(channelId, 100_000 sat, FeeratePerKw(2500 sat), 100, fundingPubkey) -> hex"9088 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000186a0 000009c4 00000064 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798", - SpliceInit(channelId, 150_000 sat, 100, FeeratePerKw(2500 sat), fundingPubkey, 25_000_000 msat, requireConfirmedInputs = false, None) -> hex"9088 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000249f0 000009c4 00000064 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fe4700000704017d7840", - SpliceInit(channelId, 0 sat, FeeratePerKw(500 sat), 0, fundingPubkey) -> hex"9088 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000000000 000001f4 00000000 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798", - SpliceInit(channelId, (-50_000).sat, FeeratePerKw(500 sat), 0, fundingPubkey) -> hex"9088 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa ffffffffffff3cb0 000001f4 00000000 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798", - SpliceInit(channelId, 100_000 sat, 100, FeeratePerKw(2500 sat), fundingPubkey, 0 msat, requireConfirmedInputs = false, Some(LiquidityAds.RequestFunding(100_000 sat, fundingRate, LiquidityAds.PaymentDetails.FromChannelBalance))) -> hex"9088 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000186a0 000009c4 00000064 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fd053b1e00000000000186a0000186a0000186a00190009600000000000000000000", - SpliceInit(channelId, 100_000 sat, 100, FeeratePerKw(2500 sat), fundingPubkey, 0 msat, requireConfirmedInputs = false, None, Some(SimpleTaprootChannelsPhoenix)) -> hex"9088 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000186a0 000009c400000064 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fe47000011 47 1000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000", - SpliceAck(channelId, 25_000 sat, fundingPubkey) -> hex"908a aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000061a8 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798", - SpliceAck(channelId, 40_000 sat, fundingPubkey, 10_000_000 msat, requireConfirmedInputs = false, None, None) -> hex"908a aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000009c40 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fe4700000703989680", - SpliceAck(channelId, 0 sat, fundingPubkey) -> hex"908a aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000000000 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798", - SpliceAck(channelId, (-25_000).sat, fundingPubkey) -> hex"908a aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa ffffffffffff9e58 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798", - SpliceAck(channelId, 25_000 sat, fundingPubkey, 0 msat, requireConfirmedInputs = false, Some(LiquidityAds.WillFund(fundingRate, hex"deadbeef", ByteVector64.Zeroes)), None) -> hex"908a aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000061a8 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fd053b5a000186a0000186a00190009600000000000000000004deadbeef00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", - SpliceAck(channelId, 25_000 sat, fundingPubkey, TlvStream(ChannelTlv.FeeCreditUsedTlv(0 msat))) -> hex"908a aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000061a8 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fda05200", - SpliceAck(channelId, 25_000 sat, fundingPubkey, TlvStream(ChannelTlv.FeeCreditUsedTlv(1729 msat))) -> hex"908a aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000061a8 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fda0520206c1", - SpliceAck(channelId, 25_000 sat, fundingPubkey, 0 msat, requireConfirmedInputs = false, None, None, Some(SimpleTaprootChannelsPhoenix)) -> hex"908a aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000061a8 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fe47000011 47 1000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000", SpliceLocked(channelId, fundingTxId) -> hex"908c aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 24e1b2c94c4e734dd5b9c5f3c910fbb6b3b436ced6382c7186056a5a23f14566", + 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, None) -> hex"0050 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000249f0 000009c4 00000064 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 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", + SpliceInit(channelId, 100_000 sat, 100, FeeratePerKw(2500 sat), fundingPubkey, 0 msat, requireConfirmedInputs = false, Some(LiquidityAds.RequestFunding(100_000 sat, fundingRate, LiquidityAds.PaymentDetails.FromChannelBalance))) -> hex"0050 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000186a0 000009c4 00000064 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fd053b1e00000000000186a0000186a0000186a00190009600000000000000000000", + SpliceInit(channelId, 100_000 sat, 100, FeeratePerKw(2500 sat), fundingPubkey, 0 msat, requireConfirmedInputs = false, None, Some(SimpleTaprootChannelsPhoenix)) -> hex"0050 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000186a0 000009c400000064 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fe47000011 47 1000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000", + SpliceAck(channelId, 25_000 sat, fundingPubkey) -> hex"0051 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000061a8 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798", + SpliceAck(channelId, 40_000 sat, fundingPubkey, 10_000_000 msat, requireConfirmedInputs = false, None, None) -> hex"0051 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000009c40 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fe4700000703989680", + SpliceAck(channelId, 0 sat, fundingPubkey) -> hex"0051 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000000000 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798", + SpliceAck(channelId, (-25_000).sat, fundingPubkey) -> hex"0051 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa ffffffffffff9e58 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798", + SpliceAck(channelId, 25_000 sat, fundingPubkey, 0 msat, requireConfirmedInputs = false, Some(LiquidityAds.WillFund(fundingRate, hex"deadbeef", ByteVector64.Zeroes)), None) -> hex"0051 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000061a8 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fd053b5a000186a0000186a00190009600000000000000000004deadbeef00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", + SpliceAck(channelId, 25_000 sat, fundingPubkey, TlvStream(ChannelTlv.FeeCreditUsedTlv(0 msat))) -> hex"0051 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000061a8 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fda05200", + SpliceAck(channelId, 25_000 sat, fundingPubkey, TlvStream(ChannelTlv.FeeCreditUsedTlv(1729 msat))) -> hex"0051 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000061a8 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fda0520206c1", + SpliceAck(channelId, 25_000 sat, fundingPubkey, 0 msat, requireConfirmedInputs = false, None, None, Some(SimpleTaprootChannelsPhoenix)) -> hex"0051 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000000061a8 0279be667ef9dcbbac55a06295ce870b07029bfcdb2dce28d959f2815b16f81798 fe47000011 47 1000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000", + SpliceLocked(channelId, fundingTxId) -> hex"004d aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 24e1b2c94c4e734dd5b9c5f3c910fbb6b3b436ced6382c7186056a5a23f14566", // @formatter:on ) - testCases.foreach { case (message, bin) => + testCases.foreach { case (msg, bin) => val decoded = lightningMessageCodec.decode(bin.bits).require.value - assert(decoded == message) - val encoded = lightningMessageCodec.encode(message).require.bytes + assert(decoded == msg) + val encoded = lightningMessageCodec.encode(msg).require.bytes + assert(encoded == bin) + } + } + + test("encode/decode start_batch message") { + val channelId = ByteVector32(hex"aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa") + val testCases = Seq( + StartBatch(channelId, 1) -> hex"007f aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0001", + StartBatch(channelId, 7) -> hex"007f aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0007", + StartBatch.commitSigBatch(channelId, 7) -> hex"007f aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0007 01020084", + StartBatch(channelId, 7, TlvStream(StartBatchTlv.MessageType(57331))) -> hex"007f aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0007 0102dff3", + StartBatch(channelId, 32000) -> hex"007f aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 7d00", + ) + 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) } } From ff4796b188c5cf2601af50915393787c6c5d41a5 Mon Sep 17 00:00:00 2001 From: t-bast Date: Tue, 12 Aug 2025 17:03:51 +0200 Subject: [PATCH 2/4] Re-work my_current_funding_locked reestablish We introduce a `retransmit_flags` field to ask our peer to retransmit `announcement_signatures` if we're expecting them: this way we don't need to retransmit `splice_locked` on reconnection to trigger the exchange of `announcement_signatures`. We don't need to retransmit it to let our peer know that we've seen enough confirmations for the splice either, since `my_current_funding_locked` implies that. This allows us to completely remove retransmission of `splice_locked` on reconnection, and also get rid of the `your_last_funding_locked` TLV, which greatly simplifies the reconnection logic. This will work with taproot channels since we'll simply provide nonces in `channel_reestablish` when we need our peer to send announcement signatures. Note that we use a different TLV type to more easily allow migrating existing users of the previous versions of the spec to the latest one. --- .../fr/acinq/eclair/channel/fsm/Channel.scala | 98 ++++------ .../fr/acinq/eclair/io/PeerConnection.scala | 6 +- .../eclair/wire/protocol/ChannelTlv.scala | 19 +- .../wire/protocol/LightningMessageTypes.scala | 8 +- .../fr/acinq/eclair/channel/RestoreSpec.scala | 7 +- .../c/WaitForDualFundingReadyStateSpec.scala | 10 +- .../states/e/NormalSplicesStateSpec.scala | 184 +++++------------- .../channel/states/e/OfflineStateSpec.scala | 49 +++-- .../protocol/LightningMessageCodecsSpec.scala | 4 +- 9 files changed, 146 insertions(+), 239 deletions(-) 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 da3bd15455..895124099c 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 @@ -246,8 +246,6 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall var announcementSigsStash = Map.empty[RealShortChannelId, AnnouncementSignatures] // we record the announcement_signatures messages we already sent to avoid unnecessary retransmission var announcementSigsSent = Set.empty[RealShortChannelId] - // we keep track of the splice_locked we sent after channel_reestablish and it's funding tx index to avoid sending it again - private var spliceLockedSent = Map.empty[TxId, Long] private def trimAnnouncementSigsStashIfNeeded(): Unit = { if (announcementSigsStash.size >= 10) { @@ -259,17 +257,6 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall } } - private def trimSpliceLockedSentIfNeeded(): Unit = { - if (spliceLockedSent.size >= 10) { - // We shouldn't store an unbounded number of splice_locked: on long-lived connections where we do a lot of splice - // transactions, we only need to keep track of the most recent ones. - val oldestFundingTxId = spliceLockedSent.toSeq - .sortBy { case (_, fundingTxIndex) => fundingTxIndex } - .map { case (fundingTxId, _) => fundingTxId }.head - spliceLockedSent -= oldestFundingTxId - } - } - val txPublisher = txPublisherFactory.spawnTxPublisher(context, remoteNodeId) // this will be used to detect htlc timeouts @@ -1473,8 +1460,6 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall watchFundingConfirmed(w.tx.txid, Some(nodeParams.channelConf.minDepth), delay_opt = None) maybeEmitEventsPostSplice(d.aliases, d.commitments, commitments1, d.lastAnnouncement_opt) maybeUpdateMaxHtlcAmount(d.channelUpdate.htlcMaximumMsat, commitments1) - spliceLockedSent += (commitment.fundingTxId -> commitment.fundingTxIndex) - trimSpliceLockedSentIfNeeded() stay() using d.copy(commitments = commitments1) storing() sending SpliceLocked(d.channelId, w.tx.txid) case Left(_) => stay() } @@ -1485,11 +1470,7 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall // We check if this commitment was already locked before receiving the event (which happens when using 0-conf // or for the initial funding transaction). If it was previously not locked, we must send splice_locked now. val previouslyNotLocked = d.commitments.all.exists(c => c.fundingTxId == commitment.fundingTxId && c.localFundingStatus.isInstanceOf[LocalFundingStatus.NotLocked]) - val spliceLocked_opt = if (previouslyNotLocked) { - spliceLockedSent += (commitment.fundingTxId -> commitment.fundingTxIndex) - trimSpliceLockedSentIfNeeded() - Some(SpliceLocked(d.channelId, w.tx.txid)) - } else None + val spliceLocked_opt = if (previouslyNotLocked) Some(SpliceLocked(d.channelId, w.tx.txid)) else None // If the channel is public and we've received the remote splice_locked, we send our announcement_signatures // in order to generate the channel_announcement. val remoteLocked = commitment.fundingTxIndex == 0 || d.commitments.all.exists(c => c.fundingTxId == commitment.fundingTxId && c.remoteFundingStatus == RemoteFundingStatus.Locked) @@ -1512,21 +1493,6 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall case Event(msg: SpliceLocked, d: DATA_NORMAL) => d.commitments.updateRemoteFundingStatus(msg.fundingTxId, d.lastAnnouncedFundingTxId_opt) match { case Right((commitments1, commitment)) => - // If we have both already sent splice_locked for this commitment, then we are receiving splice_locked - // again after a reconnection and must retransmit our splice_locked and new announcement_signatures. Nodes - // retransmit splice_locked after a reconnection when they have received splice_locked but NOT matching signatures - // before the last disconnect. If a matching splice_locked has already been sent since reconnecting, then do not - // retransmit splice_locked to avoid a loop. - // NB: It is important both nodes retransmit splice_locked after reconnecting to ensure new Taproot nonces - // are exchanged for channel announcements. - val isLatestLocked = d.commitments.lastLocalLocked_opt.exists(_.fundingTxId == msg.fundingTxId) && d.commitments.lastRemoteLocked_opt.exists(_.fundingTxId == msg.fundingTxId) - val spliceLocked_opt = if (d.commitments.announceChannel && isLatestLocked && !spliceLockedSent.contains(commitment.fundingTxId)) { - spliceLockedSent += (commitment.fundingTxId -> commitment.fundingTxIndex) - trimSpliceLockedSentIfNeeded() - Some(SpliceLocked(d.channelId, commitment.fundingTxId)) - } else { - None - } // If the commitment is confirmed, we were waiting to receive the remote splice_locked before sending our announcement_signatures. val localAnnSigs_opt = commitment.signAnnouncement(nodeParams, commitments1.channelParams, channelKeys.fundingKey(commitment.fundingTxIndex)) match { case Some(localAnnSigs) if !announcementSigsSent.contains(localAnnSigs.shortChannelId) => @@ -1539,7 +1505,7 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall } maybeEmitEventsPostSplice(d.aliases, d.commitments, commitments1, d.lastAnnouncement_opt) maybeUpdateMaxHtlcAmount(d.channelUpdate.htlcMaximumMsat, commitments1) - stay() using d.copy(commitments = commitments1) storing() sending spliceLocked_opt.toSeq ++ localAnnSigs_opt.toSeq + stay() using d.copy(commitments = commitments1) storing() sending localAnnSigs_opt.toSeq case Left(_) => stay() } @@ -2469,8 +2435,14 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall } val remoteFeatures = d.commitments.remoteChannelParams.initFeatures val lastFundingLockedTlvs: Set[ChannelReestablishTlv] = if (remoteFeatures.hasFeature(Features.Splicing) || remoteFeatures.unknown.contains(UnknownFeature(154)) || remoteFeatures.unknown.contains(UnknownFeature(155))) { - d.commitments.lastLocalLocked_opt.map(c => ChannelReestablishTlv.MyCurrentFundingLockedTlv(c.fundingTxId)).toSet ++ - d.commitments.lastRemoteLocked_opt.map(c => ChannelReestablishTlv.YourLastFundingLockedTlv(c.fundingTxId)).toSet + d.commitments.lastLocalLocked_opt.map(c => { + // We ask our peer to retransmit their announcement_signatures if we haven't already announced that splice. + val retransmitAnnSigs = d match { + case d: DATA_NORMAL if d.commitments.announceChannel => !d.lastAnnouncedFundingTxId_opt.contains(c.fundingTxId) + case _ => false + } + ChannelReestablishTlv.MyCurrentFundingLockedTlv(c.fundingTxId, retransmitAnnSigs) + }).toSet } else Set.empty // We send our verification nonces for all active commitments. @@ -2693,8 +2665,10 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall case None => remoteNextCommitNonces = channelReestablish.nextCommitNonces // We re-send our latest splice_locked if needed. - val spliceLocked_opt = resendSpliceLockedIfNeeded(channelReestablish, commitments1, d.lastAnnouncement_opt) - sendQueue = sendQueue ++ spliceLocked_opt.toSeq + val spliceLocked_opt = resendSpliceLockedIfNeeded(commitments1) + // We retransmit our latest announcement_signatures if our peer requests it. + val spliceAnnSigs_opt = resendSpliceAnnSigsIfNeeded(channelReestablish, commitments1) + sendQueue = sendQueue ++ spliceLocked_opt.toSeq ++ spliceAnnSigs_opt.toSeq // We may need to retransmit updates and/or commit_sig and/or revocation to resume the channel. sendQueue = sendQueue ++ syncSuccess.retransmit @@ -3198,7 +3172,6 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall case _ -> OFFLINE => announcementSigsStash = Map.empty announcementSigsSent = Set.empty - spliceLockedSent = Map.empty[TxId, Long] remoteNextCommitNonces = Map.empty localCloseeNonce_opt = None remoteCloseeNonce_opt = None @@ -3391,23 +3364,22 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall // We only send channel_ready for initial funding transactions. case Some(c) if c.fundingTxIndex != 0 => (None, None) case Some(c) => + // If our peer is a phoenix wallet using the legacy splicing protocol, we always retransmit channel_ready. val remoteFeatures = d.commitments.remoteChannelParams.initFeatures - val remoteSpliceSupport = remoteFeatures.hasFeature(Features.Splicing) || remoteFeatures.unknown.contains(UnknownFeature(154)) || remoteFeatures.unknown.contains(UnknownFeature(155)) - // If our peer has not received our channel_ready, we retransmit it. - val notReceivedByRemote = remoteSpliceSupport && channelReestablish.yourLastFundingLocked_opt.isEmpty + val remoteLegacySpliceSupport = remoteFeatures.unknown.contains(UnknownFeature(154)) || remoteFeatures.unknown.contains(UnknownFeature(155)) // If next_local_commitment_number is 1 in both the channel_reestablish it sent and received, then the node // MUST retransmit channel_ready, otherwise it MUST NOT - val notReceivedByRemoteLegacy = !remoteSpliceSupport && channelReestablish.nextLocalCommitmentNumber == 1 && c.localCommit.index == 0 + val notReceivedByRemote = channelReestablish.nextLocalCommitmentNumber == 1 && c.localCommit.index == 0 // If this is a public channel and we haven't announced the channel, we retransmit our channel_ready and // will also send announcement_signatures. val notAnnouncedYet = d.commitments.announceChannel && c.shortChannelId_opt.nonEmpty && d.lastAnnouncement_opt.isEmpty - val channelReady_opt = if (notAnnouncedYet || notReceivedByRemote || notReceivedByRemoteLegacy) { + val channelReady_opt = if (notAnnouncedYet || notReceivedByRemote || remoteLegacySpliceSupport) { log.debug("re-sending channel_ready") Some(createChannelReady(d.aliases, d.commitments)) } else { None } - val announcementSigs_opt = if (notAnnouncedYet) { + val announcementSigs_opt = if (notAnnouncedYet || channelReestablish.retransmitAnnSigs) { // The funding transaction is confirmed, so we've already sent our announcement_signatures. // We haven't announced the channel yet, which means we haven't received our peer's announcement_signatures. // We retransmit our announcement_signatures to let our peer know that we're ready to announce the channel. @@ -3473,23 +3445,17 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall (spliceStatus1, sendQueue) } - private def resendSpliceLockedIfNeeded(channelReestablish: ChannelReestablish, commitments: Commitments, lastAnnouncement_opt: Option[ChannelAnnouncement]): Option[SpliceLocked] = { + private def resendSpliceLockedIfNeeded(commitments: Commitments): Option[SpliceLocked] = { commitments.lastLocalLocked_opt match { case None => None // We only send splice_locked for splice transactions. case Some(c) if c.fundingTxIndex == 0 => None case Some(c) => - // If our peer has not received our splice_locked, we retransmit it. - val notReceivedByRemote = !channelReestablish.yourLastFundingLocked_opt.contains(c.fundingTxId) - // If this is a public channel and we haven't announced the splice, we retransmit our splice_locked and - // will exchange announcement_signatures afterwards. - val notAnnouncedYet = commitments.announceChannel && lastAnnouncement_opt.forall(ann => !c.shortChannelId_opt.contains(ann.shortChannelId)) - if (notReceivedByRemote || notAnnouncedYet) { - // Retransmission of local announcement_signatures for splices are done when receiving splice_locked, no need - // to retransmit here. + // We only send splice_locked for legacy phoenix wallets using the old splicing protocol. + val remoteFeatures = commitments.remoteChannelParams.initFeatures + val remoteLegacySpliceSupport = remoteFeatures.unknown.contains(UnknownFeature(154)) || remoteFeatures.unknown.contains(UnknownFeature(155)) + if (remoteLegacySpliceSupport) { log.debug("re-sending splice_locked for fundingTxId={}", c.fundingTxId) - spliceLockedSent += (c.fundingTxId -> c.fundingTxIndex) - trimSpliceLockedSentIfNeeded() Some(SpliceLocked(commitments.channelId, c.fundingTxId)) } else { None @@ -3497,6 +3463,22 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall } } + private def resendSpliceAnnSigsIfNeeded(channelReestablish: ChannelReestablish, commitments: Commitments): Option[AnnouncementSignatures] = { + commitments.lastLocalLocked_opt match { + case None => None + // This retransmit mechanism is only available for splice transactions. + case Some(c) if c.fundingTxIndex == 0 => None + case Some(c) if channelReestablish.retransmitAnnSigs && commitments.announceChannel => + val localAnnSigs = c.signAnnouncement(nodeParams, commitments.channelParams, channelKeys.fundingKey(c.fundingTxIndex)) + localAnnSigs.foreach(annSigs => { + log.debug("re-sending announcement_signatures for fundingTxId={}", c.fundingTxId) + announcementSigsSent += annSigs.shortChannelId + }) + localAnnSigs + case _ => None + } + } + /** * Return full information about a known closing tx. */ 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 967a939e94..6b1b106bc5 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 @@ -436,13 +436,13 @@ class PeerConnection(keyPair: KeyPair, conf: PeerConnection.Conf, switchboard: A } // If our peer is using the experimental splice version, we convert splice messages. case msg: ExperimentalSpliceInit => - d.peer ! msg.toSpliceInit() + d.peer ! msg.toSpliceInit stay() case msg: ExperimentalSpliceAck => - d.peer ! msg.toSpliceAck() + d.peer ! msg.toSpliceAck stay() case msg: ExperimentalSpliceLocked => - d.peer ! msg.toSpliceLocked() + d.peer ! msg.toSpliceLocked stay() case _ => d.peer ! msg diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/ChannelTlv.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/ChannelTlv.scala index 9aad296841..5dd8cfc8fd 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/ChannelTlv.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/ChannelTlv.scala @@ -261,11 +261,11 @@ object ChannelReestablishTlv { */ case class NextFundingTlv(txId: TxId) extends ChannelReestablishTlv - /** The txid of the last [[ChannelReady]] or [[SpliceLocked]] message received before disconnecting, if any. */ - case class YourLastFundingLockedTlv(txId: TxId) extends ChannelReestablishTlv - - /** The txid of our latest outgoing [[ChannelReady]] or [[SpliceLocked]] for this channel. */ - case class MyCurrentFundingLockedTlv(txId: TxId) extends ChannelReestablishTlv + /** + * @param txId the txid of our latest outgoing [[ChannelReady]] or [[SpliceLocked]] for this channel. + * @param retransmitAnnSigs true if [[AnnouncementSignatures]] must be retransmitted. + */ + case class MyCurrentFundingLockedTlv(txId: TxId, retransmitAnnSigs: Boolean) extends ChannelReestablishTlv /** * When disconnected during an interactive tx session, we'll include a verification nonce for our *current* commitment @@ -284,12 +284,8 @@ object ChannelReestablishTlv { val codec: Codec[NextFundingTlv] = tlvField(txIdAsHash) } - object YourLastFundingLockedTlv { - val codec: Codec[YourLastFundingLockedTlv] = tlvField("your_last_funding_locked_txid" | txIdAsHash) - } - object MyCurrentFundingLockedTlv { - val codec: Codec[MyCurrentFundingLockedTlv] = tlvField("my_current_funding_locked_txid" | txIdAsHash) + val codec: Codec[MyCurrentFundingLockedTlv] = tlvField(("my_current_funding_locked_txid" | txIdAsHash) :: ("retransmit_flags" | (ignore(7) :: bool))) } object CurrentCommitNonceTlv { @@ -302,8 +298,7 @@ object ChannelReestablishTlv { val channelReestablishTlvCodec: Codec[TlvStream[ChannelReestablishTlv]] = tlvStream(discriminated[ChannelReestablishTlv].by(varint) .typecase(UInt64(0), NextFundingTlv.codec) - .typecase(UInt64(1), YourLastFundingLockedTlv.codec) - .typecase(UInt64(3), MyCurrentFundingLockedTlv.codec) + .typecase(UInt64(5), MyCurrentFundingLockedTlv.codec) .typecase(UInt64(22), NextLocalNoncesTlv.codec) .typecase(UInt64(24), CurrentCommitNonceTlv.codec) ) 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 35e2950809..7698dbd442 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 @@ -214,7 +214,7 @@ case class ChannelReestablish(channelId: ByteVector32, tlvStream: TlvStream[ChannelReestablishTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId { val nextFundingTxId_opt: Option[TxId] = tlvStream.get[ChannelReestablishTlv.NextFundingTlv].map(_.txId) val myCurrentFundingLocked_opt: Option[TxId] = tlvStream.get[ChannelReestablishTlv.MyCurrentFundingLockedTlv].map(_.txId) - val yourLastFundingLocked_opt: Option[TxId] = tlvStream.get[ChannelReestablishTlv.YourLastFundingLockedTlv].map(_.txId) + val retransmitAnnSigs: Boolean = tlvStream.get[ChannelReestablishTlv.MyCurrentFundingLockedTlv].exists(_.retransmitAnnSigs) val nextCommitNonces: Map[TxId, IndividualNonce] = tlvStream.get[ChannelReestablishTlv.NextLocalNoncesTlv].map(_.nonces.toMap).getOrElse(Map.empty) val currentCommitNonce_opt: Option[IndividualNonce] = tlvStream.get[ChannelReestablishTlv.CurrentCommitNonceTlv].map(_.nonce) } @@ -418,7 +418,7 @@ case class ExperimentalSpliceInit(channelId: ByteVector32, lockTime: Long, fundingPubKey: PublicKey, tlvStream: TlvStream[SpliceInitTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId { - def toSpliceInit(): SpliceInit = SpliceInit(channelId, fundingContribution, feerate, lockTime, fundingPubKey, tlvStream) + def toSpliceInit: SpliceInit = SpliceInit(channelId, fundingContribution, feerate, lockTime, fundingPubKey, tlvStream) } object ExperimentalSpliceInit { @@ -455,7 +455,7 @@ 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) + def toSpliceAck: SpliceAck = SpliceAck(channelId, fundingContribution, fundingPubKey, tlvStream) } object ExperimentalSpliceAck { @@ -470,7 +470,7 @@ case class SpliceLocked(channelId: ByteVector32, case class ExperimentalSpliceLocked(channelId: ByteVector32, fundingTxId: TxId, tlvStream: TlvStream[SpliceLockedTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId { - def toSpliceLocked(): SpliceLocked = SpliceLocked(channelId, fundingTxId, tlvStream) + def toSpliceLocked: SpliceLocked = SpliceLocked(channelId, fundingTxId, tlvStream) } object ExperimentalSpliceLocked { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/RestoreSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/RestoreSpec.scala index db69a0d1c6..92a9295279 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/RestoreSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/RestoreSpec.scala @@ -9,7 +9,7 @@ import fr.acinq.eclair.channel.fsm.Channel import fr.acinq.eclair.channel.states.ChannelStateTestsBase import fr.acinq.eclair.channel.states.ChannelStateTestsBase.FakeTxPublisherFactory import fr.acinq.eclair.router.Announcements -import fr.acinq.eclair.wire.protocol.{ChannelReestablish, ChannelUpdate, Init} +import fr.acinq.eclair.wire.protocol.{ChannelReady, ChannelReestablish, ChannelUpdate, Init} import fr.acinq.eclair.{TestKitBaseClass, _} import org.scalatest.Outcome import org.scalatest.funsuite.FixtureAnyFunSuiteLike @@ -130,14 +130,15 @@ class RestoreSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with Chan assert(u1.channelUpdate.feeProportionalMillionths == newConfig.relayParams.privateChannelFees.feeProportionalMillionths) assert(u1.channelUpdate.cltvExpiryDelta == newConfig.channelConf.expiryDelta) + alice2bob.ignoreMsg { case _: ChannelUpdate | _: ChannelReady => true } + bob2alice.ignoreMsg { case _: ChannelUpdate | _: ChannelReady => true } + newAlice ! INPUT_RECONNECTED(alice2bob.ref, aliceInit, bobInit) bob ! INPUT_RECONNECTED(bob2alice.ref, bobInit, aliceInit) alice2bob.expectMsgType[ChannelReestablish] bob2alice.expectMsgType[ChannelReestablish] alice2bob.forward(bob) bob2alice.forward(newAlice) - alice2bob.expectMsgType[ChannelUpdate] - bob2alice.expectMsgType[ChannelUpdate] alice2bob.expectNoMessage() bob2alice.expectNoMessage() 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 25a7022584..8eff50252e 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 @@ -238,18 +238,16 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF val bobInit = Init(TestConstants.Bob.nodeParams.features.initFeatures()) alice ! INPUT_RECONNECTED(alice2bob.ref, aliceInit, bobInit) bob ! INPUT_RECONNECTED(bob2alice.ref, bobInit, aliceInit) - alice2bob.expectMsgType[ChannelReestablish] + assert(alice2bob.expectMsgType[ChannelReestablish].retransmitAnnSigs) alice2bob.forward(bob) - bob2alice.expectMsgType[ChannelReestablish] + assert(!bob2alice.expectMsgType[ChannelReestablish].retransmitAnnSigs) bob2alice.forward(alice) - // Bob does not retransmit channel_ready and announcement_signatures because he has already received both of them from Alice. - bob2alice.expectNoMessage(100 millis) - // Alice has already received Bob's channel_ready, but not its announcement_signatures. - // She retransmits channel_ready and Bob will retransmit its announcement_signatures in response. alice2bob.expectMsgType[ChannelReady] alice2bob.forward(bob) alice2bob.expectMsgType[AnnouncementSignatures] alice2bob.forward(bob) + bob2alice.expectMsgType[ChannelReady] + bob2alice.forward(alice) bob2alice.expectMsgType[AnnouncementSignatures] bob2alice.forward(alice) awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.nonEmpty) 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 0d5bb006a7..de219d8fc2 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 @@ -1535,16 +1535,15 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik awaitAssert(assert(alice.stateName == OFFLINE)) // Alice and Bob reconnect. - reconnect(f) - bob2alice.expectNoMessage(100 millis) - assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == spliceTx.txid) // Alice resends `splice_locked` because she hasn't received Bob's announcement_signatures. - alice2bob.forward(bob) - alice2bob.expectNoMessage(100 millis) - assert(bob2alice.expectMsgType[SpliceLocked].fundingTxId == spliceTx.txid) // Bob resends `splice_locked` in response to Alice's `splice_locked` after channel_reestablish. - bob2alice.forward(alice) + val (channelReestablishAlice, channelReestablishBob) = reconnect(f) + assert(channelReestablishAlice.retransmitAnnSigs) + assert(channelReestablishAlice.myCurrentFundingLocked_opt.contains(spliceTx.txid)) + assert(!channelReestablishBob.retransmitAnnSigs) + assert(channelReestablishBob.myCurrentFundingLocked_opt.contains(spliceTx.txid)) assert(bob2alice.expectMsgType[AnnouncementSignatures].shortChannelId == spliceAnn.shortChannelId) bob2alice.forward(alice) bob2alice.expectNoMessage(100 millis) + alice2bob.expectNoMessage(100 millis) assert(aliceListener.expectMsgType[ShortChannelIdAssigned].announcement_opt.contains(spliceAnn)) awaitAssert(assert(alice.stateData.asInstanceOf[DATA_NORMAL].lastAnnouncement_opt.contains(spliceAnn))) awaitAssert(assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.all.size == 1)) @@ -1617,6 +1616,10 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik // Bob disconnects before receiving Alice's commit_sig. disconnect(f) reconnect(f) + alice2bob.expectMsgType[ChannelReady] + alice2bob.forward(bob) + bob2alice.expectMsgType[ChannelReady] + bob2alice.forward(alice) alice2bob.expectMsgType[UpdateAddHtlc] alice2bob.forward(bob) val sigsA = alice2bob.expectMsgType[CommitSigBatch] @@ -1828,6 +1831,8 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik // Bob and Alice will exchange tx_abort because Bob did not receive Alice's tx_complete before the disconnect. bob2alice.expectMsgType[TxAbort] bob2alice.forward(alice) + alice2bob.expectMsgType[ChannelReady] + alice2bob.forward(bob) alice2bob.expectMsgType[TxAbort] alice2bob.forward(bob) awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus == SpliceStatus.NoSplice) @@ -2331,8 +2336,6 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik // Alice retransmits tx_signatures. alice2bob.expectMsgType[TxSignatures] alice2bob.forward(bob) - assert(alice2bob.expectMsgType[SpliceLocked].fundingTxId == spliceTx.txid) - alice2bob.forward(bob) bob2alice.expectNoMessage(100 millis) bob ! WatchFundingConfirmedTriggered(BlockHeight(42), 0, spliceTx) bob2alice.expectMsgType[SpliceLocked] @@ -2365,13 +2368,12 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik val (channelReestablishAlice, channelReestablishBob) = reconnect(f) assert(channelReestablishAlice.nextFundingTxId_opt.isEmpty) assert(channelReestablishBob.nextFundingTxId_opt.contains(spliceTx.txid)) + bob2alice.expectMsgType[ChannelReady] bob2alice.expectNoMessage(100 millis) // Bob receives Alice's tx_signatures, which completes the splice. alice2bob.expectMsgType[TxSignatures] alice2bob.forward(bob) - alice2bob.expectMsgType[SpliceLocked] - alice2bob.forward(bob) awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].spliceStatus == SpliceStatus.NoSplice) } @@ -2652,16 +2654,15 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik // From Alice's point of view, we now have two unconfirmed splices. - alice2bob.ignoreMsg { case _: ChannelUpdate => true } - bob2alice.ignoreMsg { case _: ChannelUpdate => true } + alice2bob.ignoreMsg { case _: ChannelUpdate | _: ChannelReady => true } + bob2alice.ignoreMsg { case _: ChannelUpdate | _: ChannelReady => true } disconnect(f) - reconnect(f) - - // NB: channel_ready are not re-sent because the channel has already been used (for building splices). - // Alice has already received `splice_locked` from Bob for the first splice, so he doesn't need to resend it. - bob2alice.expectNoMessage(100 millis) - alice2bob.expectNoMessage(100 millis) + val (channelReestablishA1, channelReestablishB1) = reconnect(f) + // Alice has locked the initial funding transaction, but not the splice transaction yet. + assert(channelReestablishA1.myCurrentFundingLocked_opt.nonEmpty) + assert(!channelReestablishA1.myCurrentFundingLocked_opt.contains(fundingTx1.txid)) + assert(channelReestablishB1.myCurrentFundingLocked_opt.contains(fundingTx1.txid)) // The first splice confirms on Alice's side. alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx1) @@ -2670,11 +2671,9 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice2blockchain.expectMsgType[WatchFundingSpent] disconnect(f) - reconnect(f) - - // Alice and Bob have already exchanged `splice_locked` for the first splice, so there is need to resend it. - bob2alice.expectNoMessage(100 millis) - alice2bob.expectNoMessage(100 millis) + val (channelReestablishA2, channelReestablishB2) = reconnect(f) + assert(channelReestablishA2.myCurrentFundingLocked_opt.contains(fundingTx1.txid)) + assert(channelReestablishB2.myCurrentFundingLocked_opt.contains(fundingTx1.txid)) // The second splice confirms on Alice's side. alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) @@ -2683,10 +2682,9 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice2blockchain.expectMsgType[WatchFundingSpent] disconnect(f) - reconnect(f) - - alice2bob.expectNoMessage(100 millis) - bob2alice.expectNoMessage(100 millis) + val (channelReestablishA3, channelReestablishB3) = reconnect(f) + assert(channelReestablishA3.myCurrentFundingLocked_opt.contains(fundingTx2.txid)) + assert(channelReestablishB3.myCurrentFundingLocked_opt.contains(fundingTx1.txid)) // The second splice confirms on Bob's side. bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx2) @@ -2695,14 +2693,9 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik // NB: we disconnect *before* transmitting the splice_locked to Alice. disconnect(f) - reconnect(f) - - alice2bob.expectNoMessage(100 millis) - 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) + val (channelReestablishA4, channelReestablishB4) = reconnect(f) + assert(channelReestablishA4.myCurrentFundingLocked_opt.contains(fundingTx2.txid)) + assert(channelReestablishB4.myCurrentFundingLocked_opt.contains(fundingTx2.txid)) disconnect(f) reconnect(f) @@ -2780,6 +2773,9 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) + alice2bob.ignoreMsg { case _: ChannelUpdate | _: ChannelReady => true } + bob2alice.ignoreMsg { case _: ChannelUpdate | _: ChannelReady => true } + // Bob will not receive Alice's tx_signatures, update_add_htlc or commit_sigs before disconnecting. disconnect(f) reconnect(f) @@ -2818,8 +2814,8 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx.txid) alice2bob.forward(bob) - alice2bob.ignoreMsg { case _: ChannelUpdate => true } - bob2alice.ignoreMsg { case _: ChannelUpdate => true } + alice2bob.ignoreMsg { case _: ChannelUpdate | _: ChannelReady => true } + bob2alice.ignoreMsg { case _: ChannelUpdate | _: ChannelReady => true } disconnect(f) @@ -2833,19 +2829,15 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik assert(alice.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.active.size == 2) assert(bob.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.active.size == 1) - reconnect(f) + val (channelReestablishA, channelReestablishB) = reconnect(f) + assert(channelReestablishA.myCurrentFundingLocked_opt.contains(fundingTx.txid)) + assert(channelReestablishB.myCurrentFundingLocked_opt.contains(fundingTx.txid)) - // Because `your_last_funding_locked_txid` from Bob matches the last `splice_locked` txid sent by Alice; there is no need - // for Alice to resend `splice_locked`. Alice processes the `my_current_funding_locked` from Bob as if she received - // `splice_locked` from Bob and prunes the initial funding commitment. + // Alice processes the `my_current_funding_locked` from Bob as if she received `splice_locked` from Bob and prunes the initial funding commitment. awaitCond(alice.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.active.size == 1) assert(alice.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.active.head.fundingTxId == fundingTx.txid) alice2bob.expectNoMessage(100 millis) - // The `your_last_funding_locked_txid` from Alice does not match the last `splice_locked` sent by Bob, so Bob must resend `splice_locked`. - val bobSpliceLocked = bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx.txid) - assert(bob.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.active.size == 1) - // Alice sends an HTLC before receiving Bob's splice_locked: see https://github.com/lightning/bolts/issues/1223. addHtlc(15_000_000 msat, alice, bob, alice2bob, bob2alice) val sender = TestProbe() @@ -2853,7 +2845,6 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik sender.expectMsgType[RES_SUCCESS[CMD_SIGN]] assert(alice2bob.expectMsgType[CommitSig].fundingTxId_opt.contains(fundingTx.txid)) alice2bob.forward(bob) - bob2alice.forward(alice, bobSpliceLocked) bob2alice.expectMsgType[RevokeAndAck] bob2alice.forward(alice) assert(bob2alice.expectMsgType[CommitSig].fundingTxId_opt.contains(fundingTx.txid)) @@ -2962,6 +2953,9 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) + alice2bob.ignoreMsg { case _: ChannelUpdate | _: ChannelReady => true } + bob2alice.ignoreMsg { case _: ChannelUpdate | _: ChannelReady => true } + // Bob will not receive Alice's commit_sigs before disconnecting. disconnect(f) val (channelReestablishAlice, channelReestablishBob) = reconnect(f) @@ -3032,6 +3026,9 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) + alice2bob.ignoreMsg { case _: ChannelUpdate | _: ChannelReady => true } + bob2alice.ignoreMsg { case _: ChannelUpdate | _: ChannelReady => true } + // Alice will not receive Bob's commit_sigs before disconnecting. disconnect(f) val (channelReestablishAlice, channelReestablishBob) = reconnect(f) @@ -3099,6 +3096,9 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) + alice2bob.ignoreMsg { case _: ChannelUpdate | _: ChannelReady => true } + bob2alice.ignoreMsg { case _: ChannelUpdate | _: ChannelReady => true } + // Alice will not receive Bob's commit_sigs before disconnecting. disconnect(f) val (channelReestablishAlice, channelReestablishBob) = reconnect(f) @@ -3146,78 +3146,16 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik bob2alice.expectMsgType[AnnouncementSignatures] disconnect(f) - reconnect(f) - - // Bob will not resend `splice_locked` because he has already received `announcement_signatures` from Alice. - bob2alice.expectNoMessage(100 millis) - - // Alice resends `splice_locked` because she did not receive `announcement_signatures` from Bob before the disconnect. - val aliceSpliceLocked = alice2bob.expectMsgType[SpliceLocked] - alice2bob.forward(bob) - alice2bob.expectNoMessage(100 millis) - - // Bob receives Alice's `splice_locked` after `channel_reestablish` and must retransmit both `splice_locked` and `announcement_signatures`. - val bobSpliceLocked = bob2alice.expectMsgType[SpliceLocked] - bob2alice.forward(alice) + val (channelReestablishA, channelReestablishB) = reconnect(f) + assert(channelReestablishA.retransmitAnnSigs) + assert(!channelReestablishB.retransmitAnnSigs) bob2alice.expectMsgType[AnnouncementSignatures] bob2alice.forward(alice) bob2alice.expectNoMessage(100 millis) - - // Alice retransmits `announcement_signatures` to Bob after receiving `splice_locked` from Bob. - alice2bob.expectMsgType[AnnouncementSignatures] - alice2bob.forward(bob) - alice2bob.expectNoMessage(100 millis) - bob2alice.expectNoMessage(100 millis) - - // If either node receives `splice_locked` again, it should be ignored; `announcement_signatures have already been sent. - alice2bob.forward(bob, aliceSpliceLocked) - bob2alice.forward(alice, bobSpliceLocked) alice2bob.expectNoMessage(100 millis) - bob2alice.expectNoMessage(100 millis) - - // the splice is locked on both sides - alicePeer.fishForMessage() { - case e: ChannelReadyForPayments => e.fundingTxIndex == 1 - case _ => false - } - bobPeer.fishForMessage() { - case e: ChannelReadyForPayments => e.fundingTxIndex == 1 - case _ => false - } - } - - test("disconnect before receiving splice_locked from a legacy peer") { f => - import f._ - - val fundingTx = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat, pushAmount = 0 msat))) - checkWatchConfirmed(f, fundingTx) - - // The splice confirms for both. - alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx) - alice2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx.txid) - alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx.txid) - alice2bob.forward(bob) - bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx) - bob2blockchain.expectMsgTypeHaving[WatchFundingSpent](_.txId == fundingTx.txid) - bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx.txid) - bob2alice.forward(alice) - - alice2bob.ignoreMsg { case _: ChannelUpdate => true } - bob2alice.ignoreMsg { case _: ChannelUpdate => true } - - disconnect(f) - val (aliceReestablish, bobReestablish) = reconnect(f, sendReestablish = false) - - // remove the last_funding_locked tlv from the reestablish messages - alice2bob.forward(bob, aliceReestablish.copy(tlvStream = TlvStream.empty)) - bob2alice.forward(alice, bobReestablish.copy(tlvStream = TlvStream.empty)) - // always send last splice_locked after reconnection if the last_funding_locked tlv is not set - alice2bob.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx.txid) - bob2alice.expectMsgTypeHaving[SpliceLocked](_.fundingTxId == fundingTx.txid) - alice2bob.forward(bob) - bob2alice.forward(alice) - alice2bob.expectNoMessage(100 millis) + // If Bob receives `splice_locked` again, it should be ignored; `announcement_signatures have already been sent. + alice2bob.forward(bob, SpliceLocked(alice.commitments.channelId, fundingTx.txid)) bob2alice.expectNoMessage(100 millis) // the splice is locked on both sides @@ -3252,37 +3190,23 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik // Alice sends announcement_signatures to Bob. alice2bob.expectMsgType[AnnouncementSignatures] - // Bob sends announcement_signatures to Alice. bob2alice.expectMsgType[AnnouncementSignatures] disconnect(f) reconnect(f) - // Bob resends `splice_locked` because he did not receive `announcement_signatures` from Alice before the disconnect. - val bobSpliceLocked = bob2alice.expectMsgType[SpliceLocked] - bob2alice.expectNoMessage(100 millis) - - // Alice resends `splice_locked` because she did not receive `announcement_signatures` from Bob before the disconnect. - val aliceSpliceLocked = alice2bob.expectMsgType[SpliceLocked] - alice2bob.forward(bob) - alice2bob.expectNoMessage(100 millis) - - // Alice receives Bob's `splice_locked` after already resending their `splice_locked` and retransmits `announcement_signatures`. - bob2alice.forward(alice) alice2bob.expectMsgType[AnnouncementSignatures] alice2bob.forward(bob) alice2bob.expectNoMessage(100 millis) - // Bob retransmits `announcement_signatures` to Alice after receiving `announcement_signatures` from Alice. bob2alice.expectMsgType[AnnouncementSignatures] bob2alice.forward(alice) - alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) // If either node receives `splice_locked` again, it should be ignored; `announcement_signatures have already been sent. - alice2bob.forward(bob, aliceSpliceLocked) - bob2alice.forward(alice, bobSpliceLocked) + alice2bob.forward(bob, SpliceLocked(alice.commitments.channelId, fundingTx.txid)) + bob2alice.forward(alice, SpliceLocked(bob.commitments.channelId, fundingTx.txid)) alice2bob.expectNoMessage(100 millis) bob2alice.expectNoMessage(100 millis) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/OfflineStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/OfflineStateSpec.scala index 08f7d96e18..ac3c637a63 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/OfflineStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/OfflineStateSpec.scala @@ -71,9 +71,12 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with } } - private def lastFundingLockedTlvs(commitments: Commitments): Set[ChannelReestablishTlv] = - commitments.lastLocalLocked_opt.map(c => ChannelReestablishTlv.MyCurrentFundingLockedTlv(c.fundingTxId)).toSet ++ - commitments.lastRemoteLocked_opt.map(c => ChannelReestablishTlv.YourLastFundingLockedTlv(c.fundingTxId)).toSet + private def lastFundingLockedTlvs(commitments: Commitments): Set[ChannelReestablishTlv] = { + commitments.lastLocalLocked_opt.map(c => { + val retransmitAnnSigs = commitments.announceChannel + ChannelReestablishTlv.MyCurrentFundingLockedTlv(c.fundingTxId, retransmitAnnSigs) + }).toSet + } test("reconnect after creating channel", Tag(IgnoreChannelUpdates)) { f => import f._ @@ -120,10 +123,12 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with disconnect(alice, bob) val (aliceCurrentPerCommitmentPoint, bobCurrentPerCommitmentPoint) = reconnect(alice, bob, alice2bob, bob2alice) - val reestablishA = alice2bob.expectMsg(ChannelReestablish(htlc.channelId, 1, 0, PrivateKey(ByteVector32.Zeroes), aliceCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(alice.stateData.asInstanceOf[DATA_NORMAL].commitments)))) - val reestablishB = bob2alice.expectMsg(ChannelReestablish(htlc.channelId, 1, 0, PrivateKey(ByteVector32.Zeroes), bobCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(bob.stateData.asInstanceOf[DATA_NORMAL].commitments)))) + val reestablishA = alice2bob.expectMsg(ChannelReestablish(htlc.channelId, 1, 0, PrivateKey(ByteVector32.Zeroes), aliceCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(alice.commitments)))) + val reestablishB = bob2alice.expectMsg(ChannelReestablish(htlc.channelId, 1, 0, PrivateKey(ByteVector32.Zeroes), bobCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(bob.commitments)))) alice2bob.forward(bob, reestablishA) bob2alice.forward(alice, reestablishB) + alice2bob.expectMsgType[ChannelReady] + bob2alice.expectMsgType[ChannelReady] // alice will re-send the update and the sig alice2bob.expectMsg(htlc) @@ -175,8 +180,8 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with disconnect(alice, bob) val (aliceCurrentPerCommitmentPoint, bobCurrentPerCommitmentPoint) = reconnect(alice, bob, alice2bob, bob2alice) - val reestablishA = alice2bob.expectMsg(ChannelReestablish(htlc.channelId, 1, 0, PrivateKey(ByteVector32.Zeroes), aliceCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(alice.stateData.asInstanceOf[DATA_NORMAL].commitments)))) - val reestablishB = bob2alice.expectMsg(ChannelReestablish(htlc.channelId, 2, 0, PrivateKey(ByteVector32.Zeroes), bobCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(bob.stateData.asInstanceOf[DATA_NORMAL].commitments)))) + val reestablishA = alice2bob.expectMsg(ChannelReestablish(htlc.channelId, 1, 0, PrivateKey(ByteVector32.Zeroes), aliceCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(alice.commitments)))) + val reestablishB = bob2alice.expectMsg(ChannelReestablish(htlc.channelId, 2, 0, PrivateKey(ByteVector32.Zeroes), bobCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(bob.commitments)))) alice2bob.forward(bob, reestablishA) bob2alice.forward(alice, reestablishB) @@ -223,8 +228,8 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with { val (aliceCurrentPerCommitmentPoint, bobCurrentPerCommitmentPoint) = reconnect(alice, bob, alice2bob, bob2alice) - val reestablishA = alice2bob.expectMsg(ChannelReestablish(htlc.channelId, 1, 1, revB.perCommitmentSecret, aliceCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(alice.stateData.asInstanceOf[DATA_NORMAL].commitments)))) - val reestablishB = bob2alice.expectMsg(ChannelReestablish(htlc.channelId, 2, 0, PrivateKey(ByteVector32.Zeroes), bobCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(bob.stateData.asInstanceOf[DATA_NORMAL].commitments)))) + val reestablishA = alice2bob.expectMsg(ChannelReestablish(htlc.channelId, 1, 1, revB.perCommitmentSecret, aliceCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(alice.commitments)))) + val reestablishB = bob2alice.expectMsg(ChannelReestablish(htlc.channelId, 2, 0, PrivateKey(ByteVector32.Zeroes), bobCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(bob.commitments)))) alice2bob.forward(bob, reestablishA) bob2alice.forward(alice, reestablishB) } @@ -254,8 +259,8 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with { val (aliceCurrentPerCommitmentPoint, bobCurrentPerCommitmentPoint) = reconnect(alice, bob, alice2bob, bob2alice) - val reestablishA = alice2bob.expectMsg(ChannelReestablish(htlc.channelId, 2, 1, revB.perCommitmentSecret, aliceCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(alice.stateData.asInstanceOf[DATA_NORMAL].commitments)))) - val reestablishB = bob2alice.expectMsg(ChannelReestablish(htlc.channelId, 2, 0, PrivateKey(ByteVector32.Zeroes), bobCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(bob.stateData.asInstanceOf[DATA_NORMAL].commitments)))) + val reestablishA = alice2bob.expectMsg(ChannelReestablish(htlc.channelId, 2, 1, revB.perCommitmentSecret, aliceCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(alice.commitments)))) + val reestablishB = bob2alice.expectMsg(ChannelReestablish(htlc.channelId, 2, 0, PrivateKey(ByteVector32.Zeroes), bobCurrentPerCommitmentPoint, TlvStream(lastFundingLockedTlvs(bob.commitments)))) alice2bob.forward(bob, reestablishA) bob2alice.forward(alice, reestablishB) } @@ -715,21 +720,23 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with reconnect(alice, bob, alice2bob, bob2alice) // Alice and Bob exchange channel_reestablish and channel_ready again. - alice2bob.expectMsgType[ChannelReestablish] - bob2alice.expectMsgType[ChannelReestablish] + assert(!alice2bob.expectMsgType[ChannelReestablish].retransmitAnnSigs) + assert(bob2alice.expectMsgType[ChannelReestablish].retransmitAnnSigs) bob2alice.forward(alice) alice2bob.forward(bob) - alice2bob.expectNoMessage(100 millis) - - // Bob retransmits his channel_ready and announcement_signatures because he hasn't received Alice's announcement_signatures. + alice2bob.expectMsgType[ChannelReady] + alice2bob.forward(bob) bob2alice.expectMsgType[ChannelReady] bob2alice.forward(alice) + // Alice retransmits announcement_signatures because Bob requested it. + val annSigsAlice = alice2bob.expectMsgType[AnnouncementSignatures] + alice2bob.forward(bob) + alice2bob.expectNoMessage(100 millis) + + // Bob retransmits announcement_signatures because he hasn't received Alice's announcement_signatures. val annSigsBob = bob2alice.expectMsgType[AnnouncementSignatures] bob2alice.forward(alice, annSigsBob) - // Alice retransmits her announcement_signatures when receiving Bob's. - val annSigsAlice = alice2bob.expectMsgType[AnnouncementSignatures] - alice2bob.forward(bob, annSigsAlice) // Alice and Bob ignore redundant announcement_signatures. alice2bob.forward(bob, annSigsAlice) bob2alice.expectNoMessage(100 millis) @@ -749,6 +756,8 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with bob2alice.expectMsgType[ChannelReestablish] bob2alice.forward(alice) alice2bob.forward(bob) + alice2bob.expectMsgType[ChannelReady] + bob2alice.expectMsgType[ChannelReady] // alice and bob resend their channel update at reconnection (unannounced channel) alice2bob.expectMsgType[ChannelUpdate] @@ -907,8 +916,6 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with bob2alice.expectMsgType[ChannelReestablish] bob2alice.forward(alice) alice2bob.forward(bob) - bob2alice.expectMsgType[ChannelReady] - bob2alice.forward(alice) // Alice will NOT resend their channel_ready at reconnection because she has received bob's announcement_signatures (pre-splice behavior). alice2bob.expectNoMessage(100 millis) 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 9d502b9933..b2df648038 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 @@ -161,8 +161,8 @@ class LightningMessageCodecsSpec extends AnyFunSuite { hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point), hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"00 20" ++ txId.value.reverse -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream(ChannelReestablishTlv.NextFundingTlv(txId))), - hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"01 20" ++ txId.value.reverse -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream(ChannelReestablishTlv.YourLastFundingLockedTlv(txId))), - hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"03 20" ++ txId.value.reverse -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream(ChannelReestablishTlv.MyCurrentFundingLockedTlv(txId))), + hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"05 21" ++ txId.value.reverse ++ hex"00" -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream(ChannelReestablishTlv.MyCurrentFundingLockedTlv(txId, retransmitAnnSigs = false))), + hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"05 21" ++ txId.value.reverse ++ hex"01" -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream(ChannelReestablishTlv.MyCurrentFundingLockedTlv(txId, retransmitAnnSigs = true))), hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"18 42" ++ nonce.data -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream(ChannelReestablishTlv.CurrentCommitNonceTlv(nonce))), hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"16 c4" ++ txId.value.reverse ++ nonce.data ++ nextTxId.value.reverse ++ nextNonce.data -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream(ChannelReestablishTlv.NextLocalNoncesTlv(Seq(txId -> nonce, nextTxId -> nextNonce)))), hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"fe47010000 00" -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream[ChannelReestablishTlv](Set.empty[ChannelReestablishTlv], Set(GenericTlv(tlvTag, ByteVector.empty)))), From 0bbbf3eaa6a8d28ab7b688bca2881f4f433d8f91 Mon Sep 17 00:00:00 2001 From: t-bast Date: Wed, 27 Aug 2025 11:04:36 +0200 Subject: [PATCH 3/4] Add retransmit flags to `next_funding` TLV Instead of using the `next_commitment_number` to let our peer know that we haven't received their `commit_sig` for a partially signed funding transaction, we add retransmit flags to the `next_funding` TLV in the `channel_reestablish` message. This is similar to what was done in the previous commit for announcement signatures with the `my_current_funding_locked` TLV. --- .../fr/acinq/eclair/channel/fsm/Channel.scala | 47 +++++++------------ .../channel/fund/InteractiveTxBuilder.scala | 7 +-- .../eclair/wire/protocol/ChannelTlv.scala | 9 ++-- .../wire/protocol/LightningMessageTypes.scala | 1 + .../b/WaitForDualFundingSignedStateSpec.scala | 18 ++++--- ...WaitForDualFundingConfirmedStateSpec.scala | 15 ++++-- .../states/e/NormalSplicesStateSpec.scala | 40 +++++++++++----- .../protocol/LightningMessageCodecsSpec.scala | 3 +- 8 files changed, 78 insertions(+), 62 deletions(-) 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 895124099c..236a92049c 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 @@ -1456,7 +1456,7 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall case Event(w: WatchPublishedTriggered, d: DATA_NORMAL) => val fundingStatus = LocalFundingStatus.ZeroconfPublishedFundingTx(w.tx, d.commitments.localFundingSigs(w.tx.txid), d.commitments.liquidityPurchase(w.tx.txid)) d.commitments.updateLocalFundingStatus(w.tx.txid, fundingStatus, d.lastAnnouncedFundingTxId_opt) match { - case Right((commitments1, commitment)) => + case Right((commitments1, _)) => watchFundingConfirmed(w.tx.txid, Some(nodeParams.channelConf.minDepth), delay_opt = None) maybeEmitEventsPostSplice(d.aliases, d.commitments, commitments1, d.lastAnnouncement_opt) maybeUpdateMaxHtlcAmount(d.channelUpdate.htlcMaximumMsat, commitments1) @@ -2371,7 +2371,7 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall case Event(INPUT_RECONNECTED(r, localInit, remoteInit), d: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED) => activeConnection = r val myFirstPerCommitmentPoint = channelKeys.commitmentPoint(0) - val nextFundingTlv: Set[ChannelReestablishTlv] = Set(ChannelReestablishTlv.NextFundingTlv(d.signingSession.fundingTxId)) + val nextFundingTlv = Set[ChannelReestablishTlv](ChannelReestablishTlv.NextFundingTlv(d.signingSession.fundingTxId, d.signingSession.retransmitRemoteCommitSig)) val nonceTlvs = d.signingSession.fundingParams.commitmentFormat match { case _: SegwitV0CommitmentFormat => Set.empty case _: SimpleTaprootChannelCommitmentFormat => @@ -2389,7 +2389,7 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall } val channelReestablish = ChannelReestablish( channelId = d.channelId, - nextLocalCommitmentNumber = d.signingSession.nextLocalCommitmentNumber, + nextLocalCommitmentNumber = 1, nextRemoteRevocationNumber = 0, yourLastPerCommitmentSecret = PrivateKey(ByteVector32.Zeroes), myCurrentPerCommitmentPoint = myFirstPerCommitmentPoint, @@ -2403,31 +2403,19 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall val remotePerCommitmentSecrets = d.commitments.remotePerCommitmentSecrets val yourLastPerCommitmentSecret = remotePerCommitmentSecrets.lastIndex.flatMap(remotePerCommitmentSecrets.getHash).getOrElse(ByteVector32.Zeroes) val myCurrentPerCommitmentPoint = channelKeys.commitmentPoint(d.commitments.localCommitIndex) - // If we disconnected while signing a funding transaction, we may need our peer to retransmit their commit_sig. - val nextLocalCommitmentNumber = d match { - case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => d.status match { - case DualFundingStatus.RbfWaitingForSigs(status) => status.nextLocalCommitmentNumber - case _ => d.commitments.localCommitIndex + 1 - } - case d: DATA_NORMAL => d.spliceStatus match { - case SpliceStatus.SpliceWaitingForSigs(status) => status.nextLocalCommitmentNumber - case _ => d.commitments.localCommitIndex + 1 - } - case _ => d.commitments.localCommitIndex + 1 - } - // If we disconnected while signing a funding transaction, we may need our peer to (re)transmit their tx_signatures. + // If we disconnected while signing a funding transaction, we may need our peer to (re)transmit their tx_signatures and commit_sig. val rbfTlv: Set[ChannelReestablishTlv] = d match { case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => d.status match { - case DualFundingStatus.RbfWaitingForSigs(status) => Set(ChannelReestablishTlv.NextFundingTlv(status.fundingTx.txId)) + case DualFundingStatus.RbfWaitingForSigs(status) => Set(ChannelReestablishTlv.NextFundingTlv(status.fundingTx.txId, status.retransmitRemoteCommitSig)) case _ => d.latestFundingTx.sharedTx match { - case _: InteractiveTxBuilder.PartiallySignedSharedTransaction => Set(ChannelReestablishTlv.NextFundingTlv(d.latestFundingTx.sharedTx.txId)) + case _: InteractiveTxBuilder.PartiallySignedSharedTransaction => Set(ChannelReestablishTlv.NextFundingTlv(d.latestFundingTx.sharedTx.txId, retransmitCommitSig = false)) case _: InteractiveTxBuilder.FullySignedSharedTransaction => Set.empty } } case d: DATA_NORMAL => d.spliceStatus match { - case SpliceStatus.SpliceWaitingForSigs(status) => Set(ChannelReestablishTlv.NextFundingTlv(status.fundingTx.txId)) + case SpliceStatus.SpliceWaitingForSigs(status) => Set(ChannelReestablishTlv.NextFundingTlv(status.fundingTx.txId, status.retransmitRemoteCommitSig)) case _ => d.commitments.latest.localFundingStatus match { - case LocalFundingStatus.DualFundedUnconfirmedFundingTx(fundingTx: PartiallySignedSharedTransaction, _, _, _) => Set(ChannelReestablishTlv.NextFundingTlv(fundingTx.txId)) + case LocalFundingStatus.DualFundedUnconfirmedFundingTx(fundingTx: PartiallySignedSharedTransaction, _, _, _) => Set(ChannelReestablishTlv.NextFundingTlv(fundingTx.txId, retransmitCommitSig = false)) case _ => Set.empty } } @@ -2477,7 +2465,7 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall val channelReestablish = ChannelReestablish( channelId = d.channelId, - nextLocalCommitmentNumber = nextLocalCommitmentNumber, + nextLocalCommitmentNumber = d.commitments.localCommitIndex + 1, nextRemoteRevocationNumber = d.commitments.remoteCommitIndex, yourLastPerCommitmentSecret = PrivateKey(yourLastPerCommitmentSecret), myCurrentPerCommitmentPoint = myCurrentPerCommitmentPoint, @@ -2529,7 +2517,7 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall case _ => remoteNextCommitNonces = channelReestablish.nextCommitNonces channelReestablish.nextFundingTxId_opt match { - case Some(fundingTxId) if fundingTxId == d.signingSession.fundingTx.txId && channelReestablish.nextLocalCommitmentNumber == 0 => + case Some(fundingTxId) if fundingTxId == d.signingSession.fundingTx.txId && channelReestablish.retransmitInteractiveTxCommitSig => // They haven't received our commit_sig: we retransmit it, and will send our tx_signatures once we've received // their commit_sig or their tx_signatures (depending on who must send tx_signatures first). val fundingParams = d.signingSession.fundingParams @@ -2557,7 +2545,7 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall case Some(fundingTxId) => d.status match { case DualFundingStatus.RbfWaitingForSigs(signingSession) if signingSession.fundingTx.txId == fundingTxId => - if (channelReestablish.nextLocalCommitmentNumber == 0) { + if (channelReestablish.retransmitInteractiveTxCommitSig) { // They haven't received our commit_sig: we retransmit it. // We're also waiting for signatures from them, and will send our tx_signatures once we receive them. val fundingParams = signingSession.fundingParams @@ -2574,7 +2562,7 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall case _ if d.latestFundingTx.sharedTx.txId == fundingTxId => // We've already received their commit_sig and sent our tx_signatures. We retransmit our tx_signatures // and our commit_sig if they haven't received it already. - if (channelReestablish.nextLocalCommitmentNumber == 0) { + if (channelReestablish.retransmitInteractiveTxCommitSig) { val remoteNonce_opt = channelReestablish.currentCommitNonce_opt d.commitments.latest.remoteCommit.sign(d.commitments.channelParams, d.commitments.latest.remoteCommitParams, channelKeys, d.commitments.latest.fundingTxIndex, d.commitments.latest.remoteFundingPubKey, d.commitments.latest.commitInput(channelKeys), d.commitments.latest.commitmentFormat, remoteNonce_opt) match { case Left(e) => handleLocalError(e, d, Some(channelReestablish)) @@ -2612,7 +2600,7 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall channelReestablish.nextFundingTxId_opt match { case Some(fundingTxId) if fundingTxId == d.commitments.latest.fundingTxId => d.commitments.latest.localFundingStatus.localSigs_opt match { - case Some(txSigs) if channelReestablish.nextLocalCommitmentNumber == 0 => + case Some(txSigs) if channelReestablish.retransmitInteractiveTxCommitSig => log.info("re-sending commit_sig and tx_signatures for fundingTxIndex={} fundingTxId={}", d.commitments.latest.fundingTxIndex, d.commitments.latest.fundingTxId) val remoteNonce_opt = channelReestablish.currentCommitNonce_opt d.commitments.latest.remoteCommit.sign(d.commitments.channelParams, d.commitments.latest.remoteCommitParams, channelKeys, d.commitments.latest.fundingTxIndex, d.commitments.latest.remoteFundingPubKey, d.commitments.latest.commitInput(channelKeys), d.commitments.latest.commitmentFormat, remoteNonce_opt) match { @@ -2673,9 +2661,8 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall sendQueue = sendQueue ++ syncSuccess.retransmit commitments1.remoteNextCommitInfo match { - case Left(_) => - // we expect them to (re-)send the revocation immediately - startSingleTimer(RevocationTimeout.toString, RevocationTimeout(commitments1.remoteCommitIndex, peer), nodeParams.channelConf.revocationTimeout) + // we expect them to (re-)send their revocation immediately + case Left(_) => startSingleTimer(RevocationTimeout.toString, RevocationTimeout(commitments1.remoteCommitIndex, peer), nodeParams.channelConf.revocationTimeout) case _ => () } @@ -3399,7 +3386,7 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall case Some(fundingTxId) => d.spliceStatus match { case SpliceStatus.SpliceWaitingForSigs(signingSession) if signingSession.fundingTx.txId == fundingTxId => - if (channelReestablish.nextLocalCommitmentNumber == d.commitments.remoteCommitIndex) { + if (channelReestablish.retransmitInteractiveTxCommitSig) { // They haven't received our commit_sig: we retransmit it. // We're also waiting for signatures from them, and will send our tx_signatures once we receive them. log.info("re-sending commit_sig for splice attempt with fundingTxIndex={} fundingTxId={}", signingSession.fundingTxIndex, signingSession.fundingTx.txId) @@ -3416,7 +3403,7 @@ class Channel(val nodeParams: NodeParams, val channelKeys: ChannelKeys, val wall case dfu: LocalFundingStatus.DualFundedUnconfirmedFundingTx => // We've already received their commit_sig and sent our tx_signatures. We retransmit our // tx_signatures and our commit_sig if they haven't received it already. - if (channelReestablish.nextLocalCommitmentNumber == d.commitments.remoteCommitIndex) { + if (channelReestablish.retransmitInteractiveTxCommitSig) { log.info("re-sending commit_sig and tx_signatures for fundingTxIndex={} fundingTxId={}", d.commitments.latest.fundingTxIndex, d.commitments.latest.fundingTxId) val remoteNonce_opt = channelReestablish.currentCommitNonce_opt d.commitments.latest.remoteCommit.sign(d.commitments.channelParams, d.commitments.latest.remoteCommitParams, channelKeys, d.commitments.latest.fundingTxIndex, d.commitments.latest.remoteFundingPubKey, d.commitments.latest.commitInput(channelKeys), d.commitments.latest.commitmentFormat, remoteNonce_opt) match { 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 dee0675f8a..dffe650e1b 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 @@ -1189,11 +1189,8 @@ object InteractiveTxSigningSession { liquidityPurchase_opt: Option[LiquidityAds.PurchaseBasicInfo]) extends InteractiveTxSigningSession { val fundingTxId: TxId = fundingTx.txId val localCommitIndex: Long = localCommit.fold(_.index, _.index) - // This value tells our peer whether we need them to retransmit their commit_sig on reconnection or not. - val nextLocalCommitmentNumber: Long = localCommit match { - case Left(unsignedCommit) => unsignedCommit.index - case Right(commit) => commit.index + 1 - } + // If we haven't received the remote commit_sig, we will request a retransmission on reconnection. + val retransmitRemoteCommitSig: Boolean = localCommit.isLeft def localFundingKey(channelKeys: ChannelKeys): PrivateKey = channelKeys.fundingKey(fundingTxIndex) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/ChannelTlv.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/ChannelTlv.scala index 5dd8cfc8fd..29ddbd22ea 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/ChannelTlv.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/ChannelTlv.scala @@ -258,8 +258,11 @@ object ChannelReestablishTlv { /** * When disconnected in the middle of an interactive-tx session, this field is used to request a retransmission of * [[TxSignatures]] for the given [[txId]]. + * + * @param txId the txid of the partially signed funding transaction. + * @param retransmitCommitSig true if [[CommitSig]] must be retransmitted before [[TxSignatures]]. */ - case class NextFundingTlv(txId: TxId) extends ChannelReestablishTlv + case class NextFundingTlv(txId: TxId, retransmitCommitSig: Boolean) extends ChannelReestablishTlv /** * @param txId the txid of our latest outgoing [[ChannelReady]] or [[SpliceLocked]] for this channel. @@ -281,7 +284,7 @@ object ChannelReestablishTlv { case class NextLocalNoncesTlv(nonces: Seq[(TxId, IndividualNonce)]) extends ChannelReestablishTlv object NextFundingTlv { - val codec: Codec[NextFundingTlv] = tlvField(txIdAsHash) + val codec: Codec[NextFundingTlv] = tlvField(("next_funding_txid" | txIdAsHash) :: ("retransmit_flags" | (ignore(7) :: bool))) } object MyCurrentFundingLockedTlv { @@ -297,7 +300,7 @@ object ChannelReestablishTlv { } val channelReestablishTlvCodec: Codec[TlvStream[ChannelReestablishTlv]] = tlvStream(discriminated[ChannelReestablishTlv].by(varint) - .typecase(UInt64(0), NextFundingTlv.codec) + .typecase(UInt64(1), NextFundingTlv.codec) .typecase(UInt64(5), MyCurrentFundingLockedTlv.codec) .typecase(UInt64(22), NextLocalNoncesTlv.codec) .typecase(UInt64(24), CurrentCommitNonceTlv.codec) 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 7698dbd442..a48a63e0be 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 @@ -213,6 +213,7 @@ case class ChannelReestablish(channelId: ByteVector32, myCurrentPerCommitmentPoint: PublicKey, tlvStream: TlvStream[ChannelReestablishTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId { val nextFundingTxId_opt: Option[TxId] = tlvStream.get[ChannelReestablishTlv.NextFundingTlv].map(_.txId) + val retransmitInteractiveTxCommitSig: Boolean = tlvStream.get[ChannelReestablishTlv.NextFundingTlv].exists(_.retransmitCommitSig) val myCurrentFundingLocked_opt: Option[TxId] = tlvStream.get[ChannelReestablishTlv.MyCurrentFundingLockedTlv].map(_.txId) val retransmitAnnSigs: Boolean = tlvStream.get[ChannelReestablishTlv.MyCurrentFundingLockedTlv].exists(_.retransmitAnnSigs) val nextCommitNonces: Map[TxId, IndividualNonce] = tlvStream.get[ChannelReestablishTlv.NextLocalNoncesTlv].map(_.nonces.toMap).getOrElse(Map.empty) 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 04c6a37709..7d70993420 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 @@ -452,13 +452,15 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny alice ! INPUT_RECONNECTED(bob, aliceInit, bobInit) val channelReestablishAlice = alice2bob.expectMsgType[ChannelReestablish] - assert(channelReestablishAlice.nextLocalCommitmentNumber == 0) + assert(channelReestablishAlice.nextLocalCommitmentNumber == 1) + assert(channelReestablishAlice.retransmitInteractiveTxCommitSig) assert(channelReestablishAlice.currentCommitNonce_opt.nonEmpty) assert(channelReestablishAlice.nextCommitNonces.contains(fundingTxId)) bob ! INPUT_RECONNECTED(alice, bobInit, aliceInit) val channelReestablishBob = bob2alice.expectMsgType[ChannelReestablish] - assert(channelReestablishBob.nextLocalCommitmentNumber == 0) + assert(channelReestablishBob.nextLocalCommitmentNumber == 1) + assert(channelReestablishBob.retransmitInteractiveTxCommitSig) assert(channelReestablishBob.currentCommitNonce_opt.nonEmpty) assert(channelReestablishBob.nextCommitNonces.contains(fundingTxId)) @@ -557,12 +559,14 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny assert(channelReestablishAlice.nextCommitNonces.contains(fundingTx.txid)) assert(channelReestablishAlice.nextCommitNonces.get(fundingTx.txid) != channelReestablishAlice.currentCommitNonce_opt) assert(channelReestablishAlice.nextFundingTxId_opt.contains(fundingTx.txid)) - assert(channelReestablishAlice.nextLocalCommitmentNumber == 0) + assert(channelReestablishAlice.retransmitInteractiveTxCommitSig) + assert(channelReestablishAlice.nextLocalCommitmentNumber == 1) alice2bob.forward(bob, channelReestablishAlice) val channelReestablishBob = bob2alice.expectMsgType[ChannelReestablish] assert(channelReestablishBob.currentCommitNonce_opt.isEmpty) assert(channelReestablishBob.nextCommitNonces.get(fundingTx.txid) == channelReadyB.nextCommitNonce_opt) assert(channelReestablishBob.nextFundingTxId_opt.isEmpty) + assert(!channelReestablishBob.retransmitInteractiveTxCommitSig) assert(channelReestablishBob.nextLocalCommitmentNumber == 1) bob2alice.forward(alice, channelReestablishBob) @@ -700,14 +704,14 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny alice ! INPUT_RECONNECTED(bob, aliceInit, bobInit) bob ! INPUT_RECONNECTED(alice, bobInit, aliceInit) val channelReestablishAlice = alice2bob.expectMsgType[ChannelReestablish] - val nextLocalCommitmentNumberAlice = if (aliceExpectsCommitSig) 0 else 1 assert(channelReestablishAlice.nextFundingTxId_opt.contains(fundingTxId)) - assert(channelReestablishAlice.nextLocalCommitmentNumber == nextLocalCommitmentNumberAlice) + assert(channelReestablishAlice.retransmitInteractiveTxCommitSig == aliceExpectsCommitSig) + assert(channelReestablishAlice.nextLocalCommitmentNumber == 1) alice2bob.forward(bob, channelReestablishAlice) val channelReestablishBob = bob2alice.expectMsgType[ChannelReestablish] - val nextLocalCommitmentNumberBob = if (bobExpectsCommitSig) 0 else 1 assert(channelReestablishBob.nextFundingTxId_opt.contains(fundingTxId)) - assert(channelReestablishBob.nextLocalCommitmentNumber == nextLocalCommitmentNumberBob) + assert(channelReestablishBob.retransmitInteractiveTxCommitSig == bobExpectsCommitSig) + assert(channelReestablishBob.nextLocalCommitmentNumber == 1) bob2alice.forward(alice, channelReestablishBob) // When using taproot, we must provide nonces for the partial signatures. 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 7566087c06..41ebb3b23c 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 @@ -960,8 +960,10 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture val (channelReestablishAlice, channelReestablishBob) = reconnectRbf(f) assert(channelReestablishAlice.nextFundingTxId_opt.contains(rbfTxId)) - assert(channelReestablishAlice.nextLocalCommitmentNumber == 0) + assert(channelReestablishAlice.retransmitInteractiveTxCommitSig) + assert(channelReestablishAlice.nextLocalCommitmentNumber == 1) assert(channelReestablishBob.nextFundingTxId_opt.isEmpty) + assert(!channelReestablishBob.retransmitInteractiveTxCommitSig) assert(channelReestablishBob.nextLocalCommitmentNumber == 1) commitmentFormat match { case _: SegwitV0CommitmentFormat => () @@ -1008,9 +1010,11 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture val (channelReestablishAlice, channelReestablishBob) = reconnectRbf(f) assert(channelReestablishAlice.nextFundingTxId_opt.contains(rbfTxId)) + assert(!channelReestablishAlice.retransmitInteractiveTxCommitSig) assert(channelReestablishAlice.nextLocalCommitmentNumber == 1) assert(channelReestablishBob.nextFundingTxId_opt.contains(rbfTxId)) - assert(channelReestablishBob.nextLocalCommitmentNumber == 0) + assert(channelReestablishBob.retransmitInteractiveTxCommitSig) + assert(channelReestablishBob.nextLocalCommitmentNumber == 1) commitmentFormat match { case _: SegwitV0CommitmentFormat => () case _: TaprootCommitmentFormat => @@ -1069,8 +1073,10 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture val (channelReestablishAlice, channelReestablishBob) = reconnectRbf(f) assert(channelReestablishAlice.nextFundingTxId_opt.contains(rbfTxId)) - assert(channelReestablishAlice.nextLocalCommitmentNumber == 0) + assert(channelReestablishAlice.retransmitInteractiveTxCommitSig) + assert(channelReestablishAlice.nextLocalCommitmentNumber == 1) assert(channelReestablishBob.nextFundingTxId_opt.contains(rbfTxId)) + assert(!channelReestablishBob.retransmitInteractiveTxCommitSig) assert(channelReestablishBob.nextLocalCommitmentNumber == 1) commitmentFormat match { case _: SegwitV0CommitmentFormat => () @@ -1137,7 +1143,8 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture bob ! INPUT_RECONNECTED(alice, bobInit, aliceInit) val channelReestablishAlice = alice2bob.expectMsgType[ChannelReestablish] assert(channelReestablishAlice.nextFundingTxId_opt.nonEmpty) - assert(channelReestablishAlice.nextLocalCommitmentNumber == 0) + assert(channelReestablishAlice.retransmitInteractiveTxCommitSig) + assert(channelReestablishAlice.nextLocalCommitmentNumber == 1) assert(channelReestablishAlice.currentCommitNonce_opt.nonEmpty) bob2alice.expectMsgType[ChannelReestablish] alice2bob.forward(bob, channelReestablishAlice.copy(tlvStream = TlvStream(channelReestablishAlice.tlvStream.records.filterNot(_.isInstanceOf[ChannelReestablishTlv.CurrentCommitNonceTlv])))) 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 de219d8fc2..2a1e0ee57c 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 @@ -1829,6 +1829,8 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik reconnect(f) // Bob and Alice will exchange tx_abort because Bob did not receive Alice's tx_complete before the disconnect. + bob2alice.expectMsgType[ChannelReady] + bob2alice.forward(alice) bob2alice.expectMsgType[TxAbort] bob2alice.forward(alice) alice2bob.expectMsgType[ChannelReady] @@ -1874,14 +1876,16 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik disconnect(f) - // If Bob has not implemented https://github.com/lightning/bolts/pull/1214, he will send an incorrect next_commitment_number. + // If Bob has not implemented https://github.com/lightning/bolts/pull/1214, he will not ask for a retransmission of commit_sig. val (channelReestablishAlice1, channelReestablishBob1) = reconnect(f, sendReestablish = false) assert(channelReestablishAlice1.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId)) - assert(channelReestablishAlice1.nextLocalCommitmentNumber == aliceCommitIndex) + assert(channelReestablishAlice1.retransmitInteractiveTxCommitSig) + assert(channelReestablishAlice1.nextLocalCommitmentNumber == aliceCommitIndex + 1) assert(channelReestablishBob1.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId)) - assert(channelReestablishBob1.nextLocalCommitmentNumber == bobCommitIndex) + assert(channelReestablishBob1.retransmitInteractiveTxCommitSig) + assert(channelReestablishBob1.nextLocalCommitmentNumber == bobCommitIndex + 1) alice2bob.forward(bob, channelReestablishAlice1) - bob2alice.forward(alice, channelReestablishBob1.copy(nextLocalCommitmentNumber = bobCommitIndex + 1)) + bob2alice.forward(alice, channelReestablishBob1.copy(tlvStream = TlvStream(channelReestablishBob1.tlvStream.records.filterNot(_.isInstanceOf[ChannelReestablishTlv.NextFundingTlv])))) // In that case Alice won't retransmit commit_sig and the splice won't complete since they haven't exchanged tx_signatures. assert(bob2alice.expectMsgType[CommitSig].fundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId)) bob2alice.forward(alice) @@ -1895,13 +1899,15 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik alice ! cmd.copy(replyTo = probe.ref) probe.expectMsgType[RES_ADD_FAILED[ForbiddenDuringSplice]] - // But when correctly setting their next_commitment_number, they're able to finalize the splice. + // But when correctly setting their next_funding TLV, they're able to finalize the splice. disconnect(f) val (channelReestablishAlice2, channelReestablishBob2) = reconnect(f) assert(channelReestablishAlice2.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId)) + assert(!channelReestablishAlice2.retransmitInteractiveTxCommitSig) assert(channelReestablishAlice2.nextLocalCommitmentNumber == aliceCommitIndex + 1) assert(channelReestablishBob2.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId)) - assert(channelReestablishBob2.nextLocalCommitmentNumber == bobCommitIndex) + assert(channelReestablishBob2.retransmitInteractiveTxCommitSig) + assert(channelReestablishBob2.nextLocalCommitmentNumber == bobCommitIndex + 1) // Alice retransmits commit_sig and both retransmit tx_signatures. assert(alice2bob.expectMsgType[CommitSig].fundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId)) @@ -2041,9 +2047,11 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik disconnect(f) val (channelReestablishAlice, channelReestablishBob) = reconnect(f) assert(channelReestablishAlice.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTxId)) + assert(!channelReestablishAlice.retransmitInteractiveTxCommitSig) assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex + 1) assert(channelReestablishBob.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTxId)) - assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex) + assert(channelReestablishBob.retransmitInteractiveTxCommitSig) + assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex + 1) commitmentFormat match { case _: SegwitV0CommitmentFormat => () case _: SimpleTaprootChannelCommitmentFormat => @@ -2111,8 +2119,10 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik disconnect(f) val (channelReestablishAlice, channelReestablishBob) = reconnect(f) assert(channelReestablishAlice.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId)) - assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex) + assert(channelReestablishAlice.retransmitInteractiveTxCommitSig) + assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex + 1) assert(channelReestablishBob.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId)) + assert(!channelReestablishBob.retransmitInteractiveTxCommitSig) assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex + 1) commitmentFormat match { case _: SegwitV0CommitmentFormat => () @@ -2446,9 +2456,11 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik disconnect(f) val (channelReestablishAlice, channelReestablishBob) = reconnect(f) assert(channelReestablishAlice.nextFundingTxId_opt.contains(rbfTxId)) - assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex) + assert(channelReestablishAlice.retransmitInteractiveTxCommitSig) + assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex + 1) assert(channelReestablishBob.nextFundingTxId_opt.contains(rbfTxId)) - assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex) + assert(channelReestablishBob.retransmitInteractiveTxCommitSig) + assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex + 1) bob2blockchain.expectWatchFundingConfirmed(spliceTx.txid) // Alice and Bob retransmit commit_sig and tx_signatures. @@ -2493,9 +2505,11 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik disconnect(f) val (channelReestablishAlice, channelReestablishBob) = reconnect(f) assert(channelReestablishAlice.nextFundingTxId_opt.contains(rbfTxId)) + assert(!channelReestablishAlice.retransmitInteractiveTxCommitSig) assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex + 1) assert(channelReestablishBob.nextFundingTxId_opt.contains(rbfTxId)) - assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex) + assert(channelReestablishBob.retransmitInteractiveTxCommitSig) + assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex + 1) bob2blockchain.expectWatchFundingConfirmed(spliceTx.txid) // Alice retransmits commit_sig, and they exchange tx_signatures afterwards. @@ -2543,9 +2557,11 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik disconnect(f) val (channelReestablishAlice, channelReestablishBob) = reconnect(f) assert(channelReestablishAlice.nextFundingTxId_opt.contains(rbfTxId)) - assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex) + assert(channelReestablishAlice.retransmitInteractiveTxCommitSig) + assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex + 1) assert(channelReestablishAlice.currentCommitNonce_opt.nonEmpty) assert(channelReestablishBob.nextFundingTxId_opt.contains(rbfTxId)) + assert(!channelReestablishBob.retransmitInteractiveTxCommitSig) assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex + 1) assert(channelReestablishBob.currentCommitNonce_opt.isEmpty) Seq(channelReestablishAlice, channelReestablishBob).foreach(channelReestablish => { 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 b2df648038..b9a81b9445 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 @@ -160,7 +160,8 @@ class LightningMessageCodecsSpec extends AnyFunSuite { hex"0023" ++ channelId ++ signature ++ hex"fe47010000 07 cccccccccccccc" -> FundingSigned(channelId, signature, TlvStream[FundingSignedTlv](Set.empty[FundingSignedTlv], Set(GenericTlv(tlvTag, hex"cccccccccccccc")))), hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point), - hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"00 20" ++ txId.value.reverse -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream(ChannelReestablishTlv.NextFundingTlv(txId))), + hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"01 21" ++ txId.value.reverse ++ hex"00" -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream(ChannelReestablishTlv.NextFundingTlv(txId, retransmitCommitSig = false))), + hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"01 21" ++ txId.value.reverse ++ hex"01" -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream(ChannelReestablishTlv.NextFundingTlv(txId, retransmitCommitSig = true))), hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"05 21" ++ txId.value.reverse ++ hex"00" -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream(ChannelReestablishTlv.MyCurrentFundingLockedTlv(txId, retransmitAnnSigs = false))), hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"05 21" ++ txId.value.reverse ++ hex"01" -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream(ChannelReestablishTlv.MyCurrentFundingLockedTlv(txId, retransmitAnnSigs = true))), hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"18 42" ++ nonce.data -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream(ChannelReestablishTlv.CurrentCommitNonceTlv(nonce))), From f9d0384c27cb0c6d99a844a2837ceed4e7e40372 Mon Sep 17 00:00:00 2001 From: t-bast Date: Wed, 20 Aug 2025 11:48:34 +0200 Subject: [PATCH 4/4] Remove start-up protection Since this branch is only used on regtest, this will make it easier to test and iterate. --- eclair-node/src/main/scala/fr/acinq/eclair/Boot.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/Boot.scala b/eclair-node/src/main/scala/fr/acinq/eclair/Boot.scala index 1d6c4ec134..7e29046720 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/Boot.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/Boot.scala @@ -33,10 +33,6 @@ import scala.util.{Failure, Success} */ object Boot extends App with Logging { try { - if (!System.getProperty("eclair.allow-unsafe-startup", "false").toBooleanOption.contains(true)) { - throw new RuntimeException("This version of eclair is unsafe to use: please wait for the next official release to update your node.") - } - val datadir = new File(System.getProperty("eclair.datadir", System.getProperty("user.home") + "/.eclair")) val config = NodeParams.loadConfiguration(datadir)