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

Handle aggregated anchor outputs htlc txs (#1738)

An interesting side-effect of anchor outputs is that htlc txs can be merged
when they have the same lockTime (thanks to sighash flags).

We're not currently doing that, but our peers may do it, so we need to handle
it in the revoked commit tx case and correctly claim multiple outputs if
necessary.
This commit is contained in:
Bastien Teinturier 2021-03-25 14:27:50 +01:00 committed by GitHub
parent f202587e9c
commit c37eb1ad54
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 155 additions and 53 deletions

View File

@ -134,7 +134,7 @@ class BitcoinCoreWallet(rpcClient: BitcoinJsonRPCClient)(implicit ec: ExecutionC
// now let's sign the funding tx
SignTransactionResponse(fundingTx, true) <- signTransactionOrUnlock(fundTxResponse.tx)
// there will probably be a change output, so we need to find which output is ours
outputIndex <- Transactions.findPubKeyScriptIndex(fundingTx, pubkeyScript, amount_opt = None) match {
outputIndex <- Transactions.findPubKeyScriptIndex(fundingTx, pubkeyScript) match {
case Right(outputIndex) => Future.successful(outputIndex)
case Left(skipped) => Future.failed(new RuntimeException(skipped.toString))
}

View File

@ -1340,9 +1340,9 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
}
}
val revokedCommitPublished1 = d.revokedCommitPublished.map { rev =>
val (rev1, tx_opt) = Closing.claimRevokedHtlcTxOutputs(keyManager, d.commitments, rev, tx, nodeParams.onChainFeeConf.feeEstimator)
tx_opt.foreach(claimTx => blockchain ! PublishAsap(claimTx.tx, PublishStrategy.JustPublish))
tx_opt.foreach(claimTx => blockchain ! WatchSpent(self, tx, claimTx.input.outPoint.index.toInt, BITCOIN_OUTPUT_SPENT, hints = Set(claimTx.tx.txid)))
val (rev1, penaltyTxs) = Closing.claimRevokedHtlcTxOutputs(keyManager, d.commitments, rev, tx, nodeParams.onChainFeeConf.feeEstimator)
penaltyTxs.foreach(claimTx => blockchain ! PublishAsap(claimTx.tx, PublishStrategy.JustPublish))
penaltyTxs.foreach(claimTx => blockchain ! WatchSpent(self, tx, claimTx.input.outPoint.index.toInt, BITCOIN_OUTPUT_SPENT, hints = Set(claimTx.tx.txid)))
rev1
}
stay using d.copy(revokedCommitPublished = revokedCommitPublished1) storing()

View File

@ -850,8 +850,11 @@ object Helpers {
* - by spending the delayed output of [[HtlcSuccessTx]] and [[HtlcTimeoutTx]] if those get confirmed; because the output of these txs is protected by
* an OP_CSV delay, we will have time to spend them with a revocation key. In that case, we generate the spending transactions "on demand",
* this is the purpose of this method.
*
* NB: when anchor outputs is used, htlc transactions can be aggregated in a single transaction if they share the same
* lockTime (thanks to the use of sighash_single | sighash_anyonecanpay), so we may need to claim multiple outputs.
*/
def claimRevokedHtlcTxOutputs(keyManager: ChannelKeyManager, commitments: Commitments, revokedCommitPublished: RevokedCommitPublished, htlcTx: Transaction, feeEstimator: FeeEstimator)(implicit log: LoggingAdapter): (RevokedCommitPublished, Option[ClaimHtlcDelayedOutputPenaltyTx]) = {
def claimRevokedHtlcTxOutputs(keyManager: ChannelKeyManager, commitments: Commitments, revokedCommitPublished: RevokedCommitPublished, htlcTx: Transaction, feeEstimator: FeeEstimator)(implicit log: LoggingAdapter): (RevokedCommitPublished, Seq[ClaimHtlcDelayedOutputPenaltyTx]) = {
val isHtlcTx = htlcTx.txIn.map(_.outPoint.txid).contains(revokedCommitPublished.commitTx.txid) &&
htlcTx.txIn.map(_.witness).collect(Scripts.extractPreimageFromHtlcSuccess.orElse(Scripts.extractPaymentHashFromHtlcTimeout)).nonEmpty
if (isHtlcTx) {
@ -867,7 +870,7 @@ object Helpers {
// now we know what commit number this tx is referring to, we can derive the commitment point from the shachain
remotePerCommitmentSecrets.getHash(0xFFFFFFFFFFFFL - txNumber)
.map(d => PrivateKey(d))
.flatMap(remotePerCommitmentSecret => {
.map(remotePerCommitmentSecret => {
val remotePerCommitmentPoint = remotePerCommitmentSecret.publicKey
val remoteDelayedPaymentPubkey = Generators.derivePubKey(remoteParams.delayedPaymentBasepoint, remotePerCommitmentPoint)
val remoteRevocationPubkey = Generators.revocationPubKey(keyManager.revocationPoint(channelKeyPath).publicKey, remotePerCommitmentPoint)
@ -875,24 +878,22 @@ object Helpers {
// we need to use a high fee here for punishment txs because after a delay they can be spent by the counterparty
val feeratePerKwPenalty = feeEstimator.getFeeratePerKw(target = 1)
generateTx("claim-htlc-delayed-penalty") {
Transactions.makeClaimHtlcDelayedOutputPenaltyTx(htlcTx, localParams.dustLimit, remoteRevocationPubkey, localParams.toSelfDelay, remoteDelayedPaymentPubkey, localParams.defaultFinalScriptPubKey, feeratePerKwPenalty).map(htlcDelayedPenalty => {
val sig = keyManager.sign(htlcDelayedPenalty, keyManager.revocationPoint(channelKeyPath), remotePerCommitmentSecret, TxOwner.Local, commitmentFormat)
val signedTx = Transactions.addSigs(htlcDelayedPenalty, sig)
// we need to make sure that the tx is indeed valid
Transaction.correctlySpends(signedTx.tx, Seq(htlcTx), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
signedTx
})
}
}) match {
case Some(tx) =>
val revokedCommitPublished1 = revokedCommitPublished.copy(claimHtlcDelayedPenaltyTxs = revokedCommitPublished.claimHtlcDelayedPenaltyTxs :+ tx)
(revokedCommitPublished1, Some(tx))
case None =>
(revokedCommitPublished, None)
}
val penaltyTxs = Transactions.makeClaimHtlcDelayedOutputPenaltyTxs(htlcTx, localParams.dustLimit, remoteRevocationPubkey, localParams.toSelfDelay, remoteDelayedPaymentPubkey, localParams.defaultFinalScriptPubKey, feeratePerKwPenalty).flatMap(claimHtlcDelayedOutputPenaltyTx => {
generateTx("claim-htlc-delayed-penalty") {
claimHtlcDelayedOutputPenaltyTx.map(htlcDelayedPenalty => {
val sig = keyManager.sign(htlcDelayedPenalty, keyManager.revocationPoint(channelKeyPath), remotePerCommitmentSecret, TxOwner.Local, commitmentFormat)
val signedTx = Transactions.addSigs(htlcDelayedPenalty, sig)
// we need to make sure that the tx is indeed valid
Transaction.correctlySpends(signedTx.tx, Seq(htlcTx), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
signedTx
})
}
})
val revokedCommitPublished1 = revokedCommitPublished.copy(claimHtlcDelayedPenaltyTxs = revokedCommitPublished.claimHtlcDelayedPenaltyTxs ++ penaltyTxs)
(revokedCommitPublished1, penaltyTxs)
}).getOrElse((revokedCommitPublished, Nil))
} else {
(revokedCommitPublished, None)
(revokedCommitPublished, Nil)
}
}

View File

@ -568,7 +568,7 @@ object Transactions {
def makeClaimP2WPKHOutputTx(commitTx: Transaction, localDustLimit: Satoshi, localPaymentPubkey: PublicKey, localFinalScriptPubKey: ByteVector, feeratePerKw: FeeratePerKw): Either[TxGenerationSkipped, ClaimP2WPKHOutputTx] = {
val redeemScript = Script.pay2pkh(localPaymentPubkey)
val pubkeyScript = write(pay2wpkh(localPaymentPubkey))
findPubKeyScriptIndex(commitTx, pubkeyScript, amount_opt = None) match {
findPubKeyScriptIndex(commitTx, pubkeyScript) match {
case Left(skip) => Left(skip)
case Right(outputIndex) =>
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
@ -594,7 +594,7 @@ object Transactions {
def makeClaimRemoteDelayedOutputTx(commitTx: Transaction, localDustLimit: Satoshi, localPaymentPubkey: PublicKey, localFinalScriptPubKey: ByteVector, feeratePerKw: FeeratePerKw): Either[TxGenerationSkipped, ClaimRemoteDelayedOutputTx] = {
val redeemScript = toRemoteDelayed(localPaymentPubkey)
val pubkeyScript = write(pay2wsh(redeemScript))
findPubKeyScriptIndex(commitTx, pubkeyScript, amount_opt = None) match {
findPubKeyScriptIndex(commitTx, pubkeyScript) match {
case Left(skip) => Left(skip)
case Right(outputIndex) =>
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
@ -620,7 +620,7 @@ object Transactions {
def makeClaimLocalDelayedOutputTx(commitTx: Transaction, localDustLimit: Satoshi, localRevocationPubkey: PublicKey, toLocalDelay: CltvExpiryDelta, localDelayedPaymentPubkey: PublicKey, localFinalScriptPubKey: ByteVector, feeratePerKw: FeeratePerKw): Either[TxGenerationSkipped, ClaimLocalDelayedOutputTx] = {
val redeemScript = toLocalDelayed(localRevocationPubkey, toLocalDelay, localDelayedPaymentPubkey)
val pubkeyScript = write(pay2wsh(redeemScript))
findPubKeyScriptIndex(commitTx, pubkeyScript, amount_opt = None) match {
findPubKeyScriptIndex(commitTx, pubkeyScript) match {
case Left(skip) => Left(skip)
case Right(outputIndex) =>
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
@ -646,7 +646,7 @@ object Transactions {
private def makeClaimAnchorOutputTx(commitTx: Transaction, fundingPubkey: PublicKey): Either[TxGenerationSkipped, (InputInfo, Transaction)] = {
val redeemScript = anchor(fundingPubkey)
val pubkeyScript = write(pay2wsh(redeemScript))
findPubKeyScriptIndex(commitTx, pubkeyScript, amount_opt = None) match {
findPubKeyScriptIndex(commitTx, pubkeyScript) match {
case Left(skip) => Left(skip)
case Right(outputIndex) =>
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
@ -668,12 +668,12 @@ object Transactions {
makeClaimAnchorOutputTx(commitTx, remoteFundingPubkey).map { case (input, tx) => ClaimRemoteAnchorOutputTx(input, tx) }
}
def makeClaimHtlcDelayedOutputPenaltyTx(htlcTx: Transaction, localDustLimit: Satoshi, localRevocationPubkey: PublicKey, toLocalDelay: CltvExpiryDelta, localDelayedPaymentPubkey: PublicKey, localFinalScriptPubKey: ByteVector, feeratePerKw: FeeratePerKw): Either[TxGenerationSkipped, ClaimHtlcDelayedOutputPenaltyTx] = {
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))
findPubKeyScriptIndex(htlcTx, pubkeyScript, amount_opt = None) match {
case Left(skip) => Left(skip)
case Right(outputIndex) =>
findPubKeyScriptIndexes(htlcTx, pubkeyScript) match {
case Left(skip) => Seq(Left(skip))
case Right(outputIndexes) => outputIndexes.map(outputIndex => {
val input = InputInfo(OutPoint(htlcTx, outputIndex), htlcTx.txOut(outputIndex), write(redeemScript))
// unsigned transaction
val tx = Transaction(
@ -691,13 +691,14 @@ object Transactions {
val tx1 = tx.copy(txOut = tx.txOut.head.copy(amount = amount) :: Nil)
Right(ClaimHtlcDelayedOutputPenaltyTx(input, tx1))
}
})
}
}
def makeMainPenaltyTx(commitTx: Transaction, localDustLimit: Satoshi, remoteRevocationPubkey: PublicKey, localFinalScriptPubKey: ByteVector, toRemoteDelay: CltvExpiryDelta, remoteDelayedPaymentPubkey: PublicKey, feeratePerKw: FeeratePerKw): Either[TxGenerationSkipped, MainPenaltyTx] = {
val redeemScript = toLocalDelayed(remoteRevocationPubkey, toRemoteDelay, remoteDelayedPaymentPubkey)
val pubkeyScript = write(pay2wsh(redeemScript))
findPubKeyScriptIndex(commitTx, pubkeyScript, amount_opt = None) match {
findPubKeyScriptIndex(commitTx, pubkeyScript) match {
case Left(skip) => Left(skip)
case Right(outputIndex) =>
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
@ -760,14 +761,12 @@ object Transactions {
txIn = TxIn(commitTxInput.outPoint, ByteVector.empty, sequence = 0xffffffffL) :: Nil,
txOut = toLocalOutput_opt.toSeq ++ toRemoteOutput_opt.toSeq ++ Nil,
lockTime = 0))
val toLocalOutput = findPubKeyScriptIndex(tx, localScriptPubKey, None).map(index => OutputInfo(index, toLocalAmount, localScriptPubKey)).toOption
val toLocalOutput = findPubKeyScriptIndex(tx, localScriptPubKey).map(index => OutputInfo(index, toLocalAmount, localScriptPubKey)).toOption
ClosingTx(commitTxInput, tx, toLocalOutput)
}
def findPubKeyScriptIndex(tx: Transaction, pubkeyScript: ByteVector, amount_opt: Option[Satoshi]): Either[TxGenerationSkipped, Int] = {
val outputIndex = tx.txOut
.zipWithIndex
.indexWhere { case (txOut, _) => amount_opt.forall(_ == txOut.amount) && txOut.publicKeyScript == pubkeyScript }
def findPubKeyScriptIndex(tx: Transaction, pubkeyScript: ByteVector): Either[TxGenerationSkipped, Int] = {
val outputIndex = tx.txOut.indexWhere(_.publicKeyScript == pubkeyScript)
if (outputIndex >= 0) {
Right(outputIndex)
} else {
@ -775,6 +774,17 @@ object Transactions {
}
}
def findPubKeyScriptIndexes(tx: Transaction, pubkeyScript: ByteVector): Either[TxGenerationSkipped, Seq[Int]] = {
val outputIndexes = tx.txOut.zipWithIndex.collect {
case (txOut, index) if txOut.publicKeyScript == pubkeyScript => index
}
if (outputIndexes.nonEmpty) {
Right(outputIndexes)
} else {
Left(OutputNotFound)
}
}
/**
* Default public key used for fee estimation
*/

View File

@ -18,7 +18,7 @@ package fr.acinq.eclair.channel.states.h
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.Crypto.PrivateKey
import fr.acinq.bitcoin.{ByteVector32, OutPoint, SatoshiLong, ScriptFlags, Transaction, TxIn}
import fr.acinq.bitcoin.{ByteVector32, Crypto, OutPoint, SatoshiLong, Script, ScriptFlags, Transaction, TxIn, TxOut}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.blockchain.fee.{FeeratePerKw, FeeratesPerKw}
@ -27,10 +27,10 @@ import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.states.{StateTestsBase, StateTestsTags}
import fr.acinq.eclair.payment._
import fr.acinq.eclair.payment.relay.Relayer._
import fr.acinq.eclair.transactions.Transactions.{HtlcSuccessTx, HtlcTimeoutTx}
import fr.acinq.eclair.transactions.Transactions.{AnchorOutputsCommitmentFormat, HtlcSuccessTx, HtlcTimeoutTx}
import fr.acinq.eclair.transactions.{Scripts, Transactions}
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{CltvExpiry, MilliSatoshiLong, TestConstants, TestKitBaseClass, randomBytes32}
import fr.acinq.eclair.{CltvExpiry, MilliSatoshiLong, TestConstants, TestKitBaseClass, randomBytes32, randomKey}
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
import org.scalatest.{Outcome, Tag}
import scodec.bits.ByteVector
@ -1442,6 +1442,83 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
testOutputSpentRevokedTx(f, ChannelVersion.ANCHOR_OUTPUTS)
}
test("recv BITCOIN_OUTPUT_SPENT (one revoked tx, counterparty published aggregated htlc tx)", Tag(StateTestsTags.AnchorOutputs)) { f =>
import f._
// bob publishes one of his revoked txs
val revokedCloseFixture = prepareRevokedClose(f, ChannelVersion.ANCHOR_OUTPUTS)
val bobRevokedTxs = revokedCloseFixture.bobRevokedTxs(2)
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, bobRevokedTxs.commitTx.tx)
awaitCond(alice.stateData.isInstanceOf[DATA_CLOSING])
assert(alice.stateData.asInstanceOf[DATA_CLOSING].commitments.commitmentFormat === AnchorOutputsCommitmentFormat)
awaitCond(alice.stateData.asInstanceOf[DATA_CLOSING].revokedCommitPublished.size == 1)
val rvk = alice.stateData.asInstanceOf[DATA_CLOSING].revokedCommitPublished.head
assert(rvk.commitTx === bobRevokedTxs.commitTx.tx)
assert(rvk.htlcPenaltyTxs.size === 4)
assert(rvk.claimHtlcDelayedPenaltyTxs.isEmpty)
// alice publishes the penalty txs and watches outputs
(1 to 6).foreach(_ => alice2blockchain.expectMsgType[PublishAsap]) // 2 main outputs and 4 htlcs
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === rvk.commitTx.txid)
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === rvk.claimMainOutputTx.get.tx.txid)
(1 to 5).foreach(_ => alice2blockchain.expectMsgType[WatchSpent]) // main output penalty and 4 htlc penalties
alice2blockchain.expectNoMsg(1 second)
// bob claims multiple htlc outputs in a single transaction (this is possible with anchor outputs because signatures
// use sighash_single | sighash_anyonecanpay)
val bobHtlcTxs = bobRevokedTxs.htlcTxsAndSigs.collect {
case HtlcTxAndSigs(txInfo: HtlcSuccessTx, localSig, remoteSig) =>
val preimage = revokedCloseFixture.htlcsAlice.collectFirst { case (add, preimage) if add.id == txInfo.htlcId => preimage }.get
assert(Crypto.sha256(preimage) === txInfo.paymentHash)
Transactions.addSigs(txInfo, localSig, remoteSig, preimage, AnchorOutputsCommitmentFormat)
case HtlcTxAndSigs(txInfo: HtlcTimeoutTx, localSig, remoteSig) =>
Transactions.addSigs(txInfo, localSig, remoteSig, AnchorOutputsCommitmentFormat)
}
assert(bobHtlcTxs.map(_.input.outPoint).size === 4)
val bobHtlcTx = Transaction(
2,
Seq(
TxIn(OutPoint(randomBytes32, 4), Nil, 1), // utxo used for fee bumping
bobHtlcTxs(0).tx.txIn.head,
bobHtlcTxs(1).tx.txIn.head,
bobHtlcTxs(2).tx.txIn.head,
bobHtlcTxs(3).tx.txIn.head
),
Seq(
TxOut(10000 sat, Script.pay2wpkh(randomKey.publicKey)), // change output
bobHtlcTxs(0).tx.txOut.head,
bobHtlcTxs(1).tx.txOut.head,
bobHtlcTxs(2).tx.txOut.head,
bobHtlcTxs(3).tx.txOut.head
),
0
)
// alice reacts by publishing penalty txs that spend bob's htlc transaction
alice ! WatchEventSpent(BITCOIN_OUTPUT_SPENT, bobHtlcTx)
awaitCond(alice.stateData.asInstanceOf[DATA_CLOSING].revokedCommitPublished.head.claimHtlcDelayedPenaltyTxs.size == 4)
val claimHtlcDelayedPenaltyTxs = alice.stateData.asInstanceOf[DATA_CLOSING].revokedCommitPublished.head.claimHtlcDelayedPenaltyTxs
val spentOutpoints = Set(OutPoint(bobHtlcTx, 1), OutPoint(bobHtlcTx, 2), OutPoint(bobHtlcTx, 3), OutPoint(bobHtlcTx, 4))
assert(claimHtlcDelayedPenaltyTxs.map(_.input.outPoint).toSet === spentOutpoints)
claimHtlcDelayedPenaltyTxs.foreach(claimHtlcPenalty => Transaction.correctlySpends(claimHtlcPenalty.tx, bobHtlcTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS))
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === bobHtlcTx.txid)
val publishedPenaltyTxs = Set(
alice2blockchain.expectMsgType[PublishAsap],
alice2blockchain.expectMsgType[PublishAsap],
alice2blockchain.expectMsgType[PublishAsap],
alice2blockchain.expectMsgType[PublishAsap]
)
assert(publishedPenaltyTxs.map(_.tx) === claimHtlcDelayedPenaltyTxs.map(_.tx).toSet)
val watchedOutpoints = Seq(
alice2blockchain.expectMsgType[WatchSpent],
alice2blockchain.expectMsgType[WatchSpent],
alice2blockchain.expectMsgType[WatchSpent],
alice2blockchain.expectMsgType[WatchSpent]
).map(w => OutPoint(w.txId.reverse, w.outputIndex)).toSet
assert(watchedOutpoints === spentOutpoints)
alice2blockchain.expectNoMsg(1 second)
}
private def testRevokedTxConfirmed(f: FixtureParam, channelVersion: ChannelVersion): Unit = {
import f._
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.channelVersion === channelVersion)

View File

@ -385,13 +385,13 @@ class TransactionsSpec extends AnyFunSuite with Logging {
}
{
// remote spends htlc1's htlc-timeout tx with revocation key
val Right(claimHtlcDelayedPenaltyTx) = makeClaimHtlcDelayedOutputPenaltyTx(htlcTimeoutTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
val Seq(Right(claimHtlcDelayedPenaltyTx)) = makeClaimHtlcDelayedOutputPenaltyTxs(htlcTimeoutTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
val sig = sign(claimHtlcDelayedPenaltyTx, localRevocationPriv, TxOwner.Local, DefaultCommitmentFormat)
val signed = addSigs(claimHtlcDelayedPenaltyTx, sig)
assert(checkSpendable(signed).isSuccess)
// remote can't claim revoked output of htlc3's htlc-timeout tx because it is below the dust limit
val claimHtlcDelayedPenaltyTx1 = makeClaimHtlcDelayedOutputPenaltyTx(htlcTimeoutTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
assert(claimHtlcDelayedPenaltyTx1 === Left(AmountBelowDustLimit))
val claimHtlcDelayedPenaltyTx1 = makeClaimHtlcDelayedOutputPenaltyTxs(htlcTimeoutTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
assert(claimHtlcDelayedPenaltyTx1 === Seq(Left(AmountBelowDustLimit)))
}
{
// remote spends offered HTLC output with revocation key
@ -407,13 +407,13 @@ class TransactionsSpec extends AnyFunSuite with Logging {
}
{
// remote spends htlc2's htlc-success tx with revocation key
val Right(claimHtlcDelayedPenaltyTx) = makeClaimHtlcDelayedOutputPenaltyTx(htlcSuccessTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
val Seq(Right(claimHtlcDelayedPenaltyTx)) = makeClaimHtlcDelayedOutputPenaltyTxs(htlcSuccessTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
val sig = sign(claimHtlcDelayedPenaltyTx, localRevocationPriv, TxOwner.Local, DefaultCommitmentFormat)
val signed = addSigs(claimHtlcDelayedPenaltyTx, sig)
assert(checkSpendable(signed).isSuccess)
// remote can't claim revoked output of htlc4's htlc-success tx because it is below the dust limit
val claimHtlcDelayedPenaltyTx1 = makeClaimHtlcDelayedOutputPenaltyTx(htlcSuccessTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
assert(claimHtlcDelayedPenaltyTx1 === Left(AmountBelowDustLimit))
val claimHtlcDelayedPenaltyTx1 = makeClaimHtlcDelayedOutputPenaltyTxs(htlcSuccessTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
assert(claimHtlcDelayedPenaltyTx1 === Seq(Left(AmountBelowDustLimit)))
}
{
// remote spends received HTLC output with revocation key
@ -639,13 +639,13 @@ class TransactionsSpec extends AnyFunSuite with Logging {
}
{
// remote spends htlc1's htlc-timeout tx with revocation key
val Right(claimHtlcDelayedPenaltyTx) = makeClaimHtlcDelayedOutputPenaltyTx(htlcTimeoutTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
val Seq(Right(claimHtlcDelayedPenaltyTx)) = makeClaimHtlcDelayedOutputPenaltyTxs(htlcTimeoutTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
val sig = sign(claimHtlcDelayedPenaltyTx, localRevocationPriv, TxOwner.Local, AnchorOutputsCommitmentFormat)
val signed = addSigs(claimHtlcDelayedPenaltyTx, sig)
assert(checkSpendable(signed).isSuccess)
// remote can't claim revoked output of htlc3's htlc-timeout tx because it is below the dust limit
val claimHtlcDelayedPenaltyTx1 = makeClaimHtlcDelayedOutputPenaltyTx(htlcTimeoutTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
assert(claimHtlcDelayedPenaltyTx1 === Left(AmountBelowDustLimit))
val claimHtlcDelayedPenaltyTx1 = makeClaimHtlcDelayedOutputPenaltyTxs(htlcTimeoutTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
assert(claimHtlcDelayedPenaltyTx1 === Seq(Left(AmountBelowDustLimit)))
}
{
// remote spends remote->local htlc output directly in case of timeout
@ -658,16 +658,30 @@ class TransactionsSpec extends AnyFunSuite with Logging {
}
{
// remote spends htlc2a/htlc2b's htlc-success tx with revocation key
val Right(claimHtlcDelayedPenaltyTxA) = makeClaimHtlcDelayedOutputPenaltyTx(htlcSuccessTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
val Right(claimHtlcDelayedPenaltyTxB) = makeClaimHtlcDelayedOutputPenaltyTx(htlcSuccessTxs(2).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
val Seq(Right(claimHtlcDelayedPenaltyTxA)) = makeClaimHtlcDelayedOutputPenaltyTxs(htlcSuccessTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
val Seq(Right(claimHtlcDelayedPenaltyTxB)) = makeClaimHtlcDelayedOutputPenaltyTxs(htlcSuccessTxs(2).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
for (claimHtlcSuccessPenaltyTx <- Seq(claimHtlcDelayedPenaltyTxA, claimHtlcDelayedPenaltyTxB)) {
val sig = sign(claimHtlcSuccessPenaltyTx, localRevocationPriv, TxOwner.Local, AnchorOutputsCommitmentFormat)
val signed = addSigs(claimHtlcSuccessPenaltyTx, sig)
assert(checkSpendable(signed).isSuccess)
}
// remote can't claim revoked output of htlc4's htlc-success tx because it is below the dust limit
val claimHtlcDelayedPenaltyTx1 = makeClaimHtlcDelayedOutputPenaltyTx(htlcSuccessTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
assert(claimHtlcDelayedPenaltyTx1 === Left(AmountBelowDustLimit))
val claimHtlcDelayedPenaltyTx1 = makeClaimHtlcDelayedOutputPenaltyTxs(htlcSuccessTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
assert(claimHtlcDelayedPenaltyTx1 === Seq(Left(AmountBelowDustLimit)))
}
{
// 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)
assert(claimHtlcDelayedPenaltyTxs.size === 5)
val skipped = claimHtlcDelayedPenaltyTxs.collect { case Left(reason) => reason }
assert(skipped.size === 2)
assert(skipped.toSet === Set(AmountBelowDustLimit))
val claimed = claimHtlcDelayedPenaltyTxs.collect { case Right(tx) => tx }
assert(claimed.size === 3)
assert(claimed.map(_.input.outPoint).toSet.size === 3)
}
{
// remote spends offered htlc output with revocation key