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:
parent
c7a288b91f
commit
a7348f8060
54 changed files with 3580 additions and 609 deletions
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
|
@ -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)
|
||||
|
|
|
@ -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] = {
|
||||
|
|
|
@ -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 = {
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
*
|
||||
|
|
|
@ -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
|
||||
*
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
|
|
File diff suppressed because it is too large
Load diff
|
@ -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)
|
||||
|
|
|
@ -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) ::
|
||||
|
|
|
@ -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)
|
||||
|
||||
|
|
|
@ -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))
|
||||
|
||||
|
|
|
@ -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))
|
||||
|
||||
|
|
|
@ -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))
|
||||
|
||||
|
|
|
@ -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)) ::
|
||||
|
|
|
@ -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)
|
||||
|
||||
}
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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]))
|
||||
)
|
||||
|
||||
}
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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)
|
||||
)
|
||||
}
|
||||
|
|
|
@ -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]))
|
||||
)
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
)
|
||||
|
|
|
@ -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]
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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._
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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._
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
|
|
|
@ -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"),
|
||||
|
|
|
@ -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
|
@ -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),
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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 =>
|
||||
|
|
Loading…
Add table
Reference in a new issue