Skip to content

Commit c981940

Browse files
committed
Use official splice messages
We replace our experimental version of `splice_init`, `splice_ack` and `splice_locked` by their official version. If our peer is using the experimental feature bit, we convert our outgoing messages to use the experimental encoding and incoming messages to the official messages. We also change the TLV fields added to `tx_add_input`, `tx_signatures` and `splice_locked` to match the spec version. We always write both the official and experimental TLV to updated nodes (because the experimental one is odd and will be ignored) but we drop the official TLV if our peer is using the experimental feature, because it won't understand the even TLV field. This guarantees backwards-compatibility with peers who only support the experimental feature.
1 parent 9625a6e commit c981940

File tree

22 files changed

+258
-91
lines changed

22 files changed

+258
-91
lines changed

docs/release-notes/eclair-vnext.md

Lines changed: 33 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,39 @@ Once closing transactions are broadcast, they can be RBF-ed by calling the `clos
2121
./eclair-cli close --channelId=<channel_id> --preferredFeerateSatByte=<rbf_feerate>
2222
```
2323

24+
### Channel Splicing
25+
26+
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.
27+
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.
28+
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.
29+
30+
The size of an existing channel can be increased with the `splicein` API:
31+
32+
```sh
33+
eclair-cli splicein --channelId=<channel_id> --amountIn=<amount_satoshis>
34+
```
35+
36+
Once that transaction confirms, the additional liquidity can be used to send outgoing payments.
37+
If the transaction doesn't confirm, the node operator can speed up confirmation with the `rbfsplice` API:
38+
39+
```sh
40+
eclair-cli rbfsplice --channelId=<channel_id> --targetFeerateSatByte=<feerate_satoshis_per_byte> --fundingFeeBudgetSatoshis=<maximum_on_chain_fee_satoshis>
41+
```
42+
43+
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:
44+
45+
```sh
46+
eclair-cli spliceout --channelId=<channel_id> --amountOut=<amount_satoshis> --scriptPubKey=<on_chain_address>
47+
```
48+
49+
That operation can also be RBF-ed with the `rbfsplice` API to speed up confirmation if necessary.
50+
51+
Note that when 0-conf is used for the channel, it is not possible to RBF splice transactions.
52+
Node operators should instead create a new splice transaction (with `splicein` or `spliceout`) to CPFP the previous transaction.
53+
54+
Note that eclair had already introduced support for a splicing prototype in v0.9.0, which helped improve the BOLT proposal.
55+
We're removing support for the previous splicing prototype feature: users that depended on this protocol must upgrade to create official splice transactions.
56+
2457
### Peer storage
2558

2659
With this release, eclair supports the `option_provide_storage` feature introduced in <https://github.com/lightning/bolts/pull/1110>.

eclair-core/src/main/resources/reference.conf

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -85,6 +85,7 @@ eclair {
8585
option_zeroconf = disabled
8686
keysend = disabled
8787
option_simple_close=optional
88+
option_splice = optional
8889
trampoline_payment_prototype = disabled
8990
async_payment_prototype = disabled
9091
on_the_fly_funding = disabled

eclair-core/src/main/scala/fr/acinq/eclair/Features.scala

Lines changed: 7 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -264,8 +264,7 @@ object Features {
264264
val mandatory = 28
265265
}
266266

267-
// TODO: this should also extend NodeFeature once the spec is finalized
268-
case object Quiescence extends Feature with InitFeature {
267+
case object Quiescence extends Feature with InitFeature with NodeFeature {
269268
val rfcName = "option_quiesce"
270269
val mandatory = 34
271270
}
@@ -310,6 +309,11 @@ object Features {
310309
val mandatory = 60
311310
}
312311

312+
case object Splicing extends Feature with InitFeature with NodeFeature {
313+
val rfcName = "option_splice"
314+
val mandatory = 62
315+
}
316+
313317
/** This feature bit indicates that the node is a mobile wallet that can be woken up via push notifications. */
314318
case object WakeUpNotificationClient extends Feature with InitFeature {
315319
val rfcName = "wake_up_notification_client"
@@ -333,12 +337,6 @@ object Features {
333337
val mandatory = 152
334338
}
335339

336-
// 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)
337-
case object SplicePrototype extends Feature with InitFeature {
338-
val rfcName = "splice_prototype"
339-
val mandatory = 154
340-
}
341-
342340
/**
343341
* Activate this feature to provide on-the-fly funding to remote nodes, as specified in bLIP 36: https://github.com/lightning/blips/blob/master/blip-0036.md.
344342
* TODO: add NodeFeature once bLIP is merged.
@@ -381,10 +379,10 @@ object Features {
381379
ZeroConf,
382380
KeySend,
383381
SimpleClose,
382+
Splicing,
384383
WakeUpNotificationClient,
385384
TrampolinePaymentPrototype,
386385
AsyncPaymentPrototype,
387-
SplicePrototype,
388386
OnTheFlyFunding,
389387
FundingFeeCredit
390388
)
@@ -401,7 +399,6 @@ object Features {
401399
KeySend -> (VariableLengthOnion :: Nil),
402400
SimpleClose -> (ShutdownAnySegwit :: Nil),
403401
AsyncPaymentPrototype -> (TrampolinePaymentPrototype :: Nil),
404-
OnTheFlyFunding -> (SplicePrototype :: Nil),
405402
FundingFeeCredit -> (OnTheFlyFunding :: Nil)
406403
)
407404

eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala

Lines changed: 9 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -682,9 +682,11 @@ case class Commitment(fundingTxIndex: Long,
682682
log.info(s"built remote commit number=${remoteCommit.index + 1} toLocalMsat=${spec.toLocal.toLong} toRemoteMsat=${spec.toRemote.toLong} htlc_in={} htlc_out={} feeratePerKw=${spec.commitTxFeerate} txid=${remoteCommitTx.tx.txid} fundingTxId=$fundingTxId", spec.htlcs.collect(DirectedHtlc.outgoing).map(_.id).mkString(","), spec.htlcs.collect(DirectedHtlc.incoming).map(_.id).mkString(","))
683683
Metrics.recordHtlcsInFlight(spec, remoteCommit.spec)
684684

685-
val commitSig = CommitSig(params.channelId, sig, htlcSigs.toList, TlvStream(Set(
686-
if (batchSize > 1) Some(CommitSigTlv.BatchTlv(batchSize)) else None
687-
).flatten[CommitSigTlv]))
685+
val tlvs = Set(
686+
if (batchSize > 1) Some(CommitSigTlv.BatchTlv(batchSize, fundingTxId)) else None,
687+
if (batchSize > 1) Some(CommitSigTlv.ExperimentalBatchTlv(batchSize)) else None,
688+
).flatten[CommitSigTlv]
689+
val commitSig = CommitSig(params.channelId, sig, htlcSigs.toList, TlvStream(tlvs))
688690
val nextRemoteCommit = NextRemoteCommit(commitSig, RemoteCommit(remoteCommit.index + 1, spec, remoteCommitTx.tx.txid, remoteNextPerCommitmentPoint))
689691
(copy(nextRemoteCommit_opt = Some(nextRemoteCommit)), commitSig)
690692
}
@@ -1081,8 +1083,10 @@ case class Commitments(params: ChannelParams,
10811083
}
10821084
val channelKeyPath = keyManager.keyPath(params.localParams, params.channelConfig)
10831085
val localPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, localCommitIndex + 1)
1084-
// Signatures are sent in order (most recent first), calling `zip` will drop trailing sigs that are for deactivated/pruned commitments.
1085-
val active1 = active.zip(commits).map { case (commitment, commit) =>
1086+
val active1 = active.zipWithIndex.map { case (commitment, idx) =>
1087+
// If the funding_txid isn't provided, we assume that signatures are sent in order (most recent first).
1088+
// This matches the behavior of peers who only support the experimental version of splicing.
1089+
val commit = commits.find(_.fundingTxId_opt.contains(commitment.fundingTxId)).getOrElse(commits(idx))
10861090
commitment.receiveCommit(keyManager, params, changes, localPerCommitmentPoint, commit) match {
10871091
case Left(f) => return Left(f)
10881092
case Right(commitment1) => commitment1

eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -878,7 +878,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
878878
}
879879

880880
case Event(cmd: CMD_SPLICE, d: DATA_NORMAL) =>
881-
if (!d.commitments.params.remoteParams.initFeatures.hasFeature(Features.SplicePrototype)) {
881+
if (!d.commitments.params.remoteParams.initFeatures.hasFeature(Features.Splicing)) {
882882
log.warning("cannot initiate splice, peer doesn't support splicing")
883883
cmd.replyTo ! RES_FAILURE(cmd, CommandUnavailableInThisState(d.channelId, "splice", NORMAL))
884884
stay()
@@ -2991,7 +2991,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
29912991
/** For splices we will send one commit_sig per active commitments. */
29922992
private def aggregateSigs(commit: CommitSig): Option[Seq[CommitSig]] = {
29932993
sigStash = sigStash :+ commit
2994-
log.debug("received sig for batch of size={}", commit.batchSize)
2994+
log.debug("received sig for batch of size={} for fundingTxId={}", commit.batchSize, commit.fundingTxId_opt)
29952995
if (sigStash.size == commit.batchSize) {
29962996
val sigs = sigStash
29972997
sigStash = Nil

eclair-core/src/main/scala/fr/acinq/eclair/io/PeerConnection.scala

Lines changed: 21 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,7 @@ package fr.acinq.eclair.io
1818

1919
import akka.actor.{ActorRef, FSM, OneForOneStrategy, PoisonPill, Props, Stash, SupervisorStrategy, Terminated}
2020
import akka.event.Logging.MDC
21-
import fr.acinq.bitcoin.scalacompat.{BlockHash, ByteVector32}
21+
import fr.acinq.bitcoin.scalacompat.BlockHash
2222
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
2323
import fr.acinq.eclair.Logs.LogCategory
2424
import fr.acinq.eclair.crypto.Noise.KeyPair
@@ -28,7 +28,7 @@ import fr.acinq.eclair.remote.EclairInternalsSerializer.RemoteTypes
2828
import fr.acinq.eclair.router.Router._
2929
import fr.acinq.eclair.wire.protocol
3030
import fr.acinq.eclair.wire.protocol._
31-
import fr.acinq.eclair.{FSMDiagnosticActorLogging, FeatureCompatibilityResult, Features, InitFeature, Logs, TimestampMilli, TimestampSecond}
31+
import fr.acinq.eclair.{FSMDiagnosticActorLogging, Features, InitFeature, Logs, TimestampMilli, TimestampSecond}
3232
import scodec.Attempt
3333
import scodec.bits.ByteVector
3434

@@ -206,11 +206,20 @@ class PeerConnection(keyPair: KeyPair, conf: PeerConnection.Conf, switchboard: A
206206
stay()
207207

208208
case Event(msg: LightningMessage, d: ConnectedData) if sender() != d.transport => // if the message doesn't originate from the transport, it is an outgoing message
209-
d.transport forward msg
209+
val useExperimentalSplice = d.remoteInit.features.unknown.map(_.bitIndex).contains(155)
210+
msg match {
211+
// If our peer is using the experimental splice version, we convert splice messages.
212+
case msg: SpliceInit if useExperimentalSplice => d.transport forward ExperimentalSpliceInit.from(msg)
213+
case msg: SpliceAck if useExperimentalSplice => d.transport forward ExperimentalSpliceAck.from(msg)
214+
case msg: SpliceLocked if useExperimentalSplice => d.transport forward ExperimentalSpliceLocked.from(msg)
215+
case msg: TxAddInput if useExperimentalSplice => d.transport forward msg.copy(tlvStream = TlvStream(msg.tlvStream.records.filterNot(_.isInstanceOf[TxAddInputTlv.SharedInputTxId])))
216+
case msg: CommitSig if useExperimentalSplice => d.transport forward msg.copy(tlvStream = TlvStream(msg.tlvStream.records.filterNot(_.isInstanceOf[CommitSigTlv.BatchTlv])))
217+
case msg: TxSignatures if useExperimentalSplice => d.transport forward msg.copy(tlvStream = TlvStream(msg.tlvStream.records.filterNot(_.isInstanceOf[TxSignaturesTlv.PreviousFundingTxSig])))
218+
case _ => d.transport forward msg
219+
}
210220
msg match {
211221
// If we send any channel management message to this peer, the connection should be persistent.
212-
case _: ChannelMessage if !d.isPersistent =>
213-
stay() using d.copy(isPersistent = true)
222+
case _: ChannelMessage if !d.isPersistent => stay() using d.copy(isPersistent = true)
214223
case _ => stay()
215224
}
216225

@@ -343,7 +352,13 @@ class PeerConnection(keyPair: KeyPair, conf: PeerConnection.Conf, switchboard: A
343352
case Event(msg: LightningMessage, d: ConnectedData) =>
344353
// we acknowledge and pass all other messages to the peer
345354
d.transport ! TransportHandler.ReadAck(msg)
346-
d.peer ! msg
355+
msg match {
356+
// If our peer is using the experimental splice version, we convert splice messages.
357+
case msg: ExperimentalSpliceInit => d.peer ! msg.toSpliceInit()
358+
case msg: ExperimentalSpliceAck => d.peer ! msg.toSpliceAck()
359+
case msg: ExperimentalSpliceLocked => d.peer ! msg.toSpliceLocked()
360+
case _ => d.peer ! msg
361+
}
347362
stay()
348363

349364
case Event(readAck: TransportHandler.ReadAck, d: ConnectedData) =>

eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/HtlcTlv.scala

Lines changed: 16 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717
package fr.acinq.eclair.wire.protocol
1818

1919
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
20+
import fr.acinq.bitcoin.scalacompat.TxId
2021
import fr.acinq.eclair.UInt64
2122
import fr.acinq.eclair.wire.protocol.CommonCodecs._
2223
import fr.acinq.eclair.wire.protocol.TlvCodecs.{tlvField, tlvStream, tu16}
@@ -73,16 +74,25 @@ object UpdateFailMalformedHtlcTlv {
7374
sealed trait CommitSigTlv extends Tlv
7475

7576
object CommitSigTlv {
77+
/**
78+
* While a splice is ongoing and not locked, we have multiple valid commitments.
79+
* We send one [[CommitSig]] message for each valid commitment.
80+
*
81+
* @param size the number of [[CommitSig]] messages in the batch.
82+
* @param fundingTxId the funding transaction spent by this commitment.
83+
*/
84+
case class BatchTlv(size: Int, fundingTxId: TxId) extends CommitSigTlv
7685

77-
/** @param size the number of [[CommitSig]] messages in the batch */
78-
case class BatchTlv(size: Int) extends CommitSigTlv
86+
private val batchTlv: Codec[BatchTlv] = tlvField(uint16 :: txIdAsHash)
7987

80-
object BatchTlv {
81-
val codec: Codec[BatchTlv] = tlvField(tu16)
82-
}
88+
/** Similar to [[BatchTlv]] for peers who only support the experimental version of splicing. */
89+
case class ExperimentalBatchTlv(size: Int) extends CommitSigTlv
90+
91+
private val experimentalBatchTlv: Codec[ExperimentalBatchTlv] = tlvField(tu16)
8392

8493
val commitSigTlvCodec: Codec[TlvStream[CommitSigTlv]] = tlvStream(discriminated[CommitSigTlv].by(varint)
85-
.typecase(UInt64(0x47010005), BatchTlv.codec)
94+
.typecase(UInt64(0), batchTlv)
95+
.typecase(UInt64(0x47010005), experimentalBatchTlv)
8696
)
8797

8898
}

eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/InteractiveTxTlv.scala

Lines changed: 10 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -33,9 +33,13 @@ object TxAddInputTlv {
3333
/** When doing a splice, the initiator must provide the previous funding txId instead of the whole transaction. */
3434
case class SharedInputTxId(txId: TxId) extends TxAddInputTlv
3535

36+
/** Same as [[SharedInputTxId]] for peers who only support the experimental version of splicing. */
37+
case class ExperimentalSharedInputTxId(txId: TxId) extends TxAddInputTlv
38+
3639
val txAddInputTlvCodec: Codec[TlvStream[TxAddInputTlv]] = tlvStream(discriminated[TxAddInputTlv].by(varint)
3740
// Note that we actually encode as a tx_hash to be consistent with other lightning messages.
38-
.typecase(UInt64(1105), tlvField(txIdAsHash.as[SharedInputTxId]))
41+
.typecase(UInt64(0), tlvField(txIdAsHash.as[SharedInputTxId]))
42+
.typecase(UInt64(1105), tlvField(txIdAsHash.as[ExperimentalSharedInputTxId]))
3943
)
4044
}
4145

@@ -69,8 +73,12 @@ object TxSignaturesTlv {
6973
/** When doing a splice, each peer must provide their signature for the previous 2-of-2 funding output. */
7074
case class PreviousFundingTxSig(sig: ByteVector64) extends TxSignaturesTlv
7175

76+
/** Same as [[PreviousFundingTxSig]] for peers who only support the experimental version of splicing. */
77+
case class ExperimentalPreviousFundingTxSig(sig: ByteVector64) extends TxSignaturesTlv
78+
7279
val txSignaturesTlvCodec: Codec[TlvStream[TxSignaturesTlv]] = tlvStream(discriminated[TxSignaturesTlv].by(varint)
73-
.typecase(UInt64(601), tlvField(bytes64.as[PreviousFundingTxSig]))
80+
.typecase(UInt64(0), tlvField(bytes64.as[PreviousFundingTxSig]))
81+
.typecase(UInt64(601), tlvField(bytes64.as[ExperimentalPreviousFundingTxSig]))
7482
)
7583
}
7684

eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageCodecs.scala

Lines changed: 25 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -434,17 +434,36 @@ object LightningMessageCodecs {
434434
("fundingPubkey" | publicKey) ::
435435
("tlvStream" | SpliceInitTlv.spliceInitTlvCodec)).as[SpliceInit]
436436

437+
val experimentalSpliceInitCodec: Codec[ExperimentalSpliceInit] = (
438+
("channelId" | bytes32) ::
439+
("fundingContribution" | satoshiSigned) ::
440+
("feerate" | feeratePerKw) ::
441+
("lockTime" | uint32) ::
442+
("fundingPubkey" | publicKey) ::
443+
("tlvStream" | SpliceInitTlv.spliceInitTlvCodec)).as[ExperimentalSpliceInit]
444+
437445
val spliceAckCodec: Codec[SpliceAck] = (
438446
("channelId" | bytes32) ::
439447
("fundingContribution" | satoshiSigned) ::
440448
("fundingPubkey" | publicKey) ::
441449
("tlvStream" | SpliceAckTlv.spliceAckTlvCodec)).as[SpliceAck]
442450

451+
val experimentalSpliceAckCodec: Codec[ExperimentalSpliceAck] = (
452+
("channelId" | bytes32) ::
453+
("fundingContribution" | satoshiSigned) ::
454+
("fundingPubkey" | publicKey) ::
455+
("tlvStream" | SpliceAckTlv.spliceAckTlvCodec)).as[ExperimentalSpliceAck]
456+
443457
val spliceLockedCodec: Codec[SpliceLocked] = (
444458
("channelId" | bytes32) ::
445459
("fundingTxHash" | txIdAsHash) ::
446460
("tlvStream" | SpliceLockedTlv.spliceLockedTlvCodec)).as[SpliceLocked]
447461

462+
val experimentalSpliceLockedCodec: Codec[ExperimentalSpliceLocked] = (
463+
("channelId" | bytes32) ::
464+
("fundingTxHash" | txIdAsHash) ::
465+
("tlvStream" | SpliceLockedTlv.spliceLockedTlvCodec)).as[ExperimentalSpliceLocked]
466+
448467
val stfuCodec: Codec[Stfu] = (
449468
("channelId" | bytes32) ::
450469
("initiator" | byte.xmap[Boolean](b => b != 0, b => if (b) 1 else 0))).as[Stfu]
@@ -526,6 +545,9 @@ object LightningMessageCodecs {
526545
.typecase(72, txInitRbfCodec)
527546
.typecase(73, txAckRbfCodec)
528547
.typecase(74, txAbortCodec)
548+
.typecase(77, spliceLockedCodec)
549+
.typecase(80, spliceInitCodec)
550+
.typecase(81, spliceAckCodec)
529551
.typecase(128, updateAddHtlcCodec)
530552
.typecase(130, updateFulfillHtlcCodec)
531553
.typecase(131, updateFailHtlcCodec)
@@ -557,9 +579,9 @@ object LightningMessageCodecs {
557579
.typecase(41045, addFeeCreditCodec)
558580
.typecase(41046, currentFeeCreditCodec)
559581
//
560-
.typecase(37000, spliceInitCodec)
561-
.typecase(37002, spliceAckCodec)
562-
.typecase(37004, spliceLockedCodec)
582+
.typecase(37000, experimentalSpliceInitCodec)
583+
.typecase(37002, experimentalSpliceAckCodec)
584+
.typecase(37004, experimentalSpliceLockedCodec)
563585
//
564586

565587
//

0 commit comments

Comments
 (0)