1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-02-23 06:35:11 +01:00

Use bumpforceclose RPC to also bump remote commit fees (#2744)

We want to be able to bump a force-close regardless of which commitment
was published. If the remote commitment is in our mempool, we will use
our anchor on that commit tx to bump the fees.
This commit is contained in:
Bastien Teinturier 2023-09-22 17:53:42 +02:00 committed by GitHub
parent 55f9698714
commit d4c502a7d6
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
12 changed files with 272 additions and 65 deletions

View file

@ -23,17 +23,30 @@ import fr.acinq.eclair.channel.{ChannelTypes, SupportedChannelType}
import fr.acinq.eclair.transactions.Transactions
// @formatter:off
sealed trait ConfirmationPriority {
sealed trait ConfirmationPriority extends Ordered[ConfirmationPriority] {
def underlying: Int
def getFeerate(feerates: FeeratesPerKw): FeeratePerKw = this match {
case ConfirmationPriority.Slow => feerates.slow
case ConfirmationPriority.Medium => feerates.medium
case ConfirmationPriority.Fast => feerates.fast
}
override def compare(that: ConfirmationPriority): Int = this.underlying.compare(that.underlying)
}
object ConfirmationPriority {
case object Slow extends ConfirmationPriority { override def toString = "slow" }
case object Medium extends ConfirmationPriority { override def toString = "medium" }
case object Fast extends ConfirmationPriority { override def toString = "fast" }
case object Slow extends ConfirmationPriority {
override val underlying = 1
override def toString = "slow"
}
case object Medium extends ConfirmationPriority {
override val underlying = 2
override def toString = "medium"
}
case object Fast extends ConfirmationPriority {
override val underlying = 3
override def toString = "fast"
}
}
sealed trait ConfirmationTarget
object ConfirmationTarget {

View file

@ -718,6 +718,25 @@ object Helpers {
if (isInitiator) commitInput.txOut.amount - commitTx.txOut.map(_.amount).sum else 0 sat
}
/**
* This function checks if the proposed confirmation target is more aggressive than whatever confirmation target
* 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
}
}
}
object LocalClose {
/**
@ -763,16 +782,20 @@ object Helpers {
}
def claimAnchors(keyManager: ChannelKeyManager, commitment: FullCommitment, lcp: LocalCommitPublished, confirmationTarget: ConfirmationTarget)(implicit log: LoggingAdapter): LocalCommitPublished = {
val localFundingPubKey = keyManager.fundingPublicKey(commitment.localParams.fundingKeyPath, commitment.fundingTxIndex).publicKey
val claimAnchorTxs = List(
withTxGenerationLog("local-anchor") {
Transactions.makeClaimLocalAnchorOutputTx(lcp.commitTx, localFundingPubKey, confirmationTarget)
},
withTxGenerationLog("remote-anchor") {
Transactions.makeClaimRemoteAnchorOutputTx(lcp.commitTx, commitment.remoteFundingPubKey)
}
).flatten
lcp.copy(claimAnchorTxs = claimAnchorTxs)
if (shouldUpdateAnchorTxs(lcp.claimAnchorTxs, confirmationTarget)) {
val localFundingPubKey = keyManager.fundingPublicKey(commitment.localParams.fundingKeyPath, commitment.fundingTxIndex).publicKey
val claimAnchorTxs = List(
withTxGenerationLog("local-anchor") {
Transactions.makeClaimLocalAnchorOutputTx(lcp.commitTx, localFundingPubKey, confirmationTarget)
},
withTxGenerationLog("remote-anchor") {
Transactions.makeClaimRemoteAnchorOutputTx(lcp.commitTx, commitment.remoteFundingPubKey)
}
).flatten
lcp.copy(claimAnchorTxs = claimAnchorTxs)
} else {
lcp
}
}
/**
@ -862,30 +885,38 @@ object Helpers {
val htlcTxs: Map[OutPoint, Option[ClaimHtlcTx]] = claimHtlcOutputs(keyManager, commitment, remoteCommit, feerates, finalScriptPubKey)
val spendAnchors = htlcTxs.nonEmpty || onChainFeeConf.spendAnchorWithoutHtlcs
val claimAnchorTxs: List[ClaimAnchorOutputTx] = if (spendAnchors) {
// If we don't have pending HTLCs, we don't have funds at risk, so we use the normal closing priority.
val confirmCommitBefore = htlcTxs.values.flatten.map(htlcTx => htlcTx.confirmationTarget).minByOption(_.confirmBefore).getOrElse(ConfirmationTarget.Priority(onChainFeeConf.feeTargets.closing))
val localFundingPubkey = keyManager.fundingPublicKey(commitment.localParams.fundingKeyPath, commitment.fundingTxIndex).publicKey
List(
withTxGenerationLog("local-anchor") {
Transactions.makeClaimLocalAnchorOutputTx(tx, localFundingPubkey, confirmCommitBefore)
},
withTxGenerationLog("remote-anchor") {
Transactions.makeClaimRemoteAnchorOutputTx(tx, commitment.remoteFundingPubKey)
}
).flatten
} else {
Nil
}
RemoteCommitPublished(
val rcp = RemoteCommitPublished(
commitTx = tx,
claimMainOutputTx = claimMainOutput(keyManager, commitment.params, remoteCommit.remotePerCommitmentPoint, tx, feerates, onChainFeeConf, finalScriptPubKey),
claimHtlcTxs = htlcTxs,
claimAnchorTxs = claimAnchorTxs,
claimAnchorTxs = Nil,
irrevocablySpent = Map.empty
)
val spendAnchors = htlcTxs.nonEmpty || onChainFeeConf.spendAnchorWithoutHtlcs
if (spendAnchors) {
// If we don't have pending HTLCs, we don't have funds at risk, so we use the normal closing priority.
val confirmCommitBefore = htlcTxs.values.flatten.map(htlcTx => htlcTx.confirmationTarget).minByOption(_.confirmBefore).getOrElse(ConfirmationTarget.Priority(onChainFeeConf.feeTargets.closing))
claimAnchors(keyManager, commitment, rcp, confirmCommitBefore)
} else {
rcp
}
}
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
val claimAnchorTxs = List(
withTxGenerationLog("local-anchor") {
Transactions.makeClaimLocalAnchorOutputTx(rcp.commitTx, localFundingPubkey, confirmationTarget)
},
withTxGenerationLog("remote-anchor") {
Transactions.makeClaimRemoteAnchorOutputTx(rcp.commitTx, commitment.remoteFundingPubKey)
}
).flatten
rcp.copy(claimAnchorTxs = claimAnchorTxs)
} else {
rcp
}
}
/**

View file

@ -1707,20 +1707,23 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(c: CMD_CLOSE, d: DATA_CLOSING) => handleCommandError(ClosingAlreadyInProgress(d.channelId), c)
case Event(c: CMD_BUMP_FORCE_CLOSE_FEE, d: DATA_CLOSING) =>
d.localCommitPublished match {
case Some(lcp) => d.commitments.params.commitmentFormat match {
case _: Transactions.AnchorOutputsCommitmentFormat =>
val lcp1 = Closing.LocalClose.claimAnchors(keyManager, d.commitments.latest, lcp, c.confirmationTarget)
lcp1.claimAnchorTxs.collect { case tx: Transactions.ClaimLocalAnchorOutputTx => txPublisher ! PublishReplaceableTx(tx, d.commitments.latest) }
d.commitments.params.commitmentFormat match {
case _: Transactions.AnchorOutputsCommitmentFormat =>
val lcp1 = d.localCommitPublished.map(lcp => Closing.LocalClose.claimAnchors(keyManager, d.commitments.latest, lcp, c.confirmationTarget))
val rcp1 = d.remoteCommitPublished.map(rcp => Closing.RemoteClose.claimAnchors(keyManager, d.commitments.latest, rcp, c.confirmationTarget))
val nrcp1 = d.nextRemoteCommitPublished.map(nrcp => Closing.RemoteClose.claimAnchors(keyManager, d.commitments.latest, nrcp, c.confirmationTarget))
val claimAnchorTxs = lcp1.toSeq.flatMap(_.claimAnchorTxs) ++ rcp1.toSeq.flatMap(_.claimAnchorTxs) ++ nrcp1.toSeq.flatMap(_.claimAnchorTxs)
claimAnchorTxs.collect { case tx: Transactions.ClaimLocalAnchorOutputTx => txPublisher ! PublishReplaceableTx(tx, d.commitments.latest) }
if (claimAnchorTxs.nonEmpty) {
c.replyTo ! RES_SUCCESS(c, d.channelId)
stay() using d.copy(localCommitPublished = Some(lcp1))
case Transactions.DefaultCommitmentFormat =>
log.warning("cannot bump force-close fees, channel is not using anchor outputs")
stay() using d.copy(localCommitPublished = lcp1, remoteCommitPublished = rcp1, nextRemoteCommitPublished = nrcp1) storing()
} else {
log.warning("cannot bump force-close fees, local or remote commit not published")
c.replyTo ! RES_FAILURE(c, CommandUnavailableInThisState(d.channelId, "rbf-force-close", stateName))
stay()
}
case None =>
log.warning("cannot bump force-close fees, local commit hasn't been published")
}
case _ =>
log.warning("cannot bump force-close fees, channel is not using anchor outputs")
c.replyTo ! RES_FAILURE(c, CommandUnavailableInThisState(d.channelId, "rbf-force-close", stateName))
stay()
}

View file

@ -272,8 +272,9 @@ trait ErrorHandlers extends CommonHandlers {
def doPublish(remoteCommitPublished: RemoteCommitPublished, commitment: FullCommitment): Unit = {
import remoteCommitPublished._
val claimLocalAnchor = claimAnchorTxs.collect { case tx: Transactions.ClaimLocalAnchorOutputTx => PublishReplaceableTx(tx, commitment) }
val redeemableHtlcTxs = claimHtlcTxs.values.flatten.map(tx => PublishReplaceableTx(tx, commitment))
val publishQueue = claimMainOutputTx.map(tx => PublishFinalTx(tx, tx.fee, None)).toSeq ++ redeemableHtlcTxs
val publishQueue = claimLocalAnchor ++ claimMainOutputTx.map(tx => PublishFinalTx(tx, tx.fee, None)).toSeq ++ redeemableHtlcTxs
publishIfNeeded(publishQueue, irrevocablySpent)
// we watch:

View file

@ -128,8 +128,7 @@ private class ReplaceableTxPrePublisher(nodeParams: NodeParams,
// We verify that:
// - our commit is not confirmed (if it is, no need to claim our anchor)
// - their commit is not confirmed (if it is, no need to claim our anchor either)
// - our commit tx is in the mempool (otherwise we can't claim our anchor)
val commitTx = cmd.commitment.fullySignedLocalCommitTx(nodeParams.channelKeyManager).tx
// - the local or remote commit tx is in the mempool (otherwise we can't claim our anchor)
val fundingOutpoint = cmd.commitment.commitInput.outPoint
context.pipeToSelf(bitcoinClient.getTxConfirmations(fundingOutpoint.txid).flatMap {
case Some(_) =>
@ -137,9 +136,17 @@ private class ReplaceableTxPrePublisher(nodeParams: NodeParams,
bitcoinClient.isTransactionOutputSpendable(fundingOutpoint.txid, fundingOutpoint.index.toInt, includeMempool = false).flatMap {
case false => Future.failed(CommitTxAlreadyConfirmed)
case true =>
// We must ensure our local commit tx is in the mempool before publishing the anchor transaction.
// If it's already published, this call will be a no-op.
bitcoinClient.publishTransaction(commitTx)
val remoteCommits = Set(Some(cmd.commitment.remoteCommit.txid), cmd.commitment.nextRemoteCommit_opt.map(_.commit.txid)).flatten
if (remoteCommits.contains(localAnchorTx.input.outPoint.txid)) {
// We're trying to bump the remote commit tx: we must make sure it is in our mempool first.
// If it isn't, we will publish our local commit tx instead.
bitcoinClient.getMempoolTx(localAnchorTx.input.outPoint.txid).map(_.txid)
} else {
// We must ensure our local commit tx is in the mempool before publishing the anchor transaction.
// If it's already published, this call will be a no-op.
val commitTx = cmd.commitment.fullySignedLocalCommitTx(nodeParams.channelKeyManager).tx
bitcoinClient.publishTransaction(commitTx)
}
}
case None =>
// If the funding transaction cannot be found (e.g. when using 0-conf), we should retry later.

View file

@ -244,6 +244,9 @@ private class TxPublisher(nodeParams: NodeParams, factory: TxPublisher.ChildFact
case (_: ConfirmationTarget.Priority, ConfirmationTarget.Absolute(_)) =>
// Switch from relative priority mode to absolute blockheight mode
updateConfirmationTarget()
case (ConfirmationTarget.Priority(current), ConfirmationTarget.Priority(proposed)) if current < proposed =>
// Switch to a higher relative priority.
updateConfirmationTarget()
case _ =>
log.debug("not publishing replaceable {} spending {}:{} with confirmation target={}, publishing is already in progress with confirmation target={}", cmd.desc, cmd.input.txid, cmd.input.index, proposedConfirmationTarget, currentConfirmationTarget)
Behaviors.same

View file

@ -39,7 +39,7 @@ import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsT
import fr.acinq.eclair.crypto.keymanager.LocalOnChainKeyManager
import fr.acinq.eclair.transactions.Transactions
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.wire.protocol.{CommitSig, RevokeAndAck}
import fr.acinq.eclair.wire.protocol.{CommitSig, RevokeAndAck, UpdateFee}
import fr.acinq.eclair.{BlockHeight, MilliSatoshi, MilliSatoshiLong, NodeParams, NotificationsLogger, TestConstants, TestKitBaseClass, TimestampSecond, randomKey}
import org.scalatest.BeforeAndAfterAll
import org.scalatest.funsuite.AnyFunSuiteLike
@ -192,7 +192,24 @@ class ReplaceableTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike w
(publishCommitTx, publishAnchor.copy(txInfo = anchorTx))
}
test("commit tx feerate high enough, not spending anchor output") {
def remoteCloseChannelWithoutHtlcs(f: Fixture, overrideCommitTarget: BlockHeight): (Transaction, PublishReplaceableTx) = {
import f._
val commitTx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.fullySignedLocalCommitTx(bob.underlyingActor.nodeParams.channelKeyManager).tx
wallet.publishTransaction(commitTx).pipeTo(probe.ref)
probe.expectMsg(commitTx.txid)
probe.send(alice, WatchFundingSpentTriggered(commitTx))
// Forward the anchor tx to the publisher.
val publishAnchor = alice2blockchain.expectMsgType[PublishReplaceableTx]
assert(publishAnchor.txInfo.input.outPoint.txid == commitTx.txid)
assert(publishAnchor.txInfo.isInstanceOf[ClaimLocalAnchorOutputTx])
val anchorTx = publishAnchor.txInfo.asInstanceOf[ClaimLocalAnchorOutputTx].copy(confirmationTarget = ConfirmationTarget.Absolute(overrideCommitTarget))
(commitTx, publishAnchor.copy(txInfo = anchorTx))
}
test("commit tx feerate high enough, not spending anchor output (local commit)") {
withFixture(Seq(500 millibtc), ChannelTypes.AnchorOutputsZeroFeeHtlcTx()) { f =>
import f._
@ -207,6 +224,21 @@ class ReplaceableTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike w
}
}
test("commit tx feerate high enough, not spending anchor output (remote commit)") {
withFixture(Seq(500 millibtc), ChannelTypes.AnchorOutputsZeroFeeHtlcTx()) { f =>
import f._
val commitFeerate = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.spec.commitTxFeerate
setFeerate(commitFeerate)
val (_, anchorTx) = remoteCloseChannelWithoutHtlcs(f, aliceBlockHeight() + 24)
publisher ! Publish(probe.ref, anchorTx)
val result = probe.expectMsgType[TxRejected]
assert(result.cmd == anchorTx)
assert(result.reason == TxSkipped(retryNextBlock = true))
}
}
test("commit tx confirmed, not spending anchor output") {
withFixture(Seq(500 millibtc), ChannelTypes.AnchorOutputsZeroFeeHtlcTx()) { f =>
import f._
@ -292,7 +324,7 @@ class ReplaceableTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike w
}
}
test("remote commit tx published, not spending anchor output") {
test("remote commit tx published, not spending local anchor output") {
withFixture(Seq(500 millibtc), ChannelTypes.AnchorOutputsZeroFeeHtlcTx()) { f =>
import f._
@ -366,7 +398,7 @@ class ReplaceableTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike w
}
}
test("commit tx feerate too low, spending anchor output") {
test("commit tx feerate too low, spending anchor output (local commit)") {
withFixture(Seq(500 millibtc), ChannelTypes.AnchorOutputsZeroFeeHtlcTx()) { f =>
import f._
@ -396,6 +428,62 @@ class ReplaceableTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike w
}
}
private def testSpendRemoteCommitAnchor(f: Fixture, nextCommit: Boolean): Unit = {
import f._
if (nextCommit) {
// We make sure Bob's commitment is seen as the next commitment by Alice.
val commitFeerate = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.spec.commitTxFeerate
probe.send(alice, CMD_UPDATE_FEE(commitFeerate * 1.1, commit = true, replyTo_opt = Some(probe.ref)))
probe.expectMsgType[CommandSuccess[CMD_UPDATE_FEE]]
alice2bob.expectMsgType[UpdateFee]
alice2bob.forward(bob)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[RevokeAndAck]
bob2alice.expectMsgType[CommitSig]
}
val (commitTx, anchorTx) = remoteCloseChannelWithoutHtlcs(f, aliceBlockHeight() + 30)
assert(getMempool().length == 1)
if (nextCommit) {
assert(alice.stateData.asInstanceOf[DATA_CLOSING].commitments.latest.nextRemoteCommit_opt.nonEmpty)
assert(alice.stateData.asInstanceOf[DATA_CLOSING].commitments.latest.nextRemoteCommit_opt.map(_.commit.txid).contains(commitTx.txid))
}
val targetFeerate = FeeratePerKw(3000 sat)
// NB: we try to get transactions confirmed *before* their confirmation target, so we aim for a more aggressive block target what's provided.
setFeerate(targetFeerate, blockTarget = 12)
publisher ! Publish(probe.ref, anchorTx)
// wait for the anchor tx to be published
val mempoolTxs = getMempoolTxs(2)
assert(mempoolTxs.map(_.txid).contains(commitTx.txid))
val targetFee = Transactions.weight2fee(targetFeerate, mempoolTxs.map(_.weight).sum.toInt)
val actualFee = mempoolTxs.map(_.fees).sum
assert(targetFee * 0.9 <= actualFee && actualFee <= targetFee * 1.1, s"actualFee=$actualFee targetFee=$targetFee")
generateBlocks(5)
system.eventStream.publish(CurrentBlockHeight(currentBlockHeight(probe)))
val result = probe.expectMsgType[TxConfirmed]
assert(result.cmd == anchorTx)
assert(result.tx.txIn.map(_.outPoint.txid).contains(commitTx.txid))
assert(mempoolTxs.map(_.txid).contains(result.tx.txid))
}
test("commit tx feerate too low, spending anchor output (remote commit)") {
withFixture(Seq(500 millibtc), ChannelTypes.AnchorOutputsZeroFeeHtlcTx()) { f =>
testSpendRemoteCommitAnchor(f, nextCommit = false)
}
}
test("commit tx feerate too low, spending anchor output (next remote commit)") {
withFixture(Seq(500 millibtc), ChannelTypes.AnchorOutputsZeroFeeHtlcTx()) { f =>
testSpendRemoteCommitAnchor(f, nextCommit = true)
}
}
test("commit tx feerate too low, spending anchor output (feerate upper bound reached)") {
withFixture(Seq(500 millibtc), ChannelTypes.AnchorOutputsZeroFeeHtlcTx()) { f =>
import f._
@ -1320,6 +1408,7 @@ class ReplaceableTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike w
val remoteCommitTx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.fullySignedLocalCommitTx(bob.underlyingActor.nodeParams.channelKeyManager)
assert(remoteCommitTx.tx.txOut.size == 6)
probe.send(alice, WatchFundingSpentTriggered(remoteCommitTx.tx))
alice2blockchain.expectMsgType[PublishReplaceableTx] // claim anchor
alice2blockchain.expectMsgType[PublishFinalTx] // claim main output
val claimHtlcTimeout = alice2blockchain.expectMsgType[PublishReplaceableTx]
assert(claimHtlcTimeout.txInfo.isInstanceOf[ClaimHtlcTimeoutTx])
@ -1371,10 +1460,9 @@ class ReplaceableTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike w
// Force-close channel and verify txs sent to watcher.
val remoteCommitTx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.fullySignedLocalCommitTx(bob.underlyingActor.nodeParams.channelKeyManager)
if (bob.stateData.asInstanceOf[DATA_NORMAL].commitments.params.commitmentFormat == DefaultCommitmentFormat) {
assert(remoteCommitTx.tx.txOut.size == 4)
} else {
assert(remoteCommitTx.tx.txOut.size == 6)
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)
}
probe.send(alice, WatchFundingSpentTriggered(remoteCommitTx.tx))
@ -1383,6 +1471,10 @@ class ReplaceableTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike w
probe.expectMsg(remoteCommitTx.tx.txid)
generateBlocks(1)
bob.stateData.asInstanceOf[DATA_NORMAL].commitments.params.commitmentFormat match {
case Transactions.DefaultCommitmentFormat => ()
case _: AnchorOutputsCommitmentFormat => alice2blockchain.expectMsgType[PublishReplaceableTx] // claim anchor
}
alice2blockchain.expectMsgType[PublishFinalTx] // claim main output
val claimHtlcSuccess = alice2blockchain.expectMsgType[PublishReplaceableTx]
assert(claimHtlcSuccess.txInfo.isInstanceOf[ClaimHtlcSuccessTx])

View file

@ -22,7 +22,7 @@ import akka.actor.typed.scaladsl.adapter.{ClassicActorSystemOps, TypedActorRefOp
import akka.testkit.TestProbe
import fr.acinq.bitcoin.scalacompat.{OutPoint, SatoshiLong, Transaction, TxIn, TxOut}
import fr.acinq.eclair.blockchain.CurrentBlockHeight
import fr.acinq.eclair.blockchain.fee.ConfirmationTarget
import fr.acinq.eclair.blockchain.fee.{ConfirmationPriority, ConfirmationTarget}
import fr.acinq.eclair.channel.publish
import fr.acinq.eclair.channel.publish.TxPublisher.TxRejectedReason._
import fr.acinq.eclair.channel.publish.TxPublisher._
@ -116,16 +116,30 @@ class TxPublisherSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val confirmBefore = nodeParams.currentBlockHeight + 12
val input = OutPoint(randomBytes32(), 3)
val anchorTx = ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), ConfirmationTarget.Absolute(confirmBefore))
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 cmd = PublishReplaceableTx(anchorTx, null)
txPublisher ! cmd
val child = factory.expectMsgType[ReplaceableTxPublisherSpawned].actor
assert(child.expectMsgType[ReplaceableTxPublisher.Publish].cmd == cmd)
// We ignore duplicates that don't use a more aggressive confirmation target:
txPublisher ! PublishReplaceableTx(anchorTx, null)
// We ignore duplicates that don't use a more aggressive priority:
txPublisher ! PublishReplaceableTx(anchorTx.copy(confirmationTarget = ConfirmationTarget.Priority(ConfirmationPriority.Slow)), null)
child.expectNoMessage(100 millis)
factory.expectNoMessage(100 millis)
// But we allow duplicates with a more aggressive priority:
val cmdHigherPriority = cmd.copy(txInfo = anchorTx.copy(confirmationTarget = ConfirmationTarget.Priority(ConfirmationPriority.Fast)))
txPublisher ! cmdHigherPriority
child.expectMsg(ReplaceableTxPublisher.UpdateConfirmationTarget(ConfirmationTarget.Priority(ConfirmationPriority.Fast)))
factory.expectNoMessage(100 millis)
// Absolute confirmation targets replace relative priorities:
val cmdAbsoluteTarget = cmd.copy(txInfo = anchorTx.copy(confirmationTarget = ConfirmationTarget.Absolute(confirmBefore)))
txPublisher ! cmdAbsoluteTarget
child.expectMsg(ReplaceableTxPublisher.UpdateConfirmationTarget(ConfirmationTarget.Absolute(confirmBefore)))
factory.expectNoMessage(100 millis)
// We ignore duplicates with a less aggressive confirmation target:
val cmdHigherTarget = cmd.copy(txInfo = anchorTx.copy(confirmationTarget = ConfirmationTarget.Absolute(confirmBefore + 1)))
txPublisher ! cmdHigherTarget
child.expectNoMessage(100 millis)

View file

@ -574,6 +574,13 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
assert(closingData.localCommitPublished.isEmpty)
val remoteCommitPublished = remoteCommitPublished_opt.get
// If anchor outputs is used, we use the anchor output to bump the fees if necessary.
closingData.commitments.params.commitmentFormat match {
case _: AnchorOutputsCommitmentFormat =>
val anchorTx = s2blockchain.expectMsgType[PublishReplaceableTx]
assert(anchorTx.txInfo.isInstanceOf[ClaimLocalAnchorOutputTx])
case Transactions.DefaultCommitmentFormat => ()
}
// if s has a main output in the commit tx (when it has a non-dust balance), it should be claimed
remoteCommitPublished.claimMainOutputTx.foreach(claimMain => {
Transaction.correctlySpends(claimMain.tx, rCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)

View file

@ -27,6 +27,7 @@ import fr.acinq.eclair.channel.publish.TxPublisher
import fr.acinq.eclair.channel.publish.TxPublisher.SetChannelId
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
import fr.acinq.eclair.payment.relay.Relayer.RelayFees
import fr.acinq.eclair.transactions.Transactions.ClaimLocalAnchorOutputTx
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{BlockHeight, MilliSatoshiLong, TestConstants, TestKitBaseClass}
import org.scalatest.OptionValues.convertOptionToValuable
@ -198,6 +199,7 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF
// bob publishes his commitment tx
val bobCommitTx = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
alice ! WatchFundingSpentTriggered(bobCommitTx)
assert(alice2blockchain.expectMsgType[TxPublisher.PublishReplaceableTx].txInfo.isInstanceOf[ClaimLocalAnchorOutputTx])
alice2blockchain.expectMsgType[TxPublisher.PublishTx]
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == bobCommitTx.txid)
listener.expectMsgType[ChannelAborted]

View file

@ -34,6 +34,7 @@ import fr.acinq.eclair.channel.states.ChannelStateTestsTags.{AnchorOutputsZeroFe
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
import fr.acinq.eclair.testutils.PimpTestProbe.convert
import fr.acinq.eclair.transactions.Transactions
import fr.acinq.eclair.transactions.Transactions.ClaimLocalAnchorOutputTx
import fr.acinq.eclair.wire.protocol._
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
import org.scalatest.time.SpanSugar.convertIntToGrainOfTime
@ -1370,6 +1371,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
// bob's remote tx wins
alice ! WatchAlternativeCommitTxConfirmedTriggered(BlockHeight(400000), 42, bobCommitTx1)
// we're back to the normal handling of remote commit
assert(alice2blockchain.expectMsgType[PublishReplaceableTx].txInfo.isInstanceOf[ClaimLocalAnchorOutputTx])
val claimMain = alice2blockchain.expectMsgType[PublishFinalTx].tx
val watchConfirmedRemoteCommit = alice2blockchain.expectMsgType[WatchTxConfirmed]
assert(watchConfirmedRemoteCommit.txId == bobCommitTx1.txid)

View file

@ -394,6 +394,13 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(localAnchor2.confirmationTarget == ConfirmationTarget.Priority(ConfirmationPriority.Fast))
val localCommitPublished2 = alice.stateData.asInstanceOf[DATA_CLOSING].localCommitPublished.get
assert(localCommitPublished2.claimAnchorTxs.contains(localAnchor2))
// If we try bumping again, but with a lower priority, this won't override the previous priority.
alice ! CMD_BUMP_FORCE_CLOSE_FEE(replyTo.ref, ConfirmationTarget.Priority(ConfirmationPriority.Medium))
replyTo.expectMsgType[RES_SUCCESS[CMD_BUMP_FORCE_CLOSE_FEE]]
val localAnchor3 = alice2blockchain.expectMsgType[PublishReplaceableTx].txInfo.asInstanceOf[ClaimLocalAnchorOutputTx]
assert(localAnchor3.confirmationTarget == ConfirmationTarget.Priority(ConfirmationPriority.Fast))
assert(alice.stateData.asInstanceOf[DATA_CLOSING].localCommitPublished.contains(localCommitPublished2))
}
def testLocalCommitTxConfirmed(f: FixtureParam, channelFeatures: ChannelFeatures): Unit = {
@ -730,14 +737,38 @@ 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 WatchTxConfirmedTriggered (remote commit)") { f =>
test("recv CMD_BUMP_FORCE_CLOSE_FEE (remote commit)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._
val bobCommitTx = bobCommitTxs.last.commitTx.tx
val closingState1 = remoteClose(bobCommitTx, alice, alice2blockchain)
assert(closingState1.claimAnchorTxs.nonEmpty)
val Some(localAnchor1) = closingState1.claimAnchorTxs.collectFirst { case tx: ClaimLocalAnchorOutputTx => tx }
assert(localAnchor1.confirmationTarget == ConfirmationTarget.Priority(ConfirmationPriority.Medium))
val replyTo = TestProbe()
alice ! CMD_BUMP_FORCE_CLOSE_FEE(replyTo.ref, ConfirmationTarget.Priority(ConfirmationPriority.Fast))
replyTo.expectMsgType[RES_SUCCESS[CMD_BUMP_FORCE_CLOSE_FEE]]
val localAnchor2 = alice2blockchain.expectMsgType[PublishReplaceableTx].txInfo.asInstanceOf[ClaimLocalAnchorOutputTx]
assert(localAnchor2.confirmationTarget == ConfirmationTarget.Priority(ConfirmationPriority.Fast))
val closingState2 = alice.stateData.asInstanceOf[DATA_CLOSING].remoteCommitPublished.get
assert(closingState2.claimAnchorTxs.contains(localAnchor2))
// If we try bumping again, but with a lower priority, this won't override the previous priority.
alice ! CMD_BUMP_FORCE_CLOSE_FEE(replyTo.ref, ConfirmationTarget.Priority(ConfirmationPriority.Medium))
replyTo.expectMsgType[RES_SUCCESS[CMD_BUMP_FORCE_CLOSE_FEE]]
val localAnchor3 = alice2blockchain.expectMsgType[PublishReplaceableTx].txInfo.asInstanceOf[ClaimLocalAnchorOutputTx]
assert(localAnchor3.confirmationTarget == ConfirmationTarget.Priority(ConfirmationPriority.Fast))
assert(alice.stateData.asInstanceOf[DATA_CLOSING].remoteCommitPublished.contains(closingState2))
}
test("recv WatchTxConfirmedTriggered (remote commit)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._
mutualClose(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain)
val initialState = alice.stateData.asInstanceOf[DATA_CLOSING]
assert(initialState.commitments.params.channelFeatures == ChannelFeatures())
// bob publishes his last current commit tx, the one it had when entering NEGOTIATING state
val bobCommitTx = bobCommitTxs.last.commitTx.tx
assert(bobCommitTx.txOut.size == 2) // two main outputs
assert(bobCommitTx.txOut.size == 4) // two main outputs + two anchors
val closingState = remoteClose(bobCommitTx, alice, alice2blockchain)
// actual test starts here
@ -1081,6 +1112,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
// the commit tx hasn't been confirmed yet, so we watch the funding output first
alice2blockchain.expectMsgType[WatchFundingSpent]
// then we should re-publish unconfirmed transactions
assert(alice2blockchain.expectMsgType[PublishReplaceableTx].txInfo.isInstanceOf[ClaimLocalAnchorOutputTx])
closingState.claimMainOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[PublishFinalTx].tx == claimMain.tx))
claimHtlcTimeoutTxs.foreach(claimHtlcTimeout => assert(alice2blockchain.expectMsgType[PublishReplaceableTx].txInfo.tx == claimHtlcTimeout.tx))
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == bobCommitTx.txid)