1
0
mirror of https://github.com/ACINQ/eclair.git synced 2024-11-19 01:43:22 +01:00

Prototype zero-fee commit transactions

Very basic (and hacky!) prototype to use v3 transactions to create
commitment transactions that don't pay any mining fees.
This commit is contained in:
t-bast 2024-09-11 15:16:03 +02:00
parent 1ff5697267
commit 31f24b098a
No known key found for this signature in database
GPG Key ID: 34F377B0100ED6BB
21 changed files with 293 additions and 58 deletions

View File

@ -83,6 +83,7 @@ eclair {
keysend = disabled
trampoline_payment_prototype = disabled
async_payment_prototype = disabled
zero_fee_commitment_tx = disabled
}
// The following section lets you customize features for specific nodes.
// The overrides will be applied on top of the default features settings.

View File

@ -300,6 +300,11 @@ object Features {
val mandatory = 54
}
case object ZeroFeeCommitmentTx extends Feature with InitFeature with NodeFeature with ChannelTypeFeature {
val rfcName = "zero_fee_commitment_tx"
val mandatory = 212
}
// TODO: @t-bast: update feature bits once spec-ed (currently reserved here: https://github.com/lightningnetwork/lightning-rfc/issues/605)
// We're not advertising these bits yet in our announcements, clients have to assume support.
// This is why we haven't added them yet to `areSupported`.
@ -349,6 +354,7 @@ object Features {
TrampolinePaymentPrototype,
AsyncPaymentPrototype,
SplicePrototype,
ZeroFeeCommitmentTx,
)
// Features may depend on other features, as specified in Bolt 9.

View File

@ -20,7 +20,7 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.Satoshi
import fr.acinq.eclair.BlockHeight
import fr.acinq.eclair.transactions.Transactions
import fr.acinq.eclair.transactions.Transactions.{CommitmentFormat, UnsafeLegacyAnchorOutputsCommitmentFormat, ZeroFeeHtlcTxAnchorOutputsCommitmentFormat}
import fr.acinq.eclair.transactions.Transactions.{CommitmentFormat, UnsafeLegacyAnchorOutputsCommitmentFormat, ZeroFeeCommitTxCommitmentFormat, 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 | ZeroFeeCommitTxCommitmentFormat => 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 | ZeroFeeCommitTxCommitmentFormat => false
}
}
}
@ -121,7 +121,7 @@ case class OnChainFeeConf(feeTargets: FeeTargets,
commitmentFormat match {
case Transactions.DefaultCommitmentFormat => networkFeerate
case _: Transactions.AnchorOutputsCommitmentFormat =>
case _: Transactions.AnchorOutputsCommitmentFormat | ZeroFeeCommitTxCommitmentFormat =>
val targetFeerate = networkFeerate.min(feerateToleranceFor(remoteNodeId).anchorOutputMaxCommitFeerate)
// We make sure the feerate is always greater than the propagation threshold.
targetFeerate.max(networkMinFee * 1.25)

View File

@ -16,7 +16,7 @@
package fr.acinq.eclair.channel
import fr.acinq.eclair.transactions.Transactions.{CommitmentFormat, DefaultCommitmentFormat, UnsafeLegacyAnchorOutputsCommitmentFormat, ZeroFeeHtlcTxAnchorOutputsCommitmentFormat}
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.{ChannelTypeFeature, FeatureSupport, Features, InitFeature, PermanentChannelFeature}
/**
@ -31,9 +31,11 @@ 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.ZeroFeeCommitmentTx)
/** 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.ZeroFeeCommitmentTx)) {
ZeroFeeCommitTxCommitmentFormat
} else if (hasFeature(Features.AnchorOutputs)) {
UnsafeLegacyAnchorOutputsCommitmentFormat
} else if (hasFeature(Features.AnchorOutputsZeroFeeHtlcTx)) {
ZeroFeeHtlcTxAnchorOutputsCommitmentFormat
@ -129,6 +131,16 @@ 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 ZeroFeeCommitmentFormat(scidAlias: Boolean = false, zeroConf: Boolean = false) extends SupportedChannelType {
override def features: Set[ChannelTypeFeature] = Set(
if (scidAlias) Some(Features.ScidAlias) else None,
if (zeroConf) Some(Features.ZeroConf) else None,
Some(Features.ZeroFeeCommitmentTx)
).flatten
override def paysDirectlyToWallet: Boolean = false
override def commitmentFormat: CommitmentFormat = ZeroFeeCommitTxCommitmentFormat
override def toString: String = s"zero_fee_commitment_tx${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,7 +163,11 @@ object ChannelTypes {
AnchorOutputsZeroFeeHtlcTx(),
AnchorOutputsZeroFeeHtlcTx(zeroConf = true),
AnchorOutputsZeroFeeHtlcTx(scidAlias = true),
AnchorOutputsZeroFeeHtlcTx(scidAlias = true, zeroConf = true))
AnchorOutputsZeroFeeHtlcTx(scidAlias = true, zeroConf = true),
ZeroFeeCommitmentFormat(),
ZeroFeeCommitmentFormat(zeroConf = true),
ZeroFeeCommitmentFormat(scidAlias = true),
ZeroFeeCommitmentFormat(scidAlias = true, zeroConf = true))
.map(channelType => Features(channelType.features.map(_ -> FeatureSupport.Mandatory).toMap) -> channelType)
.toMap
@ -164,7 +180,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.ZeroFeeCommitmentTx)) {
ZeroFeeCommitmentFormat(scidAlias, zeroConf)
} else if (canUse(Features.AnchorOutputsZeroFeeHtlcTx)) {
AnchorOutputsZeroFeeHtlcTx(scidAlias, zeroConf)
} else if (canUse(Features.AnchorOutputs)) {
AnchorOutputs(scidAlias, zeroConf)

View File

@ -658,6 +658,13 @@ case class Commitment(fundingTxIndex: Long,
val spec = CommitmentSpec.reduce(localCommit.spec, changes.localChanges.acked, changes.remoteChanges.proposed)
LocalCommit.fromCommitSig(keyManager, params, fundingTxId, fundingTxIndex, remoteFundingPubKey, commitInput, commit, localCommitIndex, spec, localPerCommitmentPoint).map { localCommit1 =>
log.info(s"built local commit number=$localCommitIndex toLocalMsat=${spec.toLocal.toLong} toRemoteMsat=${spec.toRemote.toLong} htlc_in={} htlc_out={} feeratePerKw=${spec.commitTxFeerate} txid=${localCommit1.commitTxAndRemoteSig.commitTx.tx.txid} fundingTxId=$fundingTxId", spec.htlcs.collect(DirectedHtlc.incoming).map(_.id).mkString(","), spec.htlcs.collect(DirectedHtlc.outgoing).map(_.id).mkString(","))
if (params.commitmentFormat == ZeroFeeCommitTxCommitmentFormat) {
if (localCommit1.commitTxAndRemoteSig.commitTx.tx.txOut.map(_.amount).sum != commitInput.txOut.amount) {
log.warning("commitment transaction is not 0-fee")
}
log.info(s"commit output amounts = {}", localCommit1.commitTxAndRemoteSig.commitTx.tx.txOut.map(_.amount).mkString(", "))
log.info(s"fully signed commit tx = {}", copy(localCommit = localCommit1).fullySignedLocalCommitTx(params, keyManager).tx)
}
copy(localCommit = localCommit1)
}
}
@ -700,7 +707,7 @@ object Commitment {
val localRevocationPubkey = Generators.revocationPubKey(remoteParams.revocationBasepoint, localPerCommitmentPoint)
val localPaymentBasepoint = localParams.walletStaticPaymentBasepoint.getOrElse(keyManager.paymentPoint(channelKeyPath).publicKey)
val outputs = makeCommitTxOutputs(localParams.paysCommitTxFees, localParams.dustLimit, localRevocationPubkey, remoteParams.toSelfDelay, localDelayedPaymentPubkey, remotePaymentPubkey, localHtlcPubkey, remoteHtlcPubkey, localFundingPubkey, remoteFundingPubKey, spec, channelFeatures.commitmentFormat)
val commitTx = makeCommitTx(commitmentInput, commitTxNumber, localPaymentBasepoint, remoteParams.paymentBasepoint, localParams.isChannelOpener, outputs)
val commitTx = makeCommitTx(commitmentInput, commitTxNumber, localPaymentBasepoint, remoteParams.paymentBasepoint, localParams.isChannelOpener, channelFeatures.commitmentFormat, outputs)
val htlcTxs = makeHtlcTxs(commitTx.tx, localParams.dustLimit, localRevocationPubkey, remoteParams.toSelfDelay, localDelayedPaymentPubkey, spec.htlcTxFeerate(channelFeatures.commitmentFormat), outputs, channelFeatures.commitmentFormat)
(commitTx, htlcTxs)
}
@ -729,7 +736,7 @@ object Commitment {
val remoteHtlcPubkey = Generators.derivePubKey(remoteParams.htlcBasepoint, remotePerCommitmentPoint)
val remoteRevocationPubkey = Generators.revocationPubKey(keyManager.revocationPoint(channelKeyPath).publicKey, remotePerCommitmentPoint)
val outputs = makeCommitTxOutputs(!localParams.paysCommitTxFees, remoteParams.dustLimit, remoteRevocationPubkey, localParams.toSelfDelay, remoteDelayedPaymentPubkey, localPaymentPubkey, remoteHtlcPubkey, localHtlcPubkey, remoteFundingPubKey, localFundingPubkey, spec, channelFeatures.commitmentFormat)
val commitTx = makeCommitTx(commitmentInput, commitTxNumber, remoteParams.paymentBasepoint, localPaymentBasepoint, !localParams.isChannelOpener, outputs)
val commitTx = makeCommitTx(commitmentInput, commitTxNumber, remoteParams.paymentBasepoint, localPaymentBasepoint, !localParams.isChannelOpener, channelFeatures.commitmentFormat, outputs)
val htlcTxs = makeHtlcTxs(commitTx.tx, remoteParams.dustLimit, remoteRevocationPubkey, localParams.toSelfDelay, remoteDelayedPaymentPubkey, spec.htlcTxFeerate(channelFeatures.commitmentFormat), outputs, channelFeatures.commitmentFormat)
(commitTx, htlcTxs)
}

View File

@ -642,6 +642,7 @@ object Helpers {
// we "MUST set fee_satoshis less than or equal to the base fee of the final commitment transaction"
requestedFeerate.min(commitment.localCommit.spec.commitTxFeerate)
case _: AnchorOutputsCommitmentFormat => requestedFeerate
case ZeroFeeCommitTxCommitmentFormat => requestedFeerate
}
// NB: we choose a minimum fee that ensures the tx will easily propagate while allowing low fees since we can
// always use CPFP to speed up confirmation if necessary.
@ -731,18 +732,19 @@ object Helpers {
* we previously had. Note that absolute targets are always considered more aggressive than relative targets.
*/
private def shouldUpdateAnchorTxs(anchorTxs: List[ClaimAnchorOutputTx], confirmationTarget: ConfirmationTarget): Boolean = {
anchorTxs
.collect { case tx: ClaimLocalAnchorOutputTx => tx.confirmationTarget }
.forall {
case ConfirmationTarget.Absolute(current) => confirmationTarget match {
case ConfirmationTarget.Absolute(proposed) => proposed < current
case _: ConfirmationTarget.Priority => false
}
case ConfirmationTarget.Priority(current) => confirmationTarget match {
case _: ConfirmationTarget.Absolute => true
case ConfirmationTarget.Priority(proposed) => current < proposed
}
anchorTxs.collect {
case tx: ClaimLocalAnchorOutputTx => tx.confirmationTarget
case tx: ClaimSharedAnchorOutputTx => tx.confirmationTarget
}.forall {
case ConfirmationTarget.Absolute(current) => confirmationTarget match {
case ConfirmationTarget.Absolute(proposed) => proposed < current
case _: ConfirmationTarget.Priority => false
}
case ConfirmationTarget.Priority(current) => confirmationTarget match {
case _: ConfirmationTarget.Absolute => true
case ConfirmationTarget.Priority(proposed) => current < proposed
}
}
}
object LocalClose {
@ -798,6 +800,9 @@ object Helpers {
},
withTxGenerationLog("remote-anchor") {
Transactions.makeClaimRemoteAnchorOutputTx(lcp.commitTx, commitment.remoteFundingPubKey)
},
withTxGenerationLog("shared-anchor") {
Transactions.makeClaimSharedAnchorOutputTx(lcp.commitTx, confirmationTarget)
}
).flatten
lcp.copy(claimAnchorTxs = claimAnchorTxs)
@ -957,6 +962,12 @@ object Helpers {
Transactions.addSigs(claimMain, sig)
})
}
case ZeroFeeCommitTxCommitmentFormat => withTxGenerationLog("remote-main") {
Transactions.makeClaimP2WPKHOutputTx(tx, params.localParams.dustLimit, localPubkey, finalScriptPubKey, feeratePerKwMain).map(claimMain => {
val sig = keyManager.sign(claimMain, keyManager.paymentPoint(channelKeyPath), remotePerCommitmentPoint, TxOwner.Local, params.commitmentFormat)
Transactions.addSigs(claimMain, localPubkey, sig)
})
}
}
}
}
@ -1092,6 +1103,12 @@ object Helpers {
Transactions.addSigs(claimMain, sig)
})
}
case ZeroFeeCommitTxCommitmentFormat => withTxGenerationLog("remote-main") {
Transactions.makeClaimP2WPKHOutputTx(commitTx, localParams.dustLimit, localPaymentPubkey, finalScriptPubKey, feerateMain).map(claimMain => {
val sig = keyManager.sign(claimMain, keyManager.paymentPoint(channelKeyPath), remotePerCommitmentPoint, TxOwner.Local, commitmentFormat)
Transactions.addSigs(claimMain, localPaymentPubkey, sig)
})
}
}
}

View File

@ -29,6 +29,7 @@ import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient
import fr.acinq.eclair.blockchain.fee.{ConfirmationPriority, ConfirmationTarget, FeeratePerKw}
import fr.acinq.eclair.channel.Commitments.PostRevocationAction
import fr.acinq.eclair.channel.Helpers.Closing.MutualClose
import fr.acinq.eclair.channel.Helpers.Syncing.SyncResult
@ -40,6 +41,7 @@ import fr.acinq.eclair.channel.fund.InteractiveTxBuilder._
import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxFunder, InteractiveTxSigningSession}
import fr.acinq.eclair.channel.publish.TxPublisher
import fr.acinq.eclair.channel.publish.TxPublisher.{PublishFinalTx, PublishReplaceableTx, SetChannelId}
import fr.acinq.eclair.crypto.Generators
import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
import fr.acinq.eclair.db.DbEventHandler.ChannelEvent.EventType
import fr.acinq.eclair.db.PendingCommandsDb
@ -47,7 +49,7 @@ import fr.acinq.eclair.io.Peer
import fr.acinq.eclair.payment.relay.Relayer
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.Transactions.{ClosingTx, TxOwner, ZeroFeeCommitTxCommitmentFormat}
import fr.acinq.eclair.transactions._
import fr.acinq.eclair.wire.protocol._
@ -609,6 +611,25 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.shortIds, commitments1))
}
context.system.eventStream.publish(ChannelSignatureReceived(self, commitments1))
if (d.commitments.params.commitmentFormat == ZeroFeeCommitTxCommitmentFormat) {
val commitment = commitments1.latest
val commitTx = commitment.localCommit.commitTxAndRemoteSig.commitTx.tx
val channelKeyPath = keyManager.keyPath(commitment.localParams, commitment.params.channelConfig)
val localPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, commitment.localCommit.index.toInt)
val localRevocationPubkey = Generators.revocationPubKey(commitment.remoteParams.revocationBasepoint, localPerCommitmentPoint)
val localDelayedPubkey = Generators.derivePubKey(keyManager.delayedPaymentPoint(channelKeyPath).publicKey, localPerCommitmentPoint)
val finalScriptPubKey = getOrGenerateFinalScriptPubKey(d)
Transactions.makeClaimSharedAnchorOutputTx(commitTx, ConfirmationTarget.Priority(ConfirmationPriority.Slow)).map(anchorTx => {
log.info("commit-tx-id = {}, claim-anchor-tx = {}", commitTx.txid, anchorTx.tx)
})
Seq(FeeratePerKw(2500 sat), FeeratePerKw(5000 sat)).foreach(feerate => {
val claimMain = Transactions.makeClaimLocalDelayedOutputTx(commitTx, commitment.localParams.dustLimit, localRevocationPubkey, commitment.remoteParams.toSelfDelay, localDelayedPubkey, finalScriptPubKey, feerate).map(claimDelayed => {
val sig = keyManager.sign(claimDelayed, keyManager.delayedPaymentPoint(channelKeyPath), localPerCommitmentPoint, TxOwner.Local, commitment.params.commitmentFormat)
val claimTx = Transactions.addSigs(claimDelayed, sig)
log.info("commit-tx-id = {}, feerate = {} claim-main-tx = {}", commitTx.txid, feerate, claimTx.tx)
})
})
}
// If we're now quiescent, we may send our stfu message.
val (d1, toSend) = d.spliceStatus match {
case SpliceStatus.QuiescenceRequested(cmd) if commitments1.localIsQuiescent =>

View File

@ -217,7 +217,7 @@ trait ErrorHandlers extends CommonHandlers {
case Transactions.DefaultCommitmentFormat =>
val redeemableHtlcTxs = htlcTxs.values.flatten.map(tx => PublishFinalTx(tx, tx.fee, Some(commitTx.txid)))
List(PublishFinalTx(commitTx, commitment.commitInput.outPoint, "commit-tx", Closing.commitTxFee(commitment.commitInput, commitTx, localPaysCommitTxFees), None)) ++ (claimMainDelayedOutputTx.map(tx => PublishFinalTx(tx, tx.fee, None)) ++ redeemableHtlcTxs ++ claimHtlcDelayedTxs.map(tx => PublishFinalTx(tx, tx.fee, None)))
case _: Transactions.AnchorOutputsCommitmentFormat =>
case _: Transactions.AnchorOutputsCommitmentFormat | Transactions.ZeroFeeCommitTxCommitmentFormat =>
val redeemableHtlcTxs = htlcTxs.values.flatten.map(tx => PublishReplaceableTx(tx, commitment))
val claimLocalAnchor = claimAnchorTxs.collect { case tx: Transactions.ClaimLocalAnchorOutputTx if !localCommitPublished.isConfirmed => PublishReplaceableTx(tx, commitment) }
List(PublishFinalTx(commitTx, commitment.commitInput.outPoint, "commit-tx", Closing.commitTxFee(commitment.commitInput, commitTx, localPaysCommitTxFees), None)) ++ claimLocalAnchor ++ claimMainDelayedOutputTx.map(tx => PublishFinalTx(tx, tx.fee, None)) ++ redeemableHtlcTxs ++ claimHtlcDelayedTxs.map(tx => PublishFinalTx(tx, tx.fee, None))

View File

@ -19,7 +19,7 @@ package fr.acinq.eclair.transactions
import fr.acinq.bitcoin.scalacompat.SatoshiLong
import fr.acinq.eclair.MilliSatoshi
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.transactions.Transactions.{CommitmentFormat, ZeroFeeHtlcTxAnchorOutputsCommitmentFormat}
import fr.acinq.eclair.transactions.Transactions.{CommitmentFormat, ZeroFeeCommitTxCommitmentFormat, ZeroFeeHtlcTxAnchorOutputsCommitmentFormat}
import fr.acinq.eclair.wire.protocol._
/**
@ -34,6 +34,7 @@ object CommitmentOutput {
case object ToRemote extends CommitmentOutput
case object ToLocalAnchor extends CommitmentOutput
case object ToRemoteAnchor extends CommitmentOutput
case object ToSharedAnchor extends CommitmentOutput
case class InHtlc(incomingHtlc: IncomingHtlc) extends CommitmentOutput
case class OutHtlc(outgoingHtlc: OutgoingHtlc) extends CommitmentOutput
// @formatter:on
@ -75,6 +76,7 @@ final case class CommitmentSpec(htlcs: Set[DirectedHtlc], commitTxFeerate: Feera
def htlcTxFeerate(commitmentFormat: CommitmentFormat): FeeratePerKw = commitmentFormat match {
case ZeroFeeHtlcTxAnchorOutputsCommitmentFormat => FeeratePerKw(0 sat)
case ZeroFeeCommitTxCommitmentFormat => FeeratePerKw(0 sat)
case _ => commitTxFeerate
}

View File

@ -22,9 +22,9 @@ import fr.acinq.bitcoin.TxIn.{SEQUENCE_LOCKTIME_DISABLE_FLAG, SEQUENCE_LOCKTIME_
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.Script._
import fr.acinq.bitcoin.scalacompat._
import fr.acinq.eclair.transactions.Transactions.{AnchorOutputsCommitmentFormat, CommitmentFormat, DefaultCommitmentFormat}
import fr.acinq.eclair.transactions.Transactions.{AnchorOutputsCommitmentFormat, CommitmentFormat, DefaultCommitmentFormat, ZeroFeeCommitTxCommitmentFormat}
import fr.acinq.eclair.{BlockHeight, CltvExpiry, CltvExpiryDelta}
import scodec.bits.ByteVector
import scodec.bits.{ByteVector, HexStringSyntax}
/**
* Created by PM on 02/12/2016.
@ -41,7 +41,7 @@ object Scripts {
private def htlcRemoteSighash(commitmentFormat: CommitmentFormat): Int = commitmentFormat match {
case DefaultCommitmentFormat => SIGHASH_ALL
case _: AnchorOutputsCommitmentFormat => SIGHASH_SINGLE | SIGHASH_ANYONECANPAY
case _: AnchorOutputsCommitmentFormat | ZeroFeeCommitTxCommitmentFormat => SIGHASH_SINGLE | SIGHASH_ANYONECANPAY
}
def multiSig2of2(pubkey1: PublicKey, pubkey2: PublicKey): Seq[ScriptElt] =
@ -186,10 +186,13 @@ object Scripts {
*/
def witnessAnchorAfterDelay(anchorScript: ByteVector) = ScriptWitness(ByteVector.empty :: anchorScript :: Nil)
def pay2anchor(): Seq[ScriptElt] = OP_1 :: OP_PUSHDATA(hex"4e73") :: Nil
def htlcOffered(localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, revocationPubKey: PublicKey, paymentHash: ByteVector, commitmentFormat: CommitmentFormat): Seq[ScriptElt] = {
val addCsvDelay = commitmentFormat match {
case DefaultCommitmentFormat => false
case _: AnchorOutputsCommitmentFormat => true
case ZeroFeeCommitTxCommitmentFormat => false
}
// @formatter:off
// To you with revocation key
@ -241,6 +244,7 @@ object Scripts {
val addCsvDelay = commitmentFormat match {
case DefaultCommitmentFormat => false
case _: AnchorOutputsCommitmentFormat => true
case ZeroFeeCommitTxCommitmentFormat => false
}
// @formatter:off
// To you with revocation key

View File

@ -92,6 +92,18 @@ object Transactions {
*/
case object ZeroFeeHtlcTxAnchorOutputsCommitmentFormat extends AnchorOutputsCommitmentFormat
/** Similar to anchor outputs, with a single P2A output and paying no mining fees. */
case object ZeroFeeCommitTxCommitmentFormat extends CommitmentFormat {
// TODO: those values are incorrect, just copy-pasted from anchors, but we probably don't care since we don't pay fees?
override def commitWeight: Int = 1124
override def htlcOutputWeight: Int = 172
override def htlcTimeoutWeight: Int = 666
override def htlcSuccessWeight: Int = 706
override def htlcTimeoutInputWeight: Int = 452
override def htlcSuccessInputWeight: Int = 491
val anchorAmount = Satoshi(330)
}
// @formatter:off
case class OutputInfo(index: Long, amount: Satoshi, publicKeyScript: ByteVector)
case class InputInfo(outPoint: OutPoint, txOut: TxOut, redeemScript: ByteVector)
@ -143,7 +155,7 @@ object Transactions {
def htlcId: Long
override def sighash(txOwner: TxOwner, commitmentFormat: CommitmentFormat): Int = commitmentFormat match {
case DefaultCommitmentFormat => SIGHASH_ALL
case _: AnchorOutputsCommitmentFormat => txOwner match {
case _: AnchorOutputsCommitmentFormat | ZeroFeeCommitTxCommitmentFormat => txOwner match {
case TxOwner.Local => SIGHASH_ALL
case TxOwner.Remote => SIGHASH_SINGLE | SIGHASH_ANYONECANPAY
}
@ -163,6 +175,7 @@ object Transactions {
sealed trait ClaimAnchorOutputTx extends TransactionWithInputInfo
case class ClaimLocalAnchorOutputTx(input: InputInfo, tx: Transaction, confirmationTarget: ConfirmationTarget) extends ClaimAnchorOutputTx with ReplaceableTransactionWithInputInfo { override def desc: String = "local-anchor" }
case class ClaimRemoteAnchorOutputTx(input: InputInfo, tx: Transaction) extends ClaimAnchorOutputTx { override def desc: String = "remote-anchor" }
case class ClaimSharedAnchorOutputTx(input: InputInfo, tx: Transaction, confirmationTarget: ConfirmationTarget) extends ClaimAnchorOutputTx { override def desc: String = "shared-anchor" }
sealed trait ClaimRemoteCommitMainOutputTx extends TransactionWithInputInfo
case class ClaimP2WPKHOutputTx(input: InputInfo, tx: Transaction) extends ClaimRemoteCommitMainOutputTx { override def desc: String = "remote-main" }
case class ClaimRemoteDelayedOutputTx(input: InputInfo, tx: Transaction) extends ClaimRemoteCommitMainOutputTx { override def desc: String = "remote-main-delayed" }
@ -238,6 +251,7 @@ object Transactions {
def offeredHtlcTrimThreshold(dustLimit: Satoshi, feerate: FeeratePerKw, commitmentFormat: CommitmentFormat): Satoshi = {
commitmentFormat match {
case ZeroFeeHtlcTxAnchorOutputsCommitmentFormat => dustLimit
case ZeroFeeCommitTxCommitmentFormat => dustLimit
case _ => dustLimit + weight2fee(feerate, commitmentFormat.htlcTimeoutWeight)
}
}
@ -256,6 +270,7 @@ object Transactions {
def receivedHtlcTrimThreshold(dustLimit: Satoshi, feerate: FeeratePerKw, commitmentFormat: CommitmentFormat): Satoshi = {
commitmentFormat match {
case ZeroFeeHtlcTxAnchorOutputsCommitmentFormat => dustLimit
case ZeroFeeCommitTxCommitmentFormat => dustLimit
case _ => dustLimit + weight2fee(feerate, commitmentFormat.htlcSuccessWeight)
}
}
@ -289,11 +304,11 @@ object Transactions {
val txFee = commitTxFeeMsat(dustLimit, spec, commitmentFormat)
// When using anchor outputs, the channel initiator pays for *both* anchors all the time, even if only one anchor is present.
// This is not technically a fee (it doesn't go to miners) but it also has to be deduced from the channel initiator's main output.
val anchorsCost = commitmentFormat match {
case DefaultCommitmentFormat => Satoshi(0)
case _: AnchorOutputsCommitmentFormat => AnchorOutputsCommitmentFormat.anchorAmount * 2
commitmentFormat match {
case DefaultCommitmentFormat => txFee
case _: AnchorOutputsCommitmentFormat => txFee + AnchorOutputsCommitmentFormat.anchorAmount * 2
case ZeroFeeCommitTxCommitmentFormat => ZeroFeeCommitTxCommitmentFormat.anchorAmount.toMilliSatoshi
}
txFee + anchorsCost
}
def commitTxTotalCost(dustLimit: Satoshi, spec: CommitmentSpec, commitmentFormat: CommitmentFormat): Satoshi = commitTxTotalCostMsat(dustLimit, spec, commitmentFormat).truncateToSatoshi
@ -346,6 +361,7 @@ object Transactions {
def getHtlcTxInputSequence(commitmentFormat: CommitmentFormat): Long = commitmentFormat match {
case DefaultCommitmentFormat => 0 // htlc txs immediately spend the commit tx
case _: AnchorOutputsCommitmentFormat => 1 // htlc txs have a 1-block delay to allow CPFP carve-out on anchors
case ZeroFeeCommitTxCommitmentFormat => 0
}
/**
@ -399,10 +415,21 @@ object Transactions {
val hasHtlcs = outputs.nonEmpty
val dustHtlcsAmount = spec.htlcs.collect { case htlc if htlc.add.amountMsat < localDustLimit => htlc.add.amountMsat }.sum.truncateToSatoshi
val commitFee = commitmentFormat match {
case ZeroFeeCommitTxCommitmentFormat =>
if (dustHtlcsAmount < ZeroFeeCommitTxCommitmentFormat.anchorAmount) {
ZeroFeeCommitTxCommitmentFormat.anchorAmount - dustHtlcsAmount
} else {
0 sat
}
case _ => commitTxTotalCost(localDustLimit, spec, commitmentFormat)
}
val (toLocalAmount: Satoshi, toRemoteAmount: Satoshi) = if (localPaysCommitTxFees) {
(spec.toLocal.truncateToSatoshi - commitTxTotalCost(localDustLimit, spec, commitmentFormat), spec.toRemote.truncateToSatoshi)
(spec.toLocal.truncateToSatoshi - commitFee, spec.toRemote.truncateToSatoshi)
} else {
(spec.toLocal.truncateToSatoshi, spec.toRemote.truncateToSatoshi - commitTxTotalCost(localDustLimit, spec, commitmentFormat))
(spec.toLocal.truncateToSatoshi, spec.toRemote.truncateToSatoshi - commitFee)
} // NB: we don't care if values are < 0, they will be trimmed if they are < dust limit anyway
if (toLocalAmount >= localDustLimit) {
@ -422,6 +449,10 @@ object Transactions {
TxOut(toRemoteAmount, pay2wsh(toRemoteDelayed(remotePaymentPubkey))),
toRemoteDelayed(remotePaymentPubkey),
ToRemote))
case ZeroFeeCommitTxCommitmentFormat => outputs.append(CommitmentOutputLink(
TxOut(toRemoteAmount, pay2wpkh(remotePaymentPubkey)),
pay2pkh(remotePaymentPubkey),
ToRemote))
}
}
@ -433,6 +464,8 @@ object Transactions {
if (toRemoteAmount >= localDustLimit || hasHtlcs) {
outputs.append(CommitmentOutputLink(TxOut(AnchorOutputsCommitmentFormat.anchorAmount, pay2wsh(anchor(remoteFundingPubkey))), anchor(remoteFundingPubkey), ToRemoteAnchor))
}
case ZeroFeeCommitTxCommitmentFormat =>
outputs.append(CommitmentOutputLink(TxOut(ZeroFeeCommitTxCommitmentFormat.anchorAmount.max(dustHtlcsAmount), pay2anchor()), Nil, ToSharedAnchor))
case _ =>
}
@ -444,12 +477,16 @@ object Transactions {
localPaymentBasePoint: PublicKey,
remotePaymentBasePoint: PublicKey,
localIsChannelOpener: Boolean,
commitmentFormat: CommitmentFormat,
outputs: CommitmentOutputs): CommitTx = {
val txNumber = obscuredCommitTxNumber(commitTxNumber, localIsChannelOpener, localPaymentBasePoint, remotePaymentBasePoint)
val (sequence, lockTime) = encodeTxNumber(txNumber)
val tx = Transaction(
version = 2,
version = commitmentFormat match {
case ZeroFeeCommitTxCommitmentFormat => 3
case _ => 2
},
txIn = TxIn(commitTxInput.outPoint, ByteVector.empty, sequence = sequence) :: Nil,
txOut = outputs.map(_.output),
lockTime = lockTime)
@ -475,7 +512,7 @@ object Transactions {
} else {
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
val tx = Transaction(
version = 2,
version = commitTx.version,
txIn = TxIn(input.outPoint, ByteVector.empty, getHtlcTxInputSequence(commitmentFormat)) :: Nil,
txOut = TxOut(amount, pay2wsh(toLocalDelayed(localRevocationPubkey, toLocalDelay, localDelayedPaymentPubkey))) :: Nil,
lockTime = htlc.cltvExpiry.toLong
@ -502,7 +539,7 @@ object Transactions {
} else {
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
val tx = Transaction(
version = 2,
version = commitTx.version,
txIn = TxIn(input.outPoint, ByteVector.empty, getHtlcTxInputSequence(commitmentFormat)) :: Nil,
txOut = TxOut(amount, pay2wsh(toLocalDelayed(localRevocationPubkey, toLocalDelay, localDelayedPaymentPubkey))) :: Nil,
lockTime = 0
@ -550,7 +587,7 @@ object Transactions {
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
// unsigned tx
val tx = Transaction(
version = 2,
version = commitTx.version,
txIn = TxIn(input.outPoint, ByteVector.empty, getHtlcTxInputSequence(commitmentFormat)) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
lockTime = 0)
@ -585,7 +622,7 @@ object Transactions {
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
// unsigned tx
val tx = Transaction(
version = 2,
version = commitTx.version,
txIn = TxIn(input.outPoint, ByteVector.empty, getHtlcTxInputSequence(commitmentFormat)) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
lockTime = htlc.cltvExpiry.toLong)
@ -611,7 +648,7 @@ object Transactions {
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
// unsigned tx
val tx = Transaction(
version = 2,
version = commitTx.version,
txIn = TxIn(input.outPoint, ByteVector.empty, 0x00000000L) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
lockTime = 0)
@ -637,7 +674,7 @@ object Transactions {
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
// unsigned transaction
val tx = Transaction(
version = 2,
version = commitTx.version,
txIn = TxIn(input.outPoint, ByteVector.empty, 1) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
lockTime = 0)
@ -675,7 +712,7 @@ object Transactions {
val input = InputInfo(OutPoint(parentTx, outputIndex), parentTx.txOut(outputIndex), write(redeemScript))
// unsigned transaction
val tx = Transaction(
version = 2,
version = parentTx.version,
txIn = TxIn(input.outPoint, ByteVector.empty, toLocalDelay.toInt) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
lockTime = 0)
@ -701,7 +738,7 @@ object Transactions {
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
// unsigned transaction
val tx = Transaction(
version = 2,
version = commitTx.version,
txIn = TxIn(input.outPoint, ByteVector.empty, 0) :: Nil,
txOut = Nil, // anchor is only used to bump fees, the output will be added later depending on available inputs
lockTime = 0)
@ -717,6 +754,21 @@ object Transactions {
makeClaimAnchorOutputTx(commitTx, remoteFundingPubkey).map { case (input, tx) => ClaimRemoteAnchorOutputTx(input, tx) }
}
def makeClaimSharedAnchorOutputTx(commitTx: Transaction, confirmationTarget: ConfirmationTarget): Either[TxGenerationSkipped, ClaimSharedAnchorOutputTx] = {
val pubkeyScript = write(pay2anchor())
findPubKeyScriptIndex(commitTx, pubkeyScript) match {
case Left(skip) => Left(skip)
case Right(outputIndex) =>
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), ByteVector.empty)
val tx = Transaction(
version = commitTx.version,
txIn = TxIn(input.outPoint, ByteVector.empty, 0) :: Nil,
txOut = Nil, // anchor is only used to bump fees, the output will be added later depending on available inputs
lockTime = 0)
Right(ClaimSharedAnchorOutputTx(input, tx, confirmationTarget))
}
}
def makeClaimHtlcDelayedOutputPenaltyTxs(htlcTx: Transaction, localDustLimit: Satoshi, localRevocationPubkey: PublicKey, toLocalDelay: CltvExpiryDelta, localDelayedPaymentPubkey: PublicKey, localFinalScriptPubKey: ByteVector, feeratePerKw: FeeratePerKw): Seq[Either[TxGenerationSkipped, ClaimHtlcDelayedOutputPenaltyTx]] = {
val redeemScript = toLocalDelayed(localRevocationPubkey, toLocalDelay, localDelayedPaymentPubkey)
val pubkeyScript = write(pay2wsh(redeemScript))
@ -726,7 +778,7 @@ object Transactions {
val input = InputInfo(OutPoint(htlcTx, outputIndex), htlcTx.txOut(outputIndex), write(redeemScript))
// unsigned transaction
val tx = Transaction(
version = 2,
version = htlcTx.version,
txIn = TxIn(input.outPoint, ByteVector.empty, 0xffffffffL) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
lockTime = 0)
@ -753,7 +805,7 @@ object Transactions {
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
// unsigned transaction
val tx = Transaction(
version = 2,
version = commitTx.version,
txIn = TxIn(input.outPoint, ByteVector.empty, 0xffffffffL) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
lockTime = 0)
@ -777,7 +829,7 @@ object Transactions {
val input = InputInfo(OutPoint(commitTx, htlcOutputIndex), commitTx.txOut(htlcOutputIndex), redeemScript)
// unsigned transaction
val tx = Transaction(
version = 2,
version = commitTx.version,
txIn = TxIn(input.outPoint, ByteVector.empty, 0xffffffffL) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
lockTime = 0)

View File

@ -1489,6 +1489,7 @@ class ReplaceableTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike w
bob.stateData.asInstanceOf[DATA_NORMAL].commitments.params.commitmentFormat match {
case Transactions.DefaultCommitmentFormat => assert(remoteCommitTx.tx.txOut.size == 4)
case _: AnchorOutputsCommitmentFormat => assert(remoteCommitTx.tx.txOut.size == 6)
case ZeroFeeCommitTxCommitmentFormat => assert(remoteCommitTx.tx.txOut.size == 5)
}
probe.send(alice, WatchFundingSpentTriggered(remoteCommitTx.tx))
@ -1500,6 +1501,7 @@ class ReplaceableTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike w
bob.stateData.asInstanceOf[DATA_NORMAL].commitments.params.commitmentFormat match {
case Transactions.DefaultCommitmentFormat => ()
case _: AnchorOutputsCommitmentFormat => alice2blockchain.expectMsgType[PublishReplaceableTx] // claim anchor
case ZeroFeeCommitTxCommitmentFormat => alice2blockchain.expectMsgType[PublishReplaceableTx] // claim anchor
}
if (!bob.stateData.asInstanceOf[DATA_NORMAL].commitments.params.channelFeatures.paysDirectlyToWallet) alice2blockchain.expectMsgType[PublishFinalTx] // claim main output
val claimHtlcSuccess = alice2blockchain.expectMsgType[PublishReplaceableTx]

View File

@ -554,7 +554,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
// all htlcs success/timeout should be published as-is, without claiming their outputs
s2blockchain.expectMsgAllOf(localCommitPublished.htlcTxs.values.toSeq.collect { case Some(tx) => TxPublisher.PublishFinalTx(tx, tx.fee, Some(commitTx.txid)) }: _*)
assert(localCommitPublished.claimHtlcDelayedTxs.isEmpty)
case _: Transactions.AnchorOutputsCommitmentFormat =>
case _: Transactions.AnchorOutputsCommitmentFormat | ZeroFeeCommitTxCommitmentFormat =>
// all htlcs success/timeout should be published as replaceable txs, without claiming their outputs
val htlcTxs = localCommitPublished.htlcTxs.values.collect { case Some(tx: HtlcTx) => tx }
val publishedTxs = htlcTxs.map(_ => s2blockchain.expectMsgType[TxPublisher.PublishReplaceableTx])
@ -593,7 +593,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
// If anchor outputs is used, we use the anchor output to bump the fees if necessary.
closingData.commitments.params.commitmentFormat match {
case _: AnchorOutputsCommitmentFormat =>
case _: AnchorOutputsCommitmentFormat | ZeroFeeCommitTxCommitmentFormat =>
val anchorTx = s2blockchain.expectMsgType[PublishReplaceableTx]
assert(anchorTx.txInfo.isInstanceOf[ClaimLocalAnchorOutputTx])
case Transactions.DefaultCommitmentFormat => ()

View File

@ -55,6 +55,7 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny
val commitFeerate = channelType.commitmentFormat match {
case Transactions.DefaultCommitmentFormat => TestConstants.feeratePerKw
case _: Transactions.AnchorOutputsCommitmentFormat => TestConstants.anchorOutputsFeeratePerKw
case Transactions.ZeroFeeCommitTxCommitmentFormat => TestConstants.anchorOutputsFeeratePerKw
}
val aliceListener = TestProbe()
val bobListener = TestProbe()

View File

@ -62,6 +62,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
val commitFeerate = channelType.commitmentFormat match {
case Transactions.DefaultCommitmentFormat => TestConstants.feeratePerKw
case _: Transactions.AnchorOutputsCommitmentFormat => TestConstants.anchorOutputsFeeratePerKw
case Transactions.ZeroFeeCommitTxCommitmentFormat => TestConstants.anchorOutputsFeeratePerKw
}
val aliceInit = Init(aliceParams.initFeatures)
val bobInit = Init(bobParams.initFeatures)

View File

@ -54,6 +54,7 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
val commitFeerate = channelType.commitmentFormat match {
case Transactions.DefaultCommitmentFormat => TestConstants.feeratePerKw
case _: Transactions.AnchorOutputsCommitmentFormat => TestConstants.anchorOutputsFeeratePerKw
case Transactions.ZeroFeeCommitTxCommitmentFormat => TestConstants.anchorOutputsFeeratePerKw
}
val pushMsat = if (test.tags.contains(ChannelStateTestsTags.NoPushAmount)) None else Some(TestConstants.initiatorPushAmount)
val aliceInit = Init(aliceParams.initFeatures)

View File

@ -66,6 +66,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
val commitFeerate = channelType.commitmentFormat match {
case Transactions.DefaultCommitmentFormat => TestConstants.feeratePerKw
case _: Transactions.AnchorOutputsCommitmentFormat => TestConstants.anchorOutputsFeeratePerKw
case Transactions.ZeroFeeCommitTxCommitmentFormat => TestConstants.anchorOutputsFeeratePerKw
}
val aliceInit = Init(aliceParams.initFeatures)
val bobInit = Init(bobParams.initFeatures)

View File

@ -958,6 +958,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
val bobCommitTx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
channelFeatures.commitmentFormat match {
case _: AnchorOutputsCommitmentFormat => assert(bobCommitTx.txOut.length == 7) // two main outputs + two anchors + 3 HTLCs
case ZeroFeeCommitTxCommitmentFormat => assert(bobCommitTx.txOut.length == 6) // two main outputs + one anchor + 3 HTLCs
case DefaultCommitmentFormat => assert(bobCommitTx.txOut.length == 5) // two main outputs + 3 HTLCs
}
val closingState = remoteClose(bobCommitTx, alice, alice2blockchain)
@ -1092,6 +1093,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
val bobCommitTx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
channelFeatures.commitmentFormat match {
case _: AnchorOutputsCommitmentFormat => assert(bobCommitTx.txOut.length == 7) // two main outputs + two anchors + 3 HTLCs
case ZeroFeeCommitTxCommitmentFormat => assert(bobCommitTx.txOut.length == 6) // two main outputs + one anchor + 3 HTLCs
case DefaultCommitmentFormat => assert(bobCommitTx.txOut.length == 5) // two main outputs + 3 HTLCs
}
val closingState = remoteClose(bobCommitTx, alice, alice2blockchain)
@ -1284,6 +1286,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
val bobCommitTx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
channelFeatures.commitmentFormat match {
case _: AnchorOutputsCommitmentFormat => assert(bobCommitTx.txOut.length == 6) // two main outputs + two anchors + 2 HTLCs
case ZeroFeeCommitTxCommitmentFormat => assert(bobCommitTx.txOut.length == 5) // two main outputs + one anchor + 2 HTLCs
case DefaultCommitmentFormat => assert(bobCommitTx.txOut.length == 4) // two main outputs + 2 HTLCs
}
alice ! WatchFundingSpentTriggered(bobCommitTx)
@ -1358,6 +1361,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
val localCommit1 = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit
channelFeatures.commitmentFormat match {
case _: AnchorOutputsCommitmentFormat => assert(localCommit1.commitTxAndRemoteSig.commitTx.tx.txOut.size == 4) // 2 main outputs + 2 anchors
case ZeroFeeCommitTxCommitmentFormat => assert(localCommit1.commitTxAndRemoteSig.commitTx.tx.txOut.size == 3) // two main outputs + 1 anchor
case DefaultCommitmentFormat => assert(localCommit1.commitTxAndRemoteSig.commitTx.tx.txOut.size == 2) // 2 main outputs
}
@ -1374,6 +1378,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx.txOut.size == localCommit2.commitTxAndRemoteSig.commitTx.tx.txOut.size)
channelFeatures.commitmentFormat match {
case _: AnchorOutputsCommitmentFormat => assert(localCommit2.commitTxAndRemoteSig.commitTx.tx.txOut.size == 6)
case ZeroFeeCommitTxCommitmentFormat => assert(localCommit2.commitTxAndRemoteSig.commitTx.tx.txOut.size == 5)
case DefaultCommitmentFormat => assert(localCommit2.commitTxAndRemoteSig.commitTx.tx.txOut.size == 4)
}
@ -1390,6 +1395,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx.txOut.size == localCommit3.commitTxAndRemoteSig.commitTx.tx.txOut.size)
channelFeatures.commitmentFormat match {
case _: AnchorOutputsCommitmentFormat => assert(localCommit3.commitTxAndRemoteSig.commitTx.tx.txOut.size == 8)
case ZeroFeeCommitTxCommitmentFormat => assert(localCommit3.commitTxAndRemoteSig.commitTx.tx.txOut.size == 7)
case DefaultCommitmentFormat => assert(localCommit3.commitTxAndRemoteSig.commitTx.tx.txOut.size == 6)
}
@ -1404,6 +1410,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx.txOut.size == localCommit4.commitTxAndRemoteSig.commitTx.tx.txOut.size)
channelFeatures.commitmentFormat match {
case _: AnchorOutputsCommitmentFormat => assert(localCommit4.commitTxAndRemoteSig.commitTx.tx.txOut.size == 4)
case ZeroFeeCommitTxCommitmentFormat => assert(localCommit4.commitTxAndRemoteSig.commitTx.tx.txOut.size == 3)
case DefaultCommitmentFormat => assert(localCommit4.commitTxAndRemoteSig.commitTx.tx.txOut.size == 2)
}
@ -1785,6 +1792,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.params.channelFeatures == channelFeatures)
val initOutputCount = channelFeatures.commitmentFormat match {
case _: AnchorOutputsCommitmentFormat => 4
case ZeroFeeCommitTxCommitmentFormat => 3
case DefaultCommitmentFormat => 2
}
assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx.txOut.size == initOutputCount)

View File

@ -186,7 +186,7 @@ abstract class ChannelIntegrationSpec extends IntegrationSpec {
generateBlocks(25, Some(minerAddress))
val expectedTxCountC = 1 // C should have 1 recv transaction: its main output
val expectedTxCountF = commitmentFormat match {
case _: AnchorOutputsCommitmentFormat => 2 // F should have 2 recv transactions: the redeemed htlc and its main output
case _: AnchorOutputsCommitmentFormat | Transactions.ZeroFeeCommitTxCommitmentFormat => 2 // F should have 2 recv transactions: the redeemed htlc and its main output
case Transactions.DefaultCommitmentFormat => 1 // F's main output uses static_remotekey
}
awaitCond({
@ -226,7 +226,7 @@ abstract class ChannelIntegrationSpec extends IntegrationSpec {
// we then generate enough blocks so that F gets its htlc-success delayed output
generateBlocks(25, Some(minerAddress))
val expectedTxCountC = commitmentFormat match {
case _: AnchorOutputsCommitmentFormat => 1 // C should have 1 recv transaction: its main output
case _: AnchorOutputsCommitmentFormat | Transactions.ZeroFeeCommitTxCommitmentFormat => 1 // C should have 1 recv transaction: its main output
case Transactions.DefaultCommitmentFormat => 0 // C's main output uses static_remotekey
}
val expectedTxCountF = 2 // F should have 2 recv transactions: the redeemed htlc and its main output
@ -280,7 +280,7 @@ abstract class ChannelIntegrationSpec extends IntegrationSpec {
generateBlocks(25, Some(minerAddress))
val expectedTxCountC = 2 // C should have 2 recv transactions: its main output and the htlc timeout
val expectedTxCountF = commitmentFormat match {
case _: AnchorOutputsCommitmentFormat => 1 // F should have 1 recv transaction: its main output
case _: AnchorOutputsCommitmentFormat | Transactions.ZeroFeeCommitTxCommitmentFormat => 1 // F should have 1 recv transaction: its main output
case Transactions.DefaultCommitmentFormat => 0 // F's main output uses static_remotekey
}
awaitCond({
@ -335,7 +335,7 @@ abstract class ChannelIntegrationSpec extends IntegrationSpec {
// we then generate enough blocks to confirm all delayed transactions
generateBlocks(25, Some(minerAddress))
val expectedTxCountC = commitmentFormat match {
case _: AnchorOutputsCommitmentFormat => 2 // C should have 2 recv transactions: its main output and the htlc timeout
case _: AnchorOutputsCommitmentFormat | Transactions.ZeroFeeCommitTxCommitmentFormat => 2 // C should have 2 recv transactions: its main output and the htlc timeout
case Transactions.DefaultCommitmentFormat => 1 // C's main output uses static_remotekey
}
val expectedTxCountF = 1 // F should have 1 recv transaction: its main output
@ -410,6 +410,7 @@ abstract class ChannelIntegrationSpec extends IntegrationSpec {
val localCommitF = commitmentsF.latest.localCommit
commitmentFormat match {
case Transactions.DefaultCommitmentFormat => assert(localCommitF.commitTxAndRemoteSig.commitTx.tx.txOut.size == 6)
case Transactions.ZeroFeeCommitTxCommitmentFormat => assert(localCommitF.commitTxAndRemoteSig.commitTx.tx.txOut.size == 7)
case _: Transactions.AnchorOutputsCommitmentFormat => assert(localCommitF.commitTxAndRemoteSig.commitTx.tx.txOut.size == 8)
}
val outgoingHtlcExpiry = localCommitF.spec.htlcs.collect { case OutgoingHtlc(add) => add.cltvExpiry }.max

View File

@ -208,6 +208,7 @@ trait TestVectorsSpec extends AnyFunSuite with Logging {
localPaymentBasePoint = Local.payment_basepoint,
remotePaymentBasePoint = Remote.payment_basepoint,
localIsChannelOpener = true,
commitmentFormat = commitmentFormat,
outputs = outputs)
val local_sig = Transactions.sign(tx, Local.funding_privkey, TxOwner.Local, commitmentFormat)
logger.info(s"# local_signature = ${Scripts.der(local_sig).dropRight(1).toHex}")

View File

@ -16,10 +16,11 @@
package fr.acinq.eclair.transactions
import fr.acinq.bitcoin.ScriptFlags
import fr.acinq.bitcoin.SigHash._
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, ripemd160, sha256}
import fr.acinq.bitcoin.scalacompat.Script.{pay2wpkh, pay2wsh, write}
import fr.acinq.bitcoin.scalacompat.{Btc, ByteVector32, Crypto, MilliBtc, MilliBtcDouble, OutPoint, Protocol, Satoshi, SatoshiLong, Script, ScriptWitness, Transaction, TxId, TxIn, TxOut, millibtc2satoshi}
import fr.acinq.bitcoin.scalacompat.{Block, Btc, ByteVector32, Crypto, MilliBtc, MilliBtcDouble, OutPoint, Protocol, Satoshi, SatoshiLong, Script, ScriptWitness, Transaction, TxId, TxIn, TxOut, addressFromPublicKeyScript, millibtc2satoshi}
import fr.acinq.eclair.TestUtils.randomTxId
import fr.acinq.eclair._
import fr.acinq.eclair.blockchain.fee.{ConfirmationTarget, FeeratePerKw}
@ -289,7 +290,7 @@ class TransactionsSpec extends AnyFunSuite with Logging {
val commitTxNumber = 0x404142434445L
val commitTx = {
val txInfo = makeCommitTx(commitInput, commitTxNumber, localPaymentPriv.publicKey, remotePaymentPriv.publicKey, localIsChannelOpener = true, outputs)
val txInfo = makeCommitTx(commitInput, commitTxNumber, localPaymentPriv.publicKey, remotePaymentPriv.publicKey, localIsChannelOpener = true, DefaultCommitmentFormat, outputs)
val localSig = Transactions.sign(txInfo, localPaymentPriv, TxOwner.Local, DefaultCommitmentFormat)
val remoteSig = Transactions.sign(txInfo, remotePaymentPriv, TxOwner.Remote, DefaultCommitmentFormat)
Transactions.addSigs(txInfo, localFundingPriv.publicKey, remoteFundingPriv.publicKey, localSig, remoteSig)
@ -526,7 +527,7 @@ class TransactionsSpec extends AnyFunSuite with Logging {
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, UnsafeLegacyAnchorOutputsCommitmentFormat)
val txInfo = makeCommitTx(commitInput, commitTxNumber, localPaymentPriv.publicKey, remotePaymentPriv.publicKey, localIsChannelOpener = true, outputs)
val txInfo = makeCommitTx(commitInput, commitTxNumber, localPaymentPriv.publicKey, remotePaymentPriv.publicKey, localIsChannelOpener = true, UnsafeLegacyAnchorOutputsCommitmentFormat, outputs)
val localSig = Transactions.sign(txInfo, localPaymentPriv, TxOwner.Local, UnsafeLegacyAnchorOutputsCommitmentFormat)
val remoteSig = Transactions.sign(txInfo, remotePaymentPriv, TxOwner.Remote, UnsafeLegacyAnchorOutputsCommitmentFormat)
val commitTx = Transactions.addSigs(txInfo, localFundingPriv.publicKey, remoteFundingPriv.publicKey, localSig, remoteSig)
@ -543,7 +544,7 @@ class TransactionsSpec extends AnyFunSuite with Logging {
assert(htlcSuccessTxs.map(_.htlcId).toSet == Set(1, 2, 4))
val zeroFeeOutputs = makeCommitTxOutputs(localPaysCommitTxFees = true, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, remotePaymentPriv.publicKey, localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localFundingPriv.publicKey, remoteFundingPriv.publicKey, spec, ZeroFeeHtlcTxAnchorOutputsCommitmentFormat)
val zeroFeeCommitTx = makeCommitTx(commitInput, commitTxNumber, localPaymentPriv.publicKey, remotePaymentPriv.publicKey, localIsChannelOpener = true, zeroFeeOutputs)
val zeroFeeCommitTx = makeCommitTx(commitInput, commitTxNumber, localPaymentPriv.publicKey, remotePaymentPriv.publicKey, localIsChannelOpener = true, ZeroFeeHtlcTxAnchorOutputsCommitmentFormat, zeroFeeOutputs)
val zeroFeeHtlcTxs = makeHtlcTxs(zeroFeeCommitTx.tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, spec.htlcTxFeerate(ZeroFeeHtlcTxAnchorOutputsCommitmentFormat), zeroFeeOutputs, ZeroFeeHtlcTxAnchorOutputsCommitmentFormat)
assert(zeroFeeHtlcTxs.length == 7)
val zeroFeeConfirmationTargets = zeroFeeHtlcTxs.map(tx => tx.htlcId -> tx.confirmationTarget.confirmBefore.toLong).toMap
@ -743,6 +744,96 @@ class TransactionsSpec extends AnyFunSuite with Logging {
}
}
test("generate valid commitment and htlc transactions (zero-fee commitment)") {
val finalPubKeyScript = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32()).publicKey))
val commitInput = Funding.makeFundingInputInfo(randomTxId(), 0, 1_000_000 sat, localFundingPriv.publicKey, remoteFundingPriv.publicKey)
val paymentPreimage1 = randomBytes32()
val htlc1 = UpdateAddHtlc(ByteVector32.Zeroes, 0, 100_000_000 msat, sha256(paymentPreimage1), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0)
val paymentPreimage2 = randomBytes32()
val htlc2a = UpdateAddHtlc(ByteVector32.Zeroes, 1, 50_000_000 msat, sha256(paymentPreimage2), CltvExpiry(310), TestConstants.emptyOnionPacket, None, 1.0)
val htlc2b = UpdateAddHtlc(ByteVector32.Zeroes, 2, 150_000_000 msat, sha256(paymentPreimage2), CltvExpiry(310), TestConstants.emptyOnionPacket, None, 1.0)
// htlc3 and htlc4 are dust IN/OUT htlcs.
val paymentPreimage3 = randomBytes32()
val htlc3 = UpdateAddHtlc(ByteVector32.Zeroes, 3, 501_000 msat, sha256(paymentPreimage3), CltvExpiry(295), TestConstants.emptyOnionPacket, None, 1.0)
val paymentPreimage4 = randomBytes32()
val htlc4 = UpdateAddHtlc(ByteVector32.Zeroes, 4, 499_000 msat, sha256(paymentPreimage4), CltvExpiry(300), TestConstants.emptyOnionPacket, None, 1.0)
val spec = CommitmentSpec(
htlcs = Set(
OutgoingHtlc(htlc1),
IncomingHtlc(htlc2a),
IncomingHtlc(htlc2b),
OutgoingHtlc(htlc3),
IncomingHtlc(htlc4),
),
commitTxFeerate = feeratePerKw,
toLocal = 399_499_000 msat,
toRemote = 299_501_000 msat
)
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, ZeroFeeCommitTxCommitmentFormat)
val txInfo = makeCommitTx(commitInput, commitTxNumber, localPaymentPriv.publicKey, remotePaymentPriv.publicKey, localIsChannelOpener = true, ZeroFeeCommitTxCommitmentFormat, outputs)
val localSig = Transactions.sign(txInfo, localPaymentPriv, TxOwner.Local, ZeroFeeCommitTxCommitmentFormat)
val remoteSig = Transactions.sign(txInfo, remotePaymentPriv, TxOwner.Remote, ZeroFeeCommitTxCommitmentFormat)
val commitTx = Transactions.addSigs(txInfo, localFundingPriv.publicKey, remoteFundingPriv.publicKey, localSig, remoteSig)
assert(commitTx.tx.version == 3)
assert(commitTx.tx.txOut.map(_.amount).sum == 1_000_000.sat)
val htlcTxs = makeHtlcTxs(commitTx.tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, spec.htlcTxFeerate(ZeroFeeCommitTxCommitmentFormat), outputs, ZeroFeeCommitTxCommitmentFormat)
assert(htlcTxs.length == 3)
val htlcSuccessTxs = htlcTxs.collect { case tx: HtlcSuccessTx => tx }
val htlcTimeoutTxs = htlcTxs.collect { case tx: HtlcTimeoutTx => tx }
assert(htlcTimeoutTxs.map(_.htlcId).toSet == Set(0))
assert(htlcSuccessTxs.map(_.htlcId).toSet == Set(1, 2))
(commitTx, outputs, htlcTimeoutTxs, htlcSuccessTxs)
}
{
// either side spends shared anchor
val Right(claimAnchorOutputTx) = makeClaimSharedAnchorOutputTx(commitTx.tx, ConfirmationTarget.Absolute(BlockHeight(1105)))
assert(claimAnchorOutputTx.input.txOut.amount == 1_000.sat)
assert(claimAnchorOutputTx.tx.version == 3)
assert(addressFromPublicKeyScript(Block.LivenetGenesisBlock.hash, Script.parse(claimAnchorOutputTx.input.txOut.publicKeyScript)).contains("bc1pfeessrawgf"))
}
{
// local spends main delayed output
val Right(claimMainOutputTx) = makeClaimLocalDelayedOutputTx(commitTx.tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
val localSig = sign(claimMainOutputTx, localDelayedPaymentPriv, TxOwner.Local, ZeroFeeCommitTxCommitmentFormat)
val signedTx = addSigs(claimMainOutputTx, localSig)
assert(signedTx.tx.version == 3)
assert(checkSpendable(signedTx).isSuccess)
}
{
// remote spends main output
val Right(claimP2WPKHOutputTx) = makeClaimP2WPKHOutputTx(commitTx.tx, localDustLimit, remotePaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
val localSig = sign(claimP2WPKHOutputTx, remotePaymentPriv, TxOwner.Local, ZeroFeeCommitTxCommitmentFormat)
val signedTx = addSigs(claimP2WPKHOutputTx, remotePaymentPriv.publicKey, localSig)
assert(signedTx.tx.version == 3)
assert(checkSpendable(signedTx).isSuccess)
}
{
// either party spends local->remote htlc output with htlc timeout tx
for (htlcTimeoutTx <- htlcTimeoutTxs) {
val localSig = sign(htlcTimeoutTx, localHtlcPriv, TxOwner.Local, ZeroFeeCommitTxCommitmentFormat)
val remoteSig = sign(htlcTimeoutTx, remoteHtlcPriv, TxOwner.Remote, ZeroFeeCommitTxCommitmentFormat)
val signed = addSigs(htlcTimeoutTx, localSig, remoteSig, ZeroFeeCommitTxCommitmentFormat)
assert(signed.tx.version == 3)
assert(checkSpendable(signed).isSuccess)
}
}
{
// 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 = sign(htlcDelayed, localDelayedPaymentPriv, TxOwner.Local, ZeroFeeCommitTxCommitmentFormat)
val signedTx = addSigs(htlcDelayed, localSig)
assert(signedTx.tx.version == 3)
assert(checkSpendable(signedTx).isSuccess)
}
}
test("sort the htlc outputs using BIP69 and cltv expiry") {
val localFundingPriv = PrivateKey(hex"a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1a1")
val remoteFundingPriv = PrivateKey(hex"a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2a2")
@ -781,7 +872,7 @@ class TransactionsSpec extends AnyFunSuite with Logging {
val commitTxNumber = 0x404142434446L
val (commitTx, outputs, htlcTxs) = {
val outputs = makeCommitTxOutputs(localPaysCommitTxFees = true, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, remotePaymentPriv.publicKey, localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, localFundingPriv.publicKey, remoteFundingPriv.publicKey, spec, DefaultCommitmentFormat)
val txInfo = makeCommitTx(commitInput, commitTxNumber, localPaymentPriv.publicKey, remotePaymentPriv.publicKey, localIsChannelOpener = true, outputs)
val txInfo = makeCommitTx(commitInput, commitTxNumber, localPaymentPriv.publicKey, remotePaymentPriv.publicKey, localIsChannelOpener = true, DefaultCommitmentFormat, outputs)
val localSig = Transactions.sign(txInfo, localPaymentPriv, TxOwner.Local, DefaultCommitmentFormat)
val remoteSig = Transactions.sign(txInfo, remotePaymentPriv, TxOwner.Remote, DefaultCommitmentFormat)
val commitTx = Transactions.addSigs(txInfo, localFundingPriv.publicKey, remoteFundingPriv.publicKey, localSig, remoteSig)