Skip to content

Commit a77aa5f

Browse files
committed
Get ready to store partial signatures
We currently store our peer's signature for our remote commit tx, so we can publish it if needed. If we upgrade funding tx to use musig2 instead of multisig 2-of-2 we will need to store a partial signature instead.
1 parent ef1a029 commit a77aa5f

File tree

13 files changed

+45
-22
lines changed

13 files changed

+45
-22
lines changed

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

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

33
import akka.event.LoggingAdapter
4+
import fr.acinq.bitcoin.crypto.musig2.IndividualNonce
45
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
56
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Crypto, Satoshi, SatoshiLong, Script, Transaction, TxId}
67
import fr.acinq.eclair.blockchain.fee.{FeeratePerByte, FeeratePerKw, FeeratesPerKw, OnChainFeeConf}
@@ -214,8 +215,10 @@ object CommitmentChanges {
214215

215216
case class HtlcTxAndRemoteSig(htlcTx: HtlcTx, remoteSig: ByteVector64)
216217

218+
case class PartialSignatureWithNonce(partialSig: ByteVector32, nonce: IndividualNonce)
219+
217220
/** We don't store the fully signed transaction, otherwise someone with read access to our database could force-close our channels. */
218-
case class CommitTxAndRemoteSig(commitTx: CommitTx, remoteSig: ByteVector64)
221+
case class CommitTxAndRemoteSig(commitTx: CommitTx, remoteSig: Either[ByteVector64, PartialSignatureWithNonce])
219222

220223
/** The local commitment maps to a commitment transaction that we can sign and broadcast if necessary. */
221224
case class LocalCommit(index: Long, spec: CommitmentSpec, commitTxAndRemoteSig: CommitTxAndRemoteSig, htlcTxsAndRemoteSigs: List[HtlcTxAndRemoteSig])
@@ -240,7 +243,7 @@ object LocalCommit {
240243
}
241244
HtlcTxAndRemoteSig(htlcTx, remoteSig)
242245
}
243-
Right(LocalCommit(localCommitIndex, spec, CommitTxAndRemoteSig(localCommitTx, commit.signature), htlcTxsAndRemoteSigs))
246+
Right(LocalCommit(localCommitIndex, spec, CommitTxAndRemoteSig(localCommitTx, Left(commit.signature)), htlcTxsAndRemoteSigs))
244247
}
245248
}
246249

@@ -663,7 +666,7 @@ case class Commitment(fundingTxIndex: Long,
663666
def fullySignedLocalCommitTx(params: ChannelParams, keyManager: ChannelKeyManager): CommitTx = {
664667
val unsignedCommitTx = localCommit.commitTxAndRemoteSig.commitTx
665668
val localSig = keyManager.sign(unsignedCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex), TxOwner.Local, params.commitmentFormat)
666-
val remoteSig = localCommit.commitTxAndRemoteSig.remoteSig
669+
val Left(remoteSig) = localCommit.commitTxAndRemoteSig.remoteSig
667670
val commitTx = addSigs(unsignedCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex).publicKey, remoteFundingPubKey, localSig, remoteSig)
668671
// We verify the remote signature when receiving their commit_sig, so this check should always pass.
669672
require(checkSpendable(commitTx).isSuccess, "commit signatures are invalid")
@@ -1151,7 +1154,7 @@ case class Commitments(params: ChannelParams,
11511154

11521155
/** This function should be used to ignore a commit_sig that we've already received. */
11531156
def ignoreRetransmittedCommitSig(commitSig: CommitSig): Boolean = {
1154-
val latestRemoteSig = latest.localCommit.commitTxAndRemoteSig.remoteSig
1157+
val Left(latestRemoteSig) = latest.localCommit.commitTxAndRemoteSig.remoteSig
11551158
params.channelFeatures.hasFeature(Features.DualFunding) && commitSig.batchSize == 1 && latestRemoteSig == commitSig.signature
11561159
}
11571160

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -281,7 +281,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
281281
remoteFundingPubKey = remoteFundingPubKey,
282282
localFundingStatus = SingleFundedUnconfirmedFundingTx(None),
283283
remoteFundingStatus = RemoteFundingStatus.NotLocked,
284-
localCommit = LocalCommit(0, localSpec, CommitTxAndRemoteSig(localCommitTx, remoteSig), htlcTxsAndRemoteSigs = Nil),
284+
localCommit = LocalCommit(0, localSpec, CommitTxAndRemoteSig(localCommitTx, Left(remoteSig)), htlcTxsAndRemoteSigs = Nil),
285285
remoteCommit = RemoteCommit(0, remoteSpec, remoteCommitTx.tx.txid, remoteFirstPerCommitmentPoint),
286286
nextRemoteCommit_opt = None)
287287
val commitments = Commitments(
@@ -328,7 +328,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
328328
remoteFundingPubKey = remoteFundingPubKey,
329329
localFundingStatus = SingleFundedUnconfirmedFundingTx(Some(fundingTx)),
330330
remoteFundingStatus = RemoteFundingStatus.NotLocked,
331-
localCommit = LocalCommit(0, localSpec, CommitTxAndRemoteSig(localCommitTx, remoteSig), htlcTxsAndRemoteSigs = Nil),
331+
localCommit = LocalCommit(0, localSpec, CommitTxAndRemoteSig(localCommitTx, Left(remoteSig)), htlcTxsAndRemoteSigs = Nil),
332332
remoteCommit = remoteCommit,
333333
nextRemoteCommit_opt = None
334334
)

eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1008,7 +1008,7 @@ object InteractiveTxSigningSession {
10081008
case class UnsignedLocalCommit(index: Long, spec: CommitmentSpec, commitTx: CommitTx, htlcTxs: List[HtlcTx])
10091009

10101010
private def shouldSignFirst(isInitiator: Boolean, channelParams: ChannelParams, tx: SharedTransaction): Boolean = {
1011-
val sharedAmountIn = tx.sharedInput_opt.map(_.txOut.amount).getOrElse(0 sat)
1011+
val sharedAmountIn = tx.sharedInput_opt.map(i => i.localAmount + i.remoteAmount + i.htlcAmount).getOrElse(0 msat).truncateToSatoshi
10121012
val (localAmountIn, remoteAmountIn) = if (isInitiator) {
10131013
(sharedAmountIn + tx.localInputs.map(i => i.txOut.amount).sum, tx.remoteInputs.map(i => i.txOut.amount).sum)
10141014
} else {

eclair-core/src/main/scala/fr/acinq/eclair/remote/LightningMessageSerializer.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,6 @@
1616

1717
package fr.acinq.eclair.remote
1818

19-
import fr.acinq.eclair.wire.protocol.LightningMessageCodecs.lightningMessageCodecWithFallback
19+
import fr.acinq.eclair.wire.protocol.LightningMessageCodecs.lightningMessageCodec
2020

21-
class LightningMessageSerializer extends ScodecSerializer(42, lightningMessageCodecWithFallback)
21+
class LightningMessageSerializer extends ScodecSerializer(42, lightningMessageCodec)

eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version0/ChannelTypes0.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -121,7 +121,7 @@ private[channel] object ChannelTypes0 {
121121
def migrate(remoteFundingPubKey: PublicKey): channel.LocalCommit = {
122122
val remoteSig = extractRemoteSig(publishableTxs.commitTx, remoteFundingPubKey)
123123
val unsignedCommitTx = publishableTxs.commitTx.modify(_.tx.txIn.each.witness).setTo(ScriptWitness.empty)
124-
val commitTxAndRemoteSig = CommitTxAndRemoteSig(unsignedCommitTx, remoteSig)
124+
val commitTxAndRemoteSig = CommitTxAndRemoteSig(unsignedCommitTx, Left(remoteSig))
125125
val htlcTxsAndRemoteSigs = publishableTxs.htlcTxsAndSigs map {
126126
case HtlcTxAndSigs(htlcTx: HtlcSuccessTx, _, remoteSig) =>
127127
val unsignedHtlcTx = htlcTx.modify(_.tx.txIn.each.witness).setTo(ScriptWitness.empty)

eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -203,7 +203,7 @@ private[channel] object ChannelCodecs3 {
203203

204204
val commitTxAndRemoteSigCodec: Codec[CommitTxAndRemoteSig] = (
205205
("commitTx" | commitTxCodec) ::
206-
("remoteSig" | bytes64)).as[CommitTxAndRemoteSig]
206+
("remoteSig" | either(provide(false), bytes64, partialSignatureWithNonce))).as[CommitTxAndRemoteSig]
207207

208208
val localCommitCodec: Codec[LocalCommit] = (
209209
("index" | uint64overflow) ::

eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version4/ChannelCodecs4.scala

Lines changed: 13 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -2,7 +2,7 @@ package fr.acinq.eclair.wire.internal.channel.version4
22

33
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
44
import fr.acinq.bitcoin.scalacompat.DeterministicWallet.KeyPath
5-
import fr.acinq.bitcoin.scalacompat.{OutPoint, ScriptWitness, Transaction, TxOut}
5+
import fr.acinq.bitcoin.scalacompat.{ByteVector64, OutPoint, ScriptWitness, Transaction, TxOut}
66
import fr.acinq.eclair.blockchain.fee.{ConfirmationPriority, ConfirmationTarget}
77
import fr.acinq.eclair.channel.LocalFundingStatus._
88
import fr.acinq.eclair.channel._
@@ -184,9 +184,19 @@ private[channel] object ChannelCodecs4 {
184184
("txinfo" | htlcTxCodec) ::
185185
("remoteSig" | bytes64)).as[HtlcTxAndRemoteSig]
186186

187-
val commitTxAndRemoteSigCodec: Codec[CommitTxAndRemoteSig] = (
187+
private case class CommitTxAndRemoteSigEx(commitTx: CommitTx, remoteSig: ByteVector64, partialSig: Either[ByteVector64, PartialSignatureWithNonce], dummy: Boolean)
188+
189+
// remoteSig is now either a signature or a partial signature with nonce. To retain compatibility with the previous codec, we use remoteSig as a left/right indicator,
190+
// a value of all zeroes meaning right (a valid signature cannot be all zeroes)
191+
private val commitTxAndRemoteSigExCodec: Codec[CommitTxAndRemoteSigEx] = (
188192
("commitTx" | commitTxCodec) ::
189-
("remoteSig" | bytes64)).as[CommitTxAndRemoteSig]
193+
(("remoteSig" | bytes64) >>:~ { remoteSig => either(provide(remoteSig == ByteVector64.Zeroes), provide(remoteSig), partialSignatureWithNonce) :: ("dummy" | provide(false)) })
194+
).as[CommitTxAndRemoteSigEx]
195+
196+
val commitTxAndRemoteSigCodec: Codec[CommitTxAndRemoteSig] = commitTxAndRemoteSigExCodec.xmap(
197+
ce => CommitTxAndRemoteSig(ce.commitTx, ce.partialSig),
198+
c => CommitTxAndRemoteSigEx(c.commitTx, c.remoteSig.swap.toOption.getOrElse(fr.acinq.bitcoin.scalacompat.ByteVector64.Zeroes), c.remoteSig, false)
199+
)
190200

191201
val updateMessageCodec: Codec[UpdateMessage] = lengthDelimited(lightningMessageCodec.narrow[UpdateMessage](f => Attempt.successful(f.asInstanceOf[UpdateMessage]), g => g))
192202

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

Lines changed: 9 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -16,10 +16,11 @@
1616

1717
package fr.acinq.eclair.wire.protocol
1818

19+
import fr.acinq.bitcoin.crypto.musig2.IndividualNonce
1920
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
2021
import fr.acinq.bitcoin.scalacompat.{BlockHash, ByteVector32, ByteVector64, Satoshi, Transaction, TxHash, TxId}
2122
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
22-
import fr.acinq.eclair.channel.{ChannelFlags, RealScidStatus, ShortIds}
23+
import fr.acinq.eclair.channel.{ChannelFlags, PartialSignatureWithNonce, RealScidStatus, ShortIds}
2324
import fr.acinq.eclair.crypto.Mac32
2425
import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Feature, Features, InitFeature, MilliSatoshi, RealShortChannelId, ShortChannelId, TimestampSecond, UInt64, UnspecifiedShortChannelId}
2526
import org.apache.commons.codec.binary.Base32
@@ -167,6 +168,13 @@ object CommonCodecs {
167168
(wire: BitVector) => bytes(33).decode(wire).map(_.map(b => PublicKey(b)))
168169
)
169170

171+
val publicNonce: Codec[IndividualNonce] = Codec[IndividualNonce](
172+
(pub: IndividualNonce) => bytes(66).encode(ByteVector.view(pub.toByteArray)),
173+
(wire: BitVector) => bytes(66).decode(wire).map(_.map(b => new IndividualNonce(b.toArray)))
174+
)
175+
176+
val partialSignatureWithNonce: Codec[PartialSignatureWithNonce] = (bytes32 :: publicNonce).as[PartialSignatureWithNonce]
177+
170178
val rgb: Codec[Color] = bytes(3).xmap(buf => Color(buf(0), buf(1), buf(2)), t => ByteVector(t.r, t.g, t.b))
171179

172180
val txCodec: Codec[Transaction] = bytes.xmap(d => Transaction.read(d.toArray), d => Transaction.write(d))

eclair-core/src/test/scala/fr/acinq/eclair/channel/CommitmentsSpec.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -491,7 +491,7 @@ object CommitmentsSpec {
491491
val remoteParams = RemoteParams(randomKey().publicKey, dustLimit, UInt64.MaxValue, Some(channelReserve), 1 msat, CltvExpiryDelta(144), 50, randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, Features.empty, None)
492492
val remoteFundingPubKey = randomKey().publicKey
493493
val commitmentInput = Funding.makeFundingInputInfo(randomTxId(), 0, (toLocal + toRemote).truncateToSatoshi, randomKey().publicKey, remoteFundingPubKey)
494-
val localCommit = LocalCommit(0, CommitmentSpec(Set.empty, feeRatePerKw, toLocal, toRemote), CommitTxAndRemoteSig(CommitTx(commitmentInput, Transaction(2, Nil, Nil, 0)), ByteVector64.Zeroes), Nil)
494+
val localCommit = LocalCommit(0, CommitmentSpec(Set.empty, feeRatePerKw, toLocal, toRemote), CommitTxAndRemoteSig(CommitTx(commitmentInput, Transaction(2, Nil, Nil, 0)), Left(ByteVector64.Zeroes)), Nil)
495495
val remoteCommit = RemoteCommit(0, CommitmentSpec(Set.empty, feeRatePerKw, toRemote, toLocal), randomTxId(), randomKey().publicKey)
496496
Commitments(
497497
ChannelParams(randomBytes32(), ChannelConfig.standard, ChannelFeatures(), localParams, remoteParams, ChannelFlags(announceChannel = announceChannel)),
@@ -510,7 +510,7 @@ object CommitmentsSpec {
510510
val remoteParams = RemoteParams(remoteNodeId, 0 sat, UInt64.MaxValue, Some(channelReserve), 1 msat, CltvExpiryDelta(144), 50, randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, randomKey().publicKey, Features.empty, None)
511511
val remoteFundingPubKey = randomKey().publicKey
512512
val commitmentInput = Funding.makeFundingInputInfo(randomTxId(), 0, (toLocal + toRemote).truncateToSatoshi, randomKey().publicKey, remoteFundingPubKey)
513-
val localCommit = LocalCommit(0, CommitmentSpec(Set.empty, FeeratePerKw(0 sat), toLocal, toRemote), CommitTxAndRemoteSig(CommitTx(commitmentInput, Transaction(2, Nil, Nil, 0)), ByteVector64.Zeroes), Nil)
513+
val localCommit = LocalCommit(0, CommitmentSpec(Set.empty, FeeratePerKw(0 sat), toLocal, toRemote), CommitTxAndRemoteSig(CommitTx(commitmentInput, Transaction(2, Nil, Nil, 0)), Left(ByteVector64.Zeroes)), Nil)
514514
val remoteCommit = RemoteCommit(0, CommitmentSpec(Set.empty, FeeratePerKw(0 sat), toRemote, toLocal), randomTxId(), randomKey().publicKey)
515515
Commitments(
516516
ChannelParams(randomBytes32(), ChannelConfig.standard, ChannelFeatures(), localParams, remoteParams, ChannelFlags(announceChannel = announceChannel)),

eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/ReplaceableTxFunderSpec.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -146,7 +146,7 @@ class ReplaceableTxFunderSpec extends TestKitBaseClass with AnyFunSuiteLike {
146146
localParams.dustLimit.returns(1000 sat)
147147
commitment.localParams.returns(localParams)
148148
val localCommit = mock[LocalCommit]
149-
localCommit.commitTxAndRemoteSig.returns(CommitTxAndRemoteSig(commitTx, PlaceHolderSig))
149+
localCommit.commitTxAndRemoteSig.returns(CommitTxAndRemoteSig(commitTx, Left(PlaceHolderSig)))
150150
commitment.localCommit.returns(localCommit)
151151

152152
// We can handle a small feerate update by lowering the change output.

0 commit comments

Comments
 (0)