1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-03-12 10:30:45 +01:00

Implement simple taproot channels

This commit implements:
  - feature bits for simple taproot channels
  - TLV extensions for funding/closing wire messages
  - modifications to how we handle channel funding, splicing and mutual closing
  - changes to the commitment structures

The v1 channel establishment protocol is modified to include nonces for creating and signing taproot transactions. This is bascially the original simple taproot proposal, which does
not cover dual-funding, splices and rbf.

We assume that simple taproot channels depends on the simple close protocol, which we extend to include musig2 nonces.

Dual-funding, splices and rbf are supported by extending the interactive tx session protocol to include musig2 nonce, which are attached to the `tx_complete` message.
There are 2 types of nonces:
- "funding nonces", which are used to sign a new funding tx that spends the current funding tx (splice, rbf).
- "commit nonces", which are used to sign the commit tx that is one of the outputs of the interactive session.

"funding nonces" can be randomly generated on-the-fly: either the interactive session will fail, and they can be forgotten, or it will succeed and we'll get a new, fully signed funding tx.
"commit nonces" can be deterministically generated.

This make nonce exchange simpler to reason about:
- when we send `tx_complete`, we know exactly what the funding tx and commit tx will be (so the funding tx id can be mixed in the nonce generation process).
- dual funding, splice and rbf message do not need to be modified

Channel re-establishment becomes a bit more complex, as one node could still be waiting for signatures while the other has completed the splice workflow, but it
can be mitigated by storing the last sent commit_sig and re-sending it again if needed.
This commit is contained in:
sstone 2024-10-21 11:58:52 +02:00
parent c7a288b91f
commit a7348f8060
No known key found for this signature in database
GPG key ID: E04E48E72C205463
54 changed files with 3580 additions and 609 deletions

View file

@ -310,6 +310,11 @@ object Features {
val mandatory = 60
}
case object SimpleTaproot extends Feature with InitFeature with NodeFeature with ChannelTypeFeature {
val rfcName = "option_simple_taproot"
val mandatory = 80
}
/** This feature bit indicates that the node is a mobile wallet that can be woken up via push notifications. */
case object WakeUpNotificationClient extends Feature with InitFeature {
val rfcName = "wake_up_notification_client"
@ -339,6 +344,11 @@ object Features {
val mandatory = 154
}
case object SimpleTaprootStaging extends Feature with InitFeature with NodeFeature with ChannelTypeFeature {
val rfcName = "option_simple_taproot_staging"
val mandatory = 180
}
/**
* 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.
* TODO: add NodeFeature once bLIP is merged.
@ -381,6 +391,8 @@ object Features {
ZeroConf,
KeySend,
SimpleClose,
SimpleTaproot,
SimpleTaprootStaging,
WakeUpNotificationClient,
TrampolinePaymentPrototype,
AsyncPaymentPrototype,
@ -400,6 +412,8 @@ object Features {
TrampolinePaymentPrototype -> (PaymentSecret :: Nil),
KeySend -> (VariableLengthOnion :: Nil),
SimpleClose -> (ShutdownAnySegwit :: Nil),
SimpleTaproot -> (ChannelType :: SimpleClose :: Nil),
SimpleTaprootStaging -> (ChannelType :: SimpleClose :: Nil),
AsyncPaymentPrototype -> (TrampolinePaymentPrototype :: Nil),
OnTheFlyFunding -> (SplicePrototype :: Nil),
FundingFeeCredit -> (OnTheFlyFunding :: Nil)

View file

@ -20,7 +20,7 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.Satoshi
import fr.acinq.eclair.BlockHeight
import fr.acinq.eclair.transactions.Transactions
import fr.acinq.eclair.transactions.Transactions.{CommitmentFormat, UnsafeLegacyAnchorOutputsCommitmentFormat, ZeroFeeHtlcTxAnchorOutputsCommitmentFormat}
import fr.acinq.eclair.transactions.Transactions.{CommitmentFormat, SimpleTaprootChannelsStagingCommitmentFormat, SimpleTaprootChannelsStagingLegacyCommitmentFormat, UnsafeLegacyAnchorOutputsCommitmentFormat, ZeroFeeHtlcTxAnchorOutputsCommitmentFormat}
// @formatter:off
sealed trait ConfirmationPriority extends Ordered[ConfirmationPriority] {
@ -77,7 +77,7 @@ case class FeerateTolerance(ratioLow: Double, ratioHigh: Double, anchorOutputMax
def isProposedFeerateTooHigh(commitmentFormat: CommitmentFormat, networkFeerate: FeeratePerKw, proposedFeerate: FeeratePerKw): Boolean = {
commitmentFormat match {
case Transactions.DefaultCommitmentFormat => networkFeerate * ratioHigh < proposedFeerate
case ZeroFeeHtlcTxAnchorOutputsCommitmentFormat | UnsafeLegacyAnchorOutputsCommitmentFormat => networkFeerate * ratioHigh < proposedFeerate
case ZeroFeeHtlcTxAnchorOutputsCommitmentFormat | UnsafeLegacyAnchorOutputsCommitmentFormat | SimpleTaprootChannelsStagingCommitmentFormat | SimpleTaprootChannelsStagingLegacyCommitmentFormat => networkFeerate * ratioHigh < proposedFeerate
}
}
@ -85,7 +85,7 @@ case class FeerateTolerance(ratioLow: Double, ratioHigh: Double, anchorOutputMax
commitmentFormat match {
case Transactions.DefaultCommitmentFormat => proposedFeerate < networkFeerate * ratioLow
// When using anchor outputs, we allow low feerates: fees will be set with CPFP and RBF at broadcast time.
case ZeroFeeHtlcTxAnchorOutputsCommitmentFormat | UnsafeLegacyAnchorOutputsCommitmentFormat => false
case ZeroFeeHtlcTxAnchorOutputsCommitmentFormat | UnsafeLegacyAnchorOutputsCommitmentFormat | SimpleTaprootChannelsStagingCommitmentFormat | SimpleTaprootChannelsStagingLegacyCommitmentFormat => false
}
}
}

View file

@ -17,6 +17,7 @@
package fr.acinq.eclair.channel
import akka.actor.{ActorRef, PossiblyHarmful, typed}
import fr.acinq.bitcoin.crypto.musig2.{IndividualNonce, SecretNonce}
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.{ByteVector32, DeterministicWallet, OutPoint, Satoshi, SatoshiLong, Transaction, TxId, TxOut}
import fr.acinq.eclair.blockchain.fee.{ConfirmationTarget, FeeratePerKw}
@ -551,7 +552,7 @@ sealed trait ChannelDataWithCommitments extends PersistentChannelData {
final case class DATA_WAIT_FOR_OPEN_CHANNEL(initFundee: INPUT_INIT_CHANNEL_NON_INITIATOR) extends TransientChannelData {
val channelId: ByteVector32 = initFundee.temporaryChannelId
}
final case class DATA_WAIT_FOR_ACCEPT_CHANNEL(initFunder: INPUT_INIT_CHANNEL_INITIATOR, lastSent: OpenChannel) extends TransientChannelData {
final case class DATA_WAIT_FOR_ACCEPT_CHANNEL(initFunder: INPUT_INIT_CHANNEL_INITIATOR, lastSent: OpenChannel, nextLocalNonce: Option[kotlin.Pair[SecretNonce, IndividualNonce]] = None) extends TransientChannelData {
val channelId: ByteVector32 = initFunder.temporaryChannelId
}
final case class DATA_WAIT_FOR_FUNDING_INTERNAL(params: ChannelParams,
@ -568,7 +569,8 @@ final case class DATA_WAIT_FOR_FUNDING_CREATED(params: ChannelParams,
pushAmount: MilliSatoshi,
commitTxFeerate: FeeratePerKw,
remoteFundingPubKey: PublicKey,
remoteFirstPerCommitmentPoint: PublicKey) extends TransientChannelData {
remoteFirstPerCommitmentPoint: PublicKey,
remoteNextLocalNonce: Option[IndividualNonce]) extends TransientChannelData {
val channelId: ByteVector32 = params.channelId
}
final case class DATA_WAIT_FOR_FUNDING_SIGNED(params: ChannelParams,

View file

@ -150,4 +150,5 @@ case class CommandUnavailableInThisState (override val channelId: Byte
case class ForbiddenDuringSplice (override val channelId: ByteVector32, command: String) extends ChannelException(channelId, s"cannot process $command while splicing")
case class ForbiddenDuringQuiescence (override val channelId: ByteVector32, command: String) extends ChannelException(channelId, s"cannot process $command while quiescent")
case class ConcurrentRemoteSplice (override val channelId: ByteVector32) extends ChannelException(channelId, "splice attempt canceled, remote initiated splice before us")
case class MissingNextLocalNonce (override val channelId: ByteVector32) extends ChannelException(channelId, "next local nonce tlv is missing")
// @formatter:on

View file

@ -16,7 +16,7 @@
package fr.acinq.eclair.channel
import fr.acinq.eclair.transactions.Transactions.{CommitmentFormat, DefaultCommitmentFormat, UnsafeLegacyAnchorOutputsCommitmentFormat, ZeroFeeHtlcTxAnchorOutputsCommitmentFormat}
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.{ChannelTypeFeature, FeatureSupport, Features, InitFeature, PermanentChannelFeature}
/**
@ -31,9 +31,12 @@ import fr.acinq.eclair.{ChannelTypeFeature, FeatureSupport, Features, InitFeatur
case class ChannelFeatures(features: Set[PermanentChannelFeature]) {
/** True if our main output in the remote commitment is directly sent (without any delay) to one of our wallet addresses. */
val paysDirectlyToWallet: Boolean = hasFeature(Features.StaticRemoteKey) && !hasFeature(Features.AnchorOutputs) && !hasFeature(Features.AnchorOutputsZeroFeeHtlcTx)
val paysDirectlyToWallet: Boolean = hasFeature(Features.StaticRemoteKey) && !hasFeature(Features.AnchorOutputs) && !hasFeature(Features.AnchorOutputsZeroFeeHtlcTx) && !hasFeature((Features.SimpleTaprootStaging))
/** Legacy option_anchor_outputs is used for Phoenix, because Phoenix doesn't have an on-chain wallet to pay for fees. */
val commitmentFormat: CommitmentFormat = if (hasFeature(Features.AnchorOutputs)) {
val commitmentFormat: CommitmentFormat = if (hasFeature(Features.SimpleTaprootStaging)) {
if (hasFeature(Features.AnchorOutputs)) SimpleTaprootChannelsStagingLegacyCommitmentFormat
else SimpleTaprootChannelsStagingCommitmentFormat
} else if (hasFeature(Features.AnchorOutputs)) {
UnsafeLegacyAnchorOutputsCommitmentFormat
} else if (hasFeature(Features.AnchorOutputsZeroFeeHtlcTx)) {
ZeroFeeHtlcTxAnchorOutputsCommitmentFormat
@ -129,6 +132,18 @@ object ChannelTypes {
override def commitmentFormat: CommitmentFormat = ZeroFeeHtlcTxAnchorOutputsCommitmentFormat
override def toString: String = s"anchor_outputs_zero_fee_htlc_tx${if (scidAlias) "+scid_alias" else ""}${if (zeroConf) "+zeroconf" else ""}"
}
case class SimpleTaprootChannelsStaging(scidAlias: Boolean = false, zeroConf: Boolean = false) extends SupportedChannelType {
/** Known channel-type features */
override def features: Set[ChannelTypeFeature] = Set(
if (scidAlias) Some(Features.ScidAlias) else None,
if (zeroConf) Some(Features.ZeroConf) else None,
Some(Features.SimpleTaprootStaging),
).flatten
override def paysDirectlyToWallet: Boolean = false
override def commitmentFormat: CommitmentFormat = SimpleTaprootChannelsStagingCommitmentFormat
override def toString: String = s"simple_taproot_channel_staging${if (scidAlias) "+scid_alias" else ""}${if (zeroConf) "+zeroconf" else ""}"
}
case class UnsupportedChannelType(featureBits: Features[InitFeature]) extends ChannelType {
override def features: Set[InitFeature] = featureBits.activated.keySet
override def toString: String = s"0x${featureBits.toByteVector.toHex}"
@ -151,12 +166,19 @@ object ChannelTypes {
AnchorOutputsZeroFeeHtlcTx(),
AnchorOutputsZeroFeeHtlcTx(zeroConf = true),
AnchorOutputsZeroFeeHtlcTx(scidAlias = true),
AnchorOutputsZeroFeeHtlcTx(scidAlias = true, zeroConf = true))
AnchorOutputsZeroFeeHtlcTx(scidAlias = true, zeroConf = true),
SimpleTaprootChannelsStaging(),
SimpleTaprootChannelsStaging(zeroConf = true),
SimpleTaprootChannelsStaging(scidAlias = true),
SimpleTaprootChannelsStaging(scidAlias = true, zeroConf = true),
)
.map(channelType => Features(channelType.features.map(_ -> FeatureSupport.Mandatory).toMap) -> channelType)
.toMap
// NB: Bolt 2: features must exactly match in order to identify a channel type.
def fromFeatures(features: Features[InitFeature]): ChannelType = features2ChannelType.getOrElse(features, UnsupportedChannelType(features))
def fromFeatures(features: Features[InitFeature]): ChannelType = {
features2ChannelType.getOrElse(features, UnsupportedChannelType(features))
}
/** Pick the channel type based on local and remote feature bits, as defined by the spec. */
def defaultFromFeatures(localFeatures: Features[InitFeature], remoteFeatures: Features[InitFeature], announceChannel: Boolean): SupportedChannelType = {
@ -164,7 +186,9 @@ object ChannelTypes {
val scidAlias = canUse(Features.ScidAlias) && !announceChannel // alias feature is incompatible with public channel
val zeroConf = canUse(Features.ZeroConf)
if (canUse(Features.AnchorOutputsZeroFeeHtlcTx)) {
if (canUse(Features.SimpleTaprootStaging)) {
SimpleTaprootChannelsStaging(scidAlias, zeroConf)
} else if (canUse(Features.AnchorOutputsZeroFeeHtlcTx)) {
AnchorOutputsZeroFeeHtlcTx(scidAlias, zeroConf)
} else if (canUse(Features.AnchorOutputs)) {
AnchorOutputs(scidAlias, zeroConf)

View file

@ -1,12 +1,13 @@
package fr.acinq.eclair.channel
import akka.event.LoggingAdapter
import fr.acinq.bitcoin.crypto.musig2.IndividualNonce
import fr.acinq.bitcoin.crypto.musig2.{IndividualNonce, SecretNonce}
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Crypto, Satoshi, SatoshiLong, Script, Transaction, TxId}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Crypto, Musig2, Satoshi, SatoshiLong, Script, Transaction, TxId}
import fr.acinq.eclair.blockchain.fee.{FeeratePerByte, FeeratePerKw, FeeratesPerKw, OnChainFeeConf}
import fr.acinq.eclair.channel.Helpers.Closing
import fr.acinq.eclair.channel.Monitoring.{Metrics, Tags}
import fr.acinq.eclair.channel.RemoteSignature.PartialSignatureWithNonce
import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.channel.fsm.Channel.ChannelConf
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.SharedTransaction
@ -235,6 +236,11 @@ case class CommitTxAndRemoteSig(commitTx: CommitTx, remoteSig: RemoteSignature)
object CommitTxAndRemoteSig {
def apply(commitTx: CommitTx, remoteSig: ByteVector64): CommitTxAndRemoteSig = CommitTxAndRemoteSig(commitTx, RemoteSignature(remoteSig))
def apply(commitTx: CommitTx, sigOrPartialSig: Either[ByteVector64, RemoteSignature.PartialSignatureWithNonce]): CommitTxAndRemoteSig = sigOrPartialSig match {
case Left(sig) => CommitTxAndRemoteSig(commitTx, RemoteSignature.FullSignature(sig))
case Right(psig) => CommitTxAndRemoteSig(commitTx, RemoteSignature.PartialSignatureWithNonce(psig.partialSig, psig.nonce))
}
}
/** The local commitment maps to a commitment transaction that we can sign and broadcast if necessary. */
@ -243,11 +249,21 @@ case class LocalCommit(index: Long, spec: CommitmentSpec, commitTxAndRemoteSig:
object LocalCommit {
def fromCommitSig(keyManager: ChannelKeyManager, params: ChannelParams, fundingTxId: TxId,
fundingTxIndex: Long, remoteFundingPubKey: PublicKey, commitInput: InputInfo,
commit: CommitSig, localCommitIndex: Long, spec: CommitmentSpec, localPerCommitmentPoint: PublicKey): Either[ChannelException, LocalCommit] = {
commit: CommitSig, localCommitIndex: Long, spec: CommitmentSpec, localPerCommitmentPoint: PublicKey)(implicit log: LoggingAdapter): Either[ChannelException, LocalCommit] = {
val (localCommitTx, htlcTxs) = Commitment.makeLocalTxs(keyManager, params.channelConfig, params.channelFeatures, localCommitIndex, params.localParams, params.remoteParams, fundingTxIndex, remoteFundingPubKey, commitInput, localPerCommitmentPoint, spec)
if (!localCommitTx.checkSig(commit.signature, remoteFundingPubKey, TxOwner.Remote, params.commitmentFormat)) {
commit.sigOrPartialSig match {
case Left(sig) =>
if (!localCommitTx.checkSig(sig, remoteFundingPubKey, TxOwner.Remote, params.commitmentFormat)) {
return Left(InvalidCommitmentSignature(params.channelId, fundingTxId, fundingTxIndex, localCommitTx.tx))
}
case Right(psig) =>
val fundingPubkey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex).publicKey
val localNonce = keyManager.verificationNonce(localCommitTx.input.outPoint.txid, fundingPubkey, localCommitIndex)
if (!localCommitTx.checkPartialSignature(psig, fundingPubkey, localNonce._2, remoteFundingPubKey)) {
log.debug(s"fromCommitSig: invalid partial signature $psig fundingPubkey = $fundingPubkey, fundingTxIndex = $fundingTxIndex localCommitIndex = $localCommitIndex localNonce = $localNonce remoteFundingPubKey = $remoteFundingPubKey")
return Left(InvalidCommitmentSignature(params.channelId, fundingTxId, fundingTxIndex, localCommitTx.tx))
}
}
val sortedHtlcTxs = htlcTxs.sortBy(_.input.outPoint.index)
if (commit.htlcSignatures.size != sortedHtlcTxs.size) {
return Left(HtlcSigCountMismatch(params.channelId, sortedHtlcTxs.size, commit.htlcSignatures.size))
@ -260,19 +276,34 @@ object LocalCommit {
}
HtlcTxAndRemoteSig(htlcTx, remoteSig)
}
Right(LocalCommit(localCommitIndex, spec, CommitTxAndRemoteSig(localCommitTx, RemoteSignature.FullSignature(commit.signature)), htlcTxsAndRemoteSigs))
val remoteSig = commit.sigOrPartialSig match {
case Left(sig) => RemoteSignature.FullSignature(sig)
case Right(psig) => psig
}
Right(LocalCommit(localCommitIndex, spec, CommitTxAndRemoteSig(localCommitTx, remoteSig), htlcTxsAndRemoteSigs))
}
}
/** The remote commitment maps to a commitment transaction that only our peer can sign and broadcast. */
case class RemoteCommit(index: Long, spec: CommitmentSpec, txid: TxId, remotePerCommitmentPoint: PublicKey) {
def sign(keyManager: ChannelKeyManager, params: ChannelParams, fundingTxIndex: Long, remoteFundingPubKey: PublicKey, commitInput: InputInfo): CommitSig = {
case class RemoteCommit(index: Long, spec: CommitmentSpec, txid: TxId, remotePerCommitmentPoint: PublicKey, localSig_opt: Option[CommitSig] = None) {
def sign(keyManager: ChannelKeyManager, params: ChannelParams, fundingTxIndex: Long, remoteFundingPubKey: PublicKey, commitInput: InputInfo, remoteNonce_opt: Option[IndividualNonce])(implicit log: LoggingAdapter): CommitSig = {
val (remoteCommitTx, htlcTxs) = Commitment.makeRemoteTxs(keyManager, params.channelConfig, params.channelFeatures, index, params.localParams, params.remoteParams, fundingTxIndex, remoteFundingPubKey, commitInput, remotePerCommitmentPoint, spec)
val sig = keyManager.sign(remoteCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex), TxOwner.Remote, params.commitmentFormat)
val localFundingPubKey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex)
val (sig, tlvStream) = if (commitInput.isP2tr) {
val localNonce = keyManager.signingNonce(localFundingPubKey.publicKey)
val Some(remoteNonce) = remoteNonce_opt
val Right(localPartialSigOfRemoteTx) = keyManager.partialSign(remoteCommitTx, localFundingPubKey, remoteFundingPubKey, TxOwner.Remote, localNonce, remoteNonce)
log.debug(s"RemoteCommit.sign localPartialSigOfRemoteTx = $localPartialSigOfRemoteTx fundingTxIndex = $fundingTxIndex remote commit index = $index remote nonce = $remoteNonce")
val tlvStream: TlvStream[CommitSigTlv] = TlvStream(CommitSigTlv.PartialSignatureWithNonceTlv(PartialSignatureWithNonce(localPartialSigOfRemoteTx, localNonce._2)))
(ByteVector64.Zeroes, tlvStream)
} else {
val sig = keyManager.sign(remoteCommitTx, localFundingPubKey, TxOwner.Remote, params.commitmentFormat)
(sig, TlvStream[CommitSigTlv]())
}
val channelKeyPath = keyManager.keyPath(params.localParams, params.channelConfig)
val sortedHtlcTxs = htlcTxs.sortBy(_.input.outPoint.index)
val htlcSigs = sortedHtlcTxs.map(keyManager.sign(_, keyManager.htlcPoint(channelKeyPath), remotePerCommitmentPoint, TxOwner.Remote, params.commitmentFormat))
CommitSig(params.channelId, sig, htlcSigs.toList)
CommitSig(params.channelId, sig, htlcSigs.toList, tlvStream)
}
}
@ -668,12 +699,25 @@ case class Commitment(fundingTxIndex: Long,
Right(())
}
def sendCommit(keyManager: ChannelKeyManager, params: ChannelParams, changes: CommitmentChanges, remoteNextPerCommitmentPoint: PublicKey, batchSize: Int)(implicit log: LoggingAdapter): (Commitment, CommitSig) = {
def sendCommit(keyManager: ChannelKeyManager, params: ChannelParams, changes: CommitmentChanges, remoteNextPerCommitmentPoint: PublicKey, batchSize: Int, nextRemoteNonce_opt: Option[IndividualNonce])(implicit log: LoggingAdapter): (Commitment, CommitSig) = {
// remote commitment will include all local proposed changes + remote acked changes
val spec = CommitmentSpec.reduce(remoteCommit.spec, changes.remoteChanges.acked, changes.localChanges.proposed)
val (remoteCommitTx, htlcTxs) = Commitment.makeRemoteTxs(keyManager, params.channelConfig, params.channelFeatures, remoteCommit.index + 1, params.localParams, params.remoteParams, fundingTxIndex, remoteFundingPubKey, commitInput, remoteNextPerCommitmentPoint, spec)
val sig = keyManager.sign(remoteCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex), TxOwner.Remote, params.commitmentFormat)
val localFundingPubKey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex)
val sig = if (remoteCommitTx.input.isP2tr) {
ByteVector64.Zeroes
} else {
keyManager.sign(remoteCommitTx, localFundingPubKey, TxOwner.Remote, params.commitmentFormat)
}
val partialSig: Set[CommitSigTlv] = if (remoteCommitTx.input.isP2tr) {
val localNonce = keyManager.signingNonce(localFundingPubKey.publicKey)
val Some(remoteNonce) = nextRemoteNonce_opt
val Right(psig) = keyManager.partialSign(remoteCommitTx, localFundingPubKey, remoteFundingPubKey, TxOwner.Remote, localNonce, remoteNonce)
log.debug(s"sendCommit: creating partial sig $psig for remote commit tx ${remoteCommitTx.tx.txid} with fundingTxIndex = $fundingTxIndex remoteCommit.index (should add +1) = ${remoteCommit.index} remote nonce $remoteNonce and remoteNextPerCommitmentPoint = $remoteNextPerCommitmentPoint")
Set(CommitSigTlv.PartialSignatureWithNonceTlv(PartialSignatureWithNonce(psig, localNonce._2)))
} else {
Set.empty
}
val sortedHtlcTxs: Seq[TransactionWithInputInfo] = htlcTxs.sortBy(_.input.outPoint.index)
val channelKeyPath = keyManager.keyPath(params.localParams, params.channelConfig)
val htlcSigs = sortedHtlcTxs.map(keyManager.sign(_, keyManager.htlcPoint(channelKeyPath), remoteNextPerCommitmentPoint, TxOwner.Remote, params.commitmentFormat))
@ -684,8 +728,9 @@ case class Commitment(fundingTxIndex: Long,
val commitSig = CommitSig(params.channelId, sig, htlcSigs.toList, TlvStream(Set(
if (batchSize > 1) Some(CommitSigTlv.BatchTlv(batchSize)) else None
).flatten[CommitSigTlv]))
val nextRemoteCommit = NextRemoteCommit(commitSig, RemoteCommit(remoteCommit.index + 1, spec, remoteCommitTx.tx.txid, remoteNextPerCommitmentPoint))
).flatten[CommitSigTlv] ++ partialSig))
log.debug(s"sendCommit: setting remoteNextPerCommitmentPoint to $remoteNextPerCommitmentPoint")
val nextRemoteCommit = NextRemoteCommit(commitSig, RemoteCommit(remoteCommit.index + 1, spec, remoteCommitTx.tx.txid, remoteNextPerCommitmentPoint, Some(commitSig)))
(copy(nextRemoteCommit_opt = Some(nextRemoteCommit)), commitSig)
}
@ -709,9 +754,31 @@ case class Commitment(fundingTxIndex: Long,
/** Return a fully signed commit tx, that can be published as-is. */
def fullySignedLocalCommitTx(params: ChannelParams, keyManager: ChannelKeyManager): CommitTx = {
val unsignedCommitTx = localCommit.commitTxAndRemoteSig.commitTx
val commitTx = localCommit.commitTxAndRemoteSig.remoteSig match {
case RemoteSignature.FullSignature(remoteSig) =>
val localSig = keyManager.sign(unsignedCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex), TxOwner.Local, params.commitmentFormat)
val RemoteSignature.FullSignature(remoteSig) = localCommit.commitTxAndRemoteSig.remoteSig
val commitTx = addSigs(unsignedCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex).publicKey, remoteFundingPubKey, localSig, remoteSig)
addSigs(unsignedCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex).publicKey, remoteFundingPubKey, localSig, remoteSig)
case RemoteSignature.PartialSignatureWithNonce(remotePsig, remoteNonce) =>
val fundingPubKey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex)
val fundingTxId = if (!params.channelFeatures.hasFeature(Features.DualFunding) && this.fundingTxIndex == 0 && localCommit.index == 0) {
TxId(ByteVector32.One) // special case because for channel establishment v1 we exchange the first nonce before the funding tx id is known
} else {
unsignedCommitTx.input.outPoint.txid
}
val localNonce = keyManager.verificationNonce(fundingTxId, fundingPubKey.publicKey, localCommit.index)
(for {
partialSig <- keyManager.partialSign(unsignedCommitTx, fundingPubKey, remoteFundingPubKey, TxOwner.Local, localNonce, remoteNonce)
inputIndex = unsignedCommitTx.tx.txIn.indexWhere(_.outPoint == unsignedCommitTx.input.outPoint)
aggSig <- Musig2.aggregateTaprootSignatures(
Seq(partialSig, remotePsig),
unsignedCommitTx.tx, inputIndex, Seq(unsignedCommitTx.input.txOut),
Scripts.sort(Seq(fundingPubKey.publicKey, remoteFundingPubKey)),
Seq(localNonce._2, remoteNonce),
None)
signedCommitTx = unsignedCommitTx.copy(tx = unsignedCommitTx.tx.updateWitness(0, Script.witnessKeyPathPay2tr(aggSig)))
} yield signedCommitTx).fold(t => throw new IllegalArgumentException("commit signatures are invalid"), identity)
}
// We verify the remote signature when receiving their commit_sig, so this check should always pass.
require(checkSpendable(commitTx).isSuccess, "commit signatures are invalid")
commitTx
@ -735,7 +802,7 @@ object Commitment {
val localFundingPubkey = keyManager.fundingPublicKey(localParams.fundingKeyPath, fundingTxIndex).publicKey
val localDelayedPaymentPubkey = Generators.derivePubKey(keyManager.delayedPaymentPoint(channelKeyPath).publicKey, localPerCommitmentPoint)
val localHtlcPubkey = Generators.derivePubKey(keyManager.htlcPoint(channelKeyPath).publicKey, localPerCommitmentPoint)
val remotePaymentPubkey = if (channelFeatures.hasFeature(Features.StaticRemoteKey)) {
val remotePaymentPubkey = if (channelFeatures.hasFeature(Features.StaticRemoteKey) || channelFeatures.hasFeature(Features.SimpleTaproot) || channelFeatures.hasFeature(Features.SimpleTaprootStaging)) {
remoteParams.paymentBasepoint
} else {
Generators.derivePubKey(remoteParams.paymentBasepoint, localPerCommitmentPoint)
@ -763,7 +830,7 @@ object Commitment {
val channelKeyPath = keyManager.keyPath(localParams, channelConfig)
val localFundingPubkey = keyManager.fundingPublicKey(localParams.fundingKeyPath, fundingTxIndex).publicKey
val localPaymentBasepoint = localParams.walletStaticPaymentBasepoint.getOrElse(keyManager.paymentPoint(channelKeyPath).publicKey)
val localPaymentPubkey = if (channelFeatures.hasFeature(Features.StaticRemoteKey)) {
val localPaymentPubkey = if (channelFeatures.hasFeature(Features.StaticRemoteKey) || channelFeatures.hasFeature(Features.SimpleTaprootStaging)) {
localPaymentBasepoint
} else {
Generators.derivePubKey(localPaymentBasepoint, remotePerCommitmentPoint)
@ -1054,11 +1121,20 @@ case class Commitments(params: ChannelParams,
}
}
def sendCommit(keyManager: ChannelKeyManager)(implicit log: LoggingAdapter): Either[ChannelException, (Commitments, Seq[CommitSig])] = {
def sendCommit(keyManager: ChannelKeyManager, nextRemoteNonces: List[IndividualNonce] = List.empty)(implicit log: LoggingAdapter): Either[ChannelException, (Commitments, Seq[CommitSig])] = {
remoteNextCommitInfo match {
case Right(_) if !changes.localHasChanges => Left(CannotSignWithoutChanges(channelId))
case Right(remoteNextPerCommitmentPoint) =>
val (active1, sigs) = active.map(_.sendCommit(keyManager, params, changes, remoteNextPerCommitmentPoint, active.size)).unzip
var nonceIndex = 0
val (active1, sigs) = active.map { c =>
val remoteNonce_opt = if (c.commitInput.isP2tr) {
val n = nextRemoteNonces(nonceIndex)
nonceIndex = nonceIndex + 1
Some(n)
} else None
c.sendCommit(keyManager, params, changes, remoteNextPerCommitmentPoint, active.size, remoteNonce_opt)
} unzip
val commitments1 = copy(
changes = changes.copy(
localChanges = changes.localChanges.copy(proposed = Nil, signed = changes.localChanges.proposed),
@ -1081,6 +1157,10 @@ case class Commitments(params: ChannelParams,
}
val channelKeyPath = keyManager.keyPath(params.localParams, params.channelConfig)
val localPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, localCommitIndex + 1)
val fundingIndexes = active.map(_.fundingTxIndex).toSet
if (fundingIndexes.size > 1) log.warning(s"more than 1 funding tx index")
// Signatures are sent in order (most recent first), calling `zip` will drop trailing sigs that are for deactivated/pruned commitments.
val active1 = active.zip(commits).map { case (commitment, commit) =>
commitment.receiveCommit(keyManager, params, changes, localPerCommitmentPoint, commit) match {
@ -1091,10 +1171,20 @@ case class Commitments(params: ChannelParams,
// we will send our revocation preimage + our next revocation hash
val localPerCommitmentSecret = keyManager.commitmentSecret(channelKeyPath, localCommitIndex)
val localNextPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, localCommitIndex + 2)
val tlvStream: TlvStream[RevokeAndAckTlv] = {
val nonces = this.active.filter(_.commitInput.isP2tr).map(c => {
val fundingPubkey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, c.fundingTxIndex).publicKey
val n = keyManager.verificationNonce(c.fundingTxId, fundingPubkey, localCommitIndex + 2)
log.debug(s"revokeandack: creating verification nonce ${n._2} fundingIndex = ${c.fundingTxIndex} commit index = ${localCommitIndex + 2}")
n
})
if (nonces.isEmpty) TlvStream.empty else TlvStream(RevokeAndAckTlv.NextLocalNoncesTlv(nonces.map(_._2).toList))
}
val revocation = RevokeAndAck(
channelId = channelId,
perCommitmentSecret = localPerCommitmentSecret,
nextPerCommitmentPoint = localNextPerCommitmentPoint
nextPerCommitmentPoint = localNextPerCommitmentPoint,
tlvStream
)
val commitments1 = copy(
changes = changes.copy(
@ -1111,6 +1201,7 @@ case class Commitments(params: ChannelParams,
remoteNextCommitInfo match {
case Right(_) => Left(UnexpectedRevocation(channelId))
case Left(_) if revocation.perCommitmentSecret.publicKey != active.head.remoteCommit.remotePerCommitmentPoint => Left(InvalidRevocation(channelId))
case Left(_) if this.active.exists(_.commitInput.isP2tr) && revocation.nexLocalNonces.isEmpty => Left(MissingNextLocalNonce(channelId))
case Left(_) =>
// Since htlcs are shared across all commitments, we generate the actions only once based on the first commitment.
val receivedHtlcs = changes.remoteChanges.signed.collect {
@ -1202,18 +1293,28 @@ case class Commitments(params: ChannelParams,
active.forall { commitment =>
val localFundingKey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, commitment.fundingTxIndex).publicKey
val remoteFundingKey = commitment.remoteFundingPubKey
val fundingScript = Script.write(Scripts.multiSig2of2(localFundingKey, remoteFundingKey))
commitment.commitInput match {
case InputInfo.SegwitInput(_, _, redeemScript) => redeemScript == fundingScript
case _: InputInfo.TaprootInput => false
case s: InputInfo.SegwitInput => s.redeemScript == Script.write(Scripts.multiSig2of2(localFundingKey, remoteFundingKey))
case t: InputInfo.TaprootInput => t.internalKey == Scripts.Taproot.musig2Aggregate(localFundingKey, remoteFundingKey) && t.scriptTree_opt.isEmpty
}
}
}
/** This function should be used to ignore a commit_sig that we've already received. */
def ignoreRetransmittedCommitSig(commitSig: CommitSig): Boolean = {
val RemoteSignature.FullSignature(latestRemoteSig) = latest.localCommit.commitTxAndRemoteSig.remoteSig
params.channelFeatures.hasFeature(Features.DualFunding) && commitSig.batchSize == 1 && latestRemoteSig == commitSig.signature
def ignoreRetransmittedCommitSig(commitSig: CommitSig, keyManager: ChannelKeyManager): Boolean = commitSig.sigOrPartialSig match {
case _ if !params.channelFeatures.hasFeature(Features.DualFunding) => false
case _ if commitSig.batchSize != 1 => false
case Left(sig) =>
latest.localCommit.commitTxAndRemoteSig.remoteSig match {
case f: RemoteSignature.FullSignature => f.sig == sig
case _: RemoteSignature.PartialSignatureWithNonce => false
}
case Right(psig) =>
// we cannot compare partial signatures directly as they are not deterministic (a new signing nonce is used every time a signature is computed)
// => instead we simply check that the provided partial signature is valid for our latest commit tx
val localFundingKey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, latest.fundingTxIndex).publicKey
val (_, localNonce) = keyManager.verificationNonce(latest.fundingTxId, localFundingKey, latest.localCommit.index)
latest.localCommit.commitTxAndRemoteSig.commitTx.checkPartialSignature(psig, localFundingKey, localNonce, latest.remoteFundingPubKey)
}
def localFundingSigs(fundingTxId: TxId): Option[TxSignatures] = {

View file

@ -18,7 +18,8 @@ package fr.acinq.eclair.channel
import akka.event.{DiagnosticLoggingAdapter, LoggingAdapter}
import com.softwaremill.quicklens.ModifyPimp
import fr.acinq.bitcoin.ScriptFlags
import fr.acinq.bitcoin.{ScriptFlags, ScriptTree}
import fr.acinq.bitcoin.crypto.musig2.{IndividualNonce, SecretNonce}
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey, sha256}
import fr.acinq.bitcoin.scalacompat.Script._
import fr.acinq.bitcoin.scalacompat._
@ -119,6 +120,7 @@ object Helpers {
}
val channelFeatures = ChannelFeatures(channelType, localFeatures, remoteFeatures, open.channelFlags.announceChannel)
if ((channelFeatures.hasFeature(Features.SimpleTaproot) || channelFeatures.hasFeature(Features.SimpleTaprootStaging)) && open.nexLocalNonce_opt.isEmpty) return Left(MissingNextLocalNonce(open.temporaryChannelId))
// BOLT #2: The receiving node MUST fail the channel if: it considers feerate_per_kw too small for timely processing or unreasonably large.
val localFeeratePerKw = nodeParams.onChainFeeConf.getCommitmentFeerate(nodeParams.currentBitcoinCoreFeerates, remoteNodeId, channelFeatures.commitmentFormat, open.fundingSatoshis)
@ -226,6 +228,7 @@ object Helpers {
if (reserveToFundingRatio > nodeParams.channelConf.maxReserveToFundingRatio) return Left(ChannelReserveTooHigh(open.temporaryChannelId, accept.channelReserveSatoshis, reserveToFundingRatio, nodeParams.channelConf.maxReserveToFundingRatio))
val channelFeatures = ChannelFeatures(channelType, localFeatures, remoteFeatures, open.channelFlags.announceChannel)
if ((channelFeatures.hasFeature(Features.SimpleTaproot) || channelFeatures.hasFeature(Features.SimpleTaprootStaging)) && accept.nexLocalNonce_opt.isEmpty) return Left(MissingNextLocalNonce(open.temporaryChannelId))
extractShutdownScript(accept.temporaryChannelId, localFeatures, remoteFeatures, accept.upfrontShutdownScript_opt).map(script_opt => (channelFeatures, script_opt))
}
@ -260,7 +263,7 @@ object Helpers {
for {
script_opt <- extractShutdownScript(accept.temporaryChannelId, localFeatures, remoteFeatures, accept.upfrontShutdownScript_opt)
fundingScript = Funding.makeFundingPubKeyScript(open.fundingPubkey, accept.fundingPubkey)
fundingScript = Funding.makeFundingPubKeyScript(open.fundingPubkey, accept.fundingPubkey, channelType.commitmentFormat)
liquidityPurchase_opt <- LiquidityAds.validateRemoteFunding(open.requestFunding_opt, remoteNodeId, accept.temporaryChannelId, fundingScript, accept.fundingAmount, open.fundingFeerate, isChannelCreation = true, accept.willFund_opt)
} yield {
val channelFeatures = ChannelFeatures(channelType, localFeatures, remoteFeatures, open.channelFlags.announceChannel)
@ -357,10 +360,17 @@ object Helpers {
}
object Funding {
def makeFundingPubKeyScript(localFundingKey: PublicKey, remoteFundingKey: PublicKey, commitmentFormat: CommitmentFormat): ByteVector = if (commitmentFormat.useTaproot) {
write(Taproot.musig2FundingScript(localFundingKey, remoteFundingKey))
} else {
write(pay2wsh(multiSig2of2(localFundingKey, remoteFundingKey)))
}
def makeFundingPubKeyScript(localFundingKey: PublicKey, remoteFundingKey: PublicKey): ByteVector = write(pay2wsh(multiSig2of2(localFundingKey, remoteFundingKey)))
def makeFundingInputInfo(fundingTxId: TxId, fundingTxOutputIndex: Int, fundingSatoshis: Satoshi, fundingPubkey1: PublicKey, fundingPubkey2: PublicKey): InputInfo.SegwitInput = {
def makeFundingInputInfo(fundingTxId: TxId, fundingTxOutputIndex: Int, fundingSatoshis: Satoshi, fundingPubkey1: PublicKey, fundingPubkey2: PublicKey, commitmentFormat: CommitmentFormat): InputInfo = if (commitmentFormat.useTaproot) {
val fundingScript = Taproot.musig2FundingScript(fundingPubkey1, fundingPubkey2)
val fundingTxOut = TxOut(fundingSatoshis, fundingScript)
InputInfo.TaprootInput(OutPoint(fundingTxId, fundingTxOutputIndex), fundingTxOut, Taproot.musig2Aggregate(fundingPubkey1, fundingPubkey2), None)
} else {
val fundingScript = multiSig2of2(fundingPubkey1, fundingPubkey2)
val fundingTxOut = TxOut(fundingSatoshis, pay2wsh(fundingScript))
InputInfo.SegwitInput(OutPoint(fundingTxId, fundingTxOutputIndex), fundingTxOut, write(fundingScript))
@ -426,7 +436,7 @@ object Helpers {
val fundingPubKey = keyManager.fundingPublicKey(localParams.fundingKeyPath, fundingTxIndex)
val channelKeyPath = keyManager.keyPath(localParams, channelConfig)
val commitmentInput = makeFundingInputInfo(fundingTxId, fundingTxOutputIndex, fundingAmount, fundingPubKey.publicKey, remoteFundingPubKey)
val commitmentInput = makeFundingInputInfo(fundingTxId, fundingTxOutputIndex, fundingAmount, fundingPubKey.publicKey, remoteFundingPubKey, params.commitmentFormat)
val localPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, localCommitmentIndex)
val (localCommitTx, _) = Commitment.makeLocalTxs(keyManager, channelConfig, channelFeatures, localCommitmentIndex, localParams, remoteParams, fundingTxIndex, remoteFundingPubKey, commitmentInput, localPerCommitmentPoint, localSpec)
val (remoteCommitTx, htlcTxs) = Commitment.makeRemoteTxs(keyManager, channelConfig, channelFeatures, remoteCommitmentIndex, localParams, remoteParams, fundingTxIndex, remoteFundingPubKey, commitmentInput, remotePerCommitmentPoint, remoteSpec)
@ -516,10 +526,20 @@ object Helpers {
val channelKeyPath = keyManager.keyPath(commitments.params.localParams, commitments.params.channelConfig)
val localPerCommitmentSecret = keyManager.commitmentSecret(channelKeyPath, commitments.localCommitIndex - 1)
val localNextPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, commitments.localCommitIndex + 1)
val nonces = commitments.active.filter(_.commitInput.isP2tr).map { c =>
val fundingPubkey = keyManager.fundingPublicKey(commitments.params.localParams.fundingKeyPath, c.fundingTxIndex).publicKey
keyManager.verificationNonce(c.fundingTxId, fundingPubkey, commitments.localCommitIndex + 1)
}
val tlvStream: TlvStream[RevokeAndAckTlv] = if (nonces.isEmpty) {
TlvStream.empty
} else {
TlvStream(RevokeAndAckTlv.NextLocalNoncesTlv(nonces.map(_._2).toList))
}
val revocation = RevokeAndAck(
channelId = commitments.channelId,
perCommitmentSecret = localPerCommitmentSecret,
nextPerCommitmentPoint = localNextPerCommitmentPoint
nextPerCommitmentPoint = localNextPerCommitmentPoint,
tlvStream
)
checkRemoteCommit(remoteChannelReestablish, retransmitRevocation_opt = Some(revocation))
} else if (commitments.localCommitIndex > remoteChannelReestablish.nextRemoteRevocationNumber + 1) {
@ -697,13 +717,17 @@ object Helpers {
}
/** We are the closer: we sign closing transactions for which we pay the fees. */
def makeSimpleClosingTx(currentBlockHeight: BlockHeight, keyManager: ChannelKeyManager, commitment: FullCommitment, localScriptPubkey: ByteVector, remoteScriptPubkey: ByteVector, feerate: FeeratePerKw): Either[ChannelException, (ClosingTxs, ClosingComplete)] = {
def makeSimpleClosingTx(currentBlockHeight: BlockHeight, keyManager: ChannelKeyManager, commitment: FullCommitment, localScriptPubkey: ByteVector, remoteScriptPubkey: ByteVector, feerate: FeeratePerKw, localClosingNonce_opt: Option[(SecretNonce, IndividualNonce)] = None, remoteClosingNonce_opt: Option[IndividualNonce] = None): Either[ChannelException, (ClosingTxs, ClosingComplete)] = {
// We must convert the feerate to a fee: we must build dummy transactions to compute their weight.
val closingFee = {
val dummyClosingTxs = Transactions.makeSimpleClosingTxs(commitment.commitInput, commitment.localCommit.spec, SimpleClosingTxFee.PaidByUs(0 sat), currentBlockHeight.toLong, localScriptPubkey, remoteScriptPubkey)
dummyClosingTxs.preferred_opt match {
case Some(dummyTx) =>
val dummySignedTx = Transactions.addSigs(dummyTx, Transactions.PlaceHolderPubKey, Transactions.PlaceHolderPubKey, Transactions.PlaceHolderSig, Transactions.PlaceHolderSig)
val dummySignedTx = if (commitment.commitInput.isP2tr) {
Transactions.addAggregatedSignature(dummyTx, Transactions.PlaceHolderSig)
} else {
Transactions.addSigs(dummyTx, Transactions.PlaceHolderPubKey, Transactions.PlaceHolderPubKey, Transactions.PlaceHolderSig, Transactions.PlaceHolderSig)
}
SimpleClosingTxFee.PaidByUs(Transactions.weight2fee(feerate, dummySignedTx.tx.weight()))
case None => return Left(CannotGenerateClosingTx(commitment.channelId))
}
@ -715,11 +739,26 @@ object Helpers {
case _ => return Left(CannotGenerateClosingTx(commitment.channelId))
}
val localFundingPubKey = keyManager.fundingPublicKey(commitment.localParams.fundingKeyPath, commitment.fundingTxIndex)
val closingComplete = ClosingComplete(commitment.channelId, localScriptPubkey, remoteScriptPubkey, closingFee.fee, currentBlockHeight.toLong, TlvStream(Set(
val tlvs = if (commitment.commitInput.isP2tr) {
def partialSign(tx: ClosingTx) = {
val Right(psig) = keyManager.partialSign(tx, localFundingPubKey, commitment.remoteFundingPubKey, TxOwner.Local, localClosingNonce_opt.get, remoteClosingNonce_opt.get)
psig
}
TlvStream(Set(
closingTxs.localAndRemote_opt.map(tx => ClosingTlv.CloserAndCloseeOutputsPartialSignature(partialSign(tx))),
closingTxs.localOnly_opt.map(tx => ClosingTlv.CloserOutputOnlyPartialSignature(partialSign(tx))),
closingTxs.remoteOnly_opt.map(tx => ClosingTlv.CloseeOutputOnlyPartialSignature(partialSign(tx))),
).flatten[ClosingTlv])
} else {
TlvStream(Set(
closingTxs.localAndRemote_opt.map(tx => ClosingTlv.CloserAndCloseeOutputs(keyManager.sign(tx, localFundingPubKey, TxOwner.Local, commitment.params.commitmentFormat))),
closingTxs.localOnly_opt.map(tx => ClosingTlv.CloserOutputOnly(keyManager.sign(tx, localFundingPubKey, TxOwner.Local, commitment.params.commitmentFormat))),
closingTxs.remoteOnly_opt.map(tx => ClosingTlv.CloseeOutputOnly(keyManager.sign(tx, localFundingPubKey, TxOwner.Local, commitment.params.commitmentFormat))),
).flatten[ClosingTlv]))
).flatten[ClosingTlv])
}
val closingComplete = ClosingComplete(commitment.channelId, localScriptPubkey, remoteScriptPubkey, closingFee.fee, currentBlockHeight.toLong, tlvs)
Right(closingTxs, closingComplete)
}
@ -729,17 +768,48 @@ object Helpers {
* Callers should ignore failures: since the protocol is fully asynchronous, failures here simply mean that they
* are not using our latest script (race condition between our closing_complete and theirs).
*/
def signSimpleClosingTx(keyManager: ChannelKeyManager, commitment: FullCommitment, localScriptPubkey: ByteVector, remoteScriptPubkey: ByteVector, closingComplete: ClosingComplete): Either[ChannelException, (ClosingTx, ClosingSig)] = {
def signSimpleClosingTx(keyManager: ChannelKeyManager, commitment: FullCommitment, localScriptPubkey: ByteVector, remoteScriptPubkey: ByteVector, closingComplete: ClosingComplete, localClosingNonce_opt: Option[(SecretNonce, IndividualNonce)] = None, remoteClosingNonce_opt: Option[IndividualNonce] = None): Either[ChannelException, (ClosingTx, ClosingSig)] = {
val closingFee = SimpleClosingTxFee.PaidByThem(closingComplete.fees)
val closingTxs = Transactions.makeSimpleClosingTxs(commitment.commitInput, commitment.localCommit.spec, closingFee, closingComplete.lockTime, localScriptPubkey, remoteScriptPubkey)
// If our output isn't dust, they must provide a signature for a transaction that includes it.
// Note that we're the closee, so we look for signatures including the closee output.
if (commitment.commitInput.isP2tr) {
(closingTxs.localAndRemote_opt, closingTxs.localOnly_opt) match {
case (Some(_), Some(_)) if closingComplete.closerAndCloseeOutputsPartialSig_opt.isEmpty && closingComplete.closeeOutputOnlyPartialSig_opt.isEmpty => return Left(MissingCloseSignature(commitment.channelId))
case (Some(_), None) if closingComplete.closerAndCloseeOutputsPartialSig_opt.isEmpty => return Left(MissingCloseSignature(commitment.channelId))
case (None, Some(_)) if closingComplete.closeeOutputOnlyPartialSig_opt.isEmpty => return Left(MissingCloseSignature(commitment.channelId))
case _ => ()
}
} else {
(closingTxs.localAndRemote_opt, closingTxs.localOnly_opt) match {
case (Some(_), Some(_)) if closingComplete.closerAndCloseeOutputsSig_opt.isEmpty && closingComplete.closeeOutputOnlySig_opt.isEmpty => return Left(MissingCloseSignature(commitment.channelId))
case (Some(_), None) if closingComplete.closerAndCloseeOutputsSig_opt.isEmpty => return Left(MissingCloseSignature(commitment.channelId))
case (None, Some(_)) if closingComplete.closeeOutputOnlySig_opt.isEmpty => return Left(MissingCloseSignature(commitment.channelId))
case _ => ()
}
}
if (commitment.commitInput.isP2tr) {
// We choose the closing signature that matches our preferred closing transaction.
val closingTxsWithSigs = Seq(
closingComplete.closerAndCloseeOutputsPartialSig_opt.flatMap(remoteSig => closingTxs.localAndRemote_opt.map(tx => (tx, remoteSig, localSig => ClosingTlv.CloserAndCloseeOutputsPartialSignature(localSig)))),
closingComplete.closeeOutputOnlyPartialSig_opt.flatMap(remoteSig => closingTxs.localOnly_opt.map(tx => (tx, remoteSig, localSig => ClosingTlv.CloseeOutputOnlyPartialSignature(localSig)))),
closingComplete.closerOutputOnlyPartialSig_opt.flatMap(remoteSig => closingTxs.remoteOnly_opt.map(tx => (tx, remoteSig, localSig => ClosingTlv.CloserOutputOnlyPartialSignature(localSig)))),
).flatten
closingTxsWithSigs.headOption match {
case Some((closingTx, remoteSig, sigToTlv)) =>
val localFundingPubKey = keyManager.fundingPublicKey(commitment.localParams.fundingKeyPath, commitment.fundingTxIndex)
(for {
localSig <- keyManager.partialSign(closingTx, localFundingPubKey, commitment.remoteFundingPubKey, TxOwner.Local, localClosingNonce_opt.get, remoteClosingNonce_opt.get)
aggregatedSignature <- Transactions.aggregatePartialSignatures(closingTx, localSig, remoteSig, localFundingPubKey.publicKey, commitment.remoteFundingPubKey, localClosingNonce_opt.get._2, remoteClosingNonce_opt.get)
signedClosingTx = Transactions.addAggregatedSignature(closingTx, aggregatedSignature)
} yield (signedClosingTx, localSig)) match {
case Right((signedClosingTx, localSig)) if Transactions.checkSpendable(signedClosingTx).isSuccess => Right(signedClosingTx, ClosingSig(commitment.channelId, remoteScriptPubkey, localScriptPubkey, closingComplete.fees, closingComplete.lockTime, TlvStream(sigToTlv(localSig))))
case _ => Left(InvalidCloseSignature(commitment.channelId, closingTx.tx.txid))
}
case None => Left(MissingCloseSignature(commitment.channelId))
}
} else {
// We choose the closing signature that matches our preferred closing transaction.
val closingTxsWithSigs = Seq(
closingComplete.closerAndCloseeOutputsSig_opt.flatMap(remoteSig => closingTxs.localAndRemote_opt.map(tx => (tx, remoteSig, localSig => ClosingTlv.CloserAndCloseeOutputs(localSig)))),
@ -758,6 +828,7 @@ object Helpers {
case None => Left(MissingCloseSignature(commitment.channelId))
}
}
}
/**
* We are the closer: they sent us their signature so we should now have a fully signed closing transaction.
@ -766,7 +837,27 @@ object Helpers {
* sent another closing_complete before receiving their closing_sig, which is now obsolete: we ignore it and wait
* for their next closing_sig that will match our latest closing_complete.
*/
def receiveSimpleClosingSig(keyManager: ChannelKeyManager, commitment: FullCommitment, closingTxs: ClosingTxs, closingSig: ClosingSig): Either[ChannelException, ClosingTx] = {
def receiveSimpleClosingSig(keyManager: ChannelKeyManager, commitment: FullCommitment, closingTxs: ClosingTxs, closingSig: ClosingSig, localNonce: Option[(SecretNonce, IndividualNonce)] = None, remoteNonce: Option[IndividualNonce] = None): Either[ChannelException, ClosingTx] = {
if (commitment.commitInput.isP2tr) {
val closingTxsWithSig = Seq(
closingSig.closerAndCloseeOutputsPartialSig_opt.flatMap(sig => closingTxs.localAndRemote_opt.map(tx => (tx, sig))),
closingSig.closerOutputOnlyPartialSig_opt.flatMap(sig => closingTxs.localOnly_opt.map(tx => (tx, sig))),
closingSig.closeeOutputOnlyPartialSig_opt.flatMap(sig => closingTxs.remoteOnly_opt.map(tx => (tx, sig))),
).flatten
closingTxsWithSig.headOption match {
case Some((closingTx, remoteSig)) =>
val localFundingPubKey = keyManager.fundingPublicKey(commitment.localParams.fundingKeyPath, commitment.fundingTxIndex)
(for {
localSig <- keyManager.partialSign(closingTx, localFundingPubKey, commitment.remoteFundingPubKey, TxOwner.Local, localNonce.get, remoteNonce.get)
aggregatedSig <- Transactions.aggregatePartialSignatures(closingTx, localSig, remoteSig, localFundingPubKey.publicKey, commitment.remoteFundingPubKey, localNonce.get._2, remoteNonce.get)
signedClosingTx = Transactions.addAggregatedSignature(closingTx, aggregatedSig)
} yield signedClosingTx) match {
case Right(signedClosingTx) if Transactions.checkSpendable(signedClosingTx).isSuccess => Right(signedClosingTx)
case _ => Left(InvalidCloseSignature(commitment.channelId, closingTx.tx.txid))
}
case None => Left(MissingCloseSignature(commitment.channelId))
}
} else {
val closingTxsWithSig = Seq(
closingSig.closerAndCloseeOutputsSig_opt.flatMap(sig => closingTxs.localAndRemote_opt.map(tx => (tx, sig))),
closingSig.closerOutputOnlySig_opt.flatMap(sig => closingTxs.localOnly_opt.map(tx => (tx, sig))),
@ -784,6 +875,7 @@ object Helpers {
case None => Left(MissingCloseSignature(commitment.channelId))
}
}
}
/**
* Check that all closing outputs are above bitcoin's dust limit for their script type, otherwise there is a risk
@ -885,12 +977,25 @@ object Helpers {
def claimAnchors(keyManager: ChannelKeyManager, commitment: FullCommitment, lcp: LocalCommitPublished, confirmationTarget: ConfirmationTarget)(implicit log: LoggingAdapter): LocalCommitPublished = {
if (shouldUpdateAnchorTxs(lcp.claimAnchorTxs, confirmationTarget)) {
val localFundingPubKey = keyManager.fundingPublicKey(commitment.localParams.fundingKeyPath, commitment.fundingTxIndex).publicKey
val localPaymentKey = if (commitment.params.commitmentFormat.useTaproot) {
val channelKeyPath = keyManager.keyPath(commitment.localParams, commitment.params.channelConfig)
val localPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, commitment.localCommit.index)
val localDelayedPaymentPubkey = Generators.derivePubKey(keyManager.delayedPaymentPoint(channelKeyPath).publicKey, localPerCommitmentPoint)
localDelayedPaymentPubkey
} else {
localFundingPubKey
}
val remotePaymentKey = if (commitment.params.commitmentFormat.useTaproot) {
commitment.remoteParams.paymentBasepoint
} else {
commitment.remoteFundingPubKey
}
val claimAnchorTxs = List(
withTxGenerationLog("local-anchor") {
Transactions.makeClaimLocalAnchorOutputTx(lcp.commitTx, localFundingPubKey, confirmationTarget)
Transactions.makeClaimLocalAnchorOutputTx(lcp.commitTx, localPaymentKey, confirmationTarget)
},
withTxGenerationLog("remote-anchor") {
Transactions.makeClaimRemoteAnchorOutputTx(lcp.commitTx, commitment.remoteFundingPubKey)
Transactions.makeClaimRemoteAnchorOutputTx(lcp.commitTx, remotePaymentKey)
}
).flatten
lcp.copy(claimAnchorTxs = claimAnchorTxs)
@ -1006,12 +1111,26 @@ object Helpers {
def claimAnchors(keyManager: ChannelKeyManager, commitment: FullCommitment, rcp: RemoteCommitPublished, confirmationTarget: ConfirmationTarget)(implicit log: LoggingAdapter): RemoteCommitPublished = {
if (shouldUpdateAnchorTxs(rcp.claimAnchorTxs, confirmationTarget)) {
val localFundingPubkey = keyManager.fundingPublicKey(commitment.localParams.fundingKeyPath, commitment.fundingTxIndex).publicKey
// taproot channels do not re-use the funding pubkeys for anchor outputs
val localPaymentKey = if (commitment.params.commitmentFormat.useTaproot) {
val channelKeyPath = keyManager.keyPath(commitment.localParams, commitment.params.channelConfig)
commitment.localParams.walletStaticPaymentBasepoint.getOrElse(keyManager.paymentPoint(channelKeyPath).publicKey)
} else {
localFundingPubkey
}
val remotePaymentKey = if (commitment.params.commitmentFormat.useTaproot) {
val remoteDelayedPaymentPubkey = Generators.derivePubKey(commitment.remoteParams.delayedPaymentBasepoint, commitment.remoteCommit.remotePerCommitmentPoint)
remoteDelayedPaymentPubkey
} else {
commitment.remoteFundingPubKey
}
val claimAnchorTxs = List(
withTxGenerationLog("local-anchor") {
Transactions.makeClaimLocalAnchorOutputTx(rcp.commitTx, localFundingPubkey, confirmationTarget)
Transactions.makeClaimLocalAnchorOutputTx(rcp.commitTx, localPaymentKey, confirmationTarget)
},
withTxGenerationLog("remote-anchor") {
Transactions.makeClaimRemoteAnchorOutputTx(rcp.commitTx, commitment.remoteFundingPubKey)
Transactions.makeClaimRemoteAnchorOutputTx(rcp.commitTx, remotePaymentKey)
}
).flatten
rcp.copy(claimAnchorTxs = claimAnchorTxs)
@ -1200,6 +1319,24 @@ object Helpers {
// we retrieve the information needed to rebuild htlc scripts
val htlcInfos = db.listHtlcInfos(channelId, commitmentNumber)
log.info("got {} htlcs for commitmentNumber={}", htlcInfos.size, commitmentNumber)
val htlcPenaltyTxs = if (commitmentFormat.useTaproot) {
val scriptTrees = (
htlcInfos.map { case (paymentHash, cltvExpiry) => Taproot.receivedHtlcScriptTree(remoteHtlcPubkey, localHtlcPubkey, paymentHash, cltvExpiry) } ++
htlcInfos.map { case (paymentHash, _) => Taproot.offeredHtlcScriptTree(remoteHtlcPubkey, localHtlcPubkey, paymentHash) })
.map(scriptTree => Script.write(Script.pay2tr(remoteRevocationPubkey.xOnly, Some(scriptTree))) -> scriptTree)
.toMap
commitTx.txOut.zipWithIndex.collect { case (txOut, outputIndex) if scriptTrees.contains(txOut.publicKeyScript) =>
val scriptTree = scriptTrees(txOut.publicKeyScript)
withTxGenerationLog("htlc-penalty") {
Transactions.makeHtlcPenaltyTx(commitTx, outputIndex, remoteRevocationPubkey.xOnly, Some(scriptTree), localParams.dustLimit, finalScriptPubKey, feeratePenalty).map(htlcPenalty => {
val sig = keyManager.sign(htlcPenalty, keyManager.revocationPoint(channelKeyPath), remotePerCommitmentSecret, TxOwner.Local, commitmentFormat)
Transactions.addSigs(htlcPenalty, sig, remoteRevocationPubkey)
})
}
}.toList.flatten
} else {
val htlcsRedeemScripts = (
htlcInfos.map { case (paymentHash, cltvExpiry) => Scripts.htlcReceived(remoteHtlcPubkey, localHtlcPubkey, remoteRevocationPubkey, Crypto.ripemd160(paymentHash), cltvExpiry, commitmentFormat) } ++
htlcInfos.map { case (paymentHash, _) => Scripts.htlcOffered(remoteHtlcPubkey, localHtlcPubkey, remoteRevocationPubkey, Crypto.ripemd160(paymentHash), commitmentFormat) }
@ -1208,7 +1345,7 @@ object Helpers {
.toMap
// and finally we steal the htlc outputs
val htlcPenaltyTxs = commitTx.txOut.zipWithIndex.collect { case (txOut, outputIndex) if htlcsRedeemScripts.contains(txOut.publicKeyScript) =>
commitTx.txOut.zipWithIndex.collect { case (txOut, outputIndex) if htlcsRedeemScripts.contains(txOut.publicKeyScript) =>
val htlcRedeemScript = htlcsRedeemScripts(txOut.publicKeyScript)
withTxGenerationLog("htlc-penalty") {
Transactions.makeHtlcPenaltyTx(commitTx, outputIndex, htlcRedeemScript, localParams.dustLimit, finalScriptPubKey, feeratePenalty).map(htlcPenalty => {
@ -1217,6 +1354,7 @@ object Helpers {
})
}
}.toList.flatten
}
RevokedCommitPublished(
commitTx = commitTx,
@ -1313,25 +1451,64 @@ object Helpers {
}
}
def isHtlcTimeout(tx: Transaction, localCommitPublished: LocalCommitPublished): Boolean = {
tx.txIn.filter(txIn => localCommitPublished.htlcTxs.get(txIn.outPoint) match {
case Some(Some(_: HtlcTimeoutTx)) => true
/**
*
* @param witness input witness
* @param scriptTree taproot script tree
* @return true if witness spends the script in the left branch of the script tree
*/
def witnessSpendsLeftBranch(witness: ScriptWitness, scriptTree: ScriptTree): Boolean = {
scriptTree match {
case b: ScriptTree.Branch => b.getLeft match {
case l: ScriptTree.Leaf => witness.stack.size >= 3 && witness.stack(witness.stack.size - 2) == KotlinUtils.kmp2scala(l.getScript)
case _ => false
}).map(_.witness).collect(Scripts.extractPaymentHashFromHtlcTimeout).nonEmpty
}
case _ => false
}
}
/**
*
* @param witness input witness
* @param scriptTree taproot script tree
* @return true if witness spends the script in the right branch of the script tree
*/
def witnessSpendsRightBranch(witness: ScriptWitness, scriptTree: ScriptTree): Boolean = !witnessSpendsLeftBranch(witness, scriptTree)
def isHtlcTimeout(tx: Transaction, localCommitPublished: LocalCommitPublished): Boolean = {
tx.txIn.exists(txIn => localCommitPublished.htlcTxs.get(txIn.outPoint) match {
case Some(Some(htlcTimeOutTx: HtlcTimeoutTx)) => htlcTimeOutTx.input match {
case t: InputInfo.TaprootInput =>
// this is a HTLC time-out tx if it uses the left branch of the script tree
t.scriptTree_opt.exists(scriptTree => witnessSpendsLeftBranch(txIn.witness, scriptTree))
case _: InputInfo.SegwitInput => Scripts.extractPaymentHashFromHtlcTimeout.isDefinedAt(txIn.witness)
}
case _ => false
})
}
def isHtlcSuccess(tx: Transaction, localCommitPublished: LocalCommitPublished): Boolean = {
tx.txIn.filter(txIn => localCommitPublished.htlcTxs.get(txIn.outPoint) match {
case Some(Some(_: HtlcSuccessTx)) => true
tx.txIn.exists(txIn => localCommitPublished.htlcTxs.get(txIn.outPoint) match {
case Some(Some(htlcSuccessTx: HtlcSuccessTx)) => htlcSuccessTx.input match {
case t: InputInfo.TaprootInput =>
// this is a HTLC success tx if it uses the right branch of the script tree
t.scriptTree_opt.exists(scriptTree => witnessSpendsRightBranch(txIn.witness, scriptTree))
case _: InputInfo.SegwitInput => Scripts.extractPreimageFromHtlcSuccess.isDefinedAt(txIn.witness)
}
case _ => false
}).map(_.witness).collect(Scripts.extractPreimageFromHtlcSuccess).nonEmpty
})
}
def isClaimHtlcTimeout(tx: Transaction, remoteCommitPublished: RemoteCommitPublished): Boolean = {
tx.txIn.filter(txIn => remoteCommitPublished.claimHtlcTxs.get(txIn.outPoint) match {
case Some(Some(_: ClaimHtlcTimeoutTx)) => true
tx.txIn.exists(txIn => remoteCommitPublished.claimHtlcTxs.get(txIn.outPoint) match {
case Some(Some(c: ClaimHtlcTimeoutTx)) => c.input match {
case t: InputInfo.TaprootInput =>
// this is a HTLC timeout tx if it uses the left branch of the script tree
t.scriptTree_opt.exists(scriptTree => witnessSpendsLeftBranch(txIn.witness, scriptTree))
case _: InputInfo.SegwitInput => Scripts.extractPaymentHashFromClaimHtlcTimeout.isDefinedAt(txIn.witness)
}
case _ => false
}).map(_.witness).collect(Scripts.extractPaymentHashFromClaimHtlcTimeout).nonEmpty
})
}
def isClaimHtlcSuccess(tx: Transaction, remoteCommitPublished: RemoteCommitPublished): Boolean = {

View file

@ -20,6 +20,7 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapter}
import akka.actor.{Actor, ActorContext, ActorRef, FSM, OneForOneStrategy, PossiblyHarmful, Props, SupervisorStrategy, typed}
import akka.event.Logging.MDC
import fr.acinq.bitcoin.crypto.musig2.{IndividualNonce, SecretNonce}
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, SatoshiLong, Transaction, TxId}
import fr.acinq.eclair.Logs.LogCategory
@ -50,9 +51,11 @@ import fr.acinq.eclair.payment.{Bolt11Invoice, PaymentSettlingOnChain}
import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.transactions.Transactions.ClosingTx
import fr.acinq.eclair.transactions._
import fr.acinq.eclair.wire.protocol.ChannelTlv.NextLocalNoncesTlv
import fr.acinq.eclair.wire.protocol._
import scala.collection.immutable.Queue
import scala.collection.mutable
import scala.concurrent.ExecutionContext
import scala.concurrent.duration._
import scala.util.Random
@ -204,6 +207,17 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
val keyManager: ChannelKeyManager = nodeParams.channelKeyManager
// remote nonces, one for each active commitment, with the same ordering
var remoteNextLocalNonces: List[IndividualNonce] = List.empty
// // will be added to remoteNextLocalNonces once a splice has been completed
var pendingRemoteNextLocalNonce: Option[IndividualNonce] = None
def setRemoteNextLocalNonces(info: String, n: List[IndividualNonce]): Unit = {
this.remoteNextLocalNonces = n
log.debug("{} set remoteNextLocalNonces to {}", info, remoteNextLocalNonces)
}
// we pass these to helpers classes so that they have the logging context
implicit def implicitLog: akka.event.DiagnosticLoggingAdapter = diagLog
@ -246,6 +260,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
}
}
var localClosingNonce: Option[(SecretNonce, IndividualNonce)] = None // used to sign closing txs
var remoteClosingNonce: Option[IndividualNonce] = None
val txPublisher = txPublisherFactory.spawnTxPublisher(context, remoteNodeId)
// this will be used to detect htlc timeouts
@ -562,7 +579,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
log.debug("ignoring CMD_SIGN (nothing to sign)")
stay()
case Right(_) =>
d.commitments.sendCommit(keyManager) match {
d.commitments.sendCommit(keyManager, this.remoteNextLocalNonces) match {
case Right((commitments1, commit)) =>
log.debug("sending a new sig, spec:\n{}", commitments1.latest.specs2String)
val nextRemoteCommit = commitments1.latest.nextRemoteCommit_opt.get.commit
@ -619,7 +636,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
stay() using d1 storing() sending signingSession1.localSigs calling endQuiescence(d1)
}
}
case _ if d.commitments.ignoreRetransmittedCommitSig(commit) =>
case _ if d.commitments.ignoreRetransmittedCommitSig(commit, keyManager) =>
// We haven't received our peer's tx_signatures for the latest funding transaction and asked them to resend it on reconnection.
// They also resend their corresponding commit_sig, but we have already received it so we should ignore it.
// Note that the funding transaction may have confirmed while we were reconnecting.
@ -666,6 +683,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
d.commitments.receiveRevocation(revocation, nodeParams.onChainFeeConf.feerateToleranceFor(remoteNodeId).dustTolerance.maxExposure) match {
case Right((commitments1, actions)) =>
cancelTimer(RevocationTimeout.toString)
setRemoteNextLocalNonces("received RevokeAndAck", revocation.nexLocalNonces)
log.debug("received a new rev, spec:\n{}", commitments1.latest.specs2String)
actions.foreach {
case PostRevocationAction.RelayHtlc(add) =>
@ -685,7 +703,17 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
if (d.remoteShutdown.isDefined && !commitments1.changes.localHasUnsignedOutgoingHtlcs) {
// we were waiting for our pending htlcs to be signed before replying with our local shutdown
val finalScriptPubKey = getOrGenerateFinalScriptPubKey(d)
val localShutdown = Shutdown(d.channelId, finalScriptPubKey)
require(d.commitments.params.commitmentFormat.useTaproot == d.commitments.latest.commitInput.isP2tr, s"commitmentFormat = ${d.commitments.params.commitmentFormat.useTaproot} is not consistent with commit input") // TODO: remove this
val tlvStream: TlvStream[ShutdownTlv] = if (d.commitments.latest.commitInput.isP2tr) {
val localFundingPubKey = keyManager.fundingPublicKey(d.commitments.latest.localParams.fundingKeyPath, d.commitments.latest.fundingTxIndex).publicKey
log.info("generating closing nonce {} with fundingKeyPath = {} fundingTxIndex = {}", localClosingNonce, d.commitments.latest.localParams.fundingKeyPath, d.commitments.latest.fundingTxIndex)
localClosingNonce = Some(keyManager.signingNonce(localFundingPubKey))
TlvStream(ShutdownTlv.ShutdownNonce(localClosingNonce.get._2))
} else {
TlvStream.empty
}
val localShutdown = Shutdown(d.channelId, finalScriptPubKey, tlvStream)
// note: it means that we had pending htlcs to sign, therefore we go to SHUTDOWN, not to NEGOTIATING
require(commitments1.latest.remoteCommit.spec.htlcs.nonEmpty, "we must have just signed new htlcs, otherwise we would have sent our Shutdown earlier")
goto(SHUTDOWN) using DATA_SHUTDOWN(commitments1, localShutdown, d.remoteShutdown.get, d.closingFeerates) storing() sending localShutdown
@ -710,7 +738,15 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
d.commitments.params.validateLocalShutdownScript(localScriptPubKey) match {
case Left(e) => handleCommandError(e, c)
case Right(localShutdownScript) =>
val shutdown = Shutdown(d.channelId, localShutdownScript)
val tlvStream: TlvStream[ShutdownTlv] = if (d.commitments.latest.commitInput.isP2tr) {
val localFundingPubKey = keyManager.fundingPublicKey(d.commitments.latest.localParams.fundingKeyPath, d.commitments.latest.fundingTxIndex).publicKey
log.info("generating closing nonce {} with fundingKeyPath = {} fundingTxIndex = {}", localClosingNonce, d.commitments.latest.localParams.fundingKeyPath, d.commitments.latest.fundingTxIndex)
localClosingNonce = Some(keyManager.signingNonce(localFundingPubKey))
TlvStream(ShutdownTlv.ShutdownNonce(localClosingNonce.get._2))
} else {
TlvStream.empty
}
val shutdown = Shutdown(d.channelId, localShutdownScript, tlvStream)
handleCommandSuccess(c, d.copy(localShutdown = Some(shutdown), closingFeerates = c.feerates)) storing() sending shutdown
}
}
@ -754,12 +790,25 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
// in the meantime we won't send new changes
stay() using d.copy(remoteShutdown = Some(remoteShutdown))
} else {
if (d.commitments.latest.commitInput.isP2tr) {
require(remoteShutdown.shutdownNonce_opt.isDefined, "missing shutdown nonce")
}
remoteClosingNonce = remoteShutdown.shutdownNonce_opt
// so we don't have any unsigned outgoing changes
val (localShutdown, sendList) = d.localShutdown match {
case Some(localShutdown) =>
(localShutdown, Nil)
case None =>
val localShutdown = Shutdown(d.channelId, getOrGenerateFinalScriptPubKey(d))
val tlvStream: TlvStream[ShutdownTlv] = if (d.commitments.latest.commitInput.isP2tr) {
val localFundingPubKey = keyManager.fundingPublicKey(d.commitments.latest.localParams.fundingKeyPath, d.commitments.latest.fundingTxIndex).publicKey
log.info("generating closing nonce {} with fundingKeyPath = {} fundingTxIndex = {}", localClosingNonce, d.commitments.latest.localParams.fundingKeyPath, d.commitments.latest.fundingTxIndex)
localClosingNonce = Some(keyManager.signingNonce(localFundingPubKey))
TlvStream(ShutdownTlv.ShutdownNonce(localClosingNonce.get._2))
} else {
TlvStream.empty
}
val localShutdown = Shutdown(d.channelId, getOrGenerateFinalScriptPubKey(d), tlvStream)
// we need to send our shutdown if we didn't previously
(localShutdown, localShutdown :: Nil)
}
@ -767,7 +816,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
if (d.commitments.hasNoPendingHtlcsOrFeeUpdate) {
// there are no pending signed changes, let's directly negotiate a closing transaction
if (Features.canUseFeature(d.commitments.params.localParams.initFeatures, d.commitments.params.remoteParams.initFeatures, Features.SimpleClose)) {
val (d1, closingComplete_opt) = startSimpleClose(d.commitments, localShutdown, remoteShutdown, d.closingFeerates)
val (d1, closingComplete_opt) = startSimpleClose(d.commitments, localShutdown, remoteShutdown, d.closingFeerates, localClosingNonce)
goto(NEGOTIATING_SIMPLE) using d1 storing() sending sendList ++ closingComplete_opt.toSeq
} else if (d.commitments.params.localParams.paysClosingFees) {
// we pay the closing fees, so we initiate the negotiation by sending the first closing_signed
@ -1023,7 +1072,12 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
} else {
val parentCommitment = d.commitments.latest.commitment
val localFundingPubKey = nodeParams.channelKeyManager.fundingPublicKey(d.commitments.params.localParams.fundingKeyPath, parentCommitment.fundingTxIndex + 1).publicKey
val fundingScript = Funding.makeFundingPubKeyScript(localFundingPubKey, msg.fundingPubKey)
val fundingScript = Funding.makeFundingPubKeyScript(localFundingPubKey, msg.fundingPubKey, d.commitments.latest.params.commitmentFormat)
val sharedInput = if (d.commitments.latest.commitInput.isP2tr) {
Musig2Input(parentCommitment)
} else {
Multisig2of2Input(parentCommitment)
}
LiquidityAds.validateRequest(nodeParams.privateKey, d.channelId, fundingScript, msg.feerate, isChannelCreation = false, msg.requestFunding_opt, nodeParams.liquidityAdsConfig.rates_opt, msg.useFeeCredit_opt) match {
case Left(t) =>
log.warning("rejecting splice request with invalid liquidity ads: {}", t.getMessage)
@ -1043,7 +1097,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
isInitiator = false,
localContribution = spliceAck.fundingContribution,
remoteContribution = msg.fundingContribution,
sharedInput_opt = Some(Multisig2of2Input(parentCommitment)),
sharedInput_opt = Some(sharedInput),
remoteFundingPubKey = msg.fundingPubKey,
localOutputs = Nil,
lockTime = msg.lockTime,
@ -1052,6 +1106,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
requireConfirmedInputs = RequireConfirmedInputs(forLocal = msg.requireConfirmedInputs, forRemote = spliceAck.requireConfirmedInputs)
)
val sessionId = randomBytes32()
log.debug("spawning InteractiveTxBuilder with remoteNextLocalNonces {}", remoteNextLocalNonces)
val txBuilder = context.spawnAnonymous(InteractiveTxBuilder(
sessionId,
nodeParams, fundingParams,
@ -1081,12 +1136,17 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case SpliceStatus.SpliceRequested(cmd, spliceInit) =>
log.info("our peer accepted our splice request and will contribute {} to the funding transaction", msg.fundingContribution)
val parentCommitment = d.commitments.latest.commitment
val sharedInput = if (d.commitments.latest.commitInput.isP2tr) {
Musig2Input(parentCommitment)
} else {
Multisig2of2Input(parentCommitment)
}
val fundingParams = InteractiveTxParams(
channelId = d.channelId,
isInitiator = true,
localContribution = spliceInit.fundingContribution,
remoteContribution = msg.fundingContribution,
sharedInput_opt = Some(Multisig2of2Input(parentCommitment)),
sharedInput_opt = Some(sharedInput),
remoteFundingPubKey = msg.fundingPubKey,
localOutputs = cmd.spliceOutputs,
lockTime = spliceInit.lockTime,
@ -1094,7 +1154,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
targetFeerate = spliceInit.feerate,
requireConfirmedInputs = RequireConfirmedInputs(forLocal = msg.requireConfirmedInputs, forRemote = spliceInit.requireConfirmedInputs)
)
val fundingScript = Funding.makeFundingPubKeyScript(spliceInit.fundingPubKey, msg.fundingPubKey)
val fundingScript = Funding.makeFundingPubKeyScript(spliceInit.fundingPubKey, msg.fundingPubKey, d.commitments.latest.params.commitmentFormat)
LiquidityAds.validateRemoteFunding(spliceInit.requestFunding_opt, remoteNodeId, d.channelId, fundingScript, msg.fundingContribution, spliceInit.feerate, isChannelCreation = false, msg.willFund_opt) match {
case Left(t) =>
log.info("rejecting splice attempt: invalid liquidity ads response ({})", t.getMessage)
@ -1109,8 +1169,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
purpose = InteractiveTxBuilder.SpliceTx(parentCommitment, d.commitments.changes),
localPushAmount = cmd.pushAmount, remotePushAmount = msg.pushAmount,
liquidityPurchase_opt = liquidityPurchase_opt,
wallet
))
wallet))
txBuilder ! InteractiveTxBuilder.Start(self)
stay() using d.copy(spliceStatus = SpliceStatus.SpliceInProgress(cmd_opt = Some(cmd), sessionId, txBuilder, remoteCommitSig = None))
}
@ -1156,12 +1215,17 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
val fundingContribution = willFund_opt.map(_.purchase.amount).getOrElse(rbf.latestFundingTx.fundingParams.localContribution)
log.info("accepting rbf with remote.in.amount={} local.in.amount={}", msg.fundingContribution, fundingContribution)
val txAckRbf = TxAckRbf(d.channelId, fundingContribution, rbf.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote, willFund_opt.map(_.willFund))
val sharedInput = if (d.commitments.latest.commitInput.isP2tr) {
Musig2Input(rbf.parentCommitment)
} else {
Multisig2of2Input(rbf.parentCommitment)
}
val fundingParams = InteractiveTxParams(
channelId = d.channelId,
isInitiator = false,
localContribution = fundingContribution,
remoteContribution = msg.fundingContribution,
sharedInput_opt = Some(Multisig2of2Input(rbf.parentCommitment)),
sharedInput_opt = Some(sharedInput),
remoteFundingPubKey = rbf.latestFundingTx.fundingParams.remoteFundingPubKey,
localOutputs = rbf.latestFundingTx.fundingParams.localOutputs,
lockTime = msg.lockTime,
@ -1177,8 +1241,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
purpose = rbf,
localPushAmount = 0 msat, remotePushAmount = 0 msat,
willFund_opt.map(_.purchase),
wallet
))
wallet))
// this.pendingRemoteNextLocalNonce = msg.secondRemoteNonce
txBuilder ! InteractiveTxBuilder.Start(self)
stay() using d.copy(spliceStatus = SpliceStatus.SpliceInProgress(cmd_opt = None, sessionId, txBuilder, remoteCommitSig = None)) sending txAckRbf
}
@ -1209,12 +1273,17 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, t.getMessage)
case Right(liquidityPurchase_opt) =>
log.info("our peer accepted our rbf request and will contribute {} to the funding transaction", msg.fundingContribution)
val sharedInput = if (d.commitments.latest.commitInput.isP2tr) {
Musig2Input(rbf.parentCommitment)
} else {
Multisig2of2Input(rbf.parentCommitment)
}
val fundingParams = InteractiveTxParams(
channelId = d.channelId,
isInitiator = true,
localContribution = txInitRbf.fundingContribution,
remoteContribution = msg.fundingContribution,
sharedInput_opt = Some(Multisig2of2Input(rbf.parentCommitment)),
sharedInput_opt = Some(sharedInput),
remoteFundingPubKey = rbf.latestFundingTx.fundingParams.remoteFundingPubKey,
localOutputs = rbf.latestFundingTx.fundingParams.localOutputs,
lockTime = txInitRbf.lockTime,
@ -1230,8 +1299,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
purpose = rbf,
localPushAmount = 0 msat, remotePushAmount = 0 msat,
liquidityPurchase_opt = liquidityPurchase_opt,
wallet
))
wallet))
// this.pendingRemoteNextLocalNonce = msg.secondRemoteNonce
txBuilder ! InteractiveTxBuilder.Start(self)
stay() using d.copy(spliceStatus = SpliceStatus.SpliceInProgress(cmd_opt = Some(cmd), sessionId, txBuilder, remoteCommitSig = None))
}
@ -1291,8 +1360,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
log.info("ignoring outgoing interactive-tx message {} from previous session", msg.getClass.getSimpleName)
stay()
}
case InteractiveTxBuilder.Succeeded(signingSession, commitSig, liquidityPurchase_opt) =>
case InteractiveTxBuilder.Succeeded(signingSession, commitSig, liquidityPurchase_opt, nextRemoteNonce_opt) =>
log.info(s"splice tx created with fundingTxIndex=${signingSession.fundingTxIndex} fundingTxId=${signingSession.fundingTx.txId}")
this.pendingRemoteNextLocalNonce = nextRemoteNonce_opt
cmd_opt.foreach(cmd => cmd.replyTo ! RES_SPLICE(fundingTxIndex = signingSession.fundingTxIndex, signingSession.fundingTx.txId, signingSession.fundingParams.fundingAmount, signingSession.localCommit.fold(_.spec, _.spec).toLocal))
remoteCommitSig_opt.foreach(self ! _)
liquidityPurchase_opt.collect {
@ -1327,6 +1397,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Right((commitments1, _)) =>
log.info("publishing funding tx for channelId={} fundingTxId={}", d.channelId, fundingTx.signedTx.txid)
Metrics.recordSplice(dfu.fundingParams, fundingTx.tx)
// README: splice has been completed, update remote nonces with the one sent in splice_init/splice_ack
setRemoteNextLocalNonces("received TxSignatures", this.pendingRemoteNextLocalNonce.toList ++ this.remoteNextLocalNonces)
stay() using d.copy(commitments = commitments1) storing() calling publishFundingTx(dfu1)
case Left(_) =>
stay()
@ -1347,6 +1419,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
val d1 = d.copy(commitments = commitments1, spliceStatus = SpliceStatus.NoSplice)
log.info("publishing funding tx for channelId={} fundingTxId={}", d.channelId, signingSession1.fundingTx.sharedTx.txId)
Metrics.recordSplice(signingSession1.fundingTx.fundingParams, signingSession1.fundingTx.sharedTx.tx)
setRemoteNextLocalNonces("end of quiescence", this.pendingRemoteNextLocalNonce.toList ++ this.remoteNextLocalNonces)
stay() using d1 storing() sending signingSession1.localSigs calling publishFundingTx(signingSession1.fundingTx) calling endQuiescence(d1)
}
case _ =>
@ -1545,7 +1618,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
log.debug("ignoring CMD_SIGN (nothing to sign)")
stay()
case Right(_) =>
d.commitments.sendCommit(keyManager) match {
d.commitments.sendCommit(keyManager, this.remoteNextLocalNonces) match {
case Right((commitments1, commit)) =>
log.debug("sending a new sig, spec:\n{}", commitments1.latest.specs2String)
val nextRemoteCommit = commitments1.latest.nextRemoteCommit_opt.get.commit
@ -1739,6 +1812,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
val localClosingFees = MutualClose.firstClosingFee(d.commitments.latest, d.localShutdown.scriptPubKey, d.remoteShutdown.scriptPubKey, nodeParams.currentFeeratesForFundingClosing, nodeParams.onChainFeeConf)
val nextPreferredFee = MutualClose.nextClosingFee(lastLocalClosingFee_opt.getOrElse(localClosingFees.preferred), remoteClosingFee)
MutualClose.makeClosingTx(keyManager, d.commitments.latest, d.localShutdown.scriptPubKey, d.remoteShutdown.scriptPubKey, localClosingFees.copy(preferred = nextPreferredFee))
Closing.MutualClose.makeClosingTx(keyManager, d.commitments.latest, d.localShutdown.scriptPubKey, d.remoteShutdown.scriptPubKey, localClosingFees.copy(preferred = nextPreferredFee))
}
val closingTxProposed1 = (d.closingTxProposed: @unchecked) match {
case previousNegotiations :+ currentNegotiation => previousNegotiations :+ (currentNegotiation :+ ClosingTxProposed(closingTx, closingSigned))
@ -1815,7 +1889,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
// No need to persist their latest script, they will re-sent it on reconnection.
stay() using d.copy(remoteScriptPubKey = closingComplete.closerScriptPubKey) sending Warning(d.channelId, InvalidCloseeScript(d.channelId, closingComplete.closeeScriptPubKey, d.localScriptPubKey).getMessage)
} else {
MutualClose.signSimpleClosingTx(keyManager, d.commitments.latest, closingComplete.closeeScriptPubKey, closingComplete.closerScriptPubKey, closingComplete) match {
MutualClose.signSimpleClosingTx(keyManager, d.commitments.latest, closingComplete.closeeScriptPubKey, closingComplete.closerScriptPubKey, closingComplete, localClosingNonce, remoteClosingNonce) match {
case Left(f) =>
log.warning("invalid closing_complete: {}", f.getMessage)
stay() sending Warning(d.channelId, f.getMessage)
@ -1830,7 +1904,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
// Note that if we sent two closing_complete in a row, without waiting for their closing_sig for the first one,
// this will fail because we only care about our latest closing_complete. This is fine, we should receive their
// closing_sig for the last closing_complete afterwards.
MutualClose.receiveSimpleClosingSig(keyManager, d.commitments.latest, d.proposedClosingTxs.last, closingSig) match {
MutualClose.receiveSimpleClosingSig(keyManager, d.commitments.latest, d.proposedClosingTxs.last, closingSig, localClosingNonce, remoteClosingNonce) match {
case Left(f) =>
log.warning("invalid closing_sig: {}", f.getMessage)
stay() sending Warning(d.channelId, f.getMessage)
@ -2240,13 +2314,20 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
val channelKeyPath = keyManager.keyPath(d.channelParams.localParams, d.channelParams.channelConfig)
val myFirstPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 0)
val nextFundingTlv: Set[ChannelReestablishTlv] = Set(ChannelReestablishTlv.NextFundingTlv(d.signingSession.fundingTx.txId))
val myNextLocalNonce = if (d.signingSession.commitInput.isP2tr) {
val localFundingPubKey = nodeParams.channelKeyManager.fundingPublicKey(d.channelParams.localParams.fundingKeyPath, 0).publicKey
val (_, publicNonce) = keyManager.verificationNonce(d.signingSession.fundingTx.txId, localFundingPubKey, 1)
Set(NextLocalNoncesTlv(List(publicNonce)))
} else {
Set.empty
}
val channelReestablish = ChannelReestablish(
channelId = d.channelId,
nextLocalCommitmentNumber = 1,
nextRemoteRevocationNumber = 0,
yourLastPerCommitmentSecret = PrivateKey(ByteVector32.Zeroes),
myCurrentPerCommitmentPoint = myFirstPerCommitmentPoint,
TlvStream(nextFundingTlv),
TlvStream(nextFundingTlv ++ myNextLocalNonce),
)
val d1 = Helpers.updateFeatures(d, localInit, remoteInit)
goto(SYNCING) using d1 sending channelReestablish
@ -2278,13 +2359,32 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
d.commitments.lastLocalLocked_opt.map(c => ChannelReestablishTlv.MyCurrentFundingLockedTlv(c.fundingTxId)).toSet ++
d.commitments.lastRemoteLocked_opt.map(c => ChannelReestablishTlv.YourLastFundingLockedTlv(c.fundingTxId)).toSet
val nonces = d.commitments.active.filter(_.commitInput.isP2tr).map(c => {
val localFundingPubKey = nodeParams.channelKeyManager.fundingPublicKey(d.commitments.params.localParams.fundingKeyPath, c.fundingTxIndex).publicKey
keyManager.verificationNonce(c.fundingTxId, localFundingPubKey, d.commitments.localCommitIndex + 1)._2
})
val nonces1 = d match {
case d: DATA_NORMAL => d.spliceStatus match {
case w: SpliceStatus.SpliceWaitingForSigs if w.signingSession.commitInput.isP2tr =>
val localFundingPubKey = nodeParams.channelKeyManager.fundingPublicKey(d.commitments.params.localParams.fundingKeyPath, w.signingSession.fundingTxIndex).publicKey
val nonce = keyManager.verificationNonce(w.signingSession.fundingTx.txId, localFundingPubKey, w.signingSession.localCommitIndex + 1)._2
nonce +: nonces
case _ => nonces
}
case _ => nonces
}
val myNextLocalNonces = if (nonces1.isEmpty) Set.empty else {
Set(NextLocalNoncesTlv(nonces1.toList))
}
val channelReestablish = ChannelReestablish(
channelId = d.channelId,
nextLocalCommitmentNumber = d.commitments.localCommitIndex + 1,
nextRemoteRevocationNumber = d.commitments.remoteCommitIndex,
yourLastPerCommitmentSecret = PrivateKey(yourLastPerCommitmentSecret),
myCurrentPerCommitmentPoint = myCurrentPerCommitmentPoint,
tlvStream = TlvStream(rbfTlv ++ lastFundingLockedTlvs)
tlvStream = TlvStream(rbfTlv ++ lastFundingLockedTlvs ++ myNextLocalNonces)
)
// we update local/remote connection-local global/local features, we don't persist it right now
val d1 = Helpers.updateFeatures(d, localInit, remoteInit)
@ -2316,31 +2416,40 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
})
when(SYNCING)(handleExceptions {
case Event(_: ChannelReestablish, _: DATA_WAIT_FOR_FUNDING_CONFIRMED) =>
case Event(channelReestablish: ChannelReestablish, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) =>
require(channelReestablish.nextLocalNonces.size == d.commitments.active.count(_.commitInput.isP2tr), "missing next local nonce")
setRemoteNextLocalNonces("received channelReestablish", channelReestablish.nextLocalNonces)
goto(WAIT_FOR_FUNDING_CONFIRMED)
case Event(channelReestablish: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED) =>
if (d.signingSession.commitInput.isP2tr) {
require(channelReestablish.nextLocalNonces.size == 1, "missing next local nonce")
}
setRemoteNextLocalNonces("received ChannelReestablish", channelReestablish.nextLocalNonces)
channelReestablish.nextFundingTxId_opt match {
case Some(fundingTxId) if fundingTxId == d.signingSession.fundingTx.txId =>
// We retransmit our commit_sig, and will send our tx_signatures once we've received their commit_sig.
val commitSig = d.signingSession.remoteCommit.sign(keyManager, d.channelParams, d.signingSession.fundingTxIndex, d.signingSession.fundingParams.remoteFundingPubKey, d.signingSession.commitInput)
val commitSig = d.signingSession.remoteCommit.sign(keyManager, d.channelParams, d.signingSession.fundingTxIndex, d.signingSession.fundingParams.remoteFundingPubKey, d.signingSession.commitInput, remoteNextLocalNonces.headOption)
goto(WAIT_FOR_DUAL_FUNDING_SIGNED) sending commitSig
case _ => goto(WAIT_FOR_DUAL_FUNDING_SIGNED)
}
case Event(channelReestablish: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
require(channelReestablish.nextLocalNonces.size == d.commitments.active.count(_.commitInput.isP2tr), "missing next local nonce")
setRemoteNextLocalNonces("received ChannelReestablish", channelReestablish.nextLocalNonces)
channelReestablish.nextFundingTxId_opt match {
case Some(fundingTxId) =>
d.status match {
case DualFundingStatus.RbfWaitingForSigs(signingSession) if signingSession.fundingTx.txId == fundingTxId =>
// We retransmit our commit_sig, and will send our tx_signatures once we've received their commit_sig.
val commitSig = signingSession.remoteCommit.sign(keyManager, d.commitments.params, signingSession.fundingTxIndex, signingSession.fundingParams.remoteFundingPubKey, signingSession.commitInput)
val commitSig = signingSession.remoteCommit.sign(keyManager, d.commitments.params, signingSession.fundingTxIndex, signingSession.fundingParams.remoteFundingPubKey, signingSession.commitInput, remoteNextLocalNonces.headOption)
goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) sending commitSig
case _ if d.latestFundingTx.sharedTx.txId == fundingTxId =>
val toSend = d.latestFundingTx.sharedTx match {
case fundingTx: InteractiveTxBuilder.PartiallySignedSharedTransaction =>
// We have not received their tx_signatures: we retransmit our commit_sig because we don't know if they received it.
val commitSig = d.commitments.latest.remoteCommit.sign(keyManager, d.commitments.params, d.commitments.latest.fundingTxIndex, d.commitments.latest.remoteFundingPubKey, d.commitments.latest.commitInput)
val commitSig = d.commitments.latest.remoteCommit.sign(keyManager, d.commitments.params, d.commitments.latest.fundingTxIndex, d.commitments.latest.remoteFundingPubKey, d.commitments.latest.commitInput, remoteNextLocalNonces.headOption)
Seq(commitSig, fundingTx.localSigs)
case fundingTx: InteractiveTxBuilder.FullySignedSharedTransaction =>
// We've already received their tx_signatures, which means they've received and stored our commit_sig, we only need to retransmit our tx_signatures.
@ -2355,17 +2464,34 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case None => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED)
}
case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_CHANNEL_READY) =>
case Event(channelReestablish: ChannelReestablish, d: DATA_WAIT_FOR_CHANNEL_READY) =>
require(channelReestablish.nextLocalNonces.size == d.commitments.active.count(_.commitInput.isP2tr), "missing next local nonce")
setRemoteNextLocalNonces("received ChannelReestablish", channelReestablish.nextLocalNonces)
log.debug("re-sending channelReady")
val channelReady = createChannelReady(d.aliases, d.commitments.params)
val channelReady = createChannelReady(d.aliases, d.commitments)
goto(WAIT_FOR_CHANNEL_READY) sending channelReady
case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) =>
case Event(channelReestablish: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) =>
require(channelReestablish.nextLocalNonces.size == d.commitments.active.count(_.commitInput.isP2tr), "missing next local nonce")
log.debug("re-sending channelReady")
val channelReady = createChannelReady(d.aliases, d.commitments.params)
setRemoteNextLocalNonces("received ChannelReestablish", channelReestablish.nextLocalNonces)
val channelReady = createChannelReady(d.aliases, d.commitments)
goto(WAIT_FOR_DUAL_FUNDING_READY) sending channelReady
case Event(channelReestablish: ChannelReestablish, d: DATA_NORMAL) =>
log.debug(s"received $channelReestablish")
d.spliceStatus match {
case _: SpliceStatus.SpliceWaitingForSigs if channelReestablish.nextLocalNonces.size == d.commitments.active.count(_.commitInput.isP2tr) + 1 =>
this.pendingRemoteNextLocalNonce = channelReestablish.nextLocalNonces.headOption
setRemoteNextLocalNonces(s"received ChannelReestablish (waiting for sigs)", channelReestablish.nextLocalNonces.tail)
case _ if channelReestablish.nextLocalNonces.size == d.commitments.active.count(_.commitInput.isP2tr) - 1 =>
()
case _ =>
require(channelReestablish.nextLocalNonces.size >= d.commitments.active.count(_.commitInput.isP2tr), "missing next local nonce")
setRemoteNextLocalNonces("received ChannelReestablish", channelReestablish.nextLocalNonces)
this.pendingRemoteNextLocalNonce = None
}
Syncing.checkSync(keyManager, d.commitments, channelReestablish) match {
case syncFailure: SyncResult.Failure =>
handleSyncFailure(channelReestablish, syncFailure, d)
@ -2380,7 +2506,14 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
log.debug("re-sending channelReady")
val channelKeyPath = keyManager.keyPath(d.commitments.params.localParams, d.commitments.params.channelConfig)
val nextPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1)
val channelReady = ChannelReady(d.commitments.channelId, nextPerCommitmentPoint)
val tlvStream: TlvStream[ChannelReadyTlv] = if (d.commitments.latest.commitInput.isP2tr) {
val localFundingPubkey = keyManager.fundingPublicKey(d.commitments.params.localParams.fundingKeyPath, 0).publicKey
val (_, nextLocalNonce) = keyManager.verificationNonce(d.commitments.latest.fundingTxId, localFundingPubkey, 1) // README: check!!
TlvStream(ChannelTlv.NextLocalNonceTlv(nextLocalNonce))
} else {
TlvStream()
}
val channelReady = ChannelReady(d.commitments.channelId, nextPerCommitmentPoint, tlvStream)
sendQueue = sendQueue :+ channelReady
}
@ -2391,7 +2524,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case SpliceStatus.SpliceWaitingForSigs(signingSession) if signingSession.fundingTx.txId == fundingTxId =>
// We retransmit our commit_sig, and will send our tx_signatures once we've received their commit_sig.
log.info("re-sending commit_sig for splice attempt with fundingTxIndex={} fundingTxId={}", signingSession.fundingTxIndex, signingSession.fundingTx.txId)
val commitSig = signingSession.remoteCommit.sign(keyManager, d.commitments.params, signingSession.fundingTxIndex, signingSession.fundingParams.remoteFundingPubKey, signingSession.commitInput)
val commitSig = signingSession.remoteCommit.localSig_opt.get
sendQueue = sendQueue :+ commitSig
d.spliceStatus
case _ if d.commitments.latest.fundingTxId == fundingTxId =>
@ -2401,7 +2534,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case fundingTx: InteractiveTxBuilder.PartiallySignedSharedTransaction =>
// If we have not received their tx_signatures, we can't tell whether they had received our commit_sig, so we need to retransmit it
log.info("re-sending commit_sig and tx_signatures for fundingTxIndex={} fundingTxId={}", d.commitments.latest.fundingTxIndex, d.commitments.latest.fundingTxId)
val commitSig = d.commitments.latest.remoteCommit.sign(keyManager, d.commitments.params, d.commitments.latest.fundingTxIndex, d.commitments.latest.remoteFundingPubKey, d.commitments.latest.commitInput)
val commitSig = d.commitments.latest.remoteCommit.localSig_opt.get
sendQueue = sendQueue :+ commitSig :+ fundingTx.localSigs
case fundingTx: InteractiveTxBuilder.FullySignedSharedTransaction =>
log.info("re-sending tx_signatures for fundingTxIndex={} fundingTxId={}", d.commitments.latest.fundingTxIndex, d.commitments.latest.fundingTxId)
@ -2532,6 +2665,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(c: CMD_UPDATE_RELAY_FEE, d: DATA_NORMAL) => handleUpdateRelayFeeDisconnected(c, d)
case Event(channelReestablish: ChannelReestablish, d: DATA_SHUTDOWN) =>
require(channelReestablish.nextLocalNonces.size == d.commitments.active.count(_.commitInput.isP2tr), "missing next local nonce")
setRemoteNextLocalNonces("received ChannelReestablish", channelReestablish.nextLocalNonces)
Syncing.checkSync(keyManager, d.commitments, channelReestablish) match {
case syncFailure: SyncResult.Failure =>
handleSyncFailure(channelReestablish, syncFailure, d)
@ -2542,7 +2677,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
goto(SHUTDOWN) using d.copy(commitments = commitments1) sending sendQueue
}
case Event(_: ChannelReestablish, d: DATA_NEGOTIATING) =>
case Event(channelReestablish: ChannelReestablish, d: DATA_NEGOTIATING) =>
// BOLT 2: A node if it has sent a previous shutdown MUST retransmit shutdown.
// negotiation restarts from the beginning, and is initialized by the channel initiator
// note: in any case we still need to keep all previously sent closing_signed, because they may publish one of them
@ -3180,9 +3315,14 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
private def initiateSplice(cmd: CMD_SPLICE, d: DATA_NORMAL): Either[ChannelException, SpliceInit] = {
val parentCommitment = d.commitments.latest.commitment
val targetFeerate = nodeParams.onChainFeeConf.getFundingFeerate(nodeParams.currentFeeratesForFundingClosing)
val sharedInput = if (d.commitments.latest.commitInput.isP2tr) {
Musig2Input(parentCommitment)
} else {
Multisig2of2Input(parentCommitment)
}
val fundingContribution = InteractiveTxFunder.computeSpliceContribution(
isInitiator = true,
sharedInput = Multisig2of2Input(parentCommitment),
sharedInput = sharedInput,
spliceInAmount = cmd.additionalLocalFunding,
spliceOut = cmd.spliceOutputs,
targetFeerate = targetFeerate)

View file

@ -18,9 +18,11 @@ package fr.acinq.eclair.channel.fsm
import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapter}
import com.softwaremill.quicklens.{ModifyPimp, QuicklensAt}
import fr.acinq.bitcoin.crypto.musig2.IndividualNonce
import fr.acinq.bitcoin.scalacompat.SatoshiLong
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
import fr.acinq.eclair.channel.Helpers.Funding
import fr.acinq.eclair.channel.ChannelTypes.SimpleTaprootChannelsStaging
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel._
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTransaction, InteractiveTxParams, PartiallySignedSharedTransaction, RequireConfirmedInputs}
@ -28,6 +30,7 @@ import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningS
import fr.acinq.eclair.channel.publish.TxPublisher.SetChannelId
import fr.acinq.eclair.crypto.ShaChain
import fr.acinq.eclair.io.Peer.{LiquidityPurchaseSigned, OpenChannelResponse}
import fr.acinq.eclair.transactions.Transactions.{SimpleTaprootChannelsStagingCommitmentFormat, SimpleTaprootChannelsStagingLegacyCommitmentFormat}
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{ToMilliSatoshiConversion, UInt64, randomBytes32}
@ -143,7 +146,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case Event(open: OpenDualFundedChannel, d: DATA_WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL) =>
import d.init.{localParams, remoteInit}
val localFundingPubkey = keyManager.fundingPublicKey(localParams.fundingKeyPath, fundingTxIndex = 0).publicKey
val fundingScript = Funding.makeFundingPubKeyScript(localFundingPubkey, open.fundingPubkey)
val fundingScript = Funding.makeFundingPubKeyScript(localFundingPubkey, open.fundingPubkey, d.init.channelType.commitmentFormat)
Helpers.validateParamsDualFundedNonInitiator(nodeParams, d.init.channelType, open, fundingScript, remoteNodeId, localParams.initFeatures, remoteInit.features, d.init.fundingContribution_opt) match {
case Left(t) => handleLocalError(t, d, Some(open))
case Right((channelFeatures, remoteShutdownScript, willFund_opt)) =>
@ -183,6 +186,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
open.useFeeCredit_opt.map(c => ChannelTlv.FeeCreditUsedTlv(c)),
d.init.pushAmount_opt.map(amount => ChannelTlv.PushAmountTlv(amount)),
).flatten
log.debug("sending AcceptDualFundedChannel with {}", tlvs)
val accept = AcceptDualFundedChannel(
temporaryChannelId = open.temporaryChannelId,
fundingAmount = localAmount,
@ -340,7 +344,8 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case Event(msg: InteractiveTxBuilder.Response, d: DATA_WAIT_FOR_DUAL_FUNDING_CREATED) => msg match {
case InteractiveTxBuilder.SendMessage(_, msg) => stay() sending msg
case InteractiveTxBuilder.Succeeded(status, commitSig, liquidityPurchase_opt) =>
case InteractiveTxBuilder.Succeeded(status, commitSig, liquidityPurchase_opt, nextRemoteNonce_opt) =>
this.pendingRemoteNextLocalNonce = nextRemoteNonce_opt
d.deferred.foreach(self ! _)
d.replyTo_opt.foreach(_ ! OpenChannelResponse.Created(d.channelId, status.fundingTx.txId, status.fundingTx.tx.localFees.truncateToSatoshi))
liquidityPurchase_opt.collect {
@ -702,7 +707,8 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case DualFundingStatus.RbfInProgress(cmd_opt, _, remoteCommitSig_opt) =>
msg match {
case InteractiveTxBuilder.SendMessage(_, msg) => stay() sending msg
case InteractiveTxBuilder.Succeeded(signingSession, commitSig, liquidityPurchase_opt) =>
case InteractiveTxBuilder.Succeeded(signingSession, commitSig, liquidityPurchase_opt, nextRemoteNonce_opt) =>
this.pendingRemoteNextLocalNonce = nextRemoteNonce_opt
cmd_opt.foreach(cmd => cmd.replyTo ! RES_BUMP_FUNDING_FEE(rbfIndex = d.previousFundingTxs.length, signingSession.fundingTx.txId, signingSession.fundingTx.tx.localFees.truncateToSatoshi))
remoteCommitSig_opt.foreach(self ! _)
liquidityPurchase_opt.collect {
@ -730,7 +736,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
// But since this is a zero-conf channel, the minimum depth isn't critical: we use the default one.
watchFundingConfirmed(w.tx.txid, Some(nodeParams.channelConf.minDepth), delay_opt = None)
val shortIds = createShortIdAliases(d.channelId)
val channelReady = createChannelReady(shortIds, d.commitments.params)
val channelReady = createChannelReady(shortIds, d.commitments)
d.deferred.foreach(self ! _)
goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments1, shortIds) storing() sending channelReady
case Left(_) => stay()
@ -740,7 +746,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
acceptFundingTxConfirmed(w, d) match {
case Right((commitments1, _)) =>
val shortIds = createShortIdAliases(d.channelId)
val channelReady = createChannelReady(shortIds, d.commitments.params)
val channelReady = createChannelReady(shortIds, d.commitments)
reportRbfFailure(d.status, InvalidRbfTxConfirmed(d.channelId))
val toSend = d.status match {
case DualFundingStatus.WaitingForConfirmations | DualFundingStatus.RbfAborted => Seq(channelReady)

View file

@ -19,11 +19,13 @@ package fr.acinq.eclair.channel.fsm
import akka.actor.Status
import akka.actor.typed.scaladsl.adapter.actorRefAdapter
import akka.pattern.pipe
import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Script}
import fr.acinq.bitcoin.ScriptFlags
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Musig2, OutPoint, SatoshiLong, Script, Transaction, TxId}
import fr.acinq.eclair.blockchain.OnChainWallet.MakeFundingTxResponse
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
import fr.acinq.eclair.channel.Helpers.Funding
import fr.acinq.eclair.channel.LocalFundingStatus.SingleFundedUnconfirmedFundingTx
import fr.acinq.eclair.channel.RemoteSignature.PartialSignatureWithNonce
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel._
import fr.acinq.eclair.channel.publish.TxPublisher.SetChannelId
@ -33,9 +35,11 @@ import fr.acinq.eclair.transactions.Transactions.TxOwner
import fr.acinq.eclair.transactions.{Scripts, Transactions}
import fr.acinq.eclair.wire.protocol.{AcceptChannel, AnnouncementSignatures, ChannelReady, ChannelTlv, Error, FundingCreated, FundingSigned, OpenChannel, TlvStream}
import fr.acinq.eclair.{Features, MilliSatoshiLong, UInt64, randomKey, toLongId}
import fr.acinq.eclair.wire.protocol.{AcceptChannel, AcceptChannelTlv, AnnouncementSignatures, ChannelReady, ChannelTlv, Error, FundingCreated, FundingSigned, OpenChannel, OpenChannelTlv, PartialSignatureWithNonceTlv, TlvStream}
import fr.acinq.eclair.{Features, MilliSatoshiLong, RealShortChannelId, UInt64, randomKey, toLongId}
import scodec.bits.ByteVector
import scala.util.{Failure, Success}
import scala.util.{Failure, Success, Try}
/**
* Created by t-bast on 28/03/2022.
@ -78,6 +82,20 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
// In order to allow TLV extensions and keep backwards-compatibility, we include an empty upfront_shutdown_script if this feature is not used
// See https://github.com/lightningnetwork/lightning-rfc/pull/714.
val localShutdownScript = input.localParams.upfrontShutdownScript_opt.getOrElse(ByteVector.empty)
val tlvStream: TlvStream[OpenChannelTlv] = if (input.channelType.commitmentFormat.useTaproot) {
// include our verification nonce at funding_index = 0, commit_index = 0
val (_, localNonce) = keyManager.verificationNonce(TxId(ByteVector32.One), fundingPubKey, 0)
TlvStream(
ChannelTlv.UpfrontShutdownScriptTlv(localShutdownScript),
ChannelTlv.ChannelTypeTlv(input.channelType),
ChannelTlv.NextLocalNonceTlv(localNonce)
)
} else {
TlvStream(
ChannelTlv.UpfrontShutdownScriptTlv(localShutdownScript),
ChannelTlv.ChannelTypeTlv(input.channelType)
)
}
val open = OpenChannel(
chainHash = nodeParams.chainHash,
temporaryChannelId = input.temporaryChannelId,
@ -97,10 +115,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
htlcBasepoint = keyManager.htlcPoint(channelKeyPath).publicKey,
firstPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 0),
channelFlags = input.channelFlags,
tlvStream = TlvStream(
ChannelTlv.UpfrontShutdownScriptTlv(localShutdownScript),
ChannelTlv.ChannelTypeTlv(input.channelType)
))
tlvStream = tlvStream)
goto(WAIT_FOR_ACCEPT_CHANNEL) using DATA_WAIT_FOR_ACCEPT_CHANNEL(input, open) sending open
})
@ -133,6 +148,20 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
// In order to allow TLV extensions and keep backwards-compatibility, we include an empty upfront_shutdown_script if this feature is not used.
// See https://github.com/lightningnetwork/lightning-rfc/pull/714.
val localShutdownScript = d.initFundee.localParams.upfrontShutdownScript_opt.getOrElse(ByteVector.empty)
val tlvStream: TlvStream[AcceptChannelTlv] = if (channelFeatures.hasFeature(Features.SimpleTaproot) || channelFeatures.hasFeature(Features.SimpleTaprootStaging)) {
// include our verification nonce at funding_index = 0, commit_index = 0
val localNonce = keyManager.verificationNonce(TxId(ByteVector32.One), fundingPubkey, 0)
TlvStream(
ChannelTlv.UpfrontShutdownScriptTlv(localShutdownScript),
ChannelTlv.ChannelTypeTlv(d.initFundee.channelType),
ChannelTlv.NextLocalNonceTlv(localNonce._2)
)
} else {
TlvStream(
ChannelTlv.UpfrontShutdownScriptTlv(localShutdownScript),
ChannelTlv.ChannelTypeTlv(d.initFundee.channelType)
)
}
val accept = AcceptChannel(temporaryChannelId = open.temporaryChannelId,
dustLimitSatoshis = d.initFundee.localParams.dustLimit,
maxHtlcValueInFlightMsat = UInt64(d.initFundee.localParams.maxHtlcValueInFlightMsat.toLong),
@ -147,11 +176,8 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
delayedPaymentBasepoint = keyManager.delayedPaymentPoint(channelKeyPath).publicKey,
htlcBasepoint = keyManager.htlcPoint(channelKeyPath).publicKey,
firstPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 0),
tlvStream = TlvStream(
ChannelTlv.UpfrontShutdownScriptTlv(localShutdownScript),
ChannelTlv.ChannelTypeTlv(d.initFundee.channelType)
))
goto(WAIT_FOR_FUNDING_CREATED) using DATA_WAIT_FOR_FUNDING_CREATED(params, open.fundingSatoshis, open.pushMsat, open.feeratePerKw, open.fundingPubkey, open.firstPerCommitmentPoint) sending accept
tlvStream = tlvStream)
goto(WAIT_FOR_FUNDING_CREATED) using DATA_WAIT_FOR_FUNDING_CREATED(params, open.fundingSatoshis, open.pushMsat, open.feeratePerKw, open.fundingPubkey, open.firstPerCommitmentPoint, open.nexLocalNonce_opt) sending accept
}
case Event(c: CloseCommand, d) => handleFastClose(c, d.channelId)
@ -162,7 +188,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
})
when(WAIT_FOR_ACCEPT_CHANNEL)(handleExceptions {
case Event(accept: AcceptChannel, d@DATA_WAIT_FOR_ACCEPT_CHANNEL(init, open)) =>
case Event(accept: AcceptChannel, d@DATA_WAIT_FOR_ACCEPT_CHANNEL(init, open, _)) =>
Helpers.validateParamsSingleFundedFunder(nodeParams, init.channelType, init.localParams.initFeatures, init.remoteInit.features, open, accept) match {
case Left(t) =>
d.initFunder.replyTo ! OpenChannelResponse.Rejected(t.getMessage)
@ -185,9 +211,14 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
log.debug("remote params: {}", remoteParams)
log.info("remote will use fundingMinDepth={}", accept.minimumDepth)
val localFundingPubkey = keyManager.fundingPublicKey(init.localParams.fundingKeyPath, fundingTxIndex = 0)
val fundingPubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingPubkey.publicKey, accept.fundingPubkey)))
val fundingPubkeyScript = if (channelFeatures.hasFeature(Features.SimpleTaproot) || (channelFeatures.hasFeature(Features.SimpleTaprootStaging))) {
Script.write(Scripts.Taproot.musig2FundingScript(localFundingPubkey.publicKey, accept.fundingPubkey))
} else {
Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingPubkey.publicKey, accept.fundingPubkey)))
}
wallet.makeFundingTx(fundingPubkeyScript, init.fundingAmount, init.fundingTxFeerate, init.fundingTxFeeBudget_opt).pipeTo(self)
val params = ChannelParams(init.temporaryChannelId, init.channelConfig, channelFeatures, init.localParams, remoteParams, open.channelFlags)
setRemoteNextLocalNonces("received AcceptChannel", accept.nexLocalNonce_opt.toList)
goto(WAIT_FOR_FUNDING_INTERNAL) using DATA_WAIT_FOR_FUNDING_INTERNAL(params, init.fundingAmount, init.pushAmount_opt.getOrElse(0 msat), init.commitTxFeerate, accept.fundingPubkey, accept.firstPerCommitmentPoint, d.initFunder.replyTo)
}
@ -216,21 +247,37 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
case Left(ex) => handleLocalError(ex, d, None)
case Right((localSpec, localCommitTx, remoteSpec, remoteCommitTx)) =>
require(fundingTx.txOut(fundingTxOutputIndex).publicKeyScript == localCommitTx.input.txOut.publicKeyScript, s"pubkey script mismatch!")
val localSigOfRemoteTx = keyManager.sign(remoteCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex = 0), TxOwner.Remote, params.commitmentFormat)
val fundingPubkey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex = 0)
// signature of their initial commitment tx that pays remote pushMsat
val fundingCreated = FundingCreated(
val fundingCreated = if (localCommitTx.input.isP2tr) {
val localNonce = keyManager.verificationNonce(TxId(ByteVector32.One), fundingPubkey.publicKey, 0)
val Right(sig) = keyManager.partialSign(remoteCommitTx,
fundingPubkey, remoteFundingPubKey, TxOwner.Remote,
localNonce, remoteNextLocalNonces.head
)
FundingCreated(
temporaryChannelId = temporaryChannelId,
fundingTxId = fundingTx.txid,
fundingOutputIndex = fundingTxOutputIndex,
signature = localSigOfRemoteTx
signature = ByteVector64.Zeroes,
tlvStream = TlvStream(PartialSignatureWithNonceTlv(PartialSignatureWithNonce(sig, localNonce._2)))
)
} else {
val localSigOfRemoteTx = keyManager.sign(remoteCommitTx, fundingPubkey, TxOwner.Remote, params.commitmentFormat)
FundingCreated(
temporaryChannelId = temporaryChannelId,
fundingTxId = fundingTx.txid,
fundingOutputIndex = fundingTxOutputIndex,
signature = localSigOfRemoteTx,
)
}
val channelId = toLongId(fundingTx.txid, fundingTxOutputIndex)
val params1 = params.copy(channelId = channelId)
peer ! ChannelIdAssigned(self, remoteNodeId, temporaryChannelId, channelId) // we notify the peer asap so it knows how to route messages
txPublisher ! SetChannelId(remoteNodeId, channelId)
context.system.eventStream.publish(ChannelIdAssigned(self, remoteNodeId, temporaryChannelId, channelId))
// NB: we don't send a ChannelSignatureSent for the first commit
goto(WAIT_FOR_FUNDING_SIGNED) using DATA_WAIT_FOR_FUNDING_SIGNED(params1, remoteFundingPubKey, fundingTx, fundingTxFee, localSpec, localCommitTx, RemoteCommit(0, remoteSpec, remoteCommitTx.tx.txid, remoteFirstPerCommitmentPoint), fundingCreated, replyTo) sending fundingCreated
goto(WAIT_FOR_FUNDING_SIGNED) using DATA_WAIT_FOR_FUNDING_SIGNED(params1, remoteFundingPubKey, fundingTx, fundingTxFee, localSpec, localCommitTx, RemoteCommit(0, remoteSpec, remoteCommitTx.tx.txid, remoteFirstPerCommitmentPoint, None), fundingCreated, replyTo) sending fundingCreated
}
case Event(Status.Failure(t), d: DATA_WAIT_FOR_FUNDING_INTERNAL) =>
@ -256,7 +303,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
})
when(WAIT_FOR_FUNDING_CREATED)(handleExceptions {
case Event(FundingCreated(_, fundingTxId, fundingTxOutputIndex, remoteSig, _), d@DATA_WAIT_FOR_FUNDING_CREATED(params, fundingAmount, pushMsat, commitTxFeerate, remoteFundingPubKey, remoteFirstPerCommitmentPoint)) =>
case Event(fc@FundingCreated(_, fundingTxId, fundingTxOutputIndex, remoteSig, _), d@DATA_WAIT_FOR_FUNDING_CREATED(params, fundingAmount, pushMsat, commitTxFeerate, remoteFundingPubKey, remoteFirstPerCommitmentPoint, remoteNextLocalNonce)) =>
val temporaryChannelId = params.channelId
// they fund the channel with their funding tx, so the money is theirs (but we are paid pushMsat)
Funding.makeFirstCommitTxs(keyManager, params, localFundingAmount = 0 sat, remoteFundingAmount = fundingAmount, localPushAmount = 0 msat, remotePushAmount = pushMsat, commitTxFeerate, fundingTxId, fundingTxOutputIndex, remoteFundingPubKey = remoteFundingPubKey, remoteFirstPerCommitmentPoint = remoteFirstPerCommitmentPoint) match {
@ -264,25 +311,52 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
case Right((localSpec, localCommitTx, remoteSpec, remoteCommitTx)) =>
// check remote signature validity
val fundingPubKey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex = 0)
val signedLocalCommitTx = if (localCommitTx.input.isP2tr) {
val localNonce = keyManager.verificationNonce(TxId(ByteVector32.One), fundingPubKey.publicKey, 0)
val Right(signedLocalCommitTx) = for {
localPartialSigOfLocalTx <- keyManager.partialSign(localCommitTx, fundingPubKey, remoteFundingPubKey, TxOwner.Local, localNonce, remoteNextLocalNonce.get)
remoteSigOfLocalTx <- fc.sigOrPartialSig
aggSig <- Musig2.aggregateTaprootSignatures(
Seq(localPartialSigOfLocalTx, remoteSigOfLocalTx.partialSig),
localCommitTx.tx, localCommitTx.tx.txIn.indexWhere(_.outPoint == localCommitTx.input.outPoint), Seq(localCommitTx.input.txOut),
Scripts.sort(Seq(fundingPubKey.publicKey, remoteFundingPubKey)),
Seq(localNonce._2, remoteNextLocalNonce.get),
None)
} yield localCommitTx.copy(tx = localCommitTx.tx.updateWitness(0, Script.witnessKeyPathPay2tr(aggSig)))
signedLocalCommitTx
} else {
val localSigOfLocalTx = keyManager.sign(localCommitTx, fundingPubKey, TxOwner.Local, params.commitmentFormat)
val signedLocalCommitTx = Transactions.addSigs(localCommitTx, fundingPubKey.publicKey, remoteFundingPubKey, localSigOfLocalTx, remoteSig)
Transactions.addSigs(localCommitTx, fundingPubKey.publicKey, remoteFundingPubKey, localSigOfLocalTx, remoteSig)
}
Transactions.checkSpendable(signedLocalCommitTx) match {
case Failure(_) => handleLocalError(InvalidCommitmentSignature(temporaryChannelId, fundingTxId, fundingTxIndex = 0, localCommitTx.tx), d, None)
case Success(_) =>
val localSigOfRemoteTx = keyManager.sign(remoteCommitTx, fundingPubKey, TxOwner.Remote, params.commitmentFormat)
val channelId = toLongId(fundingTxId, fundingTxOutputIndex)
val fundingSigned = FundingSigned(
val fundingSigned = if (signedLocalCommitTx.input.isP2tr) {
val localNonce = keyManager.verificationNonce(TxId(ByteVector32.One), fundingPubKey.publicKey, 0)
val Right(localPartialSigOfRemoteTx) = keyManager.partialSign(remoteCommitTx, fundingPubKey, remoteFundingPubKey, TxOwner.Remote, localNonce, remoteNextLocalNonce.get)
FundingSigned(
channelId = channelId,
signature = ByteVector64.Zeroes,
TlvStream(PartialSignatureWithNonceTlv(PartialSignatureWithNonce(localPartialSigOfRemoteTx, localNonce._2)))
)
} else {
val localSigOfRemoteTx = keyManager.sign(remoteCommitTx, fundingPubKey, TxOwner.Remote, params.commitmentFormat)
FundingSigned(
channelId = channelId,
signature = localSigOfRemoteTx
)
}
val commitment = Commitment(
fundingTxIndex = 0,
firstRemoteCommitIndex = 0,
remoteFundingPubKey = remoteFundingPubKey,
localFundingStatus = SingleFundedUnconfirmedFundingTx(None),
remoteFundingStatus = RemoteFundingStatus.NotLocked,
localCommit = LocalCommit(0, localSpec, CommitTxAndRemoteSig(localCommitTx, remoteSig), htlcTxsAndRemoteSigs = Nil),
remoteCommit = RemoteCommit(0, remoteSpec, remoteCommitTx.tx.txid, remoteFirstPerCommitmentPoint),
localCommit = LocalCommit(0, localSpec, CommitTxAndRemoteSig(localCommitTx, fc.sigOrPartialSig), htlcTxsAndRemoteSigs = Nil),
remoteCommit = RemoteCommit(0, remoteSpec, remoteCommitTx.tx.txid, remoteFirstPerCommitmentPoint, None),
nextRemoteCommit_opt = None)
val commitments = Commitments(
params = params.copy(channelId = channelId),
@ -310,12 +384,29 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
})
when(WAIT_FOR_FUNDING_SIGNED)(handleExceptions {
case Event(msg@FundingSigned(_, remoteSig, _), d@DATA_WAIT_FOR_FUNDING_SIGNED(params, remoteFundingPubKey, fundingTx, fundingTxFee, localSpec, localCommitTx, remoteCommit, fundingCreated, _)) =>
case Event(msg@FundingSigned(_, _, _), d@DATA_WAIT_FOR_FUNDING_SIGNED(params, remoteFundingPubKey, fundingTx, fundingTxFee, localSpec, localCommitTx, remoteCommit, fundingCreated, _)) =>
// we make sure that their sig checks out and that our first commit tx is spendable
val fundingPubKey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex = 0)
val signedLocalCommitTx = Try(if (localCommitTx.input.isP2tr) {
require(msg.sigOrPartialSig.isRight, "missing partial signature and nonce")
val localNonce = keyManager.verificationNonce(TxId(ByteVector32.One), fundingPubKey.publicKey, 0)
val Right(remotePartialSigWithNonce) = msg.sigOrPartialSig
val Right(signedCommitTx) = for {
partialSig <- keyManager.partialSign(localCommitTx, fundingPubKey, remoteFundingPubKey, TxOwner.Local, localNonce, remotePartialSigWithNonce.nonce)
aggSig <- Transactions.aggregatePartialSignatures(
localCommitTx,
partialSig, remotePartialSigWithNonce.partialSig,
fundingPubKey.publicKey, remoteFundingPubKey,
localNonce._2, remotePartialSigWithNonce.nonce)
signedTx = localCommitTx.tx.updateWitness(0, Script.witnessKeyPathPay2tr(aggSig))
} yield localCommitTx.copy(tx = signedTx)
signedCommitTx
} else {
val Left(remoteSig) = msg.sigOrPartialSig
val localSigOfLocalTx = keyManager.sign(localCommitTx, fundingPubKey, TxOwner.Local, params.commitmentFormat)
val signedLocalCommitTx = Transactions.addSigs(localCommitTx, fundingPubKey.publicKey, remoteFundingPubKey, localSigOfLocalTx, remoteSig)
Transactions.checkSpendable(signedLocalCommitTx) match {
Transactions.addSigs(localCommitTx, fundingPubKey.publicKey, remoteFundingPubKey, localSigOfLocalTx, remoteSig)
})
signedLocalCommitTx.flatMap(commitTx => Transactions.checkSpendable(commitTx)) match {
case Failure(cause) =>
// we rollback the funding tx, it will never be published
wallet.rollback(fundingTx)
@ -328,7 +419,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
remoteFundingPubKey = remoteFundingPubKey,
localFundingStatus = SingleFundedUnconfirmedFundingTx(Some(fundingTx)),
remoteFundingStatus = RemoteFundingStatus.NotLocked,
localCommit = LocalCommit(0, localSpec, CommitTxAndRemoteSig(localCommitTx, remoteSig), htlcTxsAndRemoteSigs = Nil),
localCommit = LocalCommit(0, localSpec, CommitTxAndRemoteSig(localCommitTx, msg.sigOrPartialSig), htlcTxsAndRemoteSigs = Nil),
remoteCommit = remoteCommit,
nextRemoteCommit_opt = None
)
@ -375,6 +466,9 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
when(WAIT_FOR_FUNDING_CONFIRMED)(handleExceptions {
case Event(remoteChannelReady: ChannelReady, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) =>
if (d.commitments.latest.commitInput.isP2tr) {
require(remoteChannelReady.nexLocalNonce_opt.isDefined, "missing next local nonce")
}
// We are here if:
// - we're using zero-conf, but our peer was very fast and we received their channel_ready before our watcher
// notification that the funding tx has been successfully published: in that case we don't put a duplicate watch
@ -399,7 +493,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
// But since this is a zero-conf channel, the minimum depth isn't critical: we use the default one.
watchFundingConfirmed(w.tx.txid, Some(nodeParams.channelConf.minDepth), delay_opt = None)
val shortIds = createShortIdAliases(d.channelId)
val channelReady = createChannelReady(shortIds, d.commitments.params)
val channelReady = createChannelReady(shortIds, d.commitments)
d.deferred.foreach(self ! _)
goto(WAIT_FOR_CHANNEL_READY) using DATA_WAIT_FOR_CHANNEL_READY(commitments1, shortIds) storing() sending channelReady
case Left(_) => stay()
@ -409,7 +503,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
acceptFundingTxConfirmed(w, d) match {
case Right((commitments1, _)) =>
val shortIds = createShortIdAliases(d.channelId)
val channelReady = createChannelReady(shortIds, d.commitments.params)
val channelReady = createChannelReady(shortIds, d.commitments)
d.deferred.foreach(self ! _)
goto(WAIT_FOR_CHANNEL_READY) using DATA_WAIT_FOR_CHANNEL_READY(commitments1, shortIds) storing() sending channelReady
case Left(_) => stay()

View file

@ -27,8 +27,8 @@ import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel.{BroadcastChannelUpdate, PeriodicRefresh, REFRESH_CHANNEL_UPDATE_INTERVAL}
import fr.acinq.eclair.db.RevokedHtlcInfoCleaner
import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.wire.protocol.{AnnouncementSignatures, ChannelReady, ChannelReadyTlv, TlvStream}
import fr.acinq.eclair.{RealShortChannelId, ShortChannelId}
import fr.acinq.eclair.wire.protocol._
import scala.concurrent.duration.{DurationInt, FiniteDuration}
import scala.util.{Failure, Success, Try}
@ -123,11 +123,20 @@ trait CommonFundingHandlers extends CommonHandlers {
aliases
}
def createChannelReady(aliases: ShortIdAliases, params: ChannelParams): ChannelReady = {
def createChannelReady(aliases: ShortIdAliases, commitments: Commitments): ChannelReady = {
val params = commitments.params
val fundingTxId = commitments.latest.fundingTxId
val channelKeyPath = keyManager.keyPath(params.localParams, params.channelConfig)
val nextPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1)
val tlvStream: TlvStream[ChannelReadyTlv] = if (commitments.latest.commitInput.isP2tr) {
val localFundingPubkey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex = 0).publicKey
val (_, nextLocalNonce) = keyManager.verificationNonce(fundingTxId, localFundingPubkey, 1)
TlvStream(ChannelReadyTlv.ShortChannelIdTlv(aliases.localAlias), ChannelTlv.NextLocalNonceTlv(nextLocalNonce))
} else {
TlvStream(ChannelReadyTlv.ShortChannelIdTlv(aliases.localAlias))
}
// we always send our local alias, even if it isn't explicitly supported, that's an optional TLV anyway
ChannelReady(params.channelId, nextPerCommitmentPoint, TlvStream(ChannelReadyTlv.ShortChannelIdTlv(aliases.localAlias)))
ChannelReady(params.channelId, nextPerCommitmentPoint, tlvStream)
}
def receiveChannelReady(aliases: ShortIdAliases, channelReady: ChannelReady, commitments: Commitments): DATA_NORMAL = {
@ -151,6 +160,7 @@ trait CommonFundingHandlers extends CommonHandlers {
},
remoteNextCommitInfo = Right(channelReady.nextPerCommitmentPoint)
)
setRemoteNextLocalNonces("received ChannelReady", channelReady.nexLocalNonce_opt.toList) // TODO: this is wrong, there should be a different nonce for each commitment
peer ! ChannelReadyForPayments(self, remoteNodeId, commitments.channelId, fundingTxIndex = 0)
DATA_NORMAL(commitments1, aliases1, None, initialChannelUpdate, None, None, None, SpliceStatus.NoSplice)
}

View file

@ -17,6 +17,7 @@
package fr.acinq.eclair.channel.fsm
import akka.actor.FSM
import fr.acinq.bitcoin.crypto.musig2.{IndividualNonce, SecretNonce}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Script}
import fr.acinq.eclair.Features
import fr.acinq.eclair.channel.Helpers.Closing.MutualClose
@ -132,11 +133,11 @@ trait CommonHandlers {
finalScriptPubKey
}
def startSimpleClose(commitments: Commitments, localShutdown: Shutdown, remoteShutdown: Shutdown, closingFeerates: Option[ClosingFeerates]): (DATA_NEGOTIATING_SIMPLE, Option[ClosingComplete]) = {
def startSimpleClose(commitments: Commitments, localShutdown: Shutdown, remoteShutdown: Shutdown, closingFeerates: Option[ClosingFeerates], localNonce_opt: Option[(SecretNonce, IndividualNonce)] = None): (DATA_NEGOTIATING_SIMPLE, Option[ClosingComplete]) = {
val localScript = localShutdown.scriptPubKey
val remoteScript = remoteShutdown.scriptPubKey
val closingFeerate = closingFeerates.map(_.preferred).getOrElse(nodeParams.onChainFeeConf.getClosingFeerate(nodeParams.currentBitcoinCoreFeerates))
MutualClose.makeSimpleClosingTx(nodeParams.currentBlockHeight, keyManager, commitments.latest, localScript, remoteScript, closingFeerate) match {
MutualClose.makeSimpleClosingTx(nodeParams.currentBlockHeight, keyManager, commitments.latest, localScript, remoteScript, closingFeerate, localNonce_opt, remoteShutdown.shutdownNonce_opt) match {
case Left(f) =>
log.warning("cannot create local closing txs, waiting for remote closing_complete: {}", f.getMessage)
val d = DATA_NEGOTIATING_SIMPLE(commitments, closingFeerate, localScript, remoteScript, Nil, Nil)

View file

@ -22,13 +22,15 @@ import akka.actor.typed.scaladsl.{ActorContext, Behaviors, StashBuffer}
import akka.actor.typed.{ActorRef, Behavior}
import akka.event.LoggingAdapter
import fr.acinq.bitcoin.ScriptFlags
import fr.acinq.bitcoin.crypto.musig2.{IndividualNonce, SecretNonce}
import fr.acinq.bitcoin.psbt.Psbt
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, LexicographicalOrdering, OutPoint, Satoshi, SatoshiLong, Script, ScriptWitness, Transaction, TxId, TxIn, TxOut}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, LexicographicalOrdering, Musig2, OutPoint, Satoshi, SatoshiLong, Script, ScriptWitness, Transaction, TxId, TxIn, TxOut}
import fr.acinq.eclair.blockchain.OnChainChannelFunder
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel.Helpers.Closing.MutualClose
import fr.acinq.eclair.channel.Helpers.Funding
import fr.acinq.eclair.channel.RemoteSignature.PartialSignatureWithNonce
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.Output.Local
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.Purpose
@ -92,7 +94,7 @@ object InteractiveTxBuilder {
sealed trait Response
case class SendMessage(sessionId: ByteVector32, msg: LightningMessage) extends Response
case class Succeeded(signingSession: InteractiveTxSigningSession.WaitingForSigs, commitSig: CommitSig, liquidityPurchase_opt: Option[LiquidityAds.Purchase]) extends Response
case class Succeeded(signingSession: InteractiveTxSigningSession.WaitingForSigs, commitSig: CommitSig, liquidityPurchase_opt: Option[LiquidityAds.Purchase], nextRemoteNonce_opt: Option[IndividualNonce]) extends Response
sealed trait Failed extends Response { def cause: ChannelException }
case class LocalFailure(cause: ChannelException) extends Failed
case class RemoteFailure(cause: ChannelException) extends Failed
@ -126,6 +128,21 @@ object InteractiveTxBuilder {
)
}
case class Musig2Input(info: InputInfo, fundingTxIndex: Long, remoteFundingPubkey: PublicKey, commitIndex: Long) extends SharedFundingInput {
override val weight: Int = 234
override def sign(keyManager: ChannelKeyManager, params: ChannelParams, tx: Transaction): ByteVector64 = ByteVector64.Zeroes
}
object Musig2Input {
def apply(commitment: Commitment): Musig2Input = Musig2Input(
info = commitment.commitInput,
fundingTxIndex = commitment.fundingTxIndex,
remoteFundingPubkey = commitment.remoteFundingPubKey,
commitIndex = commitment.localCommit.index
)
}
/**
* @param channelId id of the channel.
* @param isInitiator true if we initiated the protocol, in which case we will pay fees for the shared parts of the transaction.
@ -317,11 +334,12 @@ object InteractiveTxBuilder {
remoteInputs: Seq[IncomingInput] = Nil,
localOutputs: Seq[OutgoingOutput] = Nil,
remoteOutputs: Seq[IncomingOutput] = Nil,
txCompleteSent: Boolean = false,
txCompleteReceived: Boolean = false,
txCompleteSent: Option[TxComplete] = None,
txCompleteReceived: Option[TxComplete] = None,
inputsReceivedCount: Int = 0,
outputsReceivedCount: Int = 0) {
val isComplete: Boolean = txCompleteSent && txCompleteReceived
outputsReceivedCount: Int = 0,
secretNonces: Map[UInt64, (SecretNonce, IndividualNonce)] = Map.empty) {
val isComplete: Boolean = txCompleteSent.isDefined && txCompleteReceived.isDefined
}
/** Unsigned transaction created collaboratively. */
@ -340,6 +358,9 @@ object InteractiveTxBuilder {
def localOnlyNonChangeOutputs: List[Output.Local.NonChange] = localOutputs.collect { case o: Local.NonChange => o }
// outputs spent by this tx
val spentOutputs: Seq[TxOut] = (sharedInput_opt.toSeq ++ localInputs ++ remoteInputs).sortBy(_.serialId).map(_.txOut)
def buildUnsignedTx(): Transaction = {
val sharedTxIn = sharedInput_opt.map(i => (i.serialId, TxIn(i.outPoint, ByteVector.empty, i.sequence))).toSeq
val localTxIn = localInputs.map(i => (i.serialId, TxIn(i.outPoint, ByteVector.empty, i.sequence)))
@ -456,15 +477,27 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
private val log = context.log
private val keyManager = nodeParams.channelKeyManager
private val localFundingPubKey: PublicKey = keyManager.fundingPublicKey(channelParams.localParams.fundingKeyPath, purpose.fundingTxIndex).publicKey
private val fundingPubkeyScript: ByteVector = Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingPubKey, fundingParams.remoteFundingPubKey)))
private val fundingPubkeyScript: ByteVector = if (channelParams.commitmentFormat.useTaproot) {
Script.write(Scripts.Taproot.musig2FundingScript(localFundingPubKey, fundingParams.remoteFundingPubKey))
} else {
Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingPubKey, fundingParams.remoteFundingPubKey)))
}
private val remoteNodeId = channelParams.remoteParams.nodeId
private val previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction] = purpose match {
case rbf: FundingTxRbf => rbf.previousTransactions
case rbf: SpliceTxRbf => rbf.previousTransactions
case _ => Nil
}
private val localNonce = fundingParams.sharedInput_opt.collect {
case s: Musig2Input => {
val localFundingPubKey1 = keyManager.fundingPublicKey(channelParams.localParams.fundingKeyPath, s.fundingTxIndex).publicKey
keyManager.signingNonce(localFundingPubKey1)
}
}
log.debug("creating local nonce {} for fundingTxIndex {}", localNonce, purpose.fundingTxIndex)
def start(): Behavior[Command] = {
log.info(s"starting funder with $fundingPubkeyScript")
val txFunder = context.spawnAnonymous(InteractiveTxFunder(remoteNodeId, fundingParams, fundingPubkeyScript, purpose, wallet))
txFunder ! InteractiveTxFunder.FundTransaction(context.messageAdapter[InteractiveTxFunder.Response](r => FundTransactionResult(r)))
Behaviors.receiveMessagePartial {
@ -516,17 +549,36 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
TxAddInput(fundingParams.channelId, i.serialId, Some(i.previousTx), i.previousTxOutput, i.sequence)
case i: Input.Shared => TxAddInput(fundingParams.channelId, i.serialId, i.outPoint, i.sequence)
}
val nextSecretNonces = addInput match {
case i: Input.Shared if localNonce.isDefined =>
session.secretNonces + (i.serialId -> localNonce.get)
case _ => session.secretNonces
}
replyTo ! SendMessage(sessionId, message)
val next = session.copy(toSend = tail, localInputs = session.localInputs :+ addInput, txCompleteSent = false)
val next = session.copy(toSend = tail, localInputs = session.localInputs :+ addInput, txCompleteSent = None, secretNonces = nextSecretNonces)
receive(next)
case (addOutput: Output) +: tail =>
val message = TxAddOutput(fundingParams.channelId, addOutput.serialId, addOutput.amount, addOutput.pubkeyScript)
replyTo ! SendMessage(sessionId, message)
val next = session.copy(toSend = tail, localOutputs = session.localOutputs :+ addOutput, txCompleteSent = false)
val next = session.copy(toSend = tail, localOutputs = session.localOutputs :+ addOutput, txCompleteSent = None)
receive(next)
case Nil =>
replyTo ! SendMessage(sessionId, TxComplete(fundingParams.channelId))
val next = session.copy(txCompleteSent = true)
val fundingNonces = (session.remoteInputs ++ session.localInputs).sortBy(_.serialId).collect {
case i: Input.Shared if this.channelParams.commitmentFormat.useTaproot => session.secretNonces.get(i.serialId).map(_._2).getOrElse(throw new RuntimeException("missing secret nonce"))
}
val commitNonces = if (this.channelParams.commitmentFormat.useTaproot) {
validateTx(session).map(_.buildUnsignedTx().txid) match {
case Right(fundingTxId) =>
List(
keyManager.verificationNonce(fundingTxId, this.localFundingPubKey, this.purpose.localCommitIndex)._2,
keyManager.verificationNonce(fundingTxId, this.localFundingPubKey, this.purpose.localCommitIndex + 1)._2,
)
case _ => Nil
}
} else Nil
val txComplete = TxComplete(fundingParams.channelId, fundingNonces.toList, commitNonces)
replyTo ! SendMessage(sessionId, txComplete)
val next = session.copy(txCompleteSent = Some(txComplete))
if (next.isComplete) {
validateAndSign(next)
} else {
@ -535,7 +587,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
}
}
private def receiveInput(session: InteractiveTxSession, addInput: TxAddInput): Either[ChannelException, IncomingInput] = {
private def receiveInput(session: InteractiveTxSession, addInput: TxAddInput): Either[ChannelException, InteractiveTxSession] = {
if (session.inputsReceivedCount + 1 >= MAX_INPUTS_OUTPUTS_RECEIVED) {
return Left(TooManyInteractiveTxRounds(fundingParams.channelId))
}
@ -566,7 +618,17 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
if (input.sequence > 0xfffffffdL) {
return Left(NonReplaceableInput(fundingParams.channelId, addInput.serialId, input.outPoint.txid, input.outPoint.index, addInput.sequence))
}
Right(input)
val session1 = session.copy(
remoteInputs = session.remoteInputs :+ input,
inputsReceivedCount = session.inputsReceivedCount + 1,
txCompleteReceived = None,
)
val session2 = input match {
case i: Input.Shared if this.localNonce.isDefined =>
session1.copy(secretNonces = session1.secretNonces + (i.serialId -> localNonce.get))
case _ => session1
}
Right(session2)
}
private def receiveOutput(session: InteractiveTxSession, addOutput: TxAddOutput): Either[ChannelException, IncomingOutput] = {
@ -598,12 +660,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
case Left(f) =>
replyTo ! RemoteFailure(f)
unlockAndStop(session)
case Right(input) =>
val next = session.copy(
remoteInputs = session.remoteInputs :+ input,
inputsReceivedCount = session.inputsReceivedCount + 1,
txCompleteReceived = false,
)
case Right(next) =>
send(next)
}
case addOutput: TxAddOutput =>
@ -615,7 +672,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
val next = session.copy(
remoteOutputs = session.remoteOutputs :+ output,
outputsReceivedCount = session.outputsReceivedCount + 1,
txCompleteReceived = false,
txCompleteReceived = None,
)
send(next)
}
@ -624,7 +681,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
case Some(_) =>
val next = session.copy(
remoteInputs = session.remoteInputs.filterNot(_.serialId == removeInput.serialId),
txCompleteReceived = false,
txCompleteReceived = None,
)
send(next)
case None =>
@ -636,15 +693,15 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
case Some(_) =>
val next = session.copy(
remoteOutputs = session.remoteOutputs.filterNot(_.serialId == removeOutput.serialId),
txCompleteReceived = false,
txCompleteReceived = None,
)
send(next)
case None =>
replyTo ! RemoteFailure(UnknownSerialId(fundingParams.channelId, removeOutput.serialId))
unlockAndStop(session)
}
case _: TxComplete =>
val next = session.copy(txCompleteReceived = true)
case txComplete: TxComplete =>
val next = session.copy(txCompleteReceived = Some(txComplete))
if (next.isComplete) {
validateAndSign(next)
} else {
@ -674,7 +731,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
replyTo ! RemoteFailure(cause)
unlockAndStop(session)
case Right(completeTx) =>
signCommitTx(completeTx)
signCommitTx(session, completeTx)
}
case _: WalletFailure =>
replyTo ! RemoteFailure(UnconfirmedInteractiveTxInputs(fundingParams.channelId))
@ -827,7 +884,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
Right(sharedTx)
}
private def signCommitTx(completeTx: SharedTransaction): Behavior[Command] = {
private def signCommitTx(session: InteractiveTxSession, completeTx: SharedTransaction): Behavior[Command] = {
val fundingTx = completeTx.buildUnsignedTx()
val fundingOutputIndex = fundingTx.txOut.indexWhere(_.publicKeyScript == fundingPubkeyScript)
val liquidityFee = fundingParams.liquidityFees(liquidityPurchase_opt)
@ -847,18 +904,33 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
case Right((localSpec, localCommitTx, remoteSpec, remoteCommitTx, sortedHtlcTxs)) =>
require(fundingTx.txOut(fundingOutputIndex).publicKeyScript == localCommitTx.input.txOut.publicKeyScript, "pubkey script mismatch!")
val fundingPubKey = keyManager.fundingPublicKey(channelParams.localParams.fundingKeyPath, purpose.fundingTxIndex)
val localSigOfRemoteTx = keyManager.sign(remoteCommitTx, fundingPubKey, TxOwner.Remote, channelParams.channelFeatures.commitmentFormat)
val localSigOfRemoteTx = if (localCommitTx.input.isP2tr) {
ByteVector64.Zeroes
} else {
keyManager.sign(remoteCommitTx, fundingPubKey, TxOwner.Remote, channelParams.channelFeatures.commitmentFormat)
}
val tlvStream: TlvStream[CommitSigTlv] = if (remoteCommitTx.input.isP2tr) {
val localFundingPubKey = keyManager.fundingPublicKey(channelParams.localParams.fundingKeyPath, purpose.fundingTxIndex).publicKey
val localNonce = keyManager.signingNonce(localFundingPubKey)
val Some(remoteNonce) = session.txCompleteReceived.flatMap(_.commitNonces.headOption)
val Right(psig) = keyManager.partialSign(remoteCommitTx, fundingPubKey, fundingParams.remoteFundingPubKey, TxOwner.Remote, localNonce, remoteNonce)
log.debug(s"signCommitTx: creating partial signature $psig for commit tx ${remoteCommitTx.tx.txid} with local nonce ${localNonce._2} remote nonce $remoteNonce")
TlvStream(CommitSigTlv.PartialSignatureWithNonceTlv(PartialSignatureWithNonce(psig, localNonce._2)))
} else {
TlvStream.empty
}
val localPerCommitmentPoint = keyManager.htlcPoint(keyManager.keyPath(channelParams.localParams, channelParams.channelConfig))
val htlcSignatures = sortedHtlcTxs.map(keyManager.sign(_, localPerCommitmentPoint, purpose.remotePerCommitmentPoint, TxOwner.Remote, channelParams.commitmentFormat)).toList
val localCommitSig = CommitSig(fundingParams.channelId, localSigOfRemoteTx, htlcSignatures)
val localCommitSig = CommitSig(fundingParams.channelId, localSigOfRemoteTx, htlcSignatures, tlvStream)
val localCommit = UnsignedLocalCommit(purpose.localCommitIndex, localSpec, localCommitTx, htlcTxs = Nil)
val remoteCommit = RemoteCommit(purpose.remoteCommitIndex, remoteSpec, remoteCommitTx.tx.txid, purpose.remotePerCommitmentPoint)
signFundingTx(completeTx, localCommitSig, localCommit, remoteCommit)
log.debug(s"signCommitTx: setting remotePerCommitmentPoint to ${purpose.remotePerCommitmentPoint}")
val remoteCommit = RemoteCommit(purpose.remoteCommitIndex, remoteSpec, remoteCommitTx.tx.txid, purpose.remotePerCommitmentPoint, Some(localCommitSig))
signFundingTx(session, completeTx, localCommitSig, localCommit, remoteCommit)
}
}
private def signFundingTx(completeTx: SharedTransaction, commitSig: CommitSig, localCommit: UnsignedLocalCommit, remoteCommit: RemoteCommit): Behavior[Command] = {
signTx(completeTx)
private def signFundingTx(session: InteractiveTxSession, completeTx: SharedTransaction, commitSig: CommitSig, localCommit: UnsignedLocalCommit, remoteCommit: RemoteCommit): Behavior[Command] = {
signFundingTx(session, completeTx)
Behaviors.receiveMessagePartial {
case SignTransactionResult(signedTx) =>
log.info(s"interactive-tx txid=${signedTx.txId} partially signed with {} local inputs, {} remote inputs, {} local outputs and {} remote outputs", signedTx.tx.localInputs.length, signedTx.tx.remoteInputs.length, signedTx.tx.localOutputs.length, signedTx.tx.remoteOutputs.length)
@ -893,7 +965,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
remoteCommit,
liquidityPurchase_opt.map(_.basicInfo(isBuyer = fundingParams.isInitiator))
)
replyTo ! Succeeded(signingSession, commitSig, liquidityPurchase_opt)
replyTo ! Succeeded(signingSession, commitSig, liquidityPurchase_opt, session.txCompleteReceived.flatMap(_.commitNonces.lastOption))
Behaviors.stopped
case WalletFailure(t) =>
log.error("could not sign funding transaction: ", t)
@ -908,13 +980,32 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
}
}
private def signTx(unsignedTx: SharedTransaction): Unit = {
private def signFundingTx(session: InteractiveTxSession, unsignedTx: SharedTransaction): Unit = {
import fr.acinq.bitcoin.scalacompat.KotlinUtils._
val tx = unsignedTx.buildUnsignedTx()
val sharedSig_opt = fundingParams.sharedInput_opt.map(_.sign(keyManager, channelParams, tx))
val sharedPartialSig_opt = fundingParams.sharedInput_opt.collect {
case m: Musig2Input =>
val sharedInputs = (session.localInputs ++ session.remoteInputs).collect { case i: Input.Shared => i }
// there should be a single shared input
val serialId = sharedInputs.head.serialId
val localNonce = session.secretNonces(serialId)
val fundingKey = keyManager.fundingPublicKey(this.channelParams.localParams.fundingKeyPath, m.fundingTxIndex)
val inputIndex = tx.txIn.indexWhere(_.outPoint == m.info.outPoint)
// there should be one remote nonce for each shared input ordered by serial id
val remoteNonces = sharedInputs.sortBy(_.serialId).zip(session.txCompleteReceived.get.fundingNonces).map { case (i, n) => i.serialId -> n }.toMap
val remoteNonce = remoteNonces(serialId)
val Right(psig) = keyManager.partialSign(tx, inputIndex, unsignedTx.spentOutputs, fundingKey, m.remoteFundingPubkey, TxOwner.Local, localNonce, remoteNonce)
log.debug(s"signFundingTx: creating partial sig $psig for ${tx.txid} inputIndex=$inputIndex")
log.debug(s"fundingKey = ${fundingKey.publicKey} fundingTxIndex = ${m.fundingTxIndex}")
log.debug(s"remoteFundingPubkey = ${m.remoteFundingPubkey}")
log.debug(s"local nonce = ${localNonce._2} fundingTxIndex = ${m.fundingTxIndex} commitIndex = ${m.commitIndex}")
log.debug(s"remote nonce = ${remoteNonce}")
PartialSignatureWithNonce(psig, localNonce._2)
}
if (unsignedTx.localInputs.isEmpty) {
context.self ! SignTransactionResult(PartiallySignedSharedTransaction(unsignedTx, TxSignatures(fundingParams.channelId, tx, Nil, sharedSig_opt)))
context.self ! SignTransactionResult(PartiallySignedSharedTransaction(unsignedTx, TxSignatures(fundingParams.channelId, tx, Nil, sharedSig_opt, sharedPartialSig_opt)))
} else {
val ourWalletInputs = unsignedTx.localInputs.map(i => tx.txIn.indexWhere(_.outPoint == i.outPoint))
val ourWalletOutputs = unsignedTx.localOutputs.flatMap {
@ -942,7 +1033,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
}.sum
require(actualLocalAmountOut == expectedLocalAmountOut, s"local output amount $actualLocalAmountOut does not match what we expect ($expectedLocalAmountOut): bitcoin core may be malicious")
val sigs = partiallySignedTx.txIn.filter(txIn => localOutpoints.contains(txIn.outPoint)).map(_.witness)
PartiallySignedSharedTransaction(unsignedTx, TxSignatures(fundingParams.channelId, partiallySignedTx, sigs, sharedSig_opt))
PartiallySignedSharedTransaction(unsignedTx, TxSignatures(fundingParams.channelId, partiallySignedTx, sigs, sharedSig_opt, sharedPartialSig_opt))
}) {
case Failure(t) => WalletFailure(t)
case Success(signedTx) => SignTransactionResult(signedTx)
@ -1041,6 +1132,31 @@ object InteractiveTxSigningSession {
log.info("invalid tx_signatures: missing shared input signatures")
return Left(InvalidFundingSignature(fundingParams.channelId, Some(partiallySignedTx.txId)))
}
case Some(sharedInput: Musig2Input) =>
(partiallySignedTx.localSigs.previousFundingTxPartialSig_opt, remoteSigs.previousFundingTxPartialSig_opt) match {
case (Some(localPartialSig), Some(remotePartialSig)) =>
val localFundingPubkey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, sharedInput.fundingTxIndex).publicKey
val unsignedTx = partiallySignedTx.tx.buildUnsignedTx()
log.debug(s"adding remote sigs for ${unsignedTx.txid}")
log.debug("local partial sig is using nonce {}", localPartialSig.nonce)
log.debug("remote partial sig is using nonce {}", remotePartialSig.nonce)
log.debug(s"local funding key = ${localFundingPubkey}")
log.debug(s"remote funding key = ${sharedInput.remoteFundingPubkey}")
log.debug(s"spent outputs = ${partiallySignedTx.tx.spentOutputs}")
val inputIndex = unsignedTx.txIn.indexWhere(_.outPoint == sharedInput.info.outPoint)
val Right(aggSig) = Musig2.aggregateTaprootSignatures(
Seq(localPartialSig.partialSig, remotePartialSig.partialSig),
unsignedTx,
inputIndex,
partiallySignedTx.tx.spentOutputs,
Scripts.sort(Seq(localFundingPubkey, sharedInput.remoteFundingPubkey)),
Seq(localPartialSig.nonce, remotePartialSig.nonce),
None)
Some(Script.witnessKeyPathPay2tr(aggSig))
case _ =>
log.info("invalid tx_signatures: missing shared input partial signatures")
return Left(InvalidFundingSignature(fundingParams.channelId, Some(partiallySignedTx.txId)))
}
case None => None
}
val txWithSigs = FullySignedSharedTransaction(partiallySignedTx.tx, partiallySignedTx.localSigs, remoteSigs, sharedSigs_opt)

View file

@ -16,9 +16,10 @@
package fr.acinq.eclair.crypto.keymanager
import fr.acinq.bitcoin.crypto.musig2.{IndividualNonce, SecretNonce}
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.scalacompat.DeterministicWallet.ExtendedPublicKey
import fr.acinq.bitcoin.scalacompat.{ByteVector64, Crypto, DeterministicWallet, Protocol}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Crypto, DeterministicWallet, Protocol, Transaction, TxId, TxOut}
import fr.acinq.eclair.channel.{ChannelConfig, LocalParams}
import fr.acinq.eclair.transactions.Transactions.{CommitmentFormat, TransactionWithInputInfo, TxOwner}
import scodec.bits.ByteVector
@ -41,6 +42,27 @@ trait ChannelKeyManager {
def commitmentPoint(channelKeyPath: DeterministicWallet.KeyPath, index: Long): Crypto.PublicKey
/**
* Create a deterministic verification nonce for a specific funding public key and commit tx index. The public nonce will be sent to our peer to create a partial signature
* of our commit tx, the private nonce is never shared (and never serialized or stored) and is used to create our local partial signature to be combined with our peer's.
*
* @param fundingTxId funding transaction id
* @param fundingPubKey funding public key
* @param commitIndex commit tx index
* @return a verification nonce that is used to create a partial musig2 signature for our commit tx.
*/
def verificationNonce(fundingTxId: TxId, fundingPubKey: PublicKey, commitIndex: Long): (SecretNonce, IndividualNonce)
/**
* Create a new, randomized singing nonce for a specific funding public key. These nonces are used to create a partial musig2 signature for our peer's commit tx and are sent
* alongside the partial signature. They are created on the fly, and never stored.
*
* @param fundingPubKey funding public key
* @return a signing nonce that can be used to create a musig2 signature with the funding private key that matches the provided key.
* Each call to this methode will return a different, randomized signing nonce.
*/
def signingNonce(fundingPubKey: PublicKey): (SecretNonce, IndividualNonce)
def keyPath(localParams: LocalParams, channelConfig: ChannelConfig): DeterministicWallet.KeyPath = {
if (channelConfig.hasOption(ChannelConfig.FundingPubKeyBasedChannelKeyPath)) {
// deterministic mode: use the funding pubkey to compute the channel key path
@ -68,6 +90,12 @@ trait ChannelKeyManager {
*/
def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey, txOwner: TxOwner, commitmentFormat: CommitmentFormat): ByteVector64
def partialSign(tx: TransactionWithInputInfo, localPublicKey: ExtendedPublicKey, remotePublicKey: PublicKey, txOwner: TxOwner, localNonce: (SecretNonce, IndividualNonce), remoteNextLocalNonce: IndividualNonce): Either[Throwable, ByteVector32] = {
partialSign(tx.tx, tx.tx.txIn.indexWhere(_.outPoint == tx.input.outPoint), Seq(tx.input.txOut), localPublicKey, remotePublicKey, txOwner, localNonce, remoteNextLocalNonce)
}
def partialSign(tx: Transaction, inputIndex: Int, spentOutputs: Seq[TxOut], localPublicKey: ExtendedPublicKey, remotePublicKey: PublicKey, txOwner: TxOwner, localNonce: (SecretNonce, IndividualNonce), remoteNextLocalNonce: IndividualNonce): Either[Throwable, ByteVector32]
/**
* This method is used to spend funds sent to htlc keys/delayed keys
*

View file

@ -17,14 +17,16 @@
package fr.acinq.eclair.crypto.keymanager
import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
import fr.acinq.bitcoin.crypto.musig2.{IndividualNonce, KeyAggCache, SecretNonce}
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.scalacompat.DeterministicWallet._
import fr.acinq.bitcoin.scalacompat.{Block, BlockHash, ByteVector32, ByteVector64, Crypto, DeterministicWallet}
import fr.acinq.bitcoin.scalacompat.{Block, BlockHash, ByteVector32, ByteVector64, Crypto, DeterministicWallet, Musig2, Transaction, TxId, TxOut}
import fr.acinq.eclair.crypto.Generators
import fr.acinq.eclair.crypto.Monitoring.{Metrics, Tags}
import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.transactions.Transactions
import fr.acinq.eclair.transactions.Transactions.{CommitmentFormat, TransactionWithInputInfo, TxOwner}
import fr.acinq.eclair.{KamonExt, randomLong}
import fr.acinq.eclair.{KamonExt, randomBytes32, randomLong}
import grizzled.slf4j.Logging
import kamon.tag.TagSet
import scodec.bits.ByteVector
@ -95,10 +97,32 @@ class LocalChannelKeyManager(seed: ByteVector, chainHash: BlockHash) extends Cha
private def shaSeed(channelKeyPath: DeterministicWallet.KeyPath): ByteVector32 = Crypto.sha256(privateKeys.get(internalKeyPath(channelKeyPath, hardened(5))).privateKey.value :+ 1.toByte)
private def nonceSeed(channelKeyPath: DeterministicWallet.KeyPath): ByteVector32 = Crypto.sha256(shaSeed(channelKeyPath))
override def commitmentSecret(channelKeyPath: DeterministicWallet.KeyPath, index: Long): PrivateKey = Generators.perCommitSecret(shaSeed(channelKeyPath), index)
override def commitmentPoint(channelKeyPath: DeterministicWallet.KeyPath, index: Long): PublicKey = Generators.perCommitPoint(shaSeed(channelKeyPath), index)
private def generateNonce(sessionId: ByteVector32, publicKey: PublicKey, extraInput: Option[ByteVector32] = None): (SecretNonce, IndividualNonce) = {
import fr.acinq.bitcoin.scalacompat.KotlinUtils._
val keyAggCache = KeyAggCache.create(java.util.List.of(publicKey)).getSecond
val nonces = fr.acinq.bitcoin.crypto.musig2.SecretNonce.generate(sessionId, null, publicKey, null, keyAggCache, extraInput.map(scala2kmp).orNull)
nonces.getFirst -> nonces.getSecond
}
override def verificationNonce(fundingTxId: TxId, fundingPubKey: PublicKey, index: Long): (SecretNonce, IndividualNonce) = {
val keyPath = ChannelKeyManager.keyPath(fundingPubKey)
val sessionId = Generators.perCommitSecret(nonceSeed(keyPath), index).value
val nonce = generateNonce(sessionId, fundingPubKey, Some(fundingTxId.value))
nonce
}
override def signingNonce(fundingPubKey: PublicKey): (SecretNonce, IndividualNonce) = {
val sessionId = randomBytes32()
val nonce = generateNonce(sessionId, fundingPubKey)
nonce
}
/**
* @param tx input transaction
* @param publicKey extended public key
@ -116,6 +140,16 @@ class LocalChannelKeyManager(seed: ByteVector, chainHash: BlockHash) extends Cha
}
}
override def partialSign(tx: Transaction, inputIndex: Int, spentOutputs: Seq[TxOut], localPublicKey: ExtendedPublicKey, remotePublicKey: PublicKey, txOwner: TxOwner, localNonce: (SecretNonce, IndividualNonce), remoteNextLocalNonce: IndividualNonce): Either[Throwable, ByteVector32] = {
val tags = TagSet.Empty.withTag(Tags.TxOwner, txOwner.toString).withTag(Tags.TxType, Tags.TxTypes.CommitTx)
Metrics.SignTxCount.withTags(tags).increment()
KamonExt.time(Metrics.SignTxDuration.withTags(tags)) {
val privateKey = privateKeys.get(localPublicKey.path).privateKey
val psig = Transactions.partialSign(privateKey, tx, inputIndex, spentOutputs, localPublicKey.publicKey, remotePublicKey, localNonce, remoteNextLocalNonce)
psig
}
}
/**
* This method is used to spend funds sent to htlc keys/delayed keys
*

View file

@ -19,7 +19,7 @@ package fr.acinq.eclair.transactions
import fr.acinq.bitcoin.scalacompat.SatoshiLong
import fr.acinq.eclair.MilliSatoshi
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.transactions.Transactions.{CommitmentFormat, ZeroFeeHtlcTxAnchorOutputsCommitmentFormat}
import fr.acinq.eclair.transactions.Transactions.{CommitmentFormat, SimpleTaprootChannelsStagingCommitmentFormat, ZeroFeeHtlcTxAnchorOutputsCommitmentFormat}
import fr.acinq.eclair.wire.protocol._
/**
@ -74,7 +74,7 @@ case class OutgoingHtlc(add: UpdateAddHtlc) extends DirectedHtlc
final case class CommitmentSpec(htlcs: Set[DirectedHtlc], commitTxFeerate: FeeratePerKw, toLocal: MilliSatoshi, toRemote: MilliSatoshi) {
def htlcTxFeerate(commitmentFormat: CommitmentFormat): FeeratePerKw = commitmentFormat match {
case ZeroFeeHtlcTxAnchorOutputsCommitmentFormat => FeeratePerKw(0 sat)
case ZeroFeeHtlcTxAnchorOutputsCommitmentFormat | SimpleTaprootChannelsStagingCommitmentFormat => FeeratePerKw(0 sat)
case _ => commitTxFeerate
}

View file

@ -223,7 +223,8 @@ object Scripts {
/** Extract the payment preimage from a 2nd-stage HTLC Success transaction's witness script */
def extractPreimageFromHtlcSuccess: PartialFunction[ScriptWitness, ByteVector32] = {
case ScriptWitness(Seq(ByteVector.empty, _, _, paymentPreimage, _)) if paymentPreimage.size == 32 => ByteVector32(paymentPreimage)
case ScriptWitness(Seq(ByteVector.empty, _, _, paymentPreimage, _)) if paymentPreimage.size == 32 => ByteVector32(paymentPreimage) // standard channels
case ScriptWitness(Seq(remoteSig, localSig, paymentPreimage, _, _)) if remoteSig.size == 65 && localSig.size == 65 && paymentPreimage.size == 32 => ByteVector32(paymentPreimage) // simple taproot channels
}
/**
@ -307,6 +308,8 @@ object Scripts {
implicit def scala2kmpscript(input: Seq[fr.acinq.bitcoin.scalacompat.ScriptElt]): java.util.List[fr.acinq.bitcoin.ScriptElt] = input.map(e => scala2kmp(e)).asJava
def musig2FundingScript(pubkey1: PublicKey, pubkey2: PublicKey): Seq[ScriptElt] = Script.pay2tr(musig2Aggregate(pubkey1, pubkey2), None)
/**
* Taproot signatures are usually 64 bytes, unless a non-default sighash is used, in which case it is appended.
*/

View file

@ -22,6 +22,7 @@ import fr.acinq.eclair.wire.internal.channel.version1.ChannelCodecs1
import fr.acinq.eclair.wire.internal.channel.version2.ChannelCodecs2
import fr.acinq.eclair.wire.internal.channel.version3.ChannelCodecs3
import fr.acinq.eclair.wire.internal.channel.version4.ChannelCodecs4
import fr.acinq.eclair.wire.internal.channel.version5.ChannelCodecs5
import grizzled.slf4j.Logging
import scodec.Codec
import scodec.codecs.{byte, discriminated}
@ -67,7 +68,8 @@ object ChannelCodecs extends Logging {
* More info here: https://github.com/scodec/scodec/issues/122
*/
val channelDataCodec: Codec[PersistentChannelData] = discriminated[PersistentChannelData].by(byte)
.typecase(4, ChannelCodecs4.channelDataCodec)
.typecase(5, ChannelCodecs5.channelDataCodec)
.typecase(4, ChannelCodecs4.channelDataCodec.decodeOnly)
.typecase(3, ChannelCodecs3.channelDataCodec.decodeOnly)
.typecase(2, ChannelCodecs2.channelDataCodec.decodeOnly)
.typecase(1, ChannelCodecs1.channelDataCodec.decodeOnly)

View file

@ -176,7 +176,8 @@ private[channel] object ChannelCodecs0 {
("index" | uint64overflow) ::
("spec" | commitmentSpecCodec) ::
("txid" | txId) ::
("remotePerCommitmentPoint" | publicKey)).as[RemoteCommit].decodeOnly
("remotePerCommitmentPoint" | publicKey) ::
("localCommitSig_opt" | provide[Option[CommitSig]](None))).as[RemoteCommit].decodeOnly
val updateFulfillHtlcCodec: Codec[UpdateFulfillHtlc] = (
("channelId" | bytes32) ::

View file

@ -48,7 +48,7 @@ private[channel] object ChannelTypes0 {
// modified: we don't use the InputInfo in closing business logic, so we don't need to fill everything (this part
// assumes that we only have standard channels, no anchor output channels - which was the case before version2).
val input = childTx.txIn.head.outPoint
InputInfo(input, parentTx.txOut(input.index.toInt), Nil)
InputInfo(input, parentTx.txOut(input.index.toInt), ByteVector.fromValidHex("deadbeef"))
}
case class LocalCommitPublished(commitTx: Transaction, claimMainDelayedOutputTx: Option[Transaction], htlcSuccessTxs: List[Transaction], htlcTimeoutTxs: List[Transaction], claimHtlcDelayedTxs: List[Transaction], irrevocablySpent: Map[OutPoint, TxId]) {
@ -108,7 +108,7 @@ private[channel] object ChannelTypes0 {
* the raw transaction. It provides more information for auditing but is not used for business logic, so we can safely
* put dummy values in the migration.
*/
def migrateClosingTx(tx: Transaction): ClosingTx = ClosingTx(InputInfo(tx.txIn.head.outPoint, TxOut(Satoshi(0), Nil), Nil), tx, None)
def migrateClosingTx(tx: Transaction): ClosingTx = ClosingTx(InputInfo(tx.txIn.head.outPoint, TxOut(Satoshi(0), Nil), ByteVector.fromValidHex("beefdead")), tx, None)
case class HtlcTxAndSigs(txinfo: HtlcTx, localSig: ByteVector64, remoteSig: ByteVector64)

View file

@ -136,7 +136,8 @@ private[channel] object ChannelCodecs1 {
("index" | uint64overflow) ::
("spec" | commitmentSpecCodec) ::
("txid" | txId) ::
("remotePerCommitmentPoint" | publicKey)).as[RemoteCommit]
("remotePerCommitmentPoint" | publicKey) ::
("localCommitSig_opt" | provide[Option[CommitSig]](None))).as[RemoteCommit]
val updateMessageCodec: Codec[UpdateMessage] = lengthDelimited(lightningMessageCodec.narrow[UpdateMessage](f => Attempt.successful(f.asInstanceOf[UpdateMessage]), g => g))

View file

@ -163,7 +163,8 @@ private[channel] object ChannelCodecs2 {
("index" | uint64overflow) ::
("spec" | commitmentSpecCodec) ::
("txid" | txId) ::
("remotePerCommitmentPoint" | publicKey)).as[RemoteCommit]
("remotePerCommitmentPoint" | publicKey) ::
("localCommitSig_opt" | provide[Option[CommitSig]](None))).as[RemoteCommit]
val updateMessageCodec: Codec[UpdateMessage] = lengthDelimited(lightningMessageCodec.narrow[UpdateMessage](f => Attempt.successful(f.asInstanceOf[UpdateMessage]), g => g))

View file

@ -29,7 +29,7 @@ import fr.acinq.eclair.transactions.{CommitmentSpec, DirectedHtlc, IncomingHtlc,
import fr.acinq.eclair.wire.internal.channel.version0.ChannelTypes0
import fr.acinq.eclair.wire.protocol.CommonCodecs._
import fr.acinq.eclair.wire.protocol.LightningMessageCodecs._
import fr.acinq.eclair.wire.protocol.UpdateMessage
import fr.acinq.eclair.wire.protocol.{CommitSig, UpdateMessage}
import fr.acinq.eclair.{Alias, BlockHeight, FeatureSupport, Features, MilliSatoshiLong, PermanentChannelFeature}
import scodec.bits.{BitVector, ByteVector}
import scodec.codecs._
@ -215,7 +215,8 @@ private[channel] object ChannelCodecs3 {
("index" | uint64overflow) ::
("spec" | commitmentSpecCodec) ::
("txid" | txId) ::
("remotePerCommitmentPoint" | publicKey)).as[RemoteCommit]
("remotePerCommitmentPoint" | publicKey) ::
("localCommitSig_opt" | provide[Option[CommitSig]](None))).as[RemoteCommit]
val updateMessageCodec: Codec[UpdateMessage] = lengthDelimited(lightningMessageCodec.narrow[UpdateMessage](f => Attempt.successful(f.asInstanceOf[UpdateMessage]), g => g))

View file

@ -466,7 +466,8 @@ private[channel] object ChannelCodecs4 {
("index" | uint64overflow) ::
("spec" | commitmentSpecCodec) ::
("txid" | txId) ::
("remotePerCommitmentPoint" | publicKey)).as[RemoteCommit]
("remotePerCommitmentPoint" | publicKey) ::
("localCommitSig_opt" | provide[Option[CommitSig]](None))).as[RemoteCommit]
private def nextRemoteCommitCodec(commitmentSpecCodec: Codec[CommitmentSpec]): Codec[NextRemoteCommit] = (
("sig" | lengthDelimited(commitSigCodec)) ::

View file

@ -0,0 +1,901 @@
package fr.acinq.eclair.wire.internal.channel.version5
import fr.acinq.bitcoin.ScriptTree
import fr.acinq.bitcoin.io.ByteArrayInput
import fr.acinq.bitcoin.scalacompat.Crypto.{PublicKey, XonlyPublicKey}
import fr.acinq.bitcoin.scalacompat.DeterministicWallet.KeyPath
import fr.acinq.bitcoin.scalacompat.{ByteVector64, OutPoint, ScriptWitness, Transaction, TxOut}
import fr.acinq.eclair.blockchain.fee.{ConfirmationPriority, ConfirmationTarget}
import fr.acinq.eclair.channel.LocalFundingStatus._
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTransaction, PartiallySignedSharedTransaction}
import fr.acinq.eclair.channel.fund.InteractiveTxSigningSession.UnsignedLocalCommit
import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession}
import fr.acinq.eclair.crypto.ShaChain
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.transactions.{CommitmentSpec, DirectedHtlc, IncomingHtlc, OutgoingHtlc}
import fr.acinq.eclair.wire.protocol.CommonCodecs._
import fr.acinq.eclair.wire.protocol.LightningMessageCodecs._
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{Alias, BlockHeight, FeatureSupport, Features, MilliSatoshiLong, PermanentChannelFeature, RealShortChannelId, channel}
import scodec.bits.{BitVector, ByteVector}
import scodec.codecs._
import scodec.{Attempt, Codec}
private[channel] object ChannelCodecs5 {
private[version5] object Codecs {
val keyPathCodec: Codec[KeyPath] = ("path" | listOfN(uint16, uint32)).xmap[KeyPath](l => KeyPath(l), keyPath => keyPath.path.toList).as[KeyPath]
val channelConfigCodec: Codec[ChannelConfig] = lengthDelimited(bytes).xmap(b => {
val activated: Set[ChannelConfigOption] = b.bits.toIndexedSeq.reverse.zipWithIndex.collect {
case (true, 0) => ChannelConfig.FundingPubKeyBasedChannelKeyPath
}.toSet
ChannelConfig(activated)
}, cfg => {
val indices = cfg.options.map(_.supportBit)
if (indices.isEmpty) {
ByteVector.empty
} else {
// NB: when converting from BitVector to ByteVector, scodec pads right instead of left, so we make sure we pad to bytes *before* setting bits.
var buffer = BitVector.fill(indices.max + 1)(high = false).bytes.bits
indices.foreach(i => buffer = buffer.set(i))
buffer.reverse.bytes
}
})
/** We use the same encoding as init features, even if we don't need the distinction between mandatory and optional */
val channelFeaturesCodec: Codec[ChannelFeatures] = lengthDelimited(bytes).xmap(
(b: ByteVector) => ChannelFeatures(Features(b).activated.keySet.collect { case f: PermanentChannelFeature => f }), // we make no difference between mandatory/optional, both are considered activated
(cf: ChannelFeatures) => Features(cf.features.map(f => f -> FeatureSupport.Mandatory).toMap).toByteVector // we encode features as mandatory, by convention
)
def localParamsCodec(channelFeatures: ChannelFeatures): Codec[LocalParams] = (
("nodeId" | publicKey) ::
("channelPath" | keyPathCodec) ::
("dustLimit" | satoshi) ::
("maxHtlcValueInFlightMsat" | millisatoshi) ::
("channelReserve" | conditional(!channelFeatures.hasFeature(Features.DualFunding), satoshi)) ::
("htlcMinimum" | millisatoshi) ::
("toSelfDelay" | cltvExpiryDelta) ::
("maxAcceptedHtlcs" | uint16) ::
// We pad to keep codecs byte-aligned.
("isChannelOpener" | bool) :: ("paysCommitTxFees" | bool) :: ignore(6) ::
("upfrontShutdownScript_opt" | optional(bool8, lengthDelimited(bytes))) ::
("walletStaticPaymentBasepoint" | optional(provide(channelFeatures.paysDirectlyToWallet), publicKey)) ::
("features" | combinedFeaturesCodec)).as[LocalParams]
def remoteParamsCodec(channelFeatures: ChannelFeatures): Codec[RemoteParams] = (
("nodeId" | publicKey) ::
("dustLimit" | satoshi) ::
("maxHtlcValueInFlightMsat" | uint64) ::
("channelReserve" | conditional(!channelFeatures.hasFeature(Features.DualFunding), satoshi)) ::
("htlcMinimum" | millisatoshi) ::
("toSelfDelay" | cltvExpiryDelta) ::
("maxAcceptedHtlcs" | uint16) ::
("revocationBasepoint" | publicKey) ::
("paymentBasepoint" | publicKey) ::
("delayedPaymentBasepoint" | publicKey) ::
("htlcBasepoint" | publicKey) ::
("features" | combinedFeaturesCodec) ::
("shutdownScript" | optional(bool8, lengthDelimited(bytes)))).as[RemoteParams]
def setCodec[T](codec: Codec[T]): Codec[Set[T]] = listOfN(uint16, codec).xmap(_.toSet, _.toList)
val htlcCodec: Codec[DirectedHtlc] = discriminated[DirectedHtlc].by(bool8)
.typecase(true, lengthDelimited(updateAddHtlcCodec).as[IncomingHtlc])
.typecase(false, lengthDelimited(updateAddHtlcCodec).as[OutgoingHtlc])
def minimalHtlcCodec(htlcs: Set[UpdateAddHtlc]): Codec[UpdateAddHtlc] = uint64overflow.xmap[UpdateAddHtlc](id => htlcs.find(_.id == id).get, _.id)
def minimalDirectedHtlcCodec(htlcs: Set[DirectedHtlc]): Codec[DirectedHtlc] = discriminated[DirectedHtlc].by(bool8)
.typecase(true, minimalHtlcCodec(htlcs.collect(DirectedHtlc.incoming)).as[IncomingHtlc])
.typecase(false, minimalHtlcCodec(htlcs.collect(DirectedHtlc.outgoing)).as[OutgoingHtlc])
private def baseCommitmentSpecCodec(directedHtlcCodec: Codec[DirectedHtlc]): Codec[CommitmentSpec] = (
("htlcs" | setCodec(directedHtlcCodec)) ::
("feeratePerKw" | feeratePerKw) ::
("toLocal" | millisatoshi) ::
("toRemote" | millisatoshi)).as[CommitmentSpec]
/** HTLCs are stored separately to avoid duplicating data. */
def minimalCommitmentSpecCodec(htlcs: Set[DirectedHtlc]): Codec[CommitmentSpec] = baseCommitmentSpecCodec(minimalDirectedHtlcCodec(htlcs))
/** HTLCs are stored in full, the codec is stateless but creates duplication between local/remote commitment, and across commitments. */
val commitmentSpecCodec: Codec[CommitmentSpec] = baseCommitmentSpecCodec(htlcCodec)
val outPointCodec: Codec[OutPoint] = lengthDelimited(bytes.xmap(d => OutPoint.read(d.toArray), d => OutPoint.write(d)))
val txOutCodec: Codec[TxOut] = lengthDelimited(bytes.xmap(d => TxOut.read(d.toArray), d => TxOut.write(d)))
val txCodec: Codec[Transaction] = lengthDelimited(bytes.xmap(d => Transaction.read(d.toArray), d => Transaction.write(d)))
val scriptTreeCodec: Codec[ScriptTree] = lengthDelimited(bytes.xmap(d => ScriptTree.read(new ByteArrayInput(d.toArray)), d => ByteVector.view(d.write())))
case class ScriptTreeAndInternalKey(scriptTree: Option[ScriptTree], internalKey: XonlyPublicKey)
val xonlyPublicKey: Codec[XonlyPublicKey] = publicKey.xmap(p => p.xOnly, x => x.publicKey)
val scriptTreeAndInternalKey: Codec[ScriptTreeAndInternalKey] = (optional(bool8, scriptTreeCodec) :: xonlyPublicKey).as[ScriptTreeAndInternalKey]
private case class InputInfoEx(outPoint: OutPoint, txOut: TxOut, redeemScript: ByteVector, redeemScriptOrScriptTree: Either[ByteVector, ScriptTreeAndInternalKey])
// To support the change from redeemScript to "either redeem script or script tree" while remaining backwards-compatible with the previous version 4 codec, we use
// the redeem script itself as a left/write indicator: empty -> right, not empty -> left
private def scriptOrTreeCodec(redeemScript: ByteVector): Codec[Either[ByteVector, ScriptTreeAndInternalKey]] = either(provide(redeemScript.isEmpty), provide(redeemScript), scriptTreeAndInternalKey)
private val inputInfoExCodec: Codec[InputInfoEx] = {
("outPoint" | outPointCodec) ::
("txOut" | txOutCodec) ::
(("redeemScript" | lengthDelimited(bytes)) >>:~ { redeemScript => scriptOrTreeCodec(redeemScript).hlist })
}.as[InputInfoEx]
val inputInfoCodec: Codec[InputInfo] = inputInfoExCodec.xmap(
iex => iex.redeemScriptOrScriptTree match {
case Left(redeemScript) => InputInfo.SegwitInput(iex.outPoint, iex.txOut, redeemScript)
case Right(scriptTreeAndInternalKey) => InputInfo.TaprootInput(iex.outPoint, iex.txOut, scriptTreeAndInternalKey.internalKey, scriptTreeAndInternalKey.scriptTree)
},
i => i match {
case InputInfo.SegwitInput(_, _, redeemScript) => InputInfoEx(i.outPoint, i.txOut, redeemScript, Left(redeemScript))
case InputInfo.TaprootInput(_, _, internalKey, scriptTree_opt) => InputInfoEx(i.outPoint, i.txOut, ByteVector.empty, Right(ScriptTreeAndInternalKey(scriptTree_opt, internalKey)))
}
)
val outputInfoCodec: Codec[OutputInfo] = (
("index" | uint32) ::
("amount" | satoshi) ::
("scriptPubKey" | lengthDelimited(bytes))).as[OutputInfo]
private val defaultConfirmationTarget: Codec[ConfirmationTarget.Absolute] = provide(ConfirmationTarget.Absolute(BlockHeight(0)))
private val blockHeightConfirmationTarget: Codec[ConfirmationTarget.Absolute] = blockHeight.xmap(ConfirmationTarget.Absolute, _.confirmBefore)
private val confirmationPriority: Codec[ConfirmationPriority] = discriminated[ConfirmationPriority].by(uint8)
.typecase(0x01, provide(ConfirmationPriority.Slow))
.typecase(0x02, provide(ConfirmationPriority.Medium))
.typecase(0x03, provide(ConfirmationPriority.Fast))
private val priorityConfirmationTarget: Codec[ConfirmationTarget.Priority] = confirmationPriority.xmap(ConfirmationTarget.Priority, _.priority)
private val confirmationTarget: Codec[ConfirmationTarget] = discriminated[ConfirmationTarget].by(uint8)
.typecase(0x00, blockHeightConfirmationTarget)
.typecase(0x01, priorityConfirmationTarget)
val commitTxCodec: Codec[CommitTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[CommitTx]
val htlcSuccessTxCodec: Codec[HtlcSuccessTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("paymentHash" | bytes32) :: ("htlcId" | uint64overflow) :: ("confirmationTarget" | blockHeightConfirmationTarget)).as[HtlcSuccessTx]
val htlcTimeoutTxCodec: Codec[HtlcTimeoutTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("htlcId" | uint64overflow) :: ("confirmationTarget" | blockHeightConfirmationTarget)).as[HtlcTimeoutTx]
private val htlcSuccessTxNoConfirmCodec: Codec[HtlcSuccessTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("paymentHash" | bytes32) :: ("htlcId" | uint64overflow) :: ("confirmationTarget" | defaultConfirmationTarget)).as[HtlcSuccessTx]
private val htlcTimeoutTxNoConfirmCodec: Codec[HtlcTimeoutTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("htlcId" | uint64overflow) :: ("confirmationTarget" | defaultConfirmationTarget)).as[HtlcTimeoutTx]
val htlcDelayedTxCodec: Codec[HtlcDelayedTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[HtlcDelayedTx]
private val legacyClaimHtlcSuccessTxCodec: Codec[LegacyClaimHtlcSuccessTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("htlcId" | uint64overflow) :: ("confirmationTarget" | defaultConfirmationTarget)).as[LegacyClaimHtlcSuccessTx]
val claimHtlcSuccessTxCodec: Codec[ClaimHtlcSuccessTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("paymentHash" | bytes32) :: ("htlcId" | uint64overflow) :: ("confirmationTarget" | blockHeightConfirmationTarget)).as[ClaimHtlcSuccessTx]
val claimHtlcTimeoutTxCodec: Codec[ClaimHtlcTimeoutTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("htlcId" | uint64overflow) :: ("confirmationTarget" | blockHeightConfirmationTarget)).as[ClaimHtlcTimeoutTx]
private val claimHtlcSuccessTxNoConfirmCodec: Codec[ClaimHtlcSuccessTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("paymentHash" | bytes32) :: ("htlcId" | uint64overflow) :: ("confirmationTarget" | defaultConfirmationTarget)).as[ClaimHtlcSuccessTx]
private val claimHtlcTimeoutTxNoConfirmCodec: Codec[ClaimHtlcTimeoutTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("htlcId" | uint64overflow) :: ("confirmationTarget" | defaultConfirmationTarget)).as[ClaimHtlcTimeoutTx]
val claimLocalDelayedOutputTxCodec: Codec[ClaimLocalDelayedOutputTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[ClaimLocalDelayedOutputTx]
val claimP2WPKHOutputTxCodec: Codec[ClaimP2WPKHOutputTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[ClaimP2WPKHOutputTx]
val claimRemoteDelayedOutputTxCodec: Codec[ClaimRemoteDelayedOutputTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[ClaimRemoteDelayedOutputTx]
val mainPenaltyTxCodec: Codec[MainPenaltyTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[MainPenaltyTx]
val htlcPenaltyTxCodec: Codec[HtlcPenaltyTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[HtlcPenaltyTx]
val claimHtlcDelayedOutputPenaltyTxCodec: Codec[ClaimHtlcDelayedOutputPenaltyTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[ClaimHtlcDelayedOutputPenaltyTx]
val claimLocalAnchorOutputTxCodec: Codec[ClaimLocalAnchorOutputTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("confirmationTarget" | confirmationTarget)).as[ClaimLocalAnchorOutputTx]
private val claimLocalAnchorOutputTxBlockHeightConfirmCodec: Codec[ClaimLocalAnchorOutputTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("confirmationTarget" | blockHeightConfirmationTarget).upcast[ConfirmationTarget]).as[ClaimLocalAnchorOutputTx]
private val claimLocalAnchorOutputTxNoConfirmCodec: Codec[ClaimLocalAnchorOutputTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("confirmationTarget" | defaultConfirmationTarget).upcast[ConfirmationTarget]).as[ClaimLocalAnchorOutputTx]
private val claimRemoteAnchorOutputTxCodec: Codec[ClaimRemoteAnchorOutputTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[ClaimRemoteAnchorOutputTx]
val closingTxCodec: Codec[ClosingTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("outputIndex" | optional(bool8, outputInfoCodec))).as[ClosingTx]
val claimRemoteCommitMainOutputTxCodec: Codec[ClaimRemoteCommitMainOutputTx] = discriminated[ClaimRemoteCommitMainOutputTx].by(uint8)
.typecase(0x01, claimP2WPKHOutputTxCodec)
.typecase(0x02, claimRemoteDelayedOutputTxCodec)
val claimAnchorOutputTxCodec: Codec[ClaimAnchorOutputTx] = discriminated[ClaimAnchorOutputTx].by(uint8)
// Important: order matters!
.typecase(0x12, claimLocalAnchorOutputTxCodec)
.typecase(0x11, claimLocalAnchorOutputTxBlockHeightConfirmCodec)
.typecase(0x01, claimLocalAnchorOutputTxNoConfirmCodec)
.typecase(0x02, claimRemoteAnchorOutputTxCodec)
val htlcTxCodec: Codec[HtlcTx] = discriminated[HtlcTx].by(uint8)
// Important: order matters!
.typecase(0x11, htlcSuccessTxCodec)
.typecase(0x12, htlcTimeoutTxCodec)
.typecase(0x01, htlcSuccessTxNoConfirmCodec)
.typecase(0x02, htlcTimeoutTxNoConfirmCodec)
val claimHtlcTxCodec: Codec[ClaimHtlcTx] = discriminated[ClaimHtlcTx].by(uint8)
// Important: order matters!
.typecase(0x22, claimHtlcTimeoutTxCodec)
.typecase(0x23, claimHtlcSuccessTxCodec)
.typecase(0x01, legacyClaimHtlcSuccessTxCodec)
.typecase(0x02, claimHtlcTimeoutTxNoConfirmCodec)
.typecase(0x03, claimHtlcSuccessTxNoConfirmCodec)
val htlcTxsAndRemoteSigsCodec: Codec[HtlcTxAndRemoteSig] = (
("txinfo" | htlcTxCodec) ::
("remoteSig" | bytes64)).as[HtlcTxAndRemoteSig]
// 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,
// a value of all zeroes meaning right (a valid signature cannot be all zeroes)
val commitTxAndRemoteSigCodec: Codec[CommitTxAndRemoteSig] = (
commitTxCodec :: bytes64.consume {
sig => if (sig == ByteVector64.Zeroes)
partialSignatureWithNonce.as[RemoteSignature.PartialSignatureWithNonce].upcast[RemoteSignature]
else
provide(RemoteSignature.FullSignature(sig)).upcast[RemoteSignature]
} {
case RemoteSignature.FullSignature(sig) => sig
case _: RemoteSignature.PartialSignatureWithNonce => ByteVector64.Zeroes
}
).as[CommitTxAndRemoteSig]
val updateMessageCodec: Codec[UpdateMessage] = lengthDelimited(lightningMessageCodec.narrow[UpdateMessage](f => Attempt.successful(f.asInstanceOf[UpdateMessage]), g => g))
val localChangesCodec: Codec[LocalChanges] = (
("proposed" | listOfN(uint16, updateMessageCodec)) ::
("signed" | listOfN(uint16, updateMessageCodec)) ::
("acked" | listOfN(uint16, updateMessageCodec))).as[LocalChanges]
val remoteChangesCodec: Codec[RemoteChanges] = (
("proposed" | listOfN(uint16, updateMessageCodec)) ::
("acked" | listOfN(uint16, updateMessageCodec)) ::
("signed" | listOfN(uint16, updateMessageCodec))).as[RemoteChanges]
val upstreamLocalCodec: Codec[Upstream.Local] = ("id" | uuid).as[Upstream.Local]
val upstreamChannelCodec: Codec[Upstream.Cold.Channel] = (
("originChannelId" | bytes32) ::
("originHtlcId" | int64) ::
("amountIn" | millisatoshi)).as[Upstream.Cold.Channel]
val legacyUpstreamChannelCodec: Codec[Upstream.Cold.Channel] = (
("originChannelId" | bytes32) ::
("originHtlcId" | int64) ::
("amountIn" | millisatoshi) ::
("amountOut" | ignore(64))).as[Upstream.Cold.Channel]
val upstreamChannelWithoutAmountCodec: Codec[Upstream.Cold.Channel] = (
("originChannelId" | bytes32) ::
("originHtlcId" | int64) ::
("amountIn" | provide(0 msat))).as[Upstream.Cold.Channel]
val legacyUpstreamTrampolineCodec: Codec[Upstream.Cold.Trampoline] = listOfN(uint16, upstreamChannelWithoutAmountCodec).as[Upstream.Cold.Trampoline]
val upstreamTrampolineCodec: Codec[Upstream.Cold.Trampoline] = listOfN(uint16, upstreamChannelCodec).as[Upstream.Cold.Trampoline]
val coldUpstreamCodec: Codec[Upstream.Cold] = discriminated[Upstream.Cold].by(uint16)
// NB: order matters!
.typecase(0x06, upstreamChannelCodec)
.typecase(0x05, upstreamTrampolineCodec)
.typecase(0x04, legacyUpstreamTrampolineCodec)
.typecase(0x03, upstreamLocalCodec)
.typecase(0x02, legacyUpstreamChannelCodec)
val originCodec: Codec[Origin] = coldUpstreamCodec.xmap[Origin](
upstream => Origin.Cold(upstream),
{
case Origin.Hot(_, upstream) => Upstream.Cold(upstream)
case Origin.Cold(upstream) => upstream
}
)
def mapCodec[K, V](keyCodec: Codec[K], valueCodec: Codec[V]): Codec[Map[K, V]] = listOfN(uint16, keyCodec ~ valueCodec).xmap(_.toMap, _.toList)
val originsMapCodec: Codec[Map[Long, Origin]] = mapCodec(int64, originCodec)
val spentMapCodec: Codec[Map[OutPoint, Transaction]] = mapCodec(outPointCodec, txCodec)
private val multisig2of2InputCodec: Codec[InteractiveTxBuilder.Multisig2of2Input] = (
("info" | inputInfoCodec) ::
("fundingTxIndex" | uint32) ::
("remoteFundingPubkey" | publicKey)).as[InteractiveTxBuilder.Multisig2of2Input]
private val musig2of2InputCodec: Codec[InteractiveTxBuilder.Musig2Input] = (
("info" | inputInfoCodec) ::
("fundingTxIndex" | uint32) ::
("remoteFundingPubkey" | publicKey) ::
("commitIndex" | uint32)).as[InteractiveTxBuilder.Musig2Input]
private val sharedFundingInputCodec: Codec[InteractiveTxBuilder.SharedFundingInput] = discriminated[InteractiveTxBuilder.SharedFundingInput].by(uint16)
.typecase(0x01, multisig2of2InputCodec)
.typecase(0x02, musig2of2InputCodec)
private val requireConfirmedInputsCodec: Codec[InteractiveTxBuilder.RequireConfirmedInputs] = (("forLocal" | bool8) :: ("forRemote" | bool8)).as[InteractiveTxBuilder.RequireConfirmedInputs]
private val fundingParamsCodec: Codec[InteractiveTxBuilder.InteractiveTxParams] = (
("channelId" | bytes32) ::
("isInitiator" | bool8) ::
("localContribution" | satoshiSigned) ::
("remoteContribution" | satoshiSigned) ::
("sharedInput_opt" | optional(bool8, sharedFundingInputCodec)) ::
("remoteFundingPubKey" | publicKey) ::
("localOutputs" | listOfN(uint16, txOutCodec)) ::
("lockTime" | uint32) ::
("dustLimit" | satoshi) ::
("targetFeerate" | feeratePerKw) ::
("requireConfirmedInputs" | requireConfirmedInputsCodec)).as[InteractiveTxBuilder.InteractiveTxParams]
// This codec was used by a first prototype version of splicing that only worked without HTLCs.
private val sharedInteractiveTxInputWithoutHtlcsCodec: Codec[InteractiveTxBuilder.Input.Shared] = (
("serialId" | uint64) ::
("outPoint" | outPointCodec) ::
("publicKeyScript" | provide(ByteVector.empty)) ::
("sequence" | uint32) ::
("localAmount" | millisatoshi) ::
("remoteAmount" | millisatoshi) ::
("htlcAmount" | provide(0 msat))).as[InteractiveTxBuilder.Input.Shared]
private val sharedInteractiveTxInputWithHtlcsCodec: Codec[InteractiveTxBuilder.Input.Shared] = (
("serialId" | uint64) ::
("outPoint" | outPointCodec) ::
("publicKeyScript" | provide(ByteVector.empty)) ::
("sequence" | uint32) ::
("localAmount" | millisatoshi) ::
("remoteAmount" | millisatoshi) ::
("htlcAmount" | millisatoshi)).as[InteractiveTxBuilder.Input.Shared]
private val sharedInteractiveTxInputWithHtlcsAndPubkeyScriptCodec: Codec[InteractiveTxBuilder.Input.Shared] = (
("serialId" | uint64) ::
("outPoint" | outPointCodec) ::
("publicKeyScript" | lengthDelimited(bytes)) ::
("sequence" | uint32) ::
("localAmount" | millisatoshi) ::
("remoteAmount" | millisatoshi) ::
("htlcAmount" | millisatoshi)).as[InteractiveTxBuilder.Input.Shared]
private val sharedInteractiveTxInputCodec: Codec[InteractiveTxBuilder.Input.Shared] = discriminated[InteractiveTxBuilder.Input.Shared].by(byte)
.typecase(0x03, sharedInteractiveTxInputWithHtlcsAndPubkeyScriptCodec)
.typecase(0x02, sharedInteractiveTxInputWithHtlcsCodec)
.typecase(0x01, sharedInteractiveTxInputWithoutHtlcsCodec)
private val sharedInteractiveTxOutputWithoutHtlcsCodec: Codec[InteractiveTxBuilder.Output.Shared] = (
("serialId" | uint64) ::
("scriptPubKey" | lengthDelimited(bytes)) ::
("localAmount" | millisatoshi) ::
("remoteAmount" | millisatoshi) ::
("htlcAmount" | provide(0 msat))).as[InteractiveTxBuilder.Output.Shared]
private val sharedInteractiveTxOutputWithHtlcsCodec: Codec[InteractiveTxBuilder.Output.Shared] = (
("serialId" | uint64) ::
("scriptPubKey" | lengthDelimited(bytes)) ::
("localAmount" | millisatoshi) ::
("remoteAmount" | millisatoshi) ::
("htlcAmount" | millisatoshi)).as[InteractiveTxBuilder.Output.Shared]
private val sharedInteractiveTxOutputCodec: Codec[InteractiveTxBuilder.Output.Shared] = discriminated[InteractiveTxBuilder.Output.Shared].by(byte)
.typecase(0x02, sharedInteractiveTxOutputWithHtlcsCodec)
.typecase(0x01, sharedInteractiveTxOutputWithoutHtlcsCodec)
private val localOnlyInteractiveTxInputCodec: Codec[InteractiveTxBuilder.Input.Local] = (
("serialId" | uint64) ::
("previousTx" | txCodec) ::
("previousTxOutput" | uint32) ::
("sequence" | uint32)).as[InteractiveTxBuilder.Input.Local]
private val localInteractiveTxInputCodec: Codec[InteractiveTxBuilder.Input.Local] = discriminated[InteractiveTxBuilder.Input.Local].by(byte)
.typecase(0x01, localOnlyInteractiveTxInputCodec)
private val remoteOnlyInteractiveTxInputCodec: Codec[InteractiveTxBuilder.Input.Remote] = (
("serialId" | uint64) ::
("outPoint" | outPointCodec) ::
("txOut" | txOutCodec) ::
("sequence" | uint32)).as[InteractiveTxBuilder.Input.Remote]
private val remoteInteractiveTxInputCodec: Codec[InteractiveTxBuilder.Input.Remote] = discriminated[InteractiveTxBuilder.Input.Remote].by(byte)
.typecase(0x01, remoteOnlyInteractiveTxInputCodec)
private val localInteractiveTxChangeOutputCodec: Codec[InteractiveTxBuilder.Output.Local.Change] = (
("serialId" | uint64) ::
("amount" | satoshi) ::
("scriptPubKey" | lengthDelimited(bytes))).as[InteractiveTxBuilder.Output.Local.Change]
private val localInteractiveTxNonChangeOutputCodec: Codec[InteractiveTxBuilder.Output.Local.NonChange] = (
("serialId" | uint64) ::
("amount" | satoshi) ::
("scriptPubKey" | lengthDelimited(bytes))).as[InteractiveTxBuilder.Output.Local.NonChange]
private val localInteractiveTxOutputCodec: Codec[InteractiveTxBuilder.Output.Local] = discriminated[InteractiveTxBuilder.Output.Local].by(byte)
.typecase(0x01, localInteractiveTxChangeOutputCodec)
.typecase(0x02, localInteractiveTxNonChangeOutputCodec)
private val remoteStandardInteractiveTxOutputCodec: Codec[InteractiveTxBuilder.Output.Remote] = (
("serialId" | uint64) ::
("amount" | satoshi) ::
("scriptPubKey" | lengthDelimited(bytes))).as[InteractiveTxBuilder.Output.Remote]
private val remoteInteractiveTxOutputCodec: Codec[InteractiveTxBuilder.Output.Remote] = discriminated[InteractiveTxBuilder.Output.Remote].by(byte)
.typecase(0x01, remoteStandardInteractiveTxOutputCodec)
private val sharedTransactionCodec: Codec[InteractiveTxBuilder.SharedTransaction] = (
("sharedInput" | optional(bool8, sharedInteractiveTxInputCodec)) ::
("sharedOutput" | sharedInteractiveTxOutputCodec) ::
("localInputs" | listOfN(uint16, localInteractiveTxInputCodec)) ::
("remoteInputs" | listOfN(uint16, remoteInteractiveTxInputCodec)) ::
("localOutputs" | listOfN(uint16, localInteractiveTxOutputCodec)) ::
("remoteOutputs" | listOfN(uint16, remoteInteractiveTxOutputCodec)) ::
("lockTime" | uint32)).as[InteractiveTxBuilder.SharedTransaction]
private val partiallySignedSharedTransactionCodec: Codec[InteractiveTxBuilder.PartiallySignedSharedTransaction] = (
("sharedTx" | sharedTransactionCodec) ::
("localSigs" | lengthDelimited(txSignaturesCodec))).as[InteractiveTxBuilder.PartiallySignedSharedTransaction]
private val scriptWitnessCodec: Codec[ScriptWitness] = listOfN(uint16, lengthDelimited(bytes)).xmap(s => ScriptWitness(s.toSeq), w => w.stack.toList)
private val fullySignedSharedTransactionCodec: Codec[InteractiveTxBuilder.FullySignedSharedTransaction] = (
("sharedTx" | sharedTransactionCodec) ::
("localSigs" | lengthDelimited(txSignaturesCodec)) ::
("remoteSigs" | lengthDelimited(txSignaturesCodec)) ::
("sharedSigs_opt" | optional(bool8, scriptWitnessCodec))).as[InteractiveTxBuilder.FullySignedSharedTransaction]
private val signedSharedTransactionCodec: Codec[InteractiveTxBuilder.SignedSharedTransaction] = discriminated[InteractiveTxBuilder.SignedSharedTransaction].by(uint16)
.typecase(0x01, partiallySignedSharedTransactionCodec)
.typecase(0x02, fullySignedSharedTransactionCodec)
private val liquidityFeesCodec: Codec[LiquidityAds.Fees] = (("miningFees" | satoshi) :: ("serviceFees" | satoshi)).as[LiquidityAds.Fees]
private val liquidityPurchaseCodec: Codec[LiquidityAds.PurchaseBasicInfo] = (
("isBuyer" | bool8) ::
("amount" | satoshi) ::
("fees" | liquidityFeesCodec)).as[LiquidityAds.PurchaseBasicInfo]
private val dualFundedUnconfirmedFundingTxWithoutLiquidityPurchaseCodec: Codec[DualFundedUnconfirmedFundingTx] = (
("sharedTx" | signedSharedTransactionCodec) ::
("createdAt" | blockHeight) ::
("fundingParams" | fundingParamsCodec) ::
("liquidityPurchase" | provide(Option.empty[LiquidityAds.PurchaseBasicInfo]))).as[DualFundedUnconfirmedFundingTx].xmap(
dfu => fillSharedInputScript(dfu),
dfu => dfu
)
private val dualFundedUnconfirmedFundingTxCodec: Codec[DualFundedUnconfirmedFundingTx] = (
("sharedTx" | signedSharedTransactionCodec) ::
("createdAt" | blockHeight) ::
("fundingParams" | fundingParamsCodec) ::
("liquidityPurchase" | optional(bool8, liquidityPurchaseCodec))).as[DualFundedUnconfirmedFundingTx].xmap(
dfu => fillSharedInputScript(dfu),
dfu => dfu
)
// When decoding interactive-tx from older codecs, we fill the shared input publicKeyScript if necessary.
private def fillSharedInputScript(dfu: DualFundedUnconfirmedFundingTx): DualFundedUnconfirmedFundingTx = {
(dfu.sharedTx.tx.sharedInput_opt, dfu.fundingParams.sharedInput_opt) match {
case (Some(sharedTxInput), Some(sharedFundingParamsInput)) if sharedTxInput.publicKeyScript.isEmpty =>
val sharedTxInput1 = sharedTxInput.copy(publicKeyScript = sharedFundingParamsInput.info.txOut.publicKeyScript)
val sharedTx1 = dfu.sharedTx.tx.copy(sharedInput_opt = Some(sharedTxInput1))
val dfu1 = dfu.sharedTx match {
case pt: PartiallySignedSharedTransaction => dfu.copy(sharedTx = pt.copy(tx = sharedTx1))
case ft: FullySignedSharedTransaction => dfu.copy(sharedTx = ft.copy(tx = sharedTx1))
}
dfu1
case _ => dfu
}
}
val fundingTxStatusCodec: Codec[LocalFundingStatus] = discriminated[LocalFundingStatus].by(uint8)
.typecase(0x0a, (txCodec :: realshortchannelid :: optional(bool8, lengthDelimited(txSignaturesCodec)) :: optional(bool8, liquidityPurchaseCodec)).as[ConfirmedFundingTx])
.typecase(0x01, optional(bool8, txCodec).as[SingleFundedUnconfirmedFundingTx])
.typecase(0x07, dualFundedUnconfirmedFundingTxCodec)
.typecase(0x08, (txCodec :: optional(bool8, lengthDelimited(txSignaturesCodec)) :: optional(bool8, liquidityPurchaseCodec)).as[ZeroconfPublishedFundingTx])
.typecase(0x09, (txCodec :: provide(RealShortChannelId(0)) :: optional(bool8, lengthDelimited(txSignaturesCodec)) :: optional(bool8, liquidityPurchaseCodec)).as[ConfirmedFundingTx])
.typecase(0x02, dualFundedUnconfirmedFundingTxWithoutLiquidityPurchaseCodec)
.typecase(0x05, (txCodec :: optional(bool8, lengthDelimited(txSignaturesCodec)) :: provide(Option.empty[LiquidityAds.PurchaseBasicInfo])).as[ZeroconfPublishedFundingTx])
.typecase(0x06, (txCodec :: provide(RealShortChannelId(0)) :: optional(bool8, lengthDelimited(txSignaturesCodec)) :: provide(Option.empty[LiquidityAds.PurchaseBasicInfo])).as[ConfirmedFundingTx])
.typecase(0x03, (txCodec :: provide(Option.empty[TxSignatures]) :: provide(Option.empty[LiquidityAds.PurchaseBasicInfo])).as[ZeroconfPublishedFundingTx])
.typecase(0x04, (txCodec :: provide(RealShortChannelId(0)) :: provide(Option.empty[TxSignatures]) :: provide(Option.empty[LiquidityAds.PurchaseBasicInfo])).as[ConfirmedFundingTx])
val remoteFundingStatusCodec: Codec[RemoteFundingStatus] = discriminated[RemoteFundingStatus].by(uint8)
.typecase(0x01, provide(RemoteFundingStatus.NotLocked))
.typecase(0x02, provide(RemoteFundingStatus.Locked))
val paramsCodec: Codec[ChannelParams] = (
("channelId" | bytes32) ::
("channelConfig" | channelConfigCodec) ::
(("channelFeatures" | channelFeaturesCodec) >>:~ { channelFeatures =>
("localParams" | localParamsCodec(channelFeatures)) ::
("remoteParams" | remoteParamsCodec(channelFeatures)) ::
("channelFlags" | channelflags)
})).as[ChannelParams]
val waitForRevCodec: Codec[WaitForRev] = ("sentAfterLocalCommitIndex" | uint64overflow).as[WaitForRev]
val changesCodec: Codec[CommitmentChanges] = (
("localChanges" | localChangesCodec) ::
("remoteChanges" | remoteChangesCodec) ::
("localNextHtlcId" | uint64overflow) ::
("remoteNextHtlcId" | uint64overflow)).as[CommitmentChanges]
private def localCommitCodec(commitmentSpecCodec: Codec[CommitmentSpec]): Codec[LocalCommit] = (
("index" | uint64overflow) ::
("spec" | commitmentSpecCodec) ::
("commitTxAndRemoteSig" | commitTxAndRemoteSigCodec) ::
("htlcTxsAndRemoteSigs" | listOfN(uint16, htlcTxsAndRemoteSigsCodec))).as[LocalCommit]
private def remoteCommitCodec(commitmentSpecCodec: Codec[CommitmentSpec]): Codec[RemoteCommit] = (
("index" | uint64overflow) ::
("spec" | commitmentSpecCodec) ::
("txid" | txId) ::
("remotePerCommitmentPoint" | publicKey) ::
("localCommitSig_opt" | optional(bool8, lengthDelimited(commitSigCodec)))).as[RemoteCommit]
private def nextRemoteCommitCodec(commitmentSpecCodec: Codec[CommitmentSpec]): Codec[NextRemoteCommit] = (
("sig" | lengthDelimited(commitSigCodec)) ::
("commit" | remoteCommitCodec(commitmentSpecCodec))).as[NextRemoteCommit]
private def commitmentCodecWithoutFirstRemoteCommitIndex(htlcs: Set[DirectedHtlc]): Codec[Commitment] = (
("fundingTxIndex" | uint32) ::
("firstRemoteCommitIndex" | provide(0L)) ::
("fundingPubKey" | publicKey) ::
("fundingTxStatus" | fundingTxStatusCodec) ::
("remoteFundingStatus" | remoteFundingStatusCodec) ::
("localCommit" | localCommitCodec(minimalCommitmentSpecCodec(htlcs))) ::
("remoteCommit" | remoteCommitCodec(minimalCommitmentSpecCodec(htlcs.map(_.opposite)))) ::
("nextRemoteCommit_opt" | optional(bool8, nextRemoteCommitCodec(minimalCommitmentSpecCodec(htlcs.map(_.opposite)))))).as[Commitment]
private def commitmentCodec(htlcs: Set[DirectedHtlc]): Codec[Commitment] = (
("fundingTxIndex" | uint32) ::
("firstRemoteCommitIndex" | uint64overflow) ::
("fundingPubKey" | publicKey) ::
("fundingTxStatus" | fundingTxStatusCodec) ::
("remoteFundingStatus" | remoteFundingStatusCodec) ::
("localCommit" | localCommitCodec(minimalCommitmentSpecCodec(htlcs))) ::
("remoteCommit" | remoteCommitCodec(minimalCommitmentSpecCodec(htlcs.map(_.opposite)))) ::
("nextRemoteCommit_opt" | optional(bool8, nextRemoteCommitCodec(minimalCommitmentSpecCodec(htlcs.map(_.opposite)))))).as[Commitment]
/**
* When multiple commitments are active, htlcs are shared between all of these commitments.
* There may be up to 2 * 483 = 966 htlcs, and every htlc uses at least 1452 bytes and at most 65536 bytes.
* The resulting htlc set size is thus between 1,4 MB and 64 MB, which can be pretty large.
* To avoid writing that htlc set multiple times to disk, we encode it separately.
*/
case class EncodedCommitments(params: ChannelParams,
changes: CommitmentChanges,
// The direction we use is from our local point of view.
htlcs: Set[DirectedHtlc],
active: List[Commitment],
inactive: List[Commitment],
remoteNextCommitInfo: Either[WaitForRev, PublicKey],
remotePerCommitmentSecrets: ShaChain,
originChannels: Map[Long, Origin],
remoteChannelData_opt: Option[ByteVector]) {
def toCommitments: Commitments = {
Commitments(
params = params,
changes = changes,
active = active,
inactive = inactive,
remoteNextCommitInfo,
remotePerCommitmentSecrets,
originChannels,
remoteChannelData_opt
)
}
}
object EncodedCommitments {
def apply(commitments: Commitments): EncodedCommitments = {
// The direction we use is from our local point of view: we use sets, which deduplicates htlcs that are in both
// local and remote commitments.
// All active commitments have the same htlc set, but each inactive commitment may have a distinct htlc set
val commitmentsSet = (commitments.active.head +: commitments.inactive).toSet
val htlcs = commitmentsSet.flatMap(_.localCommit.spec.htlcs) ++
commitmentsSet.flatMap(_.remoteCommit.spec.htlcs.map(_.opposite)) ++
commitmentsSet.flatMap(_.nextRemoteCommit_opt.toList.flatMap(_.commit.spec.htlcs.map(_.opposite)))
EncodedCommitments(
params = commitments.params,
changes = commitments.changes,
htlcs = htlcs,
active = commitments.active.toList,
inactive = commitments.inactive.toList,
remoteNextCommitInfo = commitments.remoteNextCommitInfo,
remotePerCommitmentSecrets = commitments.remotePerCommitmentSecrets,
originChannels = commitments.originChannels,
remoteChannelData_opt = commitments.remoteChannelData_opt
)
}
}
val commitmentsCodecWithoutFirstRemoteCommitIndex: Codec[Commitments] = (
("params" | paramsCodec) ::
("changes" | changesCodec) ::
(("htlcs" | setCodec(htlcCodec)) >>:~ { htlcs =>
("active" | listOfN(uint16, commitmentCodecWithoutFirstRemoteCommitIndex(htlcs))) ::
("inactive" | listOfN(uint16, commitmentCodecWithoutFirstRemoteCommitIndex(htlcs))) ::
("remoteNextCommitInfo" | either(bool8, waitForRevCodec, publicKey)) ::
("remotePerCommitmentSecrets" | byteAligned(ShaChain.shaChainCodec)) ::
("originChannels" | originsMapCodec) ::
("remoteChannelData_opt" | optional(bool8, varsizebinarydata))
})).as[EncodedCommitments].xmap(
encoded => encoded.toCommitments,
commitments => EncodedCommitments(commitments)
)
val commitmentsCodec: Codec[Commitments] = (
("params" | paramsCodec) ::
("changes" | changesCodec) ::
(("htlcs" | setCodec(htlcCodec)) >>:~ { htlcs =>
("active" | listOfN(uint16, commitmentCodec(htlcs))) ::
("inactive" | listOfN(uint16, commitmentCodec(htlcs))) ::
("remoteNextCommitInfo" | either(bool8, waitForRevCodec, publicKey)) ::
("remotePerCommitmentSecrets" | byteAligned(ShaChain.shaChainCodec)) ::
("originChannels" | originsMapCodec) ::
("remoteChannelData_opt" | optional(bool8, varsizebinarydata))
})).as[EncodedCommitments].xmap(
encoded => encoded.toCommitments,
commitments => EncodedCommitments(commitments)
)
val versionedCommitmentsCodec: Codec[Commitments] = discriminated[Commitments].by(uint8)
.typecase(0x01, commitmentsCodec)
val closingFeeratesCodec: Codec[ClosingFeerates] = (
("preferred" | feeratePerKw) ::
("min" | feeratePerKw) ::
("max" | feeratePerKw)).as[ClosingFeerates]
val closingTxProposedCodec: Codec[ClosingTxProposed] = (
("unsignedTx" | closingTxCodec) ::
("localClosingSigned" | lengthDelimited(closingSignedCodec))).as[ClosingTxProposed]
val localCommitPublishedCodec: Codec[LocalCommitPublished] = (
("commitTx" | txCodec) ::
("claimMainDelayedOutputTx" | optional(bool8, claimLocalDelayedOutputTxCodec)) ::
("htlcTxs" | mapCodec(outPointCodec, optional(bool8, htlcTxCodec))) ::
("claimHtlcDelayedTx" | listOfN(uint16, htlcDelayedTxCodec)) ::
("claimAnchorTxs" | listOfN(uint16, claimAnchorOutputTxCodec)) ::
("spent" | spentMapCodec)).as[LocalCommitPublished]
val remoteCommitPublishedCodec: Codec[RemoteCommitPublished] = (
("commitTx" | txCodec) ::
("claimMainOutputTx" | optional(bool8, claimRemoteCommitMainOutputTxCodec)) ::
("claimHtlcTxs" | mapCodec(outPointCodec, optional(bool8, claimHtlcTxCodec))) ::
("claimAnchorTxs" | listOfN(uint16, claimAnchorOutputTxCodec)) ::
("spent" | spentMapCodec)).as[RemoteCommitPublished]
val revokedCommitPublishedCodec: Codec[RevokedCommitPublished] = (
("commitTx" | txCodec) ::
("claimMainOutputTx" | optional(bool8, claimRemoteCommitMainOutputTxCodec)) ::
("mainPenaltyTx" | optional(bool8, mainPenaltyTxCodec)) ::
("htlcPenaltyTxs" | listOfN(uint16, htlcPenaltyTxCodec)) ::
("claimHtlcDelayedPenaltyTxs" | listOfN(uint16, claimHtlcDelayedOutputPenaltyTxCodec)) ::
("spent" | spentMapCodec)).as[RevokedCommitPublished]
// We don't bother removing the duplication across HTLCs: this is a short-lived state during which the channel
// cannot be used for payments.
private val (interactiveTxWaitingForSigsWithoutLiquidityPurchaseCodec, interactiveTxWaitingForSigsCodec): (Codec[InteractiveTxSigningSession.WaitingForSigs], Codec[InteractiveTxSigningSession.WaitingForSigs]) = {
val unsignedLocalCommitCodec: Codec[UnsignedLocalCommit] = (
("index" | uint64overflow) ::
("spec" | commitmentSpecCodec) ::
("commitTx" | commitTxCodec) ::
("htlcTxs" | listOfN(uint16, htlcTxCodec))).as[UnsignedLocalCommit]
val waitingForSigsWithoutLiquidityPurchaseCodec: Codec[InteractiveTxSigningSession.WaitingForSigs] = (
("fundingParams" | fundingParamsCodec) ::
("fundingTxIndex" | uint32) ::
("fundingTx" | partiallySignedSharedTransactionCodec) ::
("localCommit" | either(bool8, unsignedLocalCommitCodec, localCommitCodec(commitmentSpecCodec))) ::
("remoteCommit" | remoteCommitCodec(commitmentSpecCodec)) ::
("liquidityPurchase" | provide(Option.empty[LiquidityAds.PurchaseBasicInfo]))).as[InteractiveTxSigningSession.WaitingForSigs]
val waitingForSigsCodec: Codec[InteractiveTxSigningSession.WaitingForSigs] = (
("fundingParams" | fundingParamsCodec) ::
("fundingTxIndex" | uint32) ::
("fundingTx" | partiallySignedSharedTransactionCodec) ::
("localCommit" | either(bool8, unsignedLocalCommitCodec, localCommitCodec(commitmentSpecCodec))) ::
("remoteCommit" | remoteCommitCodec(commitmentSpecCodec)) ::
("liquidityPurchase" | optional(bool8, liquidityPurchaseCodec))).as[InteractiveTxSigningSession.WaitingForSigs]
(waitingForSigsWithoutLiquidityPurchaseCodec, waitingForSigsCodec)
}
val dualFundingStatusCodec: Codec[DualFundingStatus] = discriminated[DualFundingStatus].by(uint8)
.\(0x01) { case status: DualFundingStatus if !status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs] => DualFundingStatus.WaitingForConfirmations }(provide(DualFundingStatus.WaitingForConfirmations))
.\(0x03) { case status: DualFundingStatus.RbfWaitingForSigs => status }(interactiveTxWaitingForSigsCodec.as[DualFundingStatus.RbfWaitingForSigs])
.\(0x02) { case status: DualFundingStatus.RbfWaitingForSigs => status }(interactiveTxWaitingForSigsWithoutLiquidityPurchaseCodec.as[DualFundingStatus.RbfWaitingForSigs])
val spliceStatusCodec: Codec[SpliceStatus] = discriminated[SpliceStatus].by(uint8)
.\(0x01) { case status: SpliceStatus if !status.isInstanceOf[SpliceStatus.SpliceWaitingForSigs] => SpliceStatus.NoSplice }(provide(SpliceStatus.NoSplice))
.\(0x03) { case status: SpliceStatus.SpliceWaitingForSigs => status }(interactiveTxWaitingForSigsCodec.as[channel.SpliceStatus.SpliceWaitingForSigs])
.\(0x02) { case status: SpliceStatus.SpliceWaitingForSigs => status }(interactiveTxWaitingForSigsWithoutLiquidityPurchaseCodec.as[channel.SpliceStatus.SpliceWaitingForSigs])
private val shortids: Codec[ChannelTypes5.ShortIds] = (
("real_opt" | optional(bool8, realshortchannelid)) ::
("localAlias" | discriminated[Alias].by(uint16).typecase(1, alias)) ::
("remoteAlias_opt" | optional(bool8, alias))
).as[ChannelTypes5.ShortIds].decodeOnly
val DATA_WAIT_FOR_FUNDING_CONFIRMED_00_Codec: Codec[DATA_WAIT_FOR_FUNDING_CONFIRMED] = (
("commitments" | commitmentsCodecWithoutFirstRemoteCommitIndex) ::
("waitingSince" | blockHeight) ::
("deferred" | optional(bool8, lengthDelimited(channelReadyCodec))) ::
("lastSent" | either(bool8, lengthDelimited(fundingCreatedCodec), lengthDelimited(fundingSignedCodec)))).as[DATA_WAIT_FOR_FUNDING_CONFIRMED]
val DATA_WAIT_FOR_FUNDING_CONFIRMED_0a_Codec: Codec[DATA_WAIT_FOR_FUNDING_CONFIRMED] = (
("commitments" | versionedCommitmentsCodec) ::
("waitingSince" | blockHeight) ::
("deferred" | optional(bool8, lengthDelimited(channelReadyCodec))) ::
("lastSent" | either(bool8, lengthDelimited(fundingCreatedCodec), lengthDelimited(fundingSignedCodec)))).as[DATA_WAIT_FOR_FUNDING_CONFIRMED]
val DATA_WAIT_FOR_CHANNEL_READY_01_Codec: Codec[DATA_WAIT_FOR_CHANNEL_READY] = (
("commitments" | commitmentsCodecWithoutFirstRemoteCommitIndex) ::
("shortIds" | shortids)).as[ChannelTypes5.DATA_WAIT_FOR_CHANNEL_READY_0b].map(_.migrate()).decodeOnly
val DATA_WAIT_FOR_CHANNEL_READY_0b_Codec: Codec[DATA_WAIT_FOR_CHANNEL_READY] = (
("commitments" | versionedCommitmentsCodec) ::
("shortIds" | shortids)).as[ChannelTypes5.DATA_WAIT_FOR_CHANNEL_READY_0b].map(_.migrate()).decodeOnly
val DATA_WAIT_FOR_CHANNEL_READY_15_Codec: Codec[DATA_WAIT_FOR_CHANNEL_READY] = (
("commitments" | versionedCommitmentsCodec) ::
("aliases" | aliases)).as[DATA_WAIT_FOR_CHANNEL_READY]
val DATA_WAIT_FOR_DUAL_FUNDING_SIGNED_09_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED] = (
("channelParams" | paramsCodec) ::
("secondRemotePerCommitmentPoint" | publicKey) ::
("localPushAmount" | millisatoshi) ::
("remotePushAmount" | millisatoshi) ::
("status" | interactiveTxWaitingForSigsWithoutLiquidityPurchaseCodec) ::
("remoteChannelData_opt" | optional(bool8, varsizebinarydata))).as[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED]
val DATA_WAIT_FOR_DUAL_FUNDING_SIGNED_13_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED] = (
("channelParams" | paramsCodec) ::
("secondRemotePerCommitmentPoint" | publicKey) ::
("localPushAmount" | millisatoshi) ::
("remotePushAmount" | millisatoshi) ::
("status" | interactiveTxWaitingForSigsCodec) ::
("remoteChannelData_opt" | optional(bool8, varsizebinarydata))).as[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED]
val DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED_02_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] = (
("commitments" | commitmentsCodecWithoutFirstRemoteCommitIndex) ::
("localPushAmount" | millisatoshi) ::
("remotePushAmount" | millisatoshi) ::
("waitingSince" | blockHeight) ::
("lastChecked" | blockHeight) ::
("status" | dualFundingStatusCodec) ::
("deferred" | optional(bool8, lengthDelimited(channelReadyCodec)))).as[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
val DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED_0c_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] = (
("commitments" | versionedCommitmentsCodec) ::
("localPushAmount" | millisatoshi) ::
("remotePushAmount" | millisatoshi) ::
("waitingSince" | blockHeight) ::
("lastChecked" | blockHeight) ::
("status" | dualFundingStatusCodec) ::
("deferred" | optional(bool8, lengthDelimited(channelReadyCodec)))).as[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
val DATA_WAIT_FOR_DUAL_FUNDING_READY_03_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_READY] = (
("commitments" | commitmentsCodecWithoutFirstRemoteCommitIndex) ::
("shortIds" | shortids)).as[ChannelTypes5.DATA_WAIT_FOR_DUAL_FUNDING_READY_0d].map(_.migrate()).decodeOnly
val DATA_WAIT_FOR_DUAL_FUNDING_READY_0d_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_READY] = (
("commitments" | versionedCommitmentsCodec) ::
("shortIds" | shortids)).as[ChannelTypes5.DATA_WAIT_FOR_DUAL_FUNDING_READY_0d].map(_.migrate()).decodeOnly
val DATA_WAIT_FOR_DUAL_FUNDING_READY_16_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_READY] = (
("commitments" | versionedCommitmentsCodec) ::
("aliases" | aliases)).as[DATA_WAIT_FOR_DUAL_FUNDING_READY]
val DATA_NORMAL_04_Codec: Codec[DATA_NORMAL] = (
("commitments" | commitmentsCodecWithoutFirstRemoteCommitIndex) ::
("shortids" | shortids) ::
("channelAnnouncement" | optional(bool8, lengthDelimited(channelAnnouncementCodec))) ::
("channelUpdate" | lengthDelimited(channelUpdateCodec)) ::
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | optional(bool8, closingFeeratesCodec)) ::
("spliceStatus" | spliceStatusCodec)).as[ChannelTypes5.DATA_NORMAL_0e].map(_.migrate()).decodeOnly
val DATA_NORMAL_0e_Codec: Codec[DATA_NORMAL] = (
("commitments" | versionedCommitmentsCodec) ::
("shortids" | shortids) ::
("channelAnnouncement" | optional(bool8, lengthDelimited(channelAnnouncementCodec))) ::
("channelUpdate" | lengthDelimited(channelUpdateCodec)) ::
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | optional(bool8, closingFeeratesCodec)) ::
("spliceStatus" | spliceStatusCodec)).as[ChannelTypes5.DATA_NORMAL_0e].map(_.migrate()).decodeOnly
val DATA_NORMAL_14_Codec: Codec[DATA_NORMAL] = (
("commitments" | versionedCommitmentsCodec) ::
("aliases" | aliases) ::
("channelAnnouncement" | optional(bool8, lengthDelimited(channelAnnouncementCodec))) ::
("channelUpdate" | lengthDelimited(channelUpdateCodec)) ::
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | optional(bool8, closingFeeratesCodec)) ::
("spliceStatus" | spliceStatusCodec)).as[DATA_NORMAL]
val DATA_SHUTDOWN_05_Codec: Codec[DATA_SHUTDOWN] = (
("commitments" | commitmentsCodecWithoutFirstRemoteCommitIndex) ::
("localShutdown" | lengthDelimited(shutdownCodec)) ::
("remoteShutdown" | lengthDelimited(shutdownCodec)) ::
("closingFeerates" | optional(bool8, closingFeeratesCodec))).as[DATA_SHUTDOWN]
val DATA_SHUTDOWN_0f_Codec: Codec[DATA_SHUTDOWN] = (
("commitments" | versionedCommitmentsCodec) ::
("localShutdown" | lengthDelimited(shutdownCodec)) ::
("remoteShutdown" | lengthDelimited(shutdownCodec)) ::
("closingFeerates" | optional(bool8, closingFeeratesCodec))).as[DATA_SHUTDOWN]
val DATA_NEGOTIATING_06_Codec: Codec[DATA_NEGOTIATING] = (
("commitments" | commitmentsCodecWithoutFirstRemoteCommitIndex) ::
("localShutdown" | lengthDelimited(shutdownCodec)) ::
("remoteShutdown" | lengthDelimited(shutdownCodec)) ::
("closingTxProposed" | listOfN(uint16, listOfN(uint16, lengthDelimited(closingTxProposedCodec)))) ::
("bestUnpublishedClosingTx_opt" | optional(bool8, closingTxCodec))).as[DATA_NEGOTIATING]
val DATA_NEGOTIATING_10_Codec: Codec[DATA_NEGOTIATING] = (
("commitments" | versionedCommitmentsCodec) ::
("localShutdown" | lengthDelimited(shutdownCodec)) ::
("remoteShutdown" | lengthDelimited(shutdownCodec)) ::
("closingTxProposed" | listOfN(uint16, listOfN(uint16, lengthDelimited(closingTxProposedCodec)))) ::
("bestUnpublishedClosingTx_opt" | optional(bool8, closingTxCodec))).as[DATA_NEGOTIATING]
private val closingTxsCodec: Codec[ClosingTxs] = (
("localAndRemote_opt" | optional(bool8, closingTxCodec)) ::
("localOnly_opt" | optional(bool8, closingTxCodec)) ::
("remoteOnly_opt" | optional(bool8, closingTxCodec))).as[ClosingTxs]
val DATA_NEGOTIATING_SIMPLE_17_Codec: Codec[DATA_NEGOTIATING_SIMPLE] = (
("commitments" | commitmentsCodec) ::
("lastClosingFeerate" | feeratePerKw) ::
("localScriptPubKey" | varsizebinarydata) ::
("remoteScriptPubKey" | varsizebinarydata) ::
("proposedClosingTxs" | listOfN(uint16, closingTxsCodec)) ::
("publishedClosingTxs" | listOfN(uint16, closingTxCodec))).as[DATA_NEGOTIATING_SIMPLE]
val DATA_CLOSING_07_Codec: Codec[DATA_CLOSING] = (
("commitments" | commitmentsCodecWithoutFirstRemoteCommitIndex) ::
("waitingSince" | blockHeight) ::
("finalScriptPubKey" | lengthDelimited(bytes)) ::
("mutualCloseProposed" | listOfN(uint16, closingTxCodec)) ::
("mutualClosePublished" | listOfN(uint16, closingTxCodec)) ::
("localCommitPublished" | optional(bool8, localCommitPublishedCodec)) ::
("remoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) ::
("nextRemoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) ::
("futureRemoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) ::
("revokedCommitPublished" | listOfN(uint16, revokedCommitPublishedCodec))).as[DATA_CLOSING]
val DATA_CLOSING_11_Codec: Codec[DATA_CLOSING] = (
("commitments" | versionedCommitmentsCodec) ::
("waitingSince" | blockHeight) ::
("finalScriptPubKey" | lengthDelimited(bytes)) ::
("mutualCloseProposed" | listOfN(uint16, closingTxCodec)) ::
("mutualClosePublished" | listOfN(uint16, closingTxCodec)) ::
("localCommitPublished" | optional(bool8, localCommitPublishedCodec)) ::
("remoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) ::
("nextRemoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) ::
("futureRemoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) ::
("revokedCommitPublished" | listOfN(uint16, revokedCommitPublishedCodec))).as[DATA_CLOSING]
val DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT_08_Codec: Codec[DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT] = (
("commitments" | commitmentsCodecWithoutFirstRemoteCommitIndex) ::
("remoteChannelReestablish" | channelReestablishCodec)).as[DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT]
val DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT_12_Codec: Codec[DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT] = (
("commitments" | versionedCommitmentsCodec) ::
("remoteChannelReestablish" | channelReestablishCodec)).as[DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT]
}
// Order matters!
val channelDataCodec: Codec[PersistentChannelData] = discriminated[PersistentChannelData].by(uint16)
.typecase(0x17, Codecs.DATA_NEGOTIATING_SIMPLE_17_Codec)
.typecase(0x16, Codecs.DATA_WAIT_FOR_DUAL_FUNDING_READY_16_Codec)
.typecase(0x15, Codecs.DATA_WAIT_FOR_CHANNEL_READY_15_Codec)
.typecase(0x14, Codecs.DATA_NORMAL_14_Codec)
.typecase(0x13, Codecs.DATA_WAIT_FOR_DUAL_FUNDING_SIGNED_13_Codec)
.typecase(0x12, Codecs.DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT_12_Codec)
.typecase(0x11, Codecs.DATA_CLOSING_11_Codec)
.typecase(0x10, Codecs.DATA_NEGOTIATING_10_Codec)
.typecase(0x0f, Codecs.DATA_SHUTDOWN_0f_Codec)
.typecase(0x0e, Codecs.DATA_NORMAL_0e_Codec)
.typecase(0x0d, Codecs.DATA_WAIT_FOR_DUAL_FUNDING_READY_0d_Codec)
.typecase(0x0c, Codecs.DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED_0c_Codec)
.typecase(0x0b, Codecs.DATA_WAIT_FOR_CHANNEL_READY_0b_Codec)
.typecase(0x0a, Codecs.DATA_WAIT_FOR_FUNDING_CONFIRMED_0a_Codec)
.typecase(0x09, Codecs.DATA_WAIT_FOR_DUAL_FUNDING_SIGNED_09_Codec)
.typecase(0x08, Codecs.DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT_08_Codec)
.typecase(0x07, Codecs.DATA_CLOSING_07_Codec)
.typecase(0x06, Codecs.DATA_NEGOTIATING_06_Codec)
.typecase(0x05, Codecs.DATA_SHUTDOWN_05_Codec)
.typecase(0x04, Codecs.DATA_NORMAL_04_Codec)
.typecase(0x03, Codecs.DATA_WAIT_FOR_DUAL_FUNDING_READY_03_Codec)
.typecase(0x02, Codecs.DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED_02_Codec)
.typecase(0x01, Codecs.DATA_WAIT_FOR_CHANNEL_READY_01_Codec)
.typecase(0x00, Codecs.DATA_WAIT_FOR_FUNDING_CONFIRMED_00_Codec)
}

View file

@ -0,0 +1,82 @@
/*
* Copyright 2024 ACINQ SAS
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package fr.acinq.eclair.wire.internal.channel.version5
import fr.acinq.eclair.channel.LocalFundingStatus.ConfirmedFundingTx
import fr.acinq.eclair.channel._
import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelUpdate, Shutdown}
import fr.acinq.eclair.{Alias, RealShortChannelId}
private[channel] object ChannelTypes5 {
// We moved the real scid inside each commitment object when adding DATA_NORMAL_14_Codec.
case class ShortIds(real_opt: Option[RealShortChannelId], localAlias: Alias, remoteAlias_opt: Option[Alias])
// We moved the channel_announcement inside each commitment object when adding DATA_NORMAL_14_Codec.
case class DATA_NORMAL_0e(commitments: Commitments,
shortIds: ShortIds,
channelAnnouncement: Option[ChannelAnnouncement],
channelUpdate: ChannelUpdate,
localShutdown: Option[Shutdown],
remoteShutdown: Option[Shutdown],
closingFeerates: Option[ClosingFeerates],
spliceStatus: SpliceStatus) {
def migrate(): DATA_NORMAL = {
val commitments1 = commitments.copy(
active = commitments.active.map(c => setScidIfMatches(c, shortIds)),
inactive = commitments.inactive.map(c => setScidIfMatches(c, shortIds)),
)
val aliases = ShortIdAliases(shortIds.localAlias, shortIds.remoteAlias_opt)
DATA_NORMAL(commitments1, aliases, channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, spliceStatus)
}
}
case class DATA_WAIT_FOR_CHANNEL_READY_0b(commitments: Commitments, shortIds: ShortIds) {
def migrate(): DATA_WAIT_FOR_CHANNEL_READY = {
val commitments1 = commitments.copy(
active = commitments.active.map(c => setScidIfMatches(c, shortIds)),
inactive = commitments.inactive.map(c => setScidIfMatches(c, shortIds)),
)
val aliases = ShortIdAliases(shortIds.localAlias, shortIds.remoteAlias_opt)
DATA_WAIT_FOR_CHANNEL_READY(commitments1, aliases)
}
}
case class DATA_WAIT_FOR_DUAL_FUNDING_READY_0d(commitments: Commitments, shortIds: ShortIds) {
def migrate(): DATA_WAIT_FOR_DUAL_FUNDING_READY = {
val commitments1 = commitments.copy(
active = commitments.active.map(c => setScidIfMatches(c, shortIds)),
inactive = commitments.inactive.map(c => setScidIfMatches(c, shortIds)),
)
val aliases = ShortIdAliases(shortIds.localAlias, shortIds.remoteAlias_opt)
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments1, aliases)
}
}
private def setScidIfMatches(c: Commitment, shortIds: ShortIds): Commitment = {
c.localFundingStatus match {
// We didn't support splicing on public channels in this version: the scid (if available) is for the initial
// funding transaction. For private channels we don't care about the real scid, it will be set correctly after
// the next splice.
case f: ConfirmedFundingTx if c.fundingTxIndex == 0 =>
val scid = shortIds.real_opt.getOrElse(f.shortChannelId)
c.copy(localFundingStatus = f.copy(shortChannelId = scid))
case _ => c
}
}
}

View file

@ -17,7 +17,11 @@
package fr.acinq.eclair.wire.protocol
import fr.acinq.bitcoin.scalacompat.{ByteVector64, Satoshi, TxId}
import fr.acinq.bitcoin.crypto.musig2.IndividualNonce
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, TxId}
import fr.acinq.eclair.channel.RemoteSignature.PartialSignatureWithNonce
import fr.acinq.eclair.channel.{ChannelType, ChannelTypes}
import fr.acinq.eclair.wire.protocol.ChannelTlv.{nexLocalNonceTlvCodec, nexLocalNoncesTlvCodec}
import fr.acinq.eclair.wire.protocol.CommonCodecs._
import fr.acinq.eclair.wire.protocol.TlvCodecs.{tlvField, tlvStream, tmillisatoshi}
import fr.acinq.eclair.{Alias, FeatureSupport, Features, MilliSatoshi, UInt64}
@ -89,6 +93,13 @@ object ChannelTlv {
*/
case class UseFeeCredit(amount: MilliSatoshi) extends OpenDualFundedChannelTlv with SpliceInitTlv
case class NextLocalNonceTlv(nonce: IndividualNonce) extends OpenChannelTlv with AcceptChannelTlv with ChannelReadyTlv with ChannelReestablishTlv
val nexLocalNonceTlvCodec: Codec[NextLocalNonceTlv] = tlvField(publicNonce)
case class NextLocalNoncesTlv(nonces: List[IndividualNonce]) extends OpenChannelTlv with AcceptChannelTlv with ChannelReadyTlv with ChannelReestablishTlv
val nexLocalNoncesTlvCodec: Codec[NextLocalNoncesTlv] = tlvField(list(publicNonce))
}
object OpenChannelTlv {
@ -98,6 +109,7 @@ object OpenChannelTlv {
val openTlvCodec: Codec[TlvStream[OpenChannelTlv]] = tlvStream(discriminated[OpenChannelTlv].by(varint)
.typecase(UInt64(0), upfrontShutdownScriptCodec)
.typecase(UInt64(1), channelTypeCodec)
.typecase(UInt64(4), nexLocalNonceTlvCodec)
)
}
@ -109,6 +121,7 @@ object AcceptChannelTlv {
val acceptTlvCodec: Codec[TlvStream[AcceptChannelTlv]] = tlvStream(discriminated[AcceptChannelTlv].by(varint)
.typecase(UInt64(0), upfrontShutdownScriptCodec)
.typecase(UInt64(1), channelTypeCodec)
.typecase(UInt64(4), nexLocalNonceTlvCodec)
)
}
@ -205,16 +218,26 @@ object AcceptDualFundedChannelTlv {
}
case class PartialSignatureWithNonceTlv(partialSigWithNonce: PartialSignatureWithNonce) extends FundingCreatedTlv with FundingSignedTlv
object PartialSignatureWithNonceTlv {
val codec: Codec[PartialSignatureWithNonceTlv] = tlvField(partialSignatureWithNonce)
}
sealed trait FundingCreatedTlv extends Tlv
object FundingCreatedTlv {
val fundingCreatedTlvCodec: Codec[TlvStream[FundingCreatedTlv]] = tlvStream(discriminated[FundingCreatedTlv].by(varint))
val fundingCreatedTlvCodec: Codec[TlvStream[FundingCreatedTlv]] = tlvStream(discriminated[FundingCreatedTlv].by(varint)
.typecase(UInt64(2), PartialSignatureWithNonceTlv.codec)
)
}
sealed trait FundingSignedTlv extends Tlv
object FundingSignedTlv {
val fundingSignedTlvCodec: Codec[TlvStream[FundingSignedTlv]] = tlvStream(discriminated[FundingSignedTlv].by(varint))
val fundingSignedTlvCodec: Codec[TlvStream[FundingSignedTlv]] = tlvStream(discriminated[FundingSignedTlv].by(varint)
.typecase(UInt64(2), PartialSignatureWithNonceTlv.codec)
)
}
sealed trait ChannelReadyTlv extends Tlv
@ -227,6 +250,7 @@ object ChannelReadyTlv {
val channelReadyTlvCodec: Codec[TlvStream[ChannelReadyTlv]] = tlvStream(discriminated[ChannelReadyTlv].by(varint)
.typecase(UInt64(1), channelAliasTlvCodec)
.typecase(UInt64(4), nexLocalNonceTlvCodec)
)
}
@ -253,6 +277,7 @@ object ChannelReestablishTlv {
.typecase(UInt64(0), NextFundingTlv.codec)
.typecase(UInt64(1), YourLastFundingLockedTlv.codec)
.typecase(UInt64(3), MyCurrentFundingLockedTlv.codec)
.typecase(UInt64(4), nexLocalNoncesTlvCodec)
)
}
@ -265,7 +290,13 @@ object UpdateFeeTlv {
sealed trait ShutdownTlv extends Tlv
object ShutdownTlv {
val shutdownTlvCodec: Codec[TlvStream[ShutdownTlv]] = tlvStream(discriminated[ShutdownTlv].by(varint))
case class ShutdownNonce(nonce: IndividualNonce) extends ShutdownTlv
private val shutdownNonceCodec: Codec[ShutdownNonce] = tlvField(publicNonce)
val shutdownTlvCodec: Codec[TlvStream[ShutdownTlv]] = tlvStream(discriminated[ShutdownTlv].by(varint)
.typecase(UInt64(8), shutdownNonceCodec)
)
}
sealed trait ClosingSignedTlv extends Tlv
@ -276,10 +307,14 @@ object ClosingSignedTlv {
private val feeRange: Codec[FeeRange] = tlvField(("min_fee_satoshis" | satoshi) :: ("max_fee_satoshis" | satoshi))
case class PartialSignature(partialSignature: ByteVector32) extends ClosingSignedTlv
private val partialSignature: Codec[PartialSignature] = tlvField(bytes32)
val closingSignedTlvCodec: Codec[TlvStream[ClosingSignedTlv]] = tlvStream(discriminated[ClosingSignedTlv].by(varint)
.typecase(UInt64(1), feeRange)
.typecase(UInt64(6), partialSignature)
)
}
sealed trait ClosingTlv extends Tlv
@ -294,10 +329,21 @@ object ClosingTlv {
/** Signature for a closing transaction containing the closer and closee's outputs. */
case class CloserAndCloseeOutputs(sig: ByteVector64) extends ClosingTlv
/** Signature for a closing transaction containing only the closer's output. */
case class CloserOutputOnlyPartialSignature(partialSignature: ByteVector32) extends ClosingTlv
/** Signature for a closing transaction containing only the closee's output. */
case class CloseeOutputOnlyPartialSignature(partialSignature: ByteVector32) extends ClosingTlv
/** Signature for a closing transaction containing the closer and closee's outputs. */
case class CloserAndCloseeOutputsPartialSignature(partialSignature: ByteVector32) extends ClosingTlv
val closingTlvCodec: Codec[TlvStream[ClosingTlv]] = tlvStream(discriminated[ClosingTlv].by(varint)
.typecase(UInt64(1), tlvField(bytes64.as[CloserOutputOnly]))
.typecase(UInt64(2), tlvField(bytes64.as[CloseeOutputOnly]))
.typecase(UInt64(3), tlvField(bytes64.as[CloserAndCloseeOutputs]))
.typecase(UInt64(4), tlvField(bytes32.as[CloserOutputOnlyPartialSignature]))
.typecase(UInt64(5), tlvField(bytes32.as[CloseeOutputOnlyPartialSignature]))
.typecase(UInt64(6), tlvField(bytes32.as[CloserAndCloseeOutputsPartialSignature]))
)
}

View file

@ -16,10 +16,12 @@
package fr.acinq.eclair.wire.protocol
import fr.acinq.bitcoin.crypto.musig2.IndividualNonce
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.scalacompat.{BlockHash, ByteVector32, ByteVector64, Satoshi, Transaction, TxHash, TxId}
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel.{ChannelFlags, ShortIdAliases}
import fr.acinq.eclair.channel.RemoteSignature.PartialSignatureWithNonce
import fr.acinq.eclair.crypto.Mac32
import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Feature, Features, InitFeature, MilliSatoshi, RealShortChannelId, ShortChannelId, TimestampSecond, UInt64, UnspecifiedShortChannelId}
import org.apache.commons.codec.binary.Base32
@ -156,6 +158,13 @@ object CommonCodecs {
val publicKey: Codec[PublicKey] = catchAllCodec(bytes(33).xmap(bin => PublicKey(bin), pub => pub.value))
val publicNonce: Codec[IndividualNonce] = Codec[IndividualNonce](
(pub: IndividualNonce) => bytes(66).encode(ByteVector.view(pub.toByteArray)),
(wire: BitVector) => bytes(66).decode(wire).map(_.map(b => new IndividualNonce(b.toArray)))
)
val partialSignatureWithNonce: Codec[PartialSignatureWithNonce] = (bytes32 :: publicNonce).as[PartialSignatureWithNonce]
val rgb: Codec[Color] = bytes(3).xmap(buf => Color(buf(0), buf(1), buf(2)), t => ByteVector(t.r, t.g, t.b))
val txCodec: Codec[Transaction] = bytes.xmap(d => Transaction.read(d.toArray), d => Transaction.write(d))

View file

@ -16,8 +16,10 @@
package fr.acinq.eclair.wire.protocol
import fr.acinq.bitcoin.crypto.musig2.IndividualNonce
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.eclair.UInt64
import fr.acinq.eclair.channel.RemoteSignature.PartialSignatureWithNonce
import fr.acinq.eclair.wire.protocol.CommonCodecs._
import fr.acinq.eclair.wire.protocol.TlvCodecs.{tlvField, tlvStream, tu16}
import scodec.{Attempt, Codec, Err}
@ -81,14 +83,28 @@ object CommitSigTlv {
val codec: Codec[BatchTlv] = tlvField(tu16)
}
case class PartialSignatureWithNonceTlv(partialSigWithNonce: PartialSignatureWithNonce) extends CommitSigTlv
object PartialSignatureWithNonceTlv {
val codec: Codec[PartialSignatureWithNonceTlv] = tlvField(partialSignatureWithNonce)
}
val commitSigTlvCodec: Codec[TlvStream[CommitSigTlv]] = tlvStream(discriminated[CommitSigTlv].by(varint)
.typecase(UInt64(2), PartialSignatureWithNonceTlv.codec)
.typecase(UInt64(0x47010005), BatchTlv.codec)
)
}
sealed trait RevokeAndAckTlv extends Tlv
object RevokeAndAckTlv {
val revokeAndAckTlvCodec: Codec[TlvStream[RevokeAndAckTlv]] = tlvStream(discriminated[RevokeAndAckTlv].by(varint))
case class NextLocalNoncesTlv(nonces: List[IndividualNonce]) extends RevokeAndAckTlv
object NextLocalNoncesTlv {
val codec: Codec[NextLocalNoncesTlv] = tlvField(list(publicNonce))
}
val revokeAndAckTlvCodec: Codec[TlvStream[RevokeAndAckTlv]] = tlvStream(discriminated[RevokeAndAckTlv].by(varint)
.typecase(UInt64(4), NextLocalNoncesTlv.codec)
)
}

View file

@ -16,12 +16,14 @@
package fr.acinq.eclair.wire.protocol
import fr.acinq.bitcoin.crypto.musig2.IndividualNonce
import fr.acinq.bitcoin.scalacompat.{ByteVector64, TxId}
import fr.acinq.eclair.UInt64
import fr.acinq.eclair.wire.protocol.CommonCodecs.{bytes64, txIdAsHash, varint}
import fr.acinq.eclair.channel.RemoteSignature.PartialSignatureWithNonce
import fr.acinq.eclair.wire.protocol.CommonCodecs.{bytes64, partialSignatureWithNonce, publicNonce, txIdAsHash, varint}
import fr.acinq.eclair.wire.protocol.TlvCodecs.{tlvField, tlvStream}
import scodec.Codec
import scodec.codecs.discriminated
import scodec.codecs.{discriminated, list}
/**
* Created by t-bast on 08/04/2022.
@ -60,7 +62,19 @@ object TxRemoveOutputTlv {
sealed trait TxCompleteTlv extends Tlv
object TxCompleteTlv {
val txCompleteTlvCodec: Codec[TlvStream[TxCompleteTlv]] = tlvStream(discriminated[TxCompleteTlv].by(varint))
case class FundingNonces(nonces: List[IndividualNonce]) extends TxCompleteTlv
object FundingNonces {
val codec: Codec[FundingNonces] = list(publicNonce).xmap(l => FundingNonces(l), _.nonces.toList)
}
case class CommitNonces(nonces: List[IndividualNonce]) extends TxCompleteTlv
object CommitNonces {
val codec: Codec[CommitNonces] = list(publicNonce).xmap(l => CommitNonces(l), _.nonces.toList)
}
val txCompleteTlvCodec: Codec[TlvStream[TxCompleteTlv]] = tlvStream(discriminated[TxCompleteTlv].by(varint)
.typecase(UInt64(4), tlvField(FundingNonces.codec))
.typecase(UInt64(6), tlvField(CommitNonces.codec))
)
}
sealed trait TxSignaturesTlv extends Tlv
@ -69,7 +83,14 @@ object TxSignaturesTlv {
/** When doing a splice, each peer must provide their signature for the previous 2-of-2 funding output. */
case class PreviousFundingTxSig(sig: ByteVector64) extends TxSignaturesTlv
case class PreviousFundingTxPartialSig(partialSigWithNonce: PartialSignatureWithNonce) extends TxSignaturesTlv
object PreviousFundingTxPartialSig {
val codec: Codec[PreviousFundingTxPartialSig] = tlvField(partialSignatureWithNonce)
}
val txSignaturesTlvCodec: Codec[TlvStream[TxSignaturesTlv]] = tlvStream(discriminated[TxSignaturesTlv].by(varint)
.typecase(UInt64(2), PreviousFundingTxPartialSig.codec)
.typecase(UInt64(601), tlvField(bytes64.as[PreviousFundingTxSig]))
)
}

View file

@ -18,11 +18,14 @@ package fr.acinq.eclair.wire.protocol
import com.google.common.base.Charsets
import com.google.common.net.InetAddresses
import fr.acinq.bitcoin.crypto.musig2.IndividualNonce
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.scalacompat.{BlockHash, ByteVector32, ByteVector64, OutPoint, Satoshi, SatoshiLong, ScriptWitness, Transaction, TxId}
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel.RemoteSignature.PartialSignatureWithNonce
import fr.acinq.eclair.channel.{ChannelFlags, ChannelType}
import fr.acinq.eclair.payment.relay.Relayer
import fr.acinq.eclair.wire.protocol
import fr.acinq.eclair.wire.protocol.ChannelReadyTlv.ShortChannelIdTlv
import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Feature, Features, InitFeature, MilliSatoshi, MilliSatoshiLong, RealShortChannelId, ShortChannelId, TimestampSecond, UInt64, isAsciiPrintable}
import scodec.bits.ByteVector
@ -116,18 +119,34 @@ case class TxRemoveOutput(channelId: ByteVector32,
tlvStream: TlvStream[TxRemoveOutputTlv] = TlvStream.empty) extends InteractiveTxConstructionMessage with HasChannelId with HasSerialId
case class TxComplete(channelId: ByteVector32,
tlvStream: TlvStream[TxCompleteTlv] = TlvStream.empty) extends InteractiveTxConstructionMessage with HasChannelId
tlvStream: TlvStream[TxCompleteTlv] = TlvStream.empty) extends InteractiveTxConstructionMessage with HasChannelId {
val fundingNonces: List[IndividualNonce] = tlvStream.get[TxCompleteTlv.FundingNonces].map(_.nonces).getOrElse(List.empty[IndividualNonce])
val commitNonces: List[IndividualNonce] = tlvStream.get[TxCompleteTlv.CommitNonces].map(_.nonces).getOrElse(List.empty[IndividualNonce])
}
object TxComplete {
def apply(channelId: ByteVector32) = new TxComplete(channelId, TlvStream.empty)
def apply(channelId: ByteVector32, tlvStream: TlvStream[TxCompleteTlv]) = new TxComplete(channelId, tlvStream)
def apply(channelId: ByteVector32, fundingNonces: List[IndividualNonce], commitNonces: List[IndividualNonce]) = new TxComplete(channelId, TlvStream(TxCompleteTlv.FundingNonces(fundingNonces), TxCompleteTlv.CommitNonces(commitNonces)))
}
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 previousFundingTxPartialSig_opt: Option[PartialSignatureWithNonce] = tlvStream.get[TxSignaturesTlv.PreviousFundingTxPartialSig].map(_.partialSigWithNonce)
}
object TxSignatures {
def apply(channelId: ByteVector32, tx: Transaction, witnesses: Seq[ScriptWitness], previousFundingSig_opt: Option[ByteVector64]): TxSignatures = {
TxSignatures(channelId, tx.txid, witnesses, TlvStream(previousFundingSig_opt.map(TxSignaturesTlv.PreviousFundingTxSig).toSet[TxSignaturesTlv]))
def apply(channelId: ByteVector32, tx: Transaction, witnesses: Seq[ScriptWitness], previousFundingSig_opt: Option[ByteVector64], previousFundingTxPartialSig_opt: Option[PartialSignatureWithNonce]): TxSignatures = {
val tlvs: Set[TxSignaturesTlv] = Set(
previousFundingSig_opt.map(TxSignaturesTlv.PreviousFundingTxSig),
previousFundingTxPartialSig_opt.map(p => TxSignaturesTlv.PreviousFundingTxPartialSig(p))
).flatten
TxSignatures(channelId, tx.txid, witnesses, TlvStream(tlvs))
}
}
@ -187,6 +206,7 @@ case class ChannelReestablish(channelId: ByteVector32,
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 nextLocalNonces: List[IndividualNonce] = tlvStream.get[ChannelTlv.NextLocalNoncesTlv].map(_.nonces).getOrElse(List.empty)
}
case class OpenChannel(chainHash: BlockHash,
@ -210,6 +230,7 @@ case class OpenChannel(chainHash: BlockHash,
tlvStream: TlvStream[OpenChannelTlv] = TlvStream.empty) extends ChannelMessage with HasTemporaryChannelId with HasChainHash {
val upfrontShutdownScript_opt: Option[ByteVector] = tlvStream.get[ChannelTlv.UpfrontShutdownScriptTlv].map(_.script)
val channelType_opt: Option[ChannelType] = tlvStream.get[ChannelTlv.ChannelTypeTlv].map(_.channelType)
val nexLocalNonce_opt: Option[IndividualNonce] = tlvStream.get[ChannelTlv.NextLocalNonceTlv].map(_.nonce)
}
case class AcceptChannel(temporaryChannelId: ByteVector32,
@ -229,6 +250,7 @@ case class AcceptChannel(temporaryChannelId: ByteVector32,
tlvStream: TlvStream[AcceptChannelTlv] = TlvStream.empty) extends ChannelMessage with HasTemporaryChannelId {
val upfrontShutdownScript_opt: Option[ByteVector] = tlvStream.get[ChannelTlv.UpfrontShutdownScriptTlv].map(_.script)
val channelType_opt: Option[ChannelType] = tlvStream.get[ChannelTlv.ChannelTypeTlv].map(_.channelType)
val nexLocalNonce_opt: Option[IndividualNonce] = tlvStream.get[ChannelTlv.NextLocalNonceTlv].map(_.nonce)
}
// NB: this message is named open_channel2 in the specification.
@ -289,16 +311,21 @@ case class FundingCreated(temporaryChannelId: ByteVector32,
fundingTxId: TxId,
fundingOutputIndex: Int,
signature: ByteVector64,
tlvStream: TlvStream[FundingCreatedTlv] = TlvStream.empty) extends ChannelMessage with HasTemporaryChannelId
tlvStream: TlvStream[FundingCreatedTlv] = TlvStream.empty) extends ChannelMessage with HasTemporaryChannelId {
val sigOrPartialSig: Either[ByteVector64, PartialSignatureWithNonce] = tlvStream.get[PartialSignatureWithNonceTlv].map(_.partialSigWithNonce).toRight(signature)
}
case class FundingSigned(channelId: ByteVector32,
signature: ByteVector64,
tlvStream: TlvStream[FundingSignedTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId
tlvStream: TlvStream[FundingSignedTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId {
val sigOrPartialSig: Either[ByteVector64, PartialSignatureWithNonce] = tlvStream.get[PartialSignatureWithNonceTlv].map(_.partialSigWithNonce).toRight(signature)
}
case class ChannelReady(channelId: ByteVector32,
nextPerCommitmentPoint: PublicKey,
tlvStream: TlvStream[ChannelReadyTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId {
val alias_opt: Option[Alias] = tlvStream.get[ShortChannelIdTlv].map(_.alias)
val nexLocalNonce_opt: Option[IndividualNonce] = tlvStream.get[ChannelTlv.NextLocalNonceTlv].map(_.nonce)
}
case class Stfu(channelId: ByteVector32, initiator: Boolean) extends SetupMessage with HasChannelId
@ -355,25 +382,51 @@ case class SpliceLocked(channelId: ByteVector32,
case class Shutdown(channelId: ByteVector32,
scriptPubKey: ByteVector,
tlvStream: TlvStream[ShutdownTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId with ForbiddenMessageWhenQuiescent
tlvStream: TlvStream[ShutdownTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId with ForbiddenMessageWhenQuiescent {
val shutdownNonce_opt: Option[IndividualNonce] = tlvStream.get[ShutdownTlv.ShutdownNonce].map(_.nonce)
}
case class ClosingSigned(channelId: ByteVector32,
feeSatoshis: Satoshi,
signature: ByteVector64,
tlvStream: TlvStream[ClosingSignedTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId {
val feeRange_opt = tlvStream.get[ClosingSignedTlv.FeeRange]
val partialSignature_opt = tlvStream.get[ClosingSignedTlv.PartialSignature]
}
case class ClosingComplete(channelId: ByteVector32, closerScriptPubKey: ByteVector, closeeScriptPubKey: ByteVector, fees: Satoshi, lockTime: Long, tlvStream: TlvStream[ClosingTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId {
val closerOutputOnlySig_opt: Option[ByteVector64] = tlvStream.get[ClosingTlv.CloserOutputOnly].map(_.sig)
val closeeOutputOnlySig_opt: Option[ByteVector64] = tlvStream.get[ClosingTlv.CloseeOutputOnly].map(_.sig)
val closerAndCloseeOutputsSig_opt: Option[ByteVector64] = tlvStream.get[ClosingTlv.CloserAndCloseeOutputs].map(_.sig)
//val closerOutputOnlySig_opt: Option[ByteVector64] = tlvStream.get[ClosingTlv.CloserOutputOnly].map(_.sig)
//val closeeOutputOnlySig_opt: Option[ByteVector64] = tlvStream.get[ClosingTlv.CloseeOutputOnly].map(_.sig)
//val closerAndCloseeOutputsSig_opt: Option[ByteVector64] = tlvStream.get[ClosingTlv.CloserAndCloseeOutputs].map(_.sig)
val closerOutputOnlyPartialSig_opt: Option[ByteVector32] = tlvStream.get[ClosingTlv.CloserOutputOnlyPartialSignature].map(_.partialSignature)
val closeeOutputOnlyPartialSig_opt: Option[ByteVector32] = tlvStream.get[ClosingTlv.CloseeOutputOnlyPartialSignature].map(_.partialSignature)
val closerAndCloseeOutputsPartialSig_opt: Option[ByteVector32] = tlvStream.get[ClosingTlv.CloserAndCloseeOutputsPartialSignature].map(_.partialSignature)
val closerOutputOnlySigOrPartialSig_opt: Option[Either[ByteVector64, ByteVector32]] = tlvStream.get[ClosingTlv.CloserOutputOnly]
.map(tlv => Some(Left(tlv.sig)))
.getOrElse(tlvStream.get[ClosingTlv.CloserOutputOnlyPartialSignature].map(tlv => Right(tlv.partialSignature)))
val closeeOutputOnlySigOrPartialSig_opt: Option[Either[ByteVector64, ByteVector32]] = tlvStream.get[ClosingTlv.CloseeOutputOnly]
.map(tlv => Some(Left(tlv.sig)))
.getOrElse(tlvStream.get[ClosingTlv.CloseeOutputOnlyPartialSignature].map(tlv => Right(tlv.partialSignature)))
val closerAndCloseeOutputsSigOrPartialSig_opt: Option[Either[ByteVector64, ByteVector32]] = tlvStream.get[ClosingTlv.CloserAndCloseeOutputs]
.map(tlv => Some(Left(tlv.sig)))
.getOrElse(tlvStream.get[ClosingTlv.CloserAndCloseeOutputsPartialSignature].map(tlv => Right(tlv.partialSignature)))
val closerOutputOnlySig_opt: Option[ByteVector64] = closerOutputOnlySigOrPartialSig_opt.flatMap(_.swap.toOption)
val closeeOutputOnlySig_opt: Option[ByteVector64] = closeeOutputOnlySigOrPartialSig_opt.flatMap(_.swap.toOption)
val closerAndCloseeOutputsSig_opt: Option[ByteVector64] = closerAndCloseeOutputsSigOrPartialSig_opt.flatMap(_.swap.toOption)
}
case class ClosingSig(channelId: ByteVector32, closerScriptPubKey: ByteVector, closeeScriptPubKey: ByteVector, fees: Satoshi, lockTime: Long, tlvStream: TlvStream[ClosingTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId {
val closerOutputOnlySig_opt: Option[ByteVector64] = tlvStream.get[ClosingTlv.CloserOutputOnly].map(_.sig)
val closeeOutputOnlySig_opt: Option[ByteVector64] = tlvStream.get[ClosingTlv.CloseeOutputOnly].map(_.sig)
val closerAndCloseeOutputsSig_opt: Option[ByteVector64] = tlvStream.get[ClosingTlv.CloserAndCloseeOutputs].map(_.sig)
val closerOutputOnlyPartialSig_opt: Option[ByteVector32] = tlvStream.get[ClosingTlv.CloserOutputOnlyPartialSignature].map(_.partialSignature)
val closeeOutputOnlyPartialSig_opt: Option[ByteVector32] = tlvStream.get[ClosingTlv.CloseeOutputOnlyPartialSignature].map(_.partialSignature)
val closerAndCloseeOutputsPartialSig_opt: Option[ByteVector32] = tlvStream.get[ClosingTlv.CloserAndCloseeOutputsPartialSignature].map(_.partialSignature)
}
case class UpdateAddHtlc(channelId: ByteVector32,
@ -432,12 +485,16 @@ case class CommitSig(channelId: ByteVector32,
htlcSignatures: List[ByteVector64],
tlvStream: TlvStream[CommitSigTlv] = TlvStream.empty) extends HtlcMessage with HasChannelId {
val batchSize: Int = tlvStream.get[CommitSigTlv.BatchTlv].map(_.size).getOrElse(1)
val partialSignature_opt: Option[PartialSignatureWithNonce] = tlvStream.get[CommitSigTlv.PartialSignatureWithNonceTlv].map(_.partialSigWithNonce)
val sigOrPartialSig: Either[ByteVector64, PartialSignatureWithNonce] = partialSignature_opt.toRight(signature)
}
case class RevokeAndAck(channelId: ByteVector32,
perCommitmentSecret: PrivateKey,
nextPerCommitmentPoint: PublicKey,
tlvStream: TlvStream[RevokeAndAckTlv] = TlvStream.empty) extends HtlcMessage with HasChannelId
tlvStream: TlvStream[RevokeAndAckTlv] = TlvStream.empty) extends HtlcMessage with HasChannelId {
val nexLocalNonces: List[IndividualNonce] = tlvStream.get[protocol.RevokeAndAckTlv.NextLocalNoncesTlv].map(_.nonces).getOrElse(List.empty)
}
case class UpdateFee(channelId: ByteVector32,
feeratePerKw: FeeratePerKw,

View file

@ -17,14 +17,14 @@
package fr.acinq.eclair.channel
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, OutPoint, SatoshiLong, Transaction, TxIn, TxOut}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, OutPoint, SatoshiLong, Script, Transaction, TxIn, TxOut}
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.WatchFundingSpentTriggered
import fr.acinq.eclair.channel.Helpers.Closing
import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.wire.protocol.{CommitSig, FailureReason, RevokeAndAck, UnknownNextPeer, UpdateAddHtlc}
import fr.acinq.eclair.{MilliSatoshiLong, NodeParams, TestKitBaseClass}
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{MilliSatoshiLong, NodeParams, TestKitBaseClass, randomKey}
import org.scalatest.funsuite.AnyFunSuiteLike
import scodec.bits.ByteVector
@ -600,8 +600,8 @@ class ChannelDataSpec extends TestKitBaseClass with AnyFunSuiteLike with Channel
case (current, tx) => Closing.updateRevokedCommitPublished(current, tx)
}.copy(
claimHtlcDelayedPenaltyTxs = List(
ClaimHtlcDelayedOutputPenaltyTx(InputInfo(OutPoint(htlcSuccess, 0), TxOut(2_500 sat, Nil), Nil), Transaction(2, Seq(TxIn(OutPoint(htlcSuccess, 0), ByteVector.empty, 0)), Seq(TxOut(5_000 sat, ByteVector.empty)), 0)),
ClaimHtlcDelayedOutputPenaltyTx(InputInfo(OutPoint(htlcTimeout, 0), TxOut(3_000 sat, Nil), Nil), Transaction(2, Seq(TxIn(OutPoint(htlcTimeout, 0), ByteVector.empty, 0)), Seq(TxOut(6_000 sat, ByteVector.empty)), 0))
ClaimHtlcDelayedOutputPenaltyTx(InputInfo(OutPoint(htlcSuccess, 0), TxOut(2_500 sat, Nil), Script.write(Script.pay2wpkh(randomKey().publicKey))), Transaction(2, Seq(TxIn(OutPoint(htlcSuccess, 0), ByteVector.empty, 0)), Seq(TxOut(5_000 sat, ByteVector.empty)), 0)),
ClaimHtlcDelayedOutputPenaltyTx(InputInfo(OutPoint(htlcTimeout, 0), TxOut(3_000 sat, Nil), Script.write(Script.pay2wpkh(randomKey().publicKey))), Transaction(2, Seq(TxIn(OutPoint(htlcTimeout, 0), ByteVector.empty, 0)), Seq(TxOut(6_000 sat, ByteVector.empty)), 0))
)
)
assert(!rvk4b.isDone)

View file

@ -28,6 +28,9 @@ import fr.acinq.eclair.crypto.keymanager.LocalChannelKeyManager
import fr.acinq.eclair.transactions.Transactions.CommitTx
import fr.acinq.eclair.transactions.{CommitmentSpec, Scripts, Transactions}
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.transactions.CommitmentSpec
import fr.acinq.eclair.transactions.Transactions.{CommitTx, DefaultCommitmentFormat}
import fr.acinq.eclair.wire.protocol.{FailureReason, IncorrectOrUnknownPaymentDetails, UpdateAddHtlc, UpdateFailHtlc}
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
import org.scalatest.{Outcome, Tag}
@ -491,7 +494,7 @@ object CommitmentsSpec {
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)
val localFundingPubKey = randomKey().publicKey
val remoteFundingPubKey = randomKey().publicKey
val fundingTx = Transaction(2, Nil, Seq(TxOut((toLocal + toRemote).truncateToSatoshi, Funding.makeFundingPubKeyScript(localFundingPubKey, remoteFundingPubKey))), 0)
val fundingTx = Transaction(2, Nil, Seq(TxOut((toLocal + toRemote).truncateToSatoshi, Funding.makeFundingPubKeyScript(localFundingPubKey, remoteFundingPubKey, DefaultCommitmentFormat))), 0)
val commitmentInput = Transactions.InputInfo(OutPoint(fundingTx, 0), fundingTx.txOut.head, Scripts.multiSig2of2(localFundingPubKey, remoteFundingPubKey))
val localCommit = LocalCommit(0, CommitmentSpec(Set.empty, feeRatePerKw, toLocal, toRemote), CommitTxAndRemoteSig(CommitTx(commitmentInput, Transaction(2, Nil, Nil, 0)), ByteVector64.Zeroes), Nil)
val remoteCommit = RemoteCommit(0, CommitmentSpec(Set.empty, feeRatePerKw, toRemote, toLocal), randomTxId(), randomKey().publicKey)
@ -516,7 +519,7 @@ object CommitmentsSpec {
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)
val localFundingPubKey = randomKey().publicKey
val remoteFundingPubKey = randomKey().publicKey
val fundingTx = Transaction(2, Nil, Seq(TxOut((toLocal + toRemote).truncateToSatoshi, Funding.makeFundingPubKeyScript(localFundingPubKey, remoteFundingPubKey))), 0)
val fundingTx = Transaction(2, Nil, Seq(TxOut((toLocal + toRemote).truncateToSatoshi, Funding.makeFundingPubKeyScript(localFundingPubKey, remoteFundingPubKey, DefaultCommitmentFormat))), 0)
val commitmentInput = Transactions.InputInfo(OutPoint(fundingTx, 0), fundingTx.txOut.head, Scripts.multiSig2of2(localFundingPubKey, remoteFundingPubKey))
val localCommit = LocalCommit(0, CommitmentSpec(Set.empty, FeeratePerKw(0 sat), toLocal, toRemote), CommitTxAndRemoteSig(CommitTx(commitmentInput, Transaction(2, Nil, Nil, 0)), ByteVector64.Zeroes), Nil)
val remoteCommit = RemoteCommit(0, CommitmentSpec(Set.empty, FeeratePerKw(0 sat), toRemote, toLocal), randomTxId(), randomKey().publicKey)

View file

@ -229,7 +229,7 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat
)
def toClosingTx(txOut: Seq[TxOut]): ClosingTx = {
ClosingTx(InputInfo(OutPoint(TxId(ByteVector32.Zeroes), 0), TxOut(1000 sat, Nil), Nil), Transaction(2, Nil, txOut, 0), None)
ClosingTx(InputInfo(OutPoint(TxId(ByteVector32.Zeroes), 0), TxOut(1000 sat, Nil), Script.write(Script.pay2wpkh(randomKey().publicKey))), Transaction(2, Nil, txOut, 0), None)
}
assert(Closing.MutualClose.checkClosingDustAmounts(toClosingTx(allOutputsAboveDust)))
@ -249,7 +249,7 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat
Transaction.read("0200000001c8a8934fb38a44b969528252bc37be66ee166c7897c57384d1e561449e110c93010000006b483045022100dc6c50f445ed53d2fb41067fdcb25686fe79492d90e6e5db43235726ace247210220773d35228af0800c257970bee9cf75175d75217de09a8ecd83521befd040c4ca012102082b751372fe7e3b012534afe0bb8d1f2f09c724b1a10a813ce704e5b9c217ccfdffffff0247ba2300000000001976a914f97a7641228e6b17d4b0b08252ae75bd62a95fe788ace3de24000000000017a914a9fefd4b9a9282a1d7a17d2f14ac7d1eb88141d287f7d50800"),
Transaction.read("010000000235a2f5c4fd48672534cce1ac063047edc38683f43c5a883f815d6026cb5f8321020000006a47304402206be5fd61b1702599acf51941560f0a1e1965aa086634b004967747f79788bd6e022002f7f719a45b8b5e89129c40a9d15e4a8ee1e33be3a891cf32e859823ecb7a510121024756c5adfbc0827478b0db042ce09d9b98e21ad80d036e73bd8e7f0ecbc254a2ffffffffb2387d3125bb8c84a2da83f4192385ce329283661dfc70191f4112c67ce7b4d0000000006b483045022100a2c737eab1c039f79238767ccb9bb3e81160e965ef0fc2ea79e8360c61b7c9f702202348b0f2c0ea2a757e25d375d9be183200ce0a79ec81d6a4ebb2ae4dc31bc3c9012102db16a822e2ec3706c58fc880c08a3617c61d8ef706cc8830cfe4561d9a5d52f0ffffffff01808d5b00000000001976a9141210c32def6b64d0d77ba8d99adeb7e9f91158b988ac00000000"),
Transaction.read("0100000001b14ba6952c83f6f8c382befbf4e44270f13e479d5a5ff3862ac3a112f103ff2a010000006b4830450221008b097fd69bfa3715fc5e119a891933c091c55eabd3d1ddae63a1c2cc36dc9a3e02205666d5299fa403a393bcbbf4b05f9c0984480384796cdebcf69171674d00809c01210335b592484a59a44f40998d65a94f9e2eecca47e8d1799342112a59fc96252830ffffffff024bf308000000000017a914440668d018e5e0ba550d6e042abcf726694f515c8798dd1801000000001976a91453a503fe151dd32e0503bd9a2fbdbf4f9a3af1da88ac00000000")
).map(tx => ClosingTx(InputInfo(tx.txIn.head.outPoint, TxOut(10_000 sat, Nil), Nil), tx, None))
).map(tx => ClosingTx(InputInfo(tx.txIn.head.outPoint, TxOut(10_000 sat, Nil), Script.write(Script.pay2wpkh(randomKey().publicKey))), tx, None))
// only mutual close
assert(Closing.isClosingTypeAlreadyKnown(

View file

@ -33,13 +33,14 @@ import fr.acinq.eclair.blockchain.fee.{FeeratePerByte, FeeratePerKw}
import fr.acinq.eclair.blockchain.{OnChainWallet, SingleKeyOnChainWallet}
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder._
import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession}
import fr.acinq.eclair.channel.states.ChannelStateTestsTags
import fr.acinq.eclair.io.OpenChannelInterceptor.makeChannelParams
import fr.acinq.eclair.transactions.Transactions.InputInfo
import fr.acinq.eclair.transactions.{Scripts, Transactions}
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{Feature, FeatureSupport, Features, InitFeature, MilliSatoshiLong, NodeParams, TestConstants, TestKitBaseClass, ToMilliSatoshiConversion, UInt64, randomBytes32, randomKey}
import org.scalatest.BeforeAndAfterAll
import org.scalatest.funsuite.AnyFunSuiteLike
import org.scalatest.{BeforeAndAfterAll, Tag}
import scodec.bits.{ByteVector, HexStringSyntax}
import java.util.UUID
@ -103,10 +104,19 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit
private val firstPerCommitmentPointA = nodeParamsA.channelKeyManager.commitmentPoint(nodeParamsA.channelKeyManager.keyPath(channelParamsA.localParams, ChannelConfig.standard), 0)
private val firstPerCommitmentPointB = nodeParamsB.channelKeyManager.commitmentPoint(nodeParamsB.channelKeyManager.keyPath(channelParamsB.localParams, ChannelConfig.standard), 0)
val fundingPubkeyScript: ByteVector = Script.write(Script.pay2wsh(Scripts.multiSig2of2(fundingParamsB.remoteFundingPubKey, fundingParamsA.remoteFundingPubKey)))
val fundingPubkeyA = nodeParamsA.channelKeyManager.fundingPublicKey(channelParamsA.localParams.fundingKeyPath, 0).publicKey
val fundingPubkeyB = nodeParamsB.channelKeyManager.fundingPublicKey(channelParamsB.localParams.fundingKeyPath, 0).publicKey
assert(channelParamsA.commitmentFormat == channelParamsB.commitmentFormat)
val fundingPubkeyScript: ByteVector = if (channelParamsA.commitmentFormat.useTaproot) {
Script.write(Scripts.Taproot.musig2FundingScript(fundingParamsB.remoteFundingPubKey, fundingParamsA.remoteFundingPubKey))
} else {
Script.write(Script.pay2wsh(Scripts.multiSig2of2(fundingParamsB.remoteFundingPubKey, fundingParamsA.remoteFundingPubKey)))
}
Script.write(Script.pay2wsh(Scripts.multiSig2of2(fundingParamsB.remoteFundingPubKey, fundingParamsA.remoteFundingPubKey)))
def dummySharedInputB(amount: Satoshi): SharedFundingInput = {
val inputInfo = InputInfo(OutPoint(randomTxId(), 3), TxOut(amount, fundingPubkeyScript), Nil)
val inputInfo = InputInfo(OutPoint(randomTxId(), 3), TxOut(amount, fundingPubkeyScript), fundingPubkeyScript)
val fundingTxIndex = fundingParamsA.sharedInput_opt match {
case Some(input: Multisig2of2Input) => input.fundingTxIndex + 1
case _ => 0
@ -211,8 +221,19 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit
}
}
private def createFixtureParams(fundingAmountA: Satoshi, fundingAmountB: Satoshi, targetFeerate: FeeratePerKw, dustLimit: Satoshi, lockTime: Long, requireConfirmedInputs: RequireConfirmedInputs = RequireConfirmedInputs(forLocal = false, forRemote = false), nonInitiatorPaysCommitTxFees: Boolean = false): FixtureParams = {
val channelFeatures = ChannelFeatures(ChannelTypes.AnchorOutputsZeroFeeHtlcTx(), Features[InitFeature](Features.DualFunding -> FeatureSupport.Optional), Features[InitFeature](Features.DualFunding -> FeatureSupport.Optional), announceChannel = true)
private def createFixtureParams(fundingAmountA: Satoshi, fundingAmountB: Satoshi, targetFeerate: FeeratePerKw, dustLimit: Satoshi, lockTime: Long, requireConfirmedInputs: RequireConfirmedInputs = RequireConfirmedInputs(forLocal = false, forRemote = false), nonInitiatorPaysCommitTxFees: Boolean = false, useTaprootChannels: Boolean = false): FixtureParams = {
val channelFeatures = if (useTaprootChannels)
ChannelFeatures(
ChannelTypes.SimpleTaprootChannelsStaging(),
Features[InitFeature](Features.SimpleTaprootStaging -> FeatureSupport.Optional, Features.DualFunding -> FeatureSupport.Optional),
Features[InitFeature](Features.SimpleTaprootStaging -> FeatureSupport.Optional, Features.DualFunding -> FeatureSupport.Optional),
announceChannel = true)
else
ChannelFeatures(
ChannelTypes.AnchorOutputsZeroFeeHtlcTx(),
Features[InitFeature](Features.DualFunding -> FeatureSupport.Optional),
Features[InitFeature](Features.DualFunding -> FeatureSupport.Optional),
announceChannel = true)
val Seq(nodeParamsA, nodeParamsB) = Seq(TestConstants.Alice.nodeParams, TestConstants.Bob.nodeParams).map(_.copy(features = Features(channelFeatures.features.map(f => f -> FeatureSupport.Optional).toMap[Feature, FeatureSupport])))
val localParamsA = makeChannelParams(nodeParamsA, nodeParamsA.features.initFeatures(), None, None, isChannelOpener = true, paysCommitTxFees = !nonInitiatorPaysCommitTxFees, dualFunded = true, fundingAmountA, unlimitedMaxHtlcValueInFlight = false)
val localParamsB = makeChannelParams(nodeParamsB, nodeParamsB.features.initFeatures(), None, None, isChannelOpener = false, paysCommitTxFees = nonInitiatorPaysCommitTxFees, dualFunded = true, fundingAmountB, unlimitedMaxHtlcValueInFlight = false)
@ -276,7 +297,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit
}
}
private def withFixture(fundingAmountA: Satoshi, utxosA: Seq[Satoshi], fundingAmountB: Satoshi, utxosB: Seq[Satoshi], targetFeerate: FeeratePerKw, dustLimit: Satoshi, lockTime: Long, requireConfirmedInputs: RequireConfirmedInputs, liquidityPurchase_opt: Option[LiquidityAds.Purchase] = None)(testFun: Fixture => Any): Unit = {
private def withFixture(fundingAmountA: Satoshi, utxosA: Seq[Satoshi], fundingAmountB: Satoshi, utxosB: Seq[Satoshi], targetFeerate: FeeratePerKw, dustLimit: Satoshi, lockTime: Long, requireConfirmedInputs: RequireConfirmedInputs, liquidityPurchase_opt: Option[LiquidityAds.Purchase] = None, useTaprootChannels: Boolean = false)(testFun: Fixture => Any): Unit = {
// Initialize wallets with a few confirmed utxos.
val probe = TestProbe()
val rpcClientA = createWallet(UUID.randomUUID().toString)
@ -287,7 +308,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit
utxosB.foreach(amount => addUtxo(walletB, amount, probe))
generateBlocks(1)
val fixtureParams = createFixtureParams(fundingAmountA, fundingAmountB, targetFeerate, dustLimit, lockTime, requireConfirmedInputs, nonInitiatorPaysCommitTxFees = liquidityPurchase_opt.nonEmpty)
val fixtureParams = createFixtureParams(fundingAmountA, fundingAmountB, targetFeerate, dustLimit, lockTime, requireConfirmedInputs, nonInitiatorPaysCommitTxFees = liquidityPurchase_opt.nonEmpty, useTaprootChannels)
val alice = fixtureParams.spawnTxBuilderAlice(walletA, liquidityPurchase_opt = liquidityPurchase_opt)
val bob = fixtureParams.spawnTxBuilderBob(walletB, liquidityPurchase_opt = liquidityPurchase_opt)
testFun(Fixture(alice, bob, fixtureParams, walletA, rpcClientA, walletB, rpcClientB, TestProbe(), TestProbe()))
@ -370,13 +391,13 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit
}
}
test("initiator funds less than non-initiator") {
test("initiator funds less than non-initiator (simple taproot channels)", Tag(ChannelStateTestsTags.OptionSimpleTaprootStaging), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) {
val targetFeerate = FeeratePerKw(3000 sat)
val fundingA = 10_000 sat
val utxosA = Seq(50_000 sat)
val fundingB = 50_000 sat
val utxosB = Seq(80_000 sat)
withFixture(fundingA, utxosA, fundingB, utxosB, targetFeerate, 660 sat, 0, RequireConfirmedInputs(forLocal = true, forRemote = true)) { f =>
withFixture(fundingA, utxosA, fundingB, utxosB, targetFeerate, 660 sat, 0, RequireConfirmedInputs(forLocal = true, forRemote = true), useTaprootChannels = true) { f =>
import f._
alice ! Start(alice2bob.ref)
@ -2614,7 +2635,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit
val params = createFixtureParams(100_000 sat, 0 sat, FeeratePerKw(5000 sat), 330 sat, 0)
val previousCommitment = CommitmentsSpec.makeCommitments(25_000_000 msat, 50_000_000 msat).active.head
val fundingTx = Transaction(2, Nil, Seq(TxOut(50_000 sat, Script.pay2wpkh(randomKey().publicKey)), TxOut(20_000 sat, Script.pay2wpkh(randomKey().publicKey))), 0)
val sharedInput = Multisig2of2Input(InputInfo(OutPoint(fundingTx, 0), fundingTx.txOut.head, Nil), 0, randomKey().publicKey)
val sharedInput = Multisig2of2Input(InputInfo(OutPoint(fundingTx, 0), fundingTx.txOut.head, Script.write(Script.pay2wpkh(randomKey().publicKey))), 0, randomKey().publicKey)
val bob = params.spawnTxBuilderSpliceBob(params.fundingParamsB.copy(sharedInput_opt = Some(sharedInput)), previousCommitment, wallet)
bob ! Start(probe.ref)
// Alice --- tx_add_input --> Bob
@ -2841,8 +2862,8 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit
assert(initiatorTx.buildUnsignedTx().txid == unsignedTx.txid)
assert(nonInitiatorTx.buildUnsignedTx().txid == unsignedTx.txid)
val initiatorSigs = TxSignatures(channelId, unsignedTx, Seq(ScriptWitness(Seq(hex"68656c6c6f2074686572652c2074686973206973206120626974636f6e212121", hex"82012088a820add57dfe5277079d069ca4ad4893c96de91f88ffb981fdc6a2a34d5336c66aff87"))), None)
val nonInitiatorSigs = TxSignatures(channelId, unsignedTx, Seq(ScriptWitness(Seq(hex"304402207de9ba56bb9f641372e805782575ee840a899e61021c8b1572b3ec1d5b5950e9022069e9ba998915dae193d3c25cb89b5e64370e6a3a7755e7f31cf6d7cbc2a49f6d01", hex"034695f5b7864c580bf11f9f8cb1a94eb336f2ce9ef872d2ae1a90ee276c772484"))), None)
val initiatorSigs = TxSignatures(channelId, unsignedTx, Seq(ScriptWitness(Seq(hex"68656c6c6f2074686572652c2074686973206973206120626974636f6e212121", hex"82012088a820add57dfe5277079d069ca4ad4893c96de91f88ffb981fdc6a2a34d5336c66aff87"))), None, None)
val nonInitiatorSigs = TxSignatures(channelId, unsignedTx, Seq(ScriptWitness(Seq(hex"304402207de9ba56bb9f641372e805782575ee840a899e61021c8b1572b3ec1d5b5950e9022069e9ba998915dae193d3c25cb89b5e64370e6a3a7755e7f31cf6d7cbc2a49f6d01", hex"034695f5b7864c580bf11f9f8cb1a94eb336f2ce9ef872d2ae1a90ee276c772484"))), None, None)
val initiatorSignedTx = FullySignedSharedTransaction(initiatorTx, initiatorSigs, nonInitiatorSigs, None)
assert(initiatorSignedTx.feerate == FeeratePerKw(262 sat))
val nonInitiatorSignedTx = FullySignedSharedTransaction(nonInitiatorTx, nonInitiatorSigs, initiatorSigs, None)

View file

@ -25,6 +25,7 @@ import fr.acinq.eclair.channel.publish.ReplaceableTxFunder._
import fr.acinq.eclair.channel.publish.ReplaceableTxPrePublisher._
import fr.acinq.eclair.channel.{CommitTxAndRemoteSig, FullCommitment, LocalCommit, LocalParams}
import fr.acinq.eclair.transactions.Scripts
import fr.acinq.eclair.transactions.Transactions.InputInfo.SegwitInput
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.{BlockHeight, CltvExpiry, TestKitBaseClass, randomBytes32}
import org.mockito.IdiomaticMockito.StubbingOps
@ -39,10 +40,10 @@ class ReplaceableTxFunderSpec extends TestKitBaseClass with AnyFunSuiteLike {
private def createAnchorTx(): (CommitTx, ClaimLocalAnchorOutputTx) = {
val anchorScript = Scripts.anchor(PlaceHolderPubKey)
val commitInput = Funding.makeFundingInputInfo(randomTxId(), 1, 500 sat, PlaceHolderPubKey, PlaceHolderPubKey)
val commitInput = Funding.makeFundingInputInfo(randomTxId(), 1, 500 sat, PlaceHolderPubKey, PlaceHolderPubKey, DefaultCommitmentFormat)
val commitTx = Transaction(
2,
Seq(TxIn(commitInput.outPoint, commitInput.redeemScript, 0, Scripts.witness2of2(PlaceHolderSig, PlaceHolderSig, PlaceHolderPubKey, PlaceHolderPubKey))),
Seq(TxIn(commitInput.outPoint, commitInput.asInstanceOf[SegwitInput].redeemScript, 0, Scripts.witness2of2(PlaceHolderSig, PlaceHolderSig, PlaceHolderPubKey, PlaceHolderPubKey))),
Seq(TxOut(330 sat, Script.pay2wsh(anchorScript))),
0
)

View file

@ -20,7 +20,7 @@ import akka.actor.typed.ActorRef
import akka.actor.typed.scaladsl.ActorContext
import akka.actor.typed.scaladsl.adapter.{ClassicActorSystemOps, TypedActorRefOps, actorRefAdapter}
import akka.testkit.TestProbe
import fr.acinq.bitcoin.scalacompat.{OutPoint, SatoshiLong, Transaction, TxIn, TxOut}
import fr.acinq.bitcoin.scalacompat.{OutPoint, SatoshiLong, Script, Transaction, TxIn, TxOut}
import fr.acinq.eclair.TestUtils.randomTxId
import fr.acinq.eclair.blockchain.CurrentBlockHeight
import fr.acinq.eclair.blockchain.fee.{ConfirmationPriority, ConfirmationTarget}
@ -39,6 +39,8 @@ class TxPublisherSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
case class FixtureParam(nodeParams: NodeParams, txPublisher: ActorRef[TxPublisher.Command], factory: TestProbe, probe: TestProbe)
private def randomScript = Script.write(Script.pay2wpkh(randomKey().publicKey))
override def withFixture(test: OneArgTest): Outcome = {
within(max = 30 seconds) {
val nodeParams = TestConstants.Alice.nodeParams
@ -105,7 +107,7 @@ class TxPublisherSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val confirmBefore = ConfirmationTarget.Absolute(nodeParams.currentBlockHeight + 12)
val input = OutPoint(randomTxId(), 3)
val cmd = PublishReplaceableTx(ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), confirmBefore), null, null)
val cmd = PublishReplaceableTx(ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), randomScript), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), confirmBefore), null, null)
txPublisher ! cmd
val child = factory.expectMsgType[ReplaceableTxPublisherSpawned].actor
val p = child.expectMsgType[ReplaceableTxPublisher.Publish]
@ -117,7 +119,7 @@ class TxPublisherSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val confirmBefore = nodeParams.currentBlockHeight + 12
val input = OutPoint(randomTxId(), 3)
val anchorTx = ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), ConfirmationTarget.Priority(ConfirmationPriority.Medium))
val anchorTx = ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), randomScript), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), ConfirmationTarget.Priority(ConfirmationPriority.Medium))
val cmd = PublishReplaceableTx(anchorTx, null, null)
txPublisher ! cmd
val child = factory.expectMsgType[ReplaceableTxPublisherSpawned].actor
@ -175,7 +177,7 @@ class TxPublisherSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val attempt2 = factory.expectMsgType[FinalTxPublisherSpawned].actor
attempt2.expectMsgType[FinalTxPublisher.Publish]
val cmd3 = PublishReplaceableTx(ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, TxOut(20_000 sat, Nil) :: Nil, 0), ConfirmationTarget.Absolute(nodeParams.currentBlockHeight)), null, null)
val cmd3 = PublishReplaceableTx(ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), randomScript), Transaction(2, TxIn(input, Nil, 0) :: Nil, TxOut(20_000 sat, Nil) :: Nil, 0), ConfirmationTarget.Absolute(nodeParams.currentBlockHeight)), null, null)
txPublisher ! cmd3
val attempt3 = factory.expectMsgType[ReplaceableTxPublisherSpawned].actor
attempt3.expectMsgType[ReplaceableTxPublisher.Publish]
@ -197,7 +199,7 @@ class TxPublisherSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val attempt1 = factory.expectMsgType[FinalTxPublisherSpawned]
attempt1.actor.expectMsgType[FinalTxPublisher.Publish]
val cmd2 = PublishReplaceableTx(ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, TxOut(20_000 sat, Nil) :: Nil, 0), ConfirmationTarget.Absolute(nodeParams.currentBlockHeight)), null, null)
val cmd2 = PublishReplaceableTx(ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), randomScript), Transaction(2, TxIn(input, Nil, 0) :: Nil, TxOut(20_000 sat, Nil) :: Nil, 0), ConfirmationTarget.Absolute(nodeParams.currentBlockHeight)), null, null)
txPublisher ! cmd2
val attempt2 = factory.expectMsgType[ReplaceableTxPublisherSpawned]
attempt2.actor.expectMsgType[ReplaceableTxPublisher.Publish]
@ -237,7 +239,7 @@ class TxPublisherSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val target = nodeParams.currentBlockHeight + 12
val input = OutPoint(randomTxId(), 7)
val paymentHash = randomBytes32()
val cmd = PublishReplaceableTx(HtlcSuccessTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), paymentHash, 3, ConfirmationTarget.Absolute(target)), null, null)
val cmd = PublishReplaceableTx(HtlcSuccessTx(InputInfo(input, TxOut(25_000 sat, Nil), randomScript), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), paymentHash, 3, ConfirmationTarget.Absolute(target)), null, null)
txPublisher ! cmd
val attempt1 = factory.expectMsgType[ReplaceableTxPublisherSpawned]
attempt1.actor.expectMsgType[ReplaceableTxPublisher.Publish]
@ -301,7 +303,7 @@ class TxPublisherSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val input = OutPoint(randomTxId(), 7)
val paymentHash = randomBytes32()
val cmd = PublishReplaceableTx(HtlcSuccessTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), paymentHash, 3, ConfirmationTarget.Absolute(nodeParams.currentBlockHeight)), null, null)
val cmd = PublishReplaceableTx(HtlcSuccessTx(InputInfo(input, TxOut(25_000 sat, Nil), randomScript), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), paymentHash, 3, ConfirmationTarget.Absolute(nodeParams.currentBlockHeight)), null, null)
txPublisher ! cmd
val attempt1 = factory.expectMsgType[ReplaceableTxPublisherSpawned]
attempt1.actor.expectMsgType[ReplaceableTxPublisher.Publish]

View file

@ -98,6 +98,8 @@ object ChannelStateTestsTags {
val AdaptMaxHtlcAmount = "adapt_max_htlc_amount"
/** If set, closing will use option_simple_close. */
val SimpleClose = "option_simple_close"
/** If set, channels weill use option_simple_taproot_staging */
val OptionSimpleTaprootStaging = "option_simple_taproot_staging"
}
trait ChannelStateTestsBase extends Assertions with Eventually {
@ -194,6 +196,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.OptionSimpleTaprootStaging))(_.updated(Features.SimpleTaprootStaging, FeatureSupport.Optional))
.initFeatures()
val bobInitFeatures = Bob.nodeParams.features
.modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.DisableWumbo))(_.removed(Features.Wumbo))
@ -207,6 +210,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.OptionSimpleTaprootStaging))(_.updated(Features.SimpleTaprootStaging, FeatureSupport.Optional))
.initFeatures()
val channelType = ChannelTypes.defaultFromFeatures(aliceInitFeatures, bobInitFeatures, announceChannel = channelFlags.announceChannel)

View file

@ -235,6 +235,38 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny
assert(aliceData.latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction].signedTx.txid == fundingTxId)
}
test("complete interactive-tx protocol (simple taproot channels, with push amount)", Tag(ChannelStateTestsTags.OptionSimpleTaprootStaging), Tag(ChannelStateTestsTags.DualFunding), Tag("both_push_amount")) { f =>
import f._
val listener = TestProbe()
alice.underlyingActor.context.system.eventStream.subscribe(listener.ref, classOf[TransactionPublished])
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
val expectedBalanceAlice = TestConstants.fundingSatoshis.toMilliSatoshi + TestConstants.nonInitiatorPushAmount - TestConstants.initiatorPushAmount
assert(expectedBalanceAlice == 900_000_000.msat)
val expectedBalanceBob = TestConstants.nonInitiatorFundingSatoshis.toMilliSatoshi + TestConstants.initiatorPushAmount - TestConstants.nonInitiatorPushAmount
assert(expectedBalanceBob == 600_000_000.msat)
// Bob sends its signatures first as he contributed less than Alice.
bob2alice.expectMsgType[TxSignatures]
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
assert(bobData.commitments.latest.localCommit.spec.toLocal == expectedBalanceBob)
assert(bobData.commitments.latest.localCommit.spec.toRemote == expectedBalanceAlice)
// Alice receives Bob's signatures and sends her own signatures.
bob2alice.forward(alice)
alice2bob.expectMsgType[TxSignatures]
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
assert(aliceData.commitments.latest.localCommit.spec.toLocal == expectedBalanceAlice)
assert(aliceData.commitments.latest.localCommit.spec.toRemote == expectedBalanceBob)
}
test("recv invalid CommitSig", Tag(ChannelStateTestsTags.DualFunding)) { f =>
import f._

View file

@ -29,11 +29,12 @@ import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.channel.fsm.Channel.ProcessCurrentBlockHeight
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.FullySignedSharedTransaction
import fr.acinq.eclair.channel.publish.TxPublisher
import fr.acinq.eclair.channel.publish.TxPublisher.{PublishFinalTx, SetChannelId}
import fr.acinq.eclair.channel.publish.TxPublisher.{PublishFinalTx, PublishReplaceableTx, SetChannelId}
import fr.acinq.eclair.channel.states.ChannelStateTestsBase.FakeTxPublisherFactory
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
import fr.acinq.eclair.transactions.Transactions
import fr.acinq.eclair.transactions.Transactions.ClaimLocalAnchorOutputTx
import fr.acinq.eclair.transactions.Transactions.AnchorOutputsCommitmentFormat
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{BlockHeight, MilliSatoshiLong, TestConstants, TestKitBaseClass, ToMilliSatoshiConversion}
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
@ -1063,16 +1064,28 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
awaitCond(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations)
}
test("recv Error", Tag(ChannelStateTestsTags.DualFunding)) { f =>
def receiveError(f: FixtureParam): Unit = {
import f._
val tx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
alice ! Error(ByteVector32.Zeroes, "dual funding d34d")
awaitCond(alice.stateName == CLOSING)
assert(alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx].tx.txid == tx.txid)
alice.stateData.asInstanceOf[DATA_CLOSING].commitments.params.commitmentFormat match {
case _: AnchorOutputsCommitmentFormat => alice2blockchain.expectMsgType[PublishReplaceableTx] // claim anchor
case Transactions.DefaultCommitmentFormat => ()
}
alice2blockchain.expectMsgType[TxPublisher.PublishTx] // claim-main-delayed
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == tx.txid)
}
test("recv Error", Tag(ChannelStateTestsTags.DualFunding)) { f =>
receiveError(f)
}
test("recv Error (simple taproot channels)", Tag(ChannelStateTestsTags.OptionSimpleTaprootStaging), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.DualFunding)) { f =>
receiveError(f)
}
test("recv Error (remote commit published)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.StaticRemoteKey), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._
val aliceCommitTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx

View file

@ -77,7 +77,7 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF
bob2alice.forward(alice)
alice2bob.expectMsgType[TxComplete]
alice2bob.forward(bob)
bob2alice.expectMsgType[CommitSig]
val sig1 = bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)

View file

@ -34,6 +34,7 @@ import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.FullySignedSharedTransaction
import fr.acinq.eclair.channel.publish.TxPublisher.{PublishFinalTx, PublishReplaceableTx, PublishTx, SetChannelId}
import fr.acinq.eclair.channel.states.ChannelStateTestsBase.{FakeTxPublisherFactory, PimpTestFSM}
import fr.acinq.eclair.channel.states.ChannelStateTestsTags.{AnchorOutputsZeroFeeHtlcTxs, OptionSimpleTaprootStaging, ZeroConf}
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
import fr.acinq.eclair.db.RevokedHtlcInfoCleaner.ForgetHtlcInfos
import fr.acinq.eclair.io.Peer.LiquidityPurchaseSigned
@ -41,6 +42,7 @@ import fr.acinq.eclair.payment.relay.Relayer
import fr.acinq.eclair.testutils.PimpTestProbe.convert
import fr.acinq.eclair.transactions.DirectedHtlc.{incoming, outgoing}
import fr.acinq.eclair.transactions.Transactions
import fr.acinq.eclair.transactions.Transactions.{AnchorOutputsCommitmentFormat, ClaimLocalAnchorOutputTx}
import fr.acinq.eclair.wire.protocol._
import org.scalatest.Inside.inside
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
@ -57,9 +59,14 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
type FixtureParam = SetupFixture
implicit val log: akka.event.LoggingAdapter = akka.event.NoLogging
val useTaproot = false
override def withFixture(test: OneArgTest): Outcome = {
val tags = test.tags + ChannelStateTestsTags.DualFunding
val tags = if (useTaproot) {
test.tags + ChannelStateTestsTags.DualFunding + ChannelStateTestsTags.OptionSimpleTaprootStaging + ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs
} else {
test.tags + ChannelStateTestsTags.DualFunding
}
val setup = init(tags = tags)
import setup._
reachNormal(setup, tags)
@ -616,7 +623,11 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
assert(commitment.localCommit.spec.toLocal == 650_000_000.msat)
assert(commitment.localChannelReserve == 15_000.sat)
val commitFees = Transactions.commitTxTotalCost(commitment.remoteParams.dustLimit, commitment.remoteCommit.spec, commitment.params.commitmentFormat)
if (commitment.commitInput.isP2tr) {
assert(commitFees > 7_000.sat)
} else {
assert(commitFees > 20_000.sat)
}
val sender = TestProbe()
val cmd = CMD_SPLICE(sender.ref, spliceIn_opt = None, Some(SpliceOut(630_000 sat, defaultSpliceOutScriptPubKey)), requestFunding_opt = None)
@ -1043,6 +1054,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
}
test("recv TxAbort (after CommitSig)") { f =>
assume(!useTaproot)
import f._
val sender = TestProbe()
@ -1569,6 +1581,38 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.forall(_.localCommit.spec.htlcs.size == 1))
}
test("recv CMD_ADD_HTLC with multiple commitments and reconnect (simple taproot channels", Tag(OptionSimpleTaprootStaging), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._
initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)))
val sender = TestProbe()
alice ! CMD_ADD_HTLC(sender.ref, 500_000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
alice2bob.expectMsgType[UpdateAddHtlc]
alice2bob.forward(bob)
alice ! CMD_SIGN()
assert(alice2bob.expectMsgType[CommitSig].batchSize == 2)
assert(alice2bob.expectMsgType[CommitSig].batchSize == 2)
// Bob disconnects before receiving Alice's commit_sig.
disconnect(f)
reconnect(f)
alice2bob.expectMsgType[UpdateAddHtlc]
alice2bob.forward(bob)
assert(alice2bob.expectMsgType[CommitSig].batchSize == 2)
alice2bob.forward(bob)
assert(alice2bob.expectMsgType[CommitSig].batchSize == 2)
alice2bob.forward(bob)
bob2alice.expectMsgType[RevokeAndAck]
bob2alice.forward(alice)
assert(bob2alice.expectMsgType[CommitSig].batchSize == 2)
bob2alice.forward(alice)
assert(bob2alice.expectMsgType[CommitSig].batchSize == 2)
bob2alice.forward(alice)
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))
}
test("recv CMD_ADD_HTLC while a splice is requested") { f =>
import f._
val sender = TestProbe()
@ -2684,6 +2728,14 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
alice2bob.expectMsgType[Error]
val commitTx2 = assertPublished(alice2blockchain, "commit-tx")
Transaction.correctlySpends(commitTx2, Seq(fundingTx2), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
val isAnchorOutputs = alice.stateData.asInstanceOf[DATA_CLOSING].commitments.params.commitmentFormat match {
case _: AnchorOutputsCommitmentFormat => true
case _ => false
}
if (isAnchorOutputs) {
val claimAnchor = assertPublished(alice2blockchain, "local-anchor")
}
val claimMainDelayed2 = assertPublished(alice2blockchain, "local-main-delayed")
// Alice publishes her htlc timeout transactions.
val htlcsTxsOut = htlcs.aliceToBob.map(_ => assertPublished(alice2blockchain, "htlc-timeout"))
@ -2691,6 +2743,10 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
val watchConfirmedCommit2 = alice2blockchain.expectWatchTxConfirmed(commitTx2.txid)
val watchConfirmedClaimMainDelayed2 = alice2blockchain.expectWatchTxConfirmed(claimMainDelayed2.txid)
// watch for all htlc outputs from local commit-tx to be spent
if (isAnchorOutputs) {
alice2blockchain.expectMsgType[WatchOutputSpent]
}
val watchHtlcsOut = htlcs.aliceToBob.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent])
htlcs.bobToAlice.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent])
@ -3432,5 +3488,8 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
assert(finalState.commitments.latest.localCommit.spec.toLocal == 805_000_000.msat)
assert(finalState.commitments.latest.localCommit.spec.toRemote == 695_000_000.msat)
}
}
class NormalSplicesStateWithTaprootChannelsSpec extends NormalSplicesStateSpec {
override val useTaproot: Boolean = true
}

View file

@ -278,7 +278,7 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
awaitCond(bob.stateName == NORMAL)
}
test("resume htlc settlement", Tag(IgnoreChannelUpdates)) { f =>
def resumeHTlcSettlement(f: FixtureParam): Unit = {
import f._
// Successfully send a first payment.
@ -325,6 +325,14 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommitIndex == 4)
}
test("resume htlc settlement", Tag(IgnoreChannelUpdates)) { f =>
resumeHTlcSettlement(f)
}
test("resume htlc settlement (simple taproot channels)", Tag(ChannelStateTestsTags.OptionSimpleTaprootStaging), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(IgnoreChannelUpdates)) { f =>
resumeHTlcSettlement(f)
}
test("reconnect with an outdated commitment", Tag(IgnoreChannelUpdates), Tag(ChannelStateTestsTags.StaticRemoteKey), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._

View file

@ -531,6 +531,54 @@ class NegotiatingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
assert(bob.stateName == NEGOTIATING_SIMPLE)
}
test("recv ClosingComplete (both outputs, simple taproot channels)", Tag(ChannelStateTestsTags.SimpleClose), Tag(ChannelStateTestsTags.OptionSimpleTaprootStaging), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._
aliceClose(f)
val aliceClosingComplete = alice2bob.expectMsgType[ClosingComplete]
assert(aliceClosingComplete.fees > 0.sat)
assert(aliceClosingComplete.closerAndCloseeOutputsPartialSig_opt.nonEmpty)
assert(aliceClosingComplete.closerOutputOnlyPartialSig_opt.nonEmpty)
assert(aliceClosingComplete.closeeOutputOnlyPartialSig_opt.isEmpty)
val bobClosingComplete = bob2alice.expectMsgType[ClosingComplete]
assert(bobClosingComplete.fees > 0.sat)
assert(bobClosingComplete.closerAndCloseeOutputsPartialSig_opt.nonEmpty)
assert(bobClosingComplete.closerOutputOnlyPartialSig_opt.nonEmpty)
assert(bobClosingComplete.closeeOutputOnlyPartialSig_opt.isEmpty)
alice2bob.forward(bob, aliceClosingComplete)
val bobClosingSig = bob2alice.expectMsgType[ClosingSig]
assert(bobClosingSig.fees == aliceClosingComplete.fees)
assert(bobClosingSig.lockTime == aliceClosingComplete.lockTime)
bob2alice.forward(alice, bobClosingSig)
val aliceTx = alice2blockchain.expectMsgType[PublishFinalTx]
assert(aliceTx.desc == "closing")
assert(aliceTx.fee > 0.sat)
alice2blockchain.expectWatchTxConfirmed(aliceTx.tx.txid)
inside(bob2blockchain.expectMsgType[PublishFinalTx]) { p =>
assert(p.tx.txid == aliceTx.tx.txid)
assert(p.fee == 0.sat)
}
bob2blockchain.expectWatchTxConfirmed(aliceTx.tx.txid)
assert(alice.stateName == NEGOTIATING_SIMPLE)
bob2alice.forward(alice, bobClosingComplete)
val aliceClosingSig = alice2bob.expectMsgType[ClosingSig]
assert(aliceClosingSig.fees == bobClosingComplete.fees)
assert(aliceClosingSig.lockTime == bobClosingComplete.lockTime)
alice2bob.forward(bob, aliceClosingSig)
val bobTx = bob2blockchain.expectMsgType[PublishFinalTx]
assert(bobTx.desc == "closing")
assert(bobTx.fee > 0.sat)
bob2blockchain.expectWatchTxConfirmed(bobTx.tx.txid)
inside(alice2blockchain.expectMsgType[PublishFinalTx]) { p =>
assert(p.tx.txid == bobTx.tx.txid)
assert(p.fee == 0.sat)
}
assert(aliceTx.tx.txid != bobTx.tx.txid)
alice2blockchain.expectWatchTxConfirmed(bobTx.tx.txid)
assert(bob.stateName == NEGOTIATING_SIMPLE)
}
test("recv ClosingComplete (single output)", Tag(ChannelStateTestsTags.SimpleClose), Tag(ChannelStateTestsTags.NoPushAmount)) { f =>
import f._
aliceClose(f)

View file

@ -412,6 +412,36 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(alice.stateData == initialState) // this was a no-op
}
test("recv WatchOutputSpentTriggered (simple taproot channels)", Tag(ChannelStateTestsTags.OptionSimpleTaprootStaging), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._
// alice sends an htlc to bob
val (ra1, htlca1) = addHtlc(50000000 msat, alice, bob, alice2bob, bob2alice)
crossSign(alice, bob, alice2bob, bob2alice)
bob2relayer.expectMsgType[RelayForward]
localClose(alice, alice2blockchain)
val initialState = alice.stateData.asInstanceOf[DATA_CLOSING]
assert(initialState.localCommitPublished.isDefined)
// actual test starts here
channelUpdateListener.expectMsgType[LocalChannelDown]
// scenario 1: bob claims the htlc output from the commit tx using its preimage
val claimHtlcSuccessFromCommitTx = Transaction(version = 0, txIn = TxIn(outPoint = OutPoint(randomTxId(), 0), signatureScript = ByteVector.empty, sequence = 0, witness = Scripts.witnessClaimHtlcSuccessFromCommitTx(Transactions.PlaceHolderSig, ra1, ByteVector.fill(130)(33))) :: Nil, txOut = Nil, lockTime = 0)
alice ! WatchOutputSpentTriggered(100_000 sat, claimHtlcSuccessFromCommitTx)
val fulfill1 = alice2relayer.expectMsgType[RES_ADD_SETTLED[Origin, HtlcResult.OnChainFulfill]]
assert(fulfill1.htlc == htlca1)
assert(fulfill1.result.paymentPreimage == ra1)
// scenario 2: bob claims the htlc output from his own commit tx using its preimage (let's assume both parties had published their commitment tx)
val claimHtlcSuccessTx = Transaction(version = 0, txIn = TxIn(outPoint = OutPoint(randomTxId(), 0), signatureScript = ByteVector.empty, sequence = 0, witness = Scripts.witnessHtlcSuccess(Transactions.PlaceHolderSig, Transactions.PlaceHolderSig, ra1, ByteVector.fill(130)(33), Transactions.DefaultCommitmentFormat)) :: Nil, txOut = Nil, lockTime = 0)
alice ! WatchOutputSpentTriggered(100_000 sat, claimHtlcSuccessTx)
val fulfill2 = alice2relayer.expectMsgType[RES_ADD_SETTLED[Origin, HtlcResult.OnChainFulfill]]
assert(fulfill2.htlc == htlca1)
assert(fulfill2.result.paymentPreimage == ra1)
assert(alice.stateData == initialState) // this was a no-op
}
test("recv CMD_BUMP_FORCE_CLOSE_FEE (local commit)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._
@ -501,6 +531,10 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
testLocalCommitTxConfirmed(f, ChannelFeatures(Features.StaticRemoteKey, Features.AnchorOutputs))
}
test("recv WatchTxConfirmedTriggered (local commit, simple taproot channels)", Tag(ChannelStateTestsTags.OptionSimpleTaprootStaging), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
testLocalCommitTxConfirmed(f, ChannelFeatures(Features.SimpleTaprootStaging))
}
test("recv WatchTxConfirmedTriggered (local commit with multiple htlcs for the same payment)") { f =>
import f._
@ -859,6 +893,18 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(txPublished.miningFee > 0.sat) // alice is funder, she pays the fee for the remote commit
}
test("recv WatchFundingSpentTriggered (remote commit) taproot channel ", Tag(ChannelStateTestsTags.OptionSimpleTaprootStaging), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._
val bobCommitTx = bobCommitTxs.last.commitTx.tx
assert(bobCommitTx.txOut.size == 4) // two main outputs
val closingState = remoteClose(bobCommitTx, alice, alice2blockchain)
assert(closingState.claimMainOutputTx.nonEmpty)
assert(closingState.claimHtlcTxs.isEmpty)
val txPublished = txListener.expectMsgType[TransactionPublished]
assert(txPublished.tx == bobCommitTx)
assert(txPublished.miningFee > 0.sat) // alice is funder, she pays the fee for the remote commit
}
test("recv WatchFundingSpentTriggered (remote commit, public channel)", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.DoNotInterceptGossip)) { f =>
import f._
@ -1027,6 +1073,10 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
testRemoteCommitTxWithHtlcsConfirmed(f, ChannelFeatures(Features.StaticRemoteKey, Features.AnchorOutputsZeroFeeHtlcTx))
}
test("recv WatchTxConfirmedTriggered (remote commit with multiple htlcs for the same payment, simple taproot channels)", Tag(ChannelStateTestsTags.OptionSimpleTaprootStaging), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
testRemoteCommitTxWithHtlcsConfirmed(f, ChannelFeatures(Features.SimpleTaprootStaging))
}
test("recv WatchTxConfirmedTriggered (remote commit) followed by CMD_FULFILL_HTLC") { f =>
import f._
// An HTLC Bob -> Alice is cross-signed that will be fulfilled later.
@ -1074,6 +1124,56 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
alice2relayer.expectNoMessage(100 millis)
}
test("recv WatchTxConfirmedTriggered (remote commit, simple taproot channels) followed by CMD_FULFILL_HTLC", Tag(ChannelStateTestsTags.OptionSimpleTaprootStaging), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._
// An HTLC Bob -> Alice is cross-signed that will be fulfilled later.
val (r1, htlc1) = addHtlc(110000000 msat, CltvExpiryDelta(48), bob, alice, bob2alice, alice2bob)
crossSign(bob, alice, bob2alice, alice2bob)
alice2relayer.expectMsgType[RelayForward]
// An HTLC Alice -> Bob is only signed by Alice: Bob has two spendable commit tx.
val (_, htlc2) = addHtlc(95000000 msat, CltvExpiryDelta(144), alice, bob, alice2bob, bob2alice)
alice ! CMD_SIGN()
alice2bob.expectMsgType[CommitSig] // We stop here: Alice sent her CommitSig, but doesn't hear back from Bob.
// Now Bob publishes the first commit tx (force-close).
val bobCommitTx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
assert(bobCommitTx.txOut.length == 5) // two main outputs + two anchor outputs + 1 HTLC
val closingState = remoteClose(bobCommitTx, alice, alice2blockchain)
assert(closingState.claimMainOutputTx.nonEmpty)
assert(closingState.claimHtlcTxs.size == 1)
assert(getClaimHtlcSuccessTxs(closingState).isEmpty) // we don't have the preimage to claim the htlc-success yet
assert(getClaimHtlcTimeoutTxs(closingState).isEmpty)
// Alice receives the preimage for the first HTLC from downstream; she can now claim the corresponding HTLC output.
alice ! CMD_FULFILL_HTLC(htlc1.id, r1, commit = true)
alice2blockchain.expectMsgType[PublishReplaceableTx]
assert(alice2blockchain.expectMsgType[PublishFinalTx].tx == closingState.claimMainOutputTx.get.tx)
val claimHtlcSuccessTx = getClaimHtlcSuccessTxs(alice.stateData.asInstanceOf[DATA_CLOSING].remoteCommitPublished.get).head.tx
Transaction.correctlySpends(claimHtlcSuccessTx, bobCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
val publishHtlcSuccessTx = alice2blockchain.expectMsgType[PublishReplaceableTx].txInfo.asInstanceOf[ClaimHtlcSuccessTx]
assert(publishHtlcSuccessTx.tx == claimHtlcSuccessTx)
assert(publishHtlcSuccessTx.confirmationTarget == ConfirmationTarget.Absolute(htlc1.cltvExpiry.blockHeight))
// Alice resets watches on all relevant transactions.
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == bobCommitTx.txid)
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == closingState.claimMainOutputTx.get.tx.txid)
val watchHtlcSuccess = alice2blockchain.expectMsgType[WatchOutputSpent]
assert(watchHtlcSuccess.txId == bobCommitTx.txid)
assert(watchHtlcSuccess.outputIndex == claimHtlcSuccessTx.txIn.head.outPoint.index)
alice2blockchain.expectNoMessage(100 millis)
alice ! WatchTxConfirmedTriggered(BlockHeight(0), 0, bobCommitTx)
// The second htlc was not included in the commit tx published on-chain, so we can consider it failed
assert(alice2relayer.expectMsgType[RES_ADD_SETTLED[Origin, HtlcResult.OnChainFail]].htlc == htlc2)
alice ! WatchTxConfirmedTriggered(BlockHeight(0), 0, closingState.claimMainOutputTx.get.tx)
alice ! WatchTxConfirmedTriggered(BlockHeight(0), 0, claimHtlcSuccessTx)
assert(alice.stateData.asInstanceOf[DATA_CLOSING].remoteCommitPublished.get.irrevocablySpent.values.toSet == Set(bobCommitTx, closingState.claimMainOutputTx.get.tx, claimHtlcSuccessTx))
awaitCond(alice.stateName == CLOSED)
alice2blockchain.expectNoMessage(100 millis)
alice2relayer.expectNoMessage(100 millis)
}
test("recv INPUT_RESTORED (remote commit)") { f =>
import f._
@ -1196,6 +1296,26 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
awaitCond(alice.stateName == CLOSED)
}
test("recv WatchTxConfirmedTriggered (next remote commit, simple taproot channels)", Tag(ChannelStateTestsTags.OptionSimpleTaprootStaging), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._
val (bobCommitTx, closingState, htlcs) = testNextRemoteCommitTxConfirmed(f, ChannelFeatures(Features.SimpleTaprootStaging))
val claimHtlcTimeoutTxs = getClaimHtlcTimeoutTxs(closingState).map(_.tx)
alice ! WatchTxConfirmedTriggered(BlockHeight(42), 0, bobCommitTx)
alice ! WatchTxConfirmedTriggered(BlockHeight(45), 0, closingState.claimMainOutputTx.get.tx)
alice2relayer.expectNoMessage(100 millis)
alice ! WatchTxConfirmedTriggered(BlockHeight(201), 0, claimHtlcTimeoutTxs(0))
val forwardedFail1 = alice2relayer.expectMsgType[RES_ADD_SETTLED[Origin, HtlcResult.OnChainFail]].htlc
alice2relayer.expectNoMessage(250 millis)
alice ! WatchTxConfirmedTriggered(BlockHeight(202), 0, claimHtlcTimeoutTxs(1))
val forwardedFail2 = alice2relayer.expectMsgType[RES_ADD_SETTLED[Origin, HtlcResult.OnChainFail]].htlc
alice2relayer.expectNoMessage(250 millis)
alice ! WatchTxConfirmedTriggered(BlockHeight(203), 1, claimHtlcTimeoutTxs(2))
val forwardedFail3 = alice2relayer.expectMsgType[RES_ADD_SETTLED[Origin, HtlcResult.OnChainFail]].htlc
assert(Set(forwardedFail1, forwardedFail2, forwardedFail3) == htlcs)
alice2relayer.expectNoMessage(250 millis)
awaitCond(alice.stateName == CLOSED)
}
test("recv WatchTxConfirmedTriggered (next remote commit) followed by CMD_FULFILL_HTLC") { f =>
import f._
// An HTLC Bob -> Alice is cross-signed that will be fulfilled later.
@ -1537,6 +1657,10 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
testFundingSpentRevokedTx(f, ChannelFeatures(Features.StaticRemoteKey, Features.AnchorOutputsZeroFeeHtlcTx))
}
test("recv WatchFundingSpentTriggered (one revoked tx, simple taproot channels)", Tag(ChannelStateTestsTags.OptionSimpleTaprootStaging), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
testFundingSpentRevokedTx(f, ChannelFeatures(Features.SimpleTaprootStaging))
}
test("recv WatchFundingSpentTriggered (multiple revoked tx)") { f =>
import f._
val revokedCloseFixture = prepareRevokedClose(f, ChannelFeatures(Features.StaticRemoteKey))

View file

@ -121,7 +121,7 @@ class JsonSerializersSpec extends TestKitBaseClass with AnyFunSuiteLike with Mat
val dummyBytes32 = ByteVector32(hex"0202020202020202020202020202020202020202020202020202020202020202")
val localParams = LocalParams(dummyPublicKey, DeterministicWallet.KeyPath(Seq(42L)), 546 sat, Long.MaxValue.msat, Some(1000 sat), 1 msat, CltvExpiryDelta(144), 50, isChannelOpener = true, paysCommitTxFees = true, None, None, Features.empty)
val remoteParams = RemoteParams(dummyPublicKey, 546 sat, UInt64.MaxValue, Some(1000 sat), 1 msat, CltvExpiryDelta(144), 50, dummyPublicKey, dummyPublicKey, dummyPublicKey, dummyPublicKey, Features.empty, None)
val commitmentInput = Funding.makeFundingInputInfo(TxId(dummyBytes32), 0, 150_000 sat, dummyPublicKey, dummyPublicKey)
val commitmentInput = Funding.makeFundingInputInfo(TxId(dummyBytes32), 0, 150_000 sat, dummyPublicKey, dummyPublicKey, DefaultCommitmentFormat)
val localCommit = LocalCommit(0, CommitmentSpec(Set.empty, FeeratePerKw(2500 sat), 100_000_000 msat, 50_000_000 msat), CommitTxAndRemoteSig(CommitTx(commitmentInput, Transaction(2, Nil, Nil, 0)), ByteVector64.Zeroes), Nil)
val remoteCommit = RemoteCommit(0, CommitmentSpec(Set.empty, FeeratePerKw(2500 sat), 50_000_000 msat, 100_000_000 msat), TxId(dummyBytes32), dummyPublicKey)
val channelInfo = RES_GET_CHANNEL_INFO(

View file

@ -513,7 +513,7 @@ class PostRestartHtlcCleanerSpec extends TestKitBaseClass with FixtureAnyFunSuit
// commit we accept it as such, so it simplifies the test.
val revokedCommitTx = normal.commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx.copy(txOut = Seq(TxOut(4500 sat, Script.pay2wpkh(randomKey().publicKey))))
val dummyClaimMainTx = Transaction(2, Seq(TxIn(OutPoint(revokedCommitTx, 0), Nil, 0)), Seq(revokedCommitTx.txOut.head.copy(amount = 4000 sat)), 0)
val dummyClaimMain = ClaimRemoteDelayedOutputTx(InputInfo(OutPoint(revokedCommitTx, 0), revokedCommitTx.txOut.head, Nil), dummyClaimMainTx)
val dummyClaimMain = ClaimRemoteDelayedOutputTx(InputInfo(OutPoint(revokedCommitTx, 0), revokedCommitTx.txOut.head, Script.write(Script.pay2wpkh(randomKey().publicKey))), dummyClaimMainTx)
val rcp = RevokedCommitPublished(revokedCommitTx, Some(dummyClaimMain), None, Nil, Nil, Map(revokedCommitTx.txIn.head.outPoint -> revokedCommitTx))
DATA_CLOSING(normal.commitments, BlockHeight(0), Script.write(Script.pay2wpkh(randomKey().publicKey)), mutualCloseProposed = Nil, revokedCommitPublished = List(rcp))
}

View file

@ -23,6 +23,7 @@ import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel.ChannelFeatures
import fr.acinq.eclair.channel.Helpers.Funding
import fr.acinq.eclair.crypto.Generators
import fr.acinq.eclair.transactions.Transactions.InputInfo.SegwitInput
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.wire.protocol.UpdateAddHtlc
import fr.acinq.eclair.{CltvExpiry, CltvExpiryDelta, Features, MilliSatoshi, MilliSatoshiLong, TestConstants}
@ -124,7 +125,7 @@ trait TestVectorsSpec extends AnyFunSuite with Logging {
val fundingAmount = fundingTx.txOut(0).amount
logger.info(s"# funding-tx: $fundingTx}")
val commitmentInput = Funding.makeFundingInputInfo(fundingTx.txid, 0, fundingAmount, Local.funding_pubkey, Remote.funding_pubkey)
val commitmentInput = Funding.makeFundingInputInfo(fundingTx.txid, 0, fundingAmount, Local.funding_pubkey, Remote.funding_pubkey, DefaultCommitmentFormat)
val obscured_tx_number = Transactions.obscuredCommitTxNumber(42, localIsChannelOpener = true, Local.payment_basepoint, Remote.payment_basepoint)
assert(obscured_tx_number == (0x2bb038521914L ^ 42L))
@ -140,8 +141,8 @@ trait TestVectorsSpec extends AnyFunSuite with Logging {
logger.info(s"remotekey: ${Remote.payment_privkey.publicKey}")
logger.info(s"local_delayedkey: ${Local.delayed_payment_privkey.publicKey}")
logger.info(s"local_revocation_key: ${Local.revocation_pubkey}")
logger.info(s"# funding wscript = ${commitmentInput.redeemScript}")
assert(commitmentInput.redeemScript == hex"5221023da092f6980e58d2c037173180e9a465476026ee50f96695963e8efe436f54eb21030e9f7b623d2ccc7c9bd44d66d5ce21ce504c0acf6385a132cec6d3c39fa711c152ae")
logger.info(s"# funding wscript = ${commitmentInput.asInstanceOf[SegwitInput].redeemScript}")
assert(commitmentInput.asInstanceOf[SegwitInput].redeemScript == hex"5221023da092f6980e58d2c037173180e9a465476026ee50f96695963e8efe436f54eb21030e9f7b623d2ccc7c9bd44d66d5ce21ce504c0acf6385a132cec6d3c39fa711c152ae")
val paymentPreimages = Seq(
ByteVector32(hex"0000000000000000000000000000000000000000000000000000000000000000"),

View file

@ -52,7 +52,7 @@ class TransactionsSpec extends AnyFunSuite with Logging {
val localHtlcPriv = PrivateKey(randomBytes32())
val remoteHtlcPriv = PrivateKey(randomBytes32())
val finalPubKeyScript = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32()).publicKey))
val commitInput = Funding.makeFundingInputInfo(randomTxId(), 0, Btc(1), localFundingPriv.publicKey, remoteFundingPriv.publicKey)
val commitInput = Funding.makeFundingInputInfo(randomTxId(), 0, Btc(1), localFundingPriv.publicKey, remoteFundingPriv.publicKey, DefaultCommitmentFormat)
val toLocalDelay = CltvExpiryDelta(144)
val localDustLimit = Satoshi(546)
val feeratePerKw = FeeratePerKw(22000 sat)
@ -258,7 +258,7 @@ class TransactionsSpec extends AnyFunSuite with Logging {
test("generate valid commitment and htlc transactions (default commitment format)") {
val finalPubKeyScript = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32()).publicKey))
val commitInput = Funding.makeFundingInputInfo(randomTxId(), 0, Btc(1), localFundingPriv.publicKey, remoteFundingPriv.publicKey)
val commitInput = Funding.makeFundingInputInfo(randomTxId(), 0, Btc(1), localFundingPriv.publicKey, remoteFundingPriv.publicKey, DefaultCommitmentFormat)
// htlc1 and htlc2 are regular IN/OUT htlcs
val paymentPreimage1 = randomBytes32()
@ -405,10 +405,9 @@ class TransactionsSpec extends AnyFunSuite with Logging {
{
// remote spends offered HTLC output with revocation key
val script = Script.write(Scripts.htlcOffered(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, Crypto.ripemd160(htlc1.paymentHash), DefaultCommitmentFormat))
val Some(htlcOutputIndex) = outputs.zipWithIndex.find {
case (CommitmentOutputLink(_, _, OutHtlc(OutgoingHtlc(someHtlc))), _) => someHtlc.id == htlc1.id
case _ => false
}.map(_._2)
val Some(htlcOutputIndex) = outputs.map(_.filter[OutHtlc]).zipWithIndex.collectFirst {
case (Some(co), outputIndex) if co.commitmentOutput.outgoingHtlc.add.id == htlc1.id => outputIndex
}
val Right(htlcPenaltyTx) = makeHtlcPenaltyTx(commitTx.tx, htlcOutputIndex, script, localDustLimit, finalPubKeyScript, feeratePerKw)
val sig = htlcPenaltyTx.sign(localRevocationPriv, TxOwner.Local, DefaultCommitmentFormat)
val signed = addSigs(htlcPenaltyTx, sig, localRevocationPriv.publicKey)
@ -427,10 +426,9 @@ class TransactionsSpec extends AnyFunSuite with Logging {
{
// remote spends received HTLC output with revocation key
val script = Script.write(Scripts.htlcReceived(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, Crypto.ripemd160(htlc2.paymentHash), htlc2.cltvExpiry, DefaultCommitmentFormat))
val Some(htlcOutputIndex) = outputs.zipWithIndex.find {
case (CommitmentOutputLink(_, _, InHtlc(IncomingHtlc(someHtlc))), _) => someHtlc.id == htlc2.id
case _ => false
}.map(_._2)
val Some(htlcOutputIndex) = outputs.map(_.filter[InHtlc]).zipWithIndex.collectFirst {
case (Some(co), outputIndex) if co.commitmentOutput.incomingHtlc.add.id == htlc2.id => outputIndex
}
val Right(htlcPenaltyTx) = makeHtlcPenaltyTx(commitTx.tx, htlcOutputIndex, script, localDustLimit, finalPubKeyScript, feeratePerKw)
val sig = htlcPenaltyTx.sign(localRevocationPriv, TxOwner.Local, DefaultCommitmentFormat)
val signed = addSigs(htlcPenaltyTx, sig, localRevocationPriv.publicKey)
@ -489,7 +487,7 @@ class TransactionsSpec extends AnyFunSuite with Logging {
test("generate valid commitment and htlc transactions (anchor outputs)") {
val finalPubKeyScript = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32()).publicKey))
val commitInput = Funding.makeFundingInputInfo(randomTxId(), 0, Btc(1), localFundingPriv.publicKey, remoteFundingPriv.publicKey)
val commitInput = Funding.makeFundingInputInfo(randomTxId(), 0, Btc(1), localFundingPriv.publicKey, remoteFundingPriv.publicKey, DefaultCommitmentFormat)
// htlc1, htlc2a and htlc2b are regular IN/OUT htlcs
val paymentPreimage1 = randomBytes32()
@ -719,10 +717,9 @@ class TransactionsSpec extends AnyFunSuite with Logging {
{
// remote spends offered htlc output with revocation key
val script = Script.write(Scripts.htlcOffered(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, Crypto.ripemd160(htlc1.paymentHash), UnsafeLegacyAnchorOutputsCommitmentFormat))
val Some(htlcOutputIndex) = commitTxOutputs.zipWithIndex.find {
case (CommitmentOutputLink(_, _, OutHtlc(OutgoingHtlc(someHtlc))), _) => someHtlc.id == htlc1.id
case _ => false
}.map(_._2)
val Some(htlcOutputIndex) = commitTxOutputs.map(_.filter[OutHtlc]).zipWithIndex.collectFirst {
case (Some(co), outputIndex) if co.commitmentOutput.outgoingHtlc.add.id == htlc1.id => outputIndex
}
val Right(htlcPenaltyTx) = makeHtlcPenaltyTx(commitTx.tx, htlcOutputIndex, script, localDustLimit, finalPubKeyScript, feeratePerKw)
val sig = htlcPenaltyTx.sign(localRevocationPriv, TxOwner.Local, UnsafeLegacyAnchorOutputsCommitmentFormat)
val signed = addSigs(htlcPenaltyTx, sig, localRevocationPriv.publicKey)
@ -732,10 +729,9 @@ class TransactionsSpec extends AnyFunSuite with Logging {
// remote spends received htlc output with revocation key
for (htlc <- Seq(htlc2a, htlc2b)) {
val script = Script.write(Scripts.htlcReceived(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, Crypto.ripemd160(htlc.paymentHash), htlc.cltvExpiry, UnsafeLegacyAnchorOutputsCommitmentFormat))
val Some(htlcOutputIndex) = commitTxOutputs.zipWithIndex.find {
case (CommitmentOutputLink(_, _, InHtlc(IncomingHtlc(someHtlc))), _) => someHtlc.id == htlc.id
case _ => false
}.map(_._2)
val Some(htlcOutputIndex) = commitTxOutputs.map(_.filter[InHtlc]).zipWithIndex.collectFirst {
case (Some(co), outputIndex) if co.commitmentOutput.incomingHtlc.add.id == htlc.id => outputIndex
}
val Right(htlcPenaltyTx) = makeHtlcPenaltyTx(commitTx.tx, htlcOutputIndex, script, localDustLimit, finalPubKeyScript, feeratePerKw)
val sig = htlcPenaltyTx.sign(localRevocationPriv, TxOwner.Local, UnsafeLegacyAnchorOutputsCommitmentFormat)
val signed = addSigs(htlcPenaltyTx, sig, localRevocationPriv.publicKey)
@ -744,7 +740,253 @@ class TransactionsSpec extends AnyFunSuite with Logging {
}
}
test("generate valid commitment and htlc transactions (taproot)") {
test("generate valid commitment and htlc transactions (simple taproot channels)") {
def test(commitmentFormat: CommitmentFormat): Unit = {
val finalPubKeyScript = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32()).publicKey))
// funding tx sends to musig2 aggregate of local and remote funding keys
val fundingTxOutpoint = OutPoint(randomTxId(), 0)
val fundingOutput = TxOut(Btc(1), Script.pay2tr(Taproot.musig2Aggregate(localFundingPriv.publicKey, remoteFundingPriv.publicKey), None))
val commitInput = Funding.makeFundingInputInfo(fundingTxOutpoint.txid, fundingTxOutpoint.index.toInt, Btc(1), localFundingPriv.publicKey, remoteFundingPriv.publicKey, commitmentFormat)
// htlc1, htlc2 are regular IN/OUT htlcs
val paymentPreimage1 = randomBytes32()
val htlc1 = UpdateAddHtlc(ByteVector32.Zeroes, 0, MilliBtc(100).toMilliSatoshi, sha256(paymentPreimage1), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0, None)
val paymentPreimage2 = randomBytes32()
val htlc2 = UpdateAddHtlc(ByteVector32.Zeroes, 1, MilliBtc(50).toMilliSatoshi, sha256(paymentPreimage2), CltvExpiry(310), TestConstants.emptyOnionPacket, None, 1.0, None)
val spec = CommitmentSpec(
htlcs = Set(
OutgoingHtlc(htlc1),
IncomingHtlc(htlc2),
),
commitTxFeerate = feeratePerKw,
toLocal = 400.millibtc.toMilliSatoshi,
toRemote = 300.millibtc.toMilliSatoshi)
val (secretLocalNonce, publicLocalNonce) = Musig2.generateNonce(randomBytes32(), localFundingPriv, Seq(localFundingPriv.publicKey))
val (secretRemoteNonce, publicRemoteNonce) = Musig2.generateNonce(randomBytes32(), remoteFundingPriv, Seq(remoteFundingPriv.publicKey))
val publicKeys = Scripts.sort(Seq(localFundingPriv.publicKey, remoteFundingPriv.publicKey))
val publicNonces = Seq(publicLocalNonce, publicRemoteNonce)
val (commitTx, commitTxOutputs, htlcTimeoutTxs, htlcSuccessTxs) = {
val commitTxNumber = 0x404142434445L
val outputs = makeCommitTxOutputs(localPaysCommitTxFees = true, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, remotePaymentPriv.publicKey, localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localFundingPriv.publicKey, remoteFundingPriv.publicKey, spec, commitmentFormat)
val txInfo = makeCommitTx(commitInput, commitTxNumber, localPaymentPriv.publicKey, remotePaymentPriv.publicKey, localIsChannelOpener = true, outputs)
val commitTx = if (commitmentFormat.useTaproot) {
val Right(sig) = for {
localPartialSig <- Musig2.signTaprootInput(localFundingPriv, txInfo.tx, 0, Seq(fundingOutput), publicKeys, secretLocalNonce, publicNonces, None)
remotePartialSig <- Musig2.signTaprootInput(remoteFundingPriv, txInfo.tx, 0, Seq(fundingOutput), publicKeys, secretRemoteNonce, publicNonces, None)
sig <- Musig2.aggregateTaprootSignatures(Seq(localPartialSig, remotePartialSig), txInfo.tx, 0, Seq(fundingOutput), publicKeys, publicNonces, None)
} yield sig
Transactions.addAggregatedSignature(txInfo, sig)
} else {
val localSig = txInfo.sign(localPaymentPriv, TxOwner.Local, commitmentFormat)
val remoteSig = txInfo.sign(remotePaymentPriv, TxOwner.Remote, commitmentFormat)
Transactions.addSigs(txInfo, localFundingPriv.publicKey, remoteFundingPriv.publicKey, localSig, remoteSig)
}
val htlcTxs = makeHtlcTxs(commitTx.tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, spec.htlcTxFeerate(commitmentFormat), outputs, commitmentFormat)
assert(htlcTxs.length == 2)
val htlcSuccessTxs = htlcTxs.collect { case tx: HtlcSuccessTx => tx }
val htlcTimeoutTxs = htlcTxs.collect { case tx: HtlcTimeoutTx => tx }
assert(htlcTimeoutTxs.size == 1) // htlc1
assert(htlcTimeoutTxs.map(_.htlcId).toSet == Set(0))
assert(htlcSuccessTxs.size == 1) // htlc2
assert(htlcSuccessTxs.map(_.htlcId).toSet == Set(1))
(commitTx, outputs, htlcTimeoutTxs, htlcSuccessTxs)
}
{
// local spends main delayed output
val Right(claimMainOutputTx) = makeClaimLocalDelayedOutputTx(commitTx.tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
val localSig = claimMainOutputTx.sign(localDelayedPaymentPriv, TxOwner.Local, commitmentFormat)
val signedTx = addSigs(claimMainOutputTx, localSig)
assert(checkSpendable(signedTx).isSuccess)
}
{
// remote cannot spend main output with default commitment format
val Left(failure) = makeClaimP2WPKHOutputTx(commitTx.tx, localDustLimit, remotePaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
assert(failure == OutputNotFound)
}
{
// remote spends main delayed output
val Right(claimRemoteDelayedOutputTx) = makeClaimRemoteDelayedOutputTx(commitTx.tx, localDustLimit, remotePaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
val localSig = claimRemoteDelayedOutputTx.sign(remotePaymentPriv, TxOwner.Local, commitmentFormat)
val signedTx = addSigs(claimRemoteDelayedOutputTx, localSig)
assert(checkSpendable(signedTx).isSuccess)
}
{
// local spends local anchor
val anchorKey = if (commitmentFormat.useTaproot) {
localDelayedPaymentPriv
} else {
localFundingPriv
}
val Right(claimAnchorOutputTx) = makeClaimLocalAnchorOutputTx(commitTx.tx, anchorKey.publicKey, ConfirmationTarget.Absolute(BlockHeight(0)))
assert(checkSpendable(claimAnchorOutputTx).isFailure)
val localSig = claimAnchorOutputTx.sign(anchorKey, TxOwner.Local, commitmentFormat)
val signedTx = addSigs(claimAnchorOutputTx, localSig)
assert(checkSpendable(signedTx).isSuccess)
}
{
// remote spends remote anchor
val anchorKey = if (commitmentFormat.useTaproot) {
remotePaymentPriv
} else {
remoteFundingPriv
}
val Right(claimAnchorOutputTx) = makeClaimLocalAnchorOutputTx(commitTx.tx, anchorKey.publicKey, ConfirmationTarget.Absolute(BlockHeight(0)))
assert(checkSpendable(claimAnchorOutputTx).isFailure)
val localSig = claimAnchorOutputTx.sign(anchorKey, TxOwner.Local, commitmentFormat)
val signedTx = addSigs(claimAnchorOutputTx, localSig)
assert(checkSpendable(signedTx).isSuccess)
}
{
// remote spends local main delayed output with revocation key
val Right(mainPenaltyTx) = makeMainPenaltyTx(commitTx.tx, localDustLimit, localRevocationPriv.publicKey, finalPubKeyScript, toLocalDelay, localDelayedPaymentPriv.publicKey, feeratePerKw)
val sig = mainPenaltyTx.sign(localRevocationPriv, TxOwner.Local, commitmentFormat)
val signed = addSigs(mainPenaltyTx, sig)
assert(checkSpendable(signed).isSuccess)
}
{
// local spends received htlc with HTLC-timeout tx
for (htlcTimeoutTx <- htlcTimeoutTxs) {
val localSig = htlcTimeoutTx.sign(localHtlcPriv, TxOwner.Local, commitmentFormat)
val remoteSig = htlcTimeoutTx.sign(remoteHtlcPriv, TxOwner.Remote, commitmentFormat)
val signedTx = addSigs(htlcTimeoutTx, localSig, remoteSig, commitmentFormat)
assert(checkSpendable(signedTx).isSuccess)
// local detects when remote doesn't use the right sighash flags
val invalidSighash = Seq(SIGHASH_ALL, SIGHASH_ALL | SIGHASH_ANYONECANPAY, SIGHASH_SINGLE, SIGHASH_NONE)
for (sighash <- invalidSighash) {
val invalidRemoteSig = htlcTimeoutTx.sign(remoteHtlcPriv, sighash)
val invalidTx = addSigs(htlcTimeoutTx, localSig, invalidRemoteSig, commitmentFormat)
assert(checkSpendable(invalidTx).isFailure)
}
}
}
{
// local spends delayed output of htlc1 timeout tx
val Right(htlcDelayed) = makeHtlcDelayedTx(htlcTimeoutTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
val localSig = htlcDelayed.sign(localDelayedPaymentPriv, TxOwner.Local, commitmentFormat)
val signedTx = addSigs(htlcDelayed, localSig)
assert(checkSpendable(signedTx).isSuccess)
// local can't claim delayed output of htlc3 timeout tx because it is below the dust limit
val htlcDelayed1 = makeHtlcDelayedTx(htlcTimeoutTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
assert(htlcDelayed1 == Left(OutputNotFound))
}
{
// local spends offered htlc with HTLC-success tx
for ((htlcSuccessTx, paymentPreimage) <- (htlcSuccessTxs(0), paymentPreimage2) :: Nil) {
val localSig = htlcSuccessTx.sign(localHtlcPriv, TxOwner.Local, commitmentFormat)
val remoteSig = htlcSuccessTx.sign(remoteHtlcPriv, TxOwner.Remote, commitmentFormat)
val signedTx = addSigs(htlcSuccessTx, localSig, remoteSig, paymentPreimage, commitmentFormat)
assert(checkSpendable(signedTx).isSuccess)
// check remote sig
assert(htlcSuccessTx.checkSig(remoteSig, remoteHtlcPriv.publicKey, TxOwner.Remote, commitmentFormat))
// local detects when remote doesn't use the right sighash flags
val invalidSighash = Seq(SIGHASH_ALL, SIGHASH_ALL | SIGHASH_ANYONECANPAY, SIGHASH_SINGLE, SIGHASH_NONE)
for (sighash <- invalidSighash) {
val invalidRemoteSig = htlcSuccessTx.sign(remoteHtlcPriv, sighash)
val invalidTx = addSigs(htlcSuccessTx, localSig, invalidRemoteSig, paymentPreimage, commitmentFormat)
assert(checkSpendable(invalidTx).isFailure)
assert(!invalidTx.checkSig(invalidRemoteSig, remoteHtlcPriv.publicKey, TxOwner.Remote, commitmentFormat))
}
}
}
{
// local spends delayed output of htlc2 success tx
val Right(htlcDelayedA) = makeHtlcDelayedTx(htlcSuccessTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
for (htlcDelayed <- Seq(htlcDelayedA)) {
val localSig = htlcDelayed.sign(localDelayedPaymentPriv, TxOwner.Local, commitmentFormat)
val signedTx = addSigs(htlcDelayed, localSig)
assert(checkSpendable(signedTx).isSuccess)
}
}
{
// remote spends local->remote htlc outputs directly in case of success
for ((htlc, paymentPreimage) <- (htlc1, paymentPreimage1) :: Nil) {
val Right(claimHtlcSuccessTx) = makeClaimHtlcSuccessTx(commitTx.tx, commitTxOutputs, localDustLimit, remoteHtlcPriv.publicKey, localHtlcPriv.publicKey, localRevocationPriv.publicKey, finalPubKeyScript, htlc, feeratePerKw, commitmentFormat)
val localSig = claimHtlcSuccessTx.sign(remoteHtlcPriv, TxOwner.Local, commitmentFormat)
val signed = addSigs(claimHtlcSuccessTx, localSig, paymentPreimage)
assert(checkSpendable(signed).isSuccess)
}
}
{
// remote spends htlc1's htlc-timeout tx with revocation key
val Seq(Right(claimHtlcDelayedPenaltyTx)) = makeClaimHtlcDelayedOutputPenaltyTxs(htlcTimeoutTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw, commitmentFormat)
val sig = claimHtlcDelayedPenaltyTx.sign(localRevocationPriv, TxOwner.Local, commitmentFormat)
val signed = addSigs(claimHtlcDelayedPenaltyTx, sig)
assert(checkSpendable(signed).isSuccess)
}
{
// remote spends remote->local htlc output directly in case of timeout
for (htlc <- Seq(htlc2)) {
val Right(claimHtlcTimeoutTx) = makeClaimHtlcTimeoutTx(commitTx.tx, commitTxOutputs, localDustLimit, remoteHtlcPriv.publicKey, localHtlcPriv.publicKey, localRevocationPriv.publicKey, finalPubKeyScript, htlc, feeratePerKw, commitmentFormat)
val localSig = claimHtlcTimeoutTx.sign(remoteHtlcPriv, TxOwner.Local, commitmentFormat)
val signed = addSigs(claimHtlcTimeoutTx, localSig)
assert(checkSpendable(signed).isSuccess)
}
}
{
// remote spends htlc2's htlc-success tx with revocation key
val Seq(Right(claimHtlcDelayedPenaltyTxA)) = makeClaimHtlcDelayedOutputPenaltyTxs(htlcSuccessTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw, commitmentFormat)
for (claimHtlcSuccessPenaltyTx <- Seq(claimHtlcDelayedPenaltyTxA)) {
val sig = claimHtlcSuccessPenaltyTx.sign(localRevocationPriv, TxOwner.Local, commitmentFormat)
val signed = addSigs(claimHtlcSuccessPenaltyTx, sig)
assert(checkSpendable(signed).isSuccess)
}
}
{
// remote spends all htlc txs aggregated in a single tx
val txIn = htlcTimeoutTxs.flatMap(_.tx.txIn) ++ htlcSuccessTxs.flatMap(_.tx.txIn)
val txOut = htlcTimeoutTxs.flatMap(_.tx.txOut) ++ htlcSuccessTxs.flatMap(_.tx.txOut)
val aggregatedHtlcTx = Transaction(2, txIn, txOut, 0)
val claimHtlcDelayedPenaltyTxs = makeClaimHtlcDelayedOutputPenaltyTxs(aggregatedHtlcTx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw, commitmentFormat)
assert(claimHtlcDelayedPenaltyTxs.size == 2)
val claimed = claimHtlcDelayedPenaltyTxs.collect { case Right(tx) => tx }
assert(claimed.size == 2)
assert(claimed.map(_.input.outPoint).toSet.size == 2)
}
{
// remote spends offered htlc output with revocation key
val Some(htlcOutputIndex) = commitTxOutputs.map(_.filter[OutHtlc]).zipWithIndex.collectFirst {
case (Some(co), outputIndex) if co.commitmentOutput.outgoingHtlc.add.id == htlc1.id => outputIndex
}
val Right(htlcPenaltyTx) = if (commitmentFormat.useTaproot) {
val scriptTree = Taproot.offeredHtlcScriptTree(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, htlc1.paymentHash)
makeHtlcPenaltyTx(commitTx.tx, htlcOutputIndex, localRevocationPriv.publicKey.xOnly, Some(scriptTree), localDustLimit, finalPubKeyScript, feeratePerKw)
} else {
val script = Script.write(Scripts.htlcOffered(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, Crypto.ripemd160(htlc1.paymentHash), commitmentFormat))
makeHtlcPenaltyTx(commitTx.tx, htlcOutputIndex, script, localDustLimit, finalPubKeyScript, feeratePerKw)
}
val sig = htlcPenaltyTx.sign(localRevocationPriv, TxOwner.Local, commitmentFormat)
val signed = addSigs(htlcPenaltyTx, sig, localRevocationPriv.publicKey)
assert(checkSpendable(signed).isSuccess)
}
{
// remote spends received htlc output with revocation key
for (htlc <- Seq(htlc2)) {
val Some(htlcOutputIndex) = commitTxOutputs.map(_.filter[InHtlc]).zipWithIndex.collectFirst {
case (Some(co), outputIndex) if co.commitmentOutput.incomingHtlc.add.id == htlc.id => outputIndex
}
val Right(htlcPenaltyTx) = if (commitmentFormat.useTaproot) {
val scriptTree = Taproot.receivedHtlcScriptTree(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, htlc.paymentHash, htlc.cltvExpiry)
makeHtlcPenaltyTx(commitTx.tx, htlcOutputIndex, localRevocationPriv.publicKey.xOnly, Some(scriptTree), localDustLimit, finalPubKeyScript, feeratePerKw)
} else {
val script = Script.write(Scripts.htlcReceived(localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localRevocationPriv.publicKey, Crypto.ripemd160(htlc.paymentHash), htlc.cltvExpiry, commitmentFormat))
makeHtlcPenaltyTx(commitTx.tx, htlcOutputIndex, script, localDustLimit, finalPubKeyScript, feeratePerKw)
}
val sig = htlcPenaltyTx.sign(localRevocationPriv, TxOwner.Local, commitmentFormat)
val signed = addSigs(htlcPenaltyTx, sig, localRevocationPriv.publicKey)
assert(checkSpendable(signed).isSuccess)
}
}
}
test(SimpleTaprootChannelsStagingCommitmentFormat)
test(SimpleTaprootChannelsStagingLegacyCommitmentFormat)
}
test("generate valid commitment and htlc transactions (taproot - unit test for low-level helpers)") {
import fr.acinq.bitcoin.scalacompat.KotlinUtils._
import fr.acinq.eclair.transactions.Scripts.Taproot
@ -1012,7 +1254,7 @@ class TransactionsSpec extends AnyFunSuite with Logging {
val remotePaymentPriv = PrivateKey(hex"a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6a6")
val localHtlcPriv = PrivateKey(hex"a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7a7")
val remoteHtlcPriv = PrivateKey(hex"a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8a8")
val commitInput = Funding.makeFundingInputInfo(TxId.fromValidHex("a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0"), 0, Btc(1), localFundingPriv.publicKey, remoteFundingPriv.publicKey)
val commitInput = Funding.makeFundingInputInfo(TxId.fromValidHex("a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0a0"), 0, Btc(1), localFundingPriv.publicKey, remoteFundingPriv.publicKey, DefaultCommitmentFormat)
// htlc1 and htlc2 are two regular incoming HTLCs with different amounts.
// htlc2 and htlc3 have the same amounts and should be sorted according to their scriptPubKey
@ -1072,7 +1314,7 @@ class TransactionsSpec extends AnyFunSuite with Logging {
}
test("find our output in closing tx") {
val commitInput = Funding.makeFundingInputInfo(randomTxId(), 0, Btc(1), localFundingPriv.publicKey, remoteFundingPriv.publicKey)
val commitInput = Funding.makeFundingInputInfo(randomTxId(), 0, Btc(1), localFundingPriv.publicKey, remoteFundingPriv.publicKey, DefaultCommitmentFormat)
val localPubKeyScript = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32()).publicKey))
val remotePubKeyScript = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32()).publicKey))

File diff suppressed because one or more lines are too long

View file

@ -124,7 +124,7 @@ class ChannelCodecs4Spec extends AnyFunSuite {
test("encode/decode rbf status") {
val channelId = randomBytes32()
val fundingInput = InputInfo(OutPoint(randomTxId(), 3), TxOut(175_000 sat, Script.pay2wpkh(randomKey().publicKey)), Nil)
val fundingInput = InputInfo(OutPoint(randomTxId(), 3), TxOut(175_000 sat, Script.pay2wpkh(randomKey().publicKey)), hex"deadbeef")
val fundingTx = SharedTransaction(
sharedInput_opt = None,
sharedOutput = InteractiveTxBuilder.Output.Shared(UInt64(8), ByteVector.empty, 100_000_600 msat, 74_000_400 msat, 0 msat),

View file

@ -212,9 +212,9 @@ class LightningMessageCodecsSpec extends AnyFunSuite {
TxRemoveOutput(channelId1, UInt64(1)) -> hex"0045 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 0000000000000001",
TxComplete(channelId1) -> hex"0046 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa",
TxComplete(channelId1, TlvStream(Set.empty[TxCompleteTlv], Set(GenericTlv(UInt64(231), hex"deadbeef"), GenericTlv(UInt64(507), hex"")))) -> hex"0046 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa e704deadbeef fd01fb00",
TxSignatures(channelId1, tx2, Seq(ScriptWitness(Seq(hex"68656c6c6f2074686572652c2074686973206973206120626974636f6e212121", hex"82012088a820add57dfe5277079d069ca4ad4893c96de91f88ffb981fdc6a2a34d5336c66aff87")), ScriptWitness(Seq(hex"304402207de9ba56bb9f641372e805782575ee840a899e61021c8b1572b3ec1d5b5950e9022069e9ba998915dae193d3c25cb89b5e64370e6a3a7755e7f31cf6d7cbc2a49f6d01", hex"034695f5b7864c580bf11f9f8cb1a94eb336f2ce9ef872d2ae1a90ee276c772484"))), None) -> hex"0047 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa fc7aa8845f192959202c1b7ff704e7cbddded463c05e844676a94ccb4bed69f1 0002 004a 022068656c6c6f2074686572652c2074686973206973206120626974636f6e2121212782012088a820add57dfe5277079d069ca4ad4893c96de91f88ffb981fdc6a2a34d5336c66aff87 006b 0247304402207de9ba56bb9f641372e805782575ee840a899e61021c8b1572b3ec1d5b5950e9022069e9ba998915dae193d3c25cb89b5e64370e6a3a7755e7f31cf6d7cbc2a49f6d0121034695f5b7864c580bf11f9f8cb1a94eb336f2ce9ef872d2ae1a90ee276c772484",
TxSignatures(channelId2, tx1, Nil, None) -> hex"0047 bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb 1f2ec025a33e39ef8e177afcdc1adc855bf128dc906182255aeb64efa825f106 0000",
TxSignatures(channelId2, tx1, Nil, Some(signature)) -> hex"0047 bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb 1f2ec025a33e39ef8e177afcdc1adc855bf128dc906182255aeb64efa825f106 0000 fd0259 40 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb",
TxSignatures(channelId1, tx2, Seq(ScriptWitness(Seq(hex"68656c6c6f2074686572652c2074686973206973206120626974636f6e212121", hex"82012088a820add57dfe5277079d069ca4ad4893c96de91f88ffb981fdc6a2a34d5336c66aff87")), ScriptWitness(Seq(hex"304402207de9ba56bb9f641372e805782575ee840a899e61021c8b1572b3ec1d5b5950e9022069e9ba998915dae193d3c25cb89b5e64370e6a3a7755e7f31cf6d7cbc2a49f6d01", hex"034695f5b7864c580bf11f9f8cb1a94eb336f2ce9ef872d2ae1a90ee276c772484"))), None, None) -> hex"0047 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa fc7aa8845f192959202c1b7ff704e7cbddded463c05e844676a94ccb4bed69f1 0002 004a 022068656c6c6f2074686572652c2074686973206973206120626974636f6e2121212782012088a820add57dfe5277079d069ca4ad4893c96de91f88ffb981fdc6a2a34d5336c66aff87 006b 0247304402207de9ba56bb9f641372e805782575ee840a899e61021c8b1572b3ec1d5b5950e9022069e9ba998915dae193d3c25cb89b5e64370e6a3a7755e7f31cf6d7cbc2a49f6d0121034695f5b7864c580bf11f9f8cb1a94eb336f2ce9ef872d2ae1a90ee276c772484",
TxSignatures(channelId2, tx1, Nil, None, None) -> hex"0047 bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb 1f2ec025a33e39ef8e177afcdc1adc855bf128dc906182255aeb64efa825f106 0000",
TxSignatures(channelId2, tx1, Nil, Some(signature), None) -> hex"0047 bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb 1f2ec025a33e39ef8e177afcdc1adc855bf128dc906182255aeb64efa825f106 0000 fd0259 40 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb",
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",
TxInitRbf(channelId1, 0, FeeratePerKw(4000 sat), 0 sat, requireConfirmedInputs = false, None) -> hex"0048 aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 00000000 00000fa0 00080000000000000000",
@ -291,6 +291,11 @@ class LightningMessageCodecsSpec extends AnyFunSuite {
}
}
test("decode open_channel with simple_taproot_channel extension") {
val raw = "06226e46111a0b59caaf126043eb5bbf28c34f3a5e332a1fc7b2b73cf188910f85c4f4bf75b2cb938d4c3e75bd53949f12d708b0b8d6db817e10ac3437ffb29f00000000000186a0000000000000000000000000000001620000000005e69ec000000000000003e80000000000000001000009c4009001e303d01507c5d81a04650898e6ce017a3ed8349b83dd1f592e7ec8b9d6bdb064950c02a54a8591a5fdc5f082f23d0f3e83ff74b6de433f71e40123c44b20a56a5bb9f502a8e31e0707b1ac67b9fd938e5c9d59e3607fb84e0ab6e0824ad582e4f8f88df802721e2a2757ff1c60a92716a366f89c3a7df6a48e71bc8824e23b1ae47d9f596503df8191d861c265ab1f0539bdc04f8ac94847511abd6c70ed0775aea3f6c3821202c2fdb53245754e0e033a71e260e64f0c0959ac4a994e9c5159708ae05559e9ad00000001171000000000000000000000000000000000000000000000044203a8c947da4dae605ee05f7894e22a9d6d51e23c5523e63f8fc5dc7aea90835a9403f68dbb02e8cba1a97ea42bd6a963942187ff0da465dda3dc35cf0d260bcdcece"
assert(openChannelCodec.decode(BitVector.fromValidHex(raw)).isSuccessful)
}
test("decode invalid open_channel") {
val defaultEncoded = hex"000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001000000000000000100000000000000010000000000000001000000000000000100000000000000010000000100010001031b84c5567b126440995d3ed5aaba0565d71e1834604819ff9c17f5e9d5dd078f024d4b6cd1361032ca9bd2aeb9d900aa4d45d9ead80ac9423374c451a7254d076602531fe6068134503d2723133227c867ac8fa6c83c537e9a44c3c5bdbdcb1fe33703462779ad4aad39514614751a71085f2f10e1c7a593e4e030efb5b8721ce55b0b0362c0a046dacce86ddd0343c6d3c7c79c2208ba0d9c9cf24a6d046d21d21f90f703f006a18d5653c4edf5391ff23a61f03ff83d237e880ee61187fa9f379a028e0a00"
val testCases = Seq(

View file

@ -48,7 +48,11 @@ trait Channel {
ChannelTypes.AnchorOutputsZeroFeeHtlcTx(),
ChannelTypes.AnchorOutputsZeroFeeHtlcTx(zeroConf = true),
ChannelTypes.AnchorOutputsZeroFeeHtlcTx(scidAlias = true),
ChannelTypes.AnchorOutputsZeroFeeHtlcTx(scidAlias = true, zeroConf = true)
ChannelTypes.AnchorOutputsZeroFeeHtlcTx(scidAlias = true, zeroConf = true),
ChannelTypes.SimpleTaprootChannelsStaging(),
ChannelTypes.SimpleTaprootChannelsStaging(zeroConf = true),
ChannelTypes.SimpleTaprootChannelsStaging(scidAlias = true),
ChannelTypes.SimpleTaprootChannelsStaging(scidAlias = true, zeroConf = true),
).map(ct => ct.toString -> ct).toMap // we use the toString method as name in the api
val open: Route = postRequest("open") { implicit t =>