Skip to content

Commit 0082aea

Browse files
committed
Update splicing protocol
The current "simple taproot channels" proposal is not compatible with splices. Supporting splices means supporting multiple commitment transactions that are valid at the same time, with the same commitment index but with different funding transactions. We need to extend the taproot proposal to include a list of musig2 nonces (one for each active commitment transaction). Similar to how commitment points are handled, `firstRemoteNonce` and `secondRemoteNonce` fields have been added to `SpliceInit` and `SpliceAck`, encoded as a list of nonces (instead of 2 expicit nonces) We also need a for the new commit tx that is being built, here it has been added to `SpliceInit` and `SpliceAck`. The funding tx that is being built during the interactive session needs to spend the current funding tx. For this, we re-use the scheme that we developped for our custome "swaproot" musig swap-ins: we add musig2 nonces to the `TxComplete` message, one nonce for each input that requires one, ordered by serial id. The life-cycle of these nonces is tied to the life-cycle of the interactive session which is never persisted (nonces here do not have to be deterministic).
1 parent 63b4d50 commit 0082aea

30 files changed

+646
-231
lines changed

eclair-core/pom.xml

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -188,6 +188,21 @@
188188
<version>4.1.94.Final</version>
189189
</dependency>
190190
<!-- BITCOIN -->
191+
<dependency>
192+
<groupId>fr.acinq.bitcoin</groupId>
193+
<artifactId>bitcoin-kmp-jvm</artifactId>
194+
<version>0.20.0-SNAPSHOT</version>
195+
</dependency>
196+
<dependency>
197+
<groupId>fr.acinq.secp256k1</groupId>
198+
<artifactId>secp256k1-kmp-jvm</artifactId>
199+
<version>0.15.0</version>
200+
</dependency>
201+
<dependency>
202+
<groupId>fr.acinq.secp256k1</groupId>
203+
<artifactId>secp256k1-kmp-jni-jvm</artifactId>
204+
<version>0.15.0</version>
205+
</dependency>
191206
<dependency>
192207
<groupId>fr.acinq</groupId>
193208
<artifactId>bitcoin-lib_${scala.version.short}</artifactId>

eclair-core/src/main/scala/fr/acinq/eclair/blockchain/fee/OnChainFeeConf.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -121,7 +121,7 @@ case class OnChainFeeConf(feeTargets: FeeTargets,
121121

122122
commitmentFormat match {
123123
case Transactions.DefaultCommitmentFormat => networkFeerate
124-
case _: Transactions.AnchorOutputsCommitmentFormat =>
124+
case _: Transactions.AnchorOutputsCommitmentFormat =>
125125
val targetFeerate = networkFeerate.min(feerateToleranceFor(remoteNodeId).anchorOutputMaxCommitFeerate)
126126
// We make sure the feerate is always greater than the propagation threshold.
127127
targetFeerate.max(networkMinFee * 1.25)

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

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -569,14 +569,15 @@ final case class DATA_WAIT_FOR_DUAL_FUNDING_CREATED(channelId: ByteVector32,
569569
remotePushAmount: MilliSatoshi,
570570
txBuilder: typed.ActorRef[InteractiveTxBuilder.Command],
571571
deferred: Option[CommitSig],
572-
remoteNextLocalNonce: Option[IndividualNonce],
572+
secondRemoteNonce_opt: Option[IndividualNonce],
573573
replyTo_opt: Option[akka.actor.typed.ActorRef[Peer.OpenChannelResponse]]) extends TransientChannelData
574574
final case class DATA_WAIT_FOR_DUAL_FUNDING_SIGNED(channelParams: ChannelParams,
575575
secondRemotePerCommitmentPoint: PublicKey,
576576
localPushAmount: MilliSatoshi,
577577
remotePushAmount: MilliSatoshi,
578578
signingSession: InteractiveTxSigningSession.WaitingForSigs,
579-
remoteChannelData_opt: Option[ByteVector]) extends ChannelDataWithoutCommitments
579+
remoteChannelData_opt: Option[ByteVector],
580+
secondRemoteNonce_opt: Option[IndividualNonce]) extends ChannelDataWithoutCommitments
580581
final case class DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments: Commitments,
581582
localPushAmount: MilliSatoshi,
582583
remotePushAmount: MilliSatoshi,

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

Lines changed: 10 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -131,18 +131,16 @@ object ChannelTypes {
131131
override def commitmentFormat: CommitmentFormat = ZeroFeeHtlcTxAnchorOutputsCommitmentFormat
132132
override def toString: String = s"anchor_outputs_zero_fee_htlc_tx${if (scidAlias) "+scid_alias" else ""}${if (zeroConf) "+zeroconf" else ""}"
133133
}
134-
case object SimpleTaprootChannelsStaging extends SupportedChannelType {
134+
case class SimpleTaprootChannelsStaging(scidAlias: Boolean = false, zeroConf: Boolean = false) extends SupportedChannelType {
135135
/** Known channel-type features */
136136
override def features: Set[ChannelTypeFeature] = Set(
137+
if (scidAlias) Some(Features.ScidAlias) else None,
138+
if (zeroConf) Some(Features.ZeroConf) else None,
137139
Some(Features.SimpleTaprootStaging)
138140
).flatten
139-
140-
/** True if our main output in the remote commitment is directly sent (without any delay) to one of our wallet addresses. */
141141
override def paysDirectlyToWallet: Boolean = false
142-
/** Format of the channel transactions. */
143142
override def commitmentFormat: CommitmentFormat = SimpleTaprootChannelsStagingCommitmentFormat
144-
145-
override def toString: String = "simple_taproot_channel_staging"
143+
override def toString: String = s"simple_taproot_channel_staging${if (scidAlias) "+scid_alias" else ""}${if (zeroConf) "+zeroconf" else ""}"
146144
}
147145

148146
case class UnsupportedChannelType(featureBits: Features[InitFeature]) extends ChannelType {
@@ -168,7 +166,11 @@ object ChannelTypes {
168166
AnchorOutputsZeroFeeHtlcTx(zeroConf = true),
169167
AnchorOutputsZeroFeeHtlcTx(scidAlias = true),
170168
AnchorOutputsZeroFeeHtlcTx(scidAlias = true, zeroConf = true),
171-
SimpleTaprootChannelsStaging)
169+
SimpleTaprootChannelsStaging(),
170+
SimpleTaprootChannelsStaging(zeroConf = true),
171+
SimpleTaprootChannelsStaging(scidAlias = true),
172+
SimpleTaprootChannelsStaging(scidAlias = true, zeroConf = true),
173+
)
172174
.map(channelType => Features(channelType.features.map(_ -> FeatureSupport.Mandatory).toMap) -> channelType)
173175
.toMap
174176

@@ -184,7 +186,7 @@ object ChannelTypes {
184186
val scidAlias = canUse(Features.ScidAlias) && !announceChannel // alias feature is incompatible with public channel
185187
val zeroConf = canUse(Features.ZeroConf)
186188
if (canUse(Features.SimpleTaprootStaging)) {
187-
SimpleTaprootChannelsStaging
189+
SimpleTaprootChannelsStaging(scidAlias, zeroConf)
188190
} else if (canUse(Features.AnchorOutputsZeroFeeHtlcTx)) {
189191
AnchorOutputsZeroFeeHtlcTx(scidAlias, zeroConf)
190192
} else if (canUse(Features.AnchorOutputs)) {

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

Lines changed: 44 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,7 @@
11
package fr.acinq.eclair.channel
22

3-
import akka.event.LoggingAdapter
4-
import fr.acinq.bitcoin.crypto.musig2.IndividualNonce
3+
import akka.event.{DiagnosticLoggingAdapter, LoggingAdapter}
4+
import fr.acinq.bitcoin.crypto.musig2.{IndividualNonce, SecretNonce}
55
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
66
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Crypto, Musig2, Satoshi, SatoshiLong, Script, Transaction, TxId}
77
import fr.acinq.eclair.blockchain.fee.{FeeratePerByte, FeeratePerKw, FeeratesPerKw, OnChainFeeConf}
@@ -17,6 +17,7 @@ import fr.acinq.eclair.transactions.Transactions._
1717
import fr.acinq.eclair.transactions._
1818
import fr.acinq.eclair.wire.protocol._
1919
import fr.acinq.eclair.{BlockHeight, CltvExpiry, CltvExpiryDelta, Features, MilliSatoshi, MilliSatoshiLong, payment}
20+
import grizzled.slf4j.Logging
2021
import scodec.bits.ByteVector
2122

2223
/** Static channel parameters shared by all commitments. */
@@ -229,11 +230,23 @@ case class LocalCommit(index: Long, spec: CommitmentSpec, commitTxAndRemoteSig:
229230
object LocalCommit {
230231
def fromCommitSig(keyManager: ChannelKeyManager, params: ChannelParams, fundingTxId: TxId,
231232
fundingTxIndex: Long, remoteFundingPubKey: PublicKey, commitInput: InputInfo,
232-
commit: CommitSig, localCommitIndex: Long, spec: CommitmentSpec, localPerCommitmentPoint: PublicKey): Either[ChannelException, LocalCommit] = {
233+
commit: CommitSig, localCommitIndex: Long, spec: CommitmentSpec, localPerCommitmentPoint: PublicKey, localNonce_opt: Option[(SecretNonce, IndividualNonce)]): Either[ChannelException, LocalCommit] = {
233234
val (localCommitTx, htlcTxs) = Commitment.makeLocalTxs(keyManager, params.channelConfig, params.channelFeatures, localCommitIndex, params.localParams, params.remoteParams, fundingTxIndex, remoteFundingPubKey, commitInput, localPerCommitmentPoint, spec)
234235
if (!localCommitTx.checkSig(commit, remoteFundingPubKey, TxOwner.Remote, params.commitmentFormat)) {
235236
return Left(InvalidCommitmentSignature(params.channelId, fundingTxId, fundingTxIndex, localCommitTx.tx))
236237
}
238+
commit.sigOrPartialSig match {
239+
case Left(_) =>
240+
if (!localCommitTx.checkSig(commit, remoteFundingPubKey, TxOwner.Remote, params.commitmentFormat)) {
241+
return Left(InvalidCommitmentSignature(params.channelId, fundingTxId, fundingTxIndex, localCommitTx.tx))
242+
}
243+
case Right(psig) =>
244+
val fundingPubkey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex).publicKey
245+
val Some(localNonce) = localNonce_opt
246+
if (!localCommitTx.checkPartialSignature(psig, fundingPubkey, localNonce._2, remoteFundingPubKey)) {
247+
return Left(InvalidCommitmentSignature(params.channelId, fundingTxId, fundingTxIndex, localCommitTx.tx))
248+
}
249+
}
237250
val sortedHtlcTxs = htlcTxs.sortBy(_.input.outPoint.index)
238251
if (commit.htlcSignatures.size != sortedHtlcTxs.size) {
239252
return Left(HtlcSigCountMismatch(params.channelId, sortedHtlcTxs.size, commit.htlcSignatures.size))
@@ -252,7 +265,7 @@ object LocalCommit {
252265

253266
/** The remote commitment maps to a commitment transaction that only our peer can sign and broadcast. */
254267
case class RemoteCommit(index: Long, spec: CommitmentSpec, txid: TxId, remotePerCommitmentPoint: PublicKey) {
255-
def sign(keyManager: ChannelKeyManager, params: ChannelParams, fundingTxIndex: Long, remoteFundingPubKey: PublicKey, commitInput: InputInfo, remoteNonce_opt: Option[IndividualNonce]): CommitSig = {
268+
def sign(keyManager: ChannelKeyManager, params: ChannelParams, fundingTxIndex: Long, remoteFundingPubKey: PublicKey, commitInput: InputInfo, remoteNonce_opt: Option[IndividualNonce])(implicit log: LoggingAdapter): CommitSig = {
256269
val (remoteCommitTx, htlcTxs) = Commitment.makeRemoteTxs(keyManager, params.channelConfig, params.channelFeatures, index, params.localParams, params.remoteParams, fundingTxIndex, remoteFundingPubKey, commitInput, remotePerCommitmentPoint, spec)
257270
val (sig, tlvStream) = params.commitmentFormat match {
258271
case SimpleTaprootChannelsStagingCommitmentFormat =>
@@ -653,6 +666,7 @@ case class Commitment(fundingTxIndex: Long,
653666
val localNonce = keyManager.signingNonce(params.localParams.fundingKeyPath, fundingTxIndex)
654667
val Some(remoteNonce) = nextRemoteNonce_opt
655668
val Right(psig) = keyManager.partialSign(remoteCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex), remoteFundingPubKey, TxOwner.Remote, localNonce, remoteNonce)
669+
log.debug(s"sendCommit: creating partial sig $psig for remote commit tx ${remoteCommitTx.tx.txid} with remote nonce $remoteNonce and remoteNextPerCommitmentPoint = $remoteNextPerCommitmentPoint")
656670
Set(CommitSigTlv.PartialSignatureWithNonceTlv(PartialSignatureWithNonce(psig, localNonce._2)))
657671
case _ =>
658672
Set.empty
@@ -668,11 +682,12 @@ case class Commitment(fundingTxIndex: Long,
668682
val commitSig = CommitSig(params.channelId, sig, htlcSigs.toList, TlvStream(Set(
669683
if (batchSize > 1) Some(CommitSigTlv.BatchTlv(batchSize)) else None
670684
).flatten[CommitSigTlv] ++ partialSig))
685+
log.debug(s"sendCommit: setting remoteNextPerCommitmentPoint to $remoteNextPerCommitmentPoint")
671686
val nextRemoteCommit = NextRemoteCommit(commitSig, RemoteCommit(remoteCommit.index + 1, spec, remoteCommitTx.tx.txid, remoteNextPerCommitmentPoint))
672687
(copy(nextRemoteCommit_opt = Some(nextRemoteCommit)), commitSig)
673688
}
674689

675-
def receiveCommit(keyManager: ChannelKeyManager, params: ChannelParams, changes: CommitmentChanges, localPerCommitmentPoint: PublicKey, commit: CommitSig)(implicit log: LoggingAdapter): Either[ChannelException, Commitment] = {
690+
def receiveCommit(keyManager: ChannelKeyManager, params: ChannelParams, changes: CommitmentChanges, localPerCommitmentPoint: PublicKey, commit: CommitSig, localNonce_opt: Option[(SecretNonce, IndividualNonce)])(implicit log: LoggingAdapter): Either[ChannelException, Commitment] = {
676691
// they sent us a signature for *their* view of *our* next commit tx
677692
// so in terms of rev.hashes and indexes we have:
678693
// ourCommit.index -> our current revocation hash, which is about to become our old revocation hash
@@ -683,7 +698,7 @@ case class Commitment(fundingTxIndex: Long,
683698
// and will increment our index
684699
val localCommitIndex = localCommit.index + 1
685700
val spec = CommitmentSpec.reduce(localCommit.spec, changes.localChanges.acked, changes.remoteChanges.proposed)
686-
LocalCommit.fromCommitSig(keyManager, params, fundingTxId, fundingTxIndex, remoteFundingPubKey, commitInput, commit, localCommitIndex, spec, localPerCommitmentPoint).map { localCommit1 =>
701+
LocalCommit.fromCommitSig(keyManager, params, fundingTxId, fundingTxIndex, remoteFundingPubKey, commitInput, commit, localCommitIndex, spec, localPerCommitmentPoint, localNonce_opt).map { localCommit1 =>
687702
log.info(s"built local commit number=$localCommitIndex toLocalMsat=${spec.toLocal.toLong} toRemoteMsat=${spec.toRemote.toLong} htlc_in={} htlc_out={} feeratePerKw=${spec.commitTxFeerate} txid=${localCommit1.commitTxAndRemoteSig.commitTx.tx.txid} fundingTxId=$fundingTxId", spec.htlcs.collect(DirectedHtlc.incoming).map(_.id).mkString(","), spec.htlcs.collect(DirectedHtlc.outgoing).map(_.id).mkString(","))
688703
copy(localCommit = localCommit1)
689704
}
@@ -698,9 +713,10 @@ case class Commitment(fundingTxIndex: Long,
698713
addSigs(unsignedCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex).publicKey, remoteFundingPubKey, localSig, remoteSig)
699714
case Right(remotePartialSigWithNonce) =>
700715
val fundingPubKey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex)
701-
val localNonce = keyManager.verificationNonce(params.localParams.fundingKeyPath, fundingTxIndex, ChannelKeyManager.keyPath(fundingPubKey.publicKey), localCommit.index)
716+
val channelKeyPath = ChannelKeyManager.keyPath(keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex = 0L))
717+
val localNonce = keyManager.verificationNonce(params.localParams.fundingKeyPath, fundingTxIndex, channelKeyPath, localCommit.index)
702718
val Right(partialSig) = keyManager.partialSign(unsignedCommitTx,
703-
keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex = 0), remoteFundingPubKey,
719+
keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex), remoteFundingPubKey,
704720
TxOwner.Local,
705721
localNonce, remotePartialSigWithNonce.nonce)
706722
val Right(aggSig) = Musig2.aggregateTaprootSignatures(
@@ -1037,11 +1053,17 @@ case class Commitments(params: ChannelParams,
10371053
}
10381054
}
10391055

1040-
def sendCommit(keyManager: ChannelKeyManager, nextRemoteNonce_opt: Option[IndividualNonce] = None)(implicit log: LoggingAdapter): Either[ChannelException, (Commitments, Seq[CommitSig])] = {
1056+
def sendCommit(keyManager: ChannelKeyManager, nextRemoteNonces: List[IndividualNonce] = List.empty)(implicit log: LoggingAdapter): Either[ChannelException, (Commitments, Seq[CommitSig])] = {
10411057
remoteNextCommitInfo match {
10421058
case Right(_) if !changes.localHasChanges => Left(CannotSignWithoutChanges(channelId))
10431059
case Right(remoteNextPerCommitmentPoint) =>
1044-
val (active1, sigs) = active.map(_.sendCommit(keyManager, params, changes, remoteNextPerCommitmentPoint, active.size, nextRemoteNonce_opt)).unzip
1060+
val (active1, sigs) = this.params.commitmentFormat match {
1061+
case SimpleTaprootChannelsStagingCommitmentFormat =>
1062+
require(active.size <= nextRemoteNonces.size, s"we have ${active.size} commitments but ${nextRemoteNonces.size} remote musig2 nonces")
1063+
active.zip(nextRemoteNonces).map { case (c, n) => c.sendCommit(keyManager, params, changes, remoteNextPerCommitmentPoint, active.size, Some(n)) } unzip
1064+
case _ =>
1065+
active.map(_.sendCommit(keyManager, params, changes, remoteNextPerCommitmentPoint, active.size, None)).unzip
1066+
}
10451067
val commitments1 = copy(
10461068
changes = changes.copy(
10471069
localChanges = changes.localChanges.copy(proposed = Nil, signed = changes.localChanges.proposed),
@@ -1066,7 +1088,11 @@ case class Commitments(params: ChannelParams,
10661088
val localPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, localCommitIndex + 1)
10671089
// Signatures are sent in order (most recent first), calling `zip` will drop trailing sigs that are for deactivated/pruned commitments.
10681090
val active1 = active.zip(commits).map { case (commitment, commit) =>
1069-
commitment.receiveCommit(keyManager, params, changes, localPerCommitmentPoint, commit) match {
1091+
val localNonce_opt = params.commitmentFormat match {
1092+
case SimpleTaprootChannelsStagingCommitmentFormat => Some(keyManager.verificationNonce(params.localParams.fundingKeyPath, commitment.fundingTxIndex, channelKeyPath, localCommitIndex + 1))
1093+
case _ => None
1094+
}
1095+
commitment.receiveCommit(keyManager, params, changes, localPerCommitmentPoint, commit, localNonce_opt) match {
10701096
case Left(f) => return Left(f)
10711097
case Right(commitment1) => commitment1
10721098
}
@@ -1076,9 +1102,12 @@ case class Commitments(params: ChannelParams,
10761102
val localNextPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, localCommitIndex + 2)
10771103
val tlvStream: TlvStream[RevokeAndAckTlv] = params.commitmentFormat match {
10781104
case SimpleTaprootChannelsStagingCommitmentFormat =>
1079-
val (_, nonce) = keyManager.verificationNonce(params.localParams.fundingKeyPath, this.latest.fundingTxIndex, channelKeyPath, localCommitIndex + 2)
1080-
log.debug("generating our next local nonce with {} {} {} {}", params.localParams.fundingKeyPath, this.latest.fundingTxIndex, channelKeyPath, localCommitIndex + 2)
1081-
TlvStream(RevokeAndAckTlv.NextLocalNonceTlv(nonce))
1105+
val nonces = this.active.map(c => {
1106+
val n = keyManager.verificationNonce(params.localParams.fundingKeyPath, c.fundingTxIndex, channelKeyPath, localCommitIndex + 2)
1107+
log.debug(s"revokeandack: creating verification nonce ${n._2} fundingIndex = ${c.fundingTxIndex} commit index = ${localCommitIndex + 2}")
1108+
n
1109+
})
1110+
TlvStream(RevokeAndAckTlv.NextLocalNoncesTlv(nonces.map(_._2).toList))
10821111
case _ =>
10831112
TlvStream.empty
10841113
}
@@ -1103,7 +1132,7 @@ case class Commitments(params: ChannelParams,
11031132
remoteNextCommitInfo match {
11041133
case Right(_) => Left(UnexpectedRevocation(channelId))
11051134
case Left(_) if revocation.perCommitmentSecret.publicKey != active.head.remoteCommit.remotePerCommitmentPoint => Left(InvalidRevocation(channelId))
1106-
case Left(_) if this.params.commitmentFormat == SimpleTaprootChannelsStagingCommitmentFormat && revocation.nexLocalNonce_opt.isEmpty => Left(MissingNextLocalNonce(channelId))
1135+
case Left(_) if this.params.commitmentFormat == SimpleTaprootChannelsStagingCommitmentFormat && revocation.nexLocalNonces.isEmpty => Left(MissingNextLocalNonce(channelId))
11071136
case Left(_) =>
11081137
// Since htlcs are shared across all commitments, we generate the actions only once based on the first commitment.
11091138
val receivedHtlcs = changes.remoteChanges.signed.collect {

0 commit comments

Comments
 (0)