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

Prioritize remote commitment instead of local one (#3019)

If we've seen the (signed) remote commitment in our mempool, we now
actively favor it instead of trying to publish our local commitment.
It is more interesting to us since it doesn't have any CSV delays
and doesn't require 2nd-stage HTLC transactions.

Now that we've updated to Bitcoin Core 28.1, which supports 1p1c
opportunistic package relay, we should be able to get those remote
commitment transactions confirmed more easily.
This commit is contained in:
Bastien Teinturier 2025-02-28 10:45:30 +01:00 committed by GitHub
parent cae22d71be
commit f6b051cf73
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
9 changed files with 104 additions and 58 deletions

View file

@ -2109,11 +2109,19 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
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) {
// We favor the remote commitment(s) because they're more interesting than the local commitment (no CSV delays).
if (rcp1.nonEmpty) {
rcp1.foreach(rcp => rcp.claimAnchorTxs.collect { case tx: Transactions.ClaimLocalAnchorOutputTx => txPublisher ! PublishReplaceableTx(tx, d.commitments.latest, rcp.commitTx) })
c.replyTo ! RES_SUCCESS(c, d.channelId)
stay() using d.copy(localCommitPublished = lcp1, remoteCommitPublished = rcp1, nextRemoteCommitPublished = nrcp1) storing()
stay() using d.copy(remoteCommitPublished = rcp1) storing()
} else if (nrcp1.nonEmpty) {
nrcp1.foreach(rcp => rcp.claimAnchorTxs.collect { case tx: Transactions.ClaimLocalAnchorOutputTx => txPublisher ! PublishReplaceableTx(tx, d.commitments.latest, rcp.commitTx) })
c.replyTo ! RES_SUCCESS(c, d.channelId)
stay() using d.copy(nextRemoteCommitPublished = nrcp1) storing()
} else if (lcp1.nonEmpty) {
lcp1.foreach(lcp => lcp.claimAnchorTxs.collect { case tx: Transactions.ClaimLocalAnchorOutputTx => txPublisher ! PublishReplaceableTx(tx, d.commitments.latest, lcp.commitTx) })
c.replyTo ! RES_SUCCESS(c, d.channelId)
stay() using d.copy(localCommitPublished = lcp1) 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))
@ -2778,7 +2786,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case (SYNCING, NORMAL, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.channelUpdate != d2.channelUpdate || d1.lastAnnouncement_opt != d2.lastAnnouncement_opt => Some(EmitLocalChannelUpdate("syncing->normal", d2, sendToPeer = d2.lastAnnouncement_opt.isEmpty))
case (NORMAL, OFFLINE, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.channelUpdate != d2.channelUpdate || d1.lastAnnouncement_opt != d2.lastAnnouncement_opt => Some(EmitLocalChannelUpdate("normal->offline", d2, sendToPeer = false))
case (OFFLINE, OFFLINE, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.channelUpdate != d2.channelUpdate || d1.lastAnnouncement_opt != d2.lastAnnouncement_opt => Some(EmitLocalChannelUpdate("offline->offline", d2, sendToPeer = false))
case (NORMAL | SYNCING | OFFLINE, SHUTDOWN | NEGOTIATING | NEGOTIATING_SIMPLE| CLOSING | CLOSED | ERR_INFORMATION_LEAK | WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT, d: DATA_NORMAL, _) => Some(EmitLocalChannelDown(d))
case (NORMAL | SYNCING | OFFLINE, SHUTDOWN | NEGOTIATING | NEGOTIATING_SIMPLE | CLOSING | CLOSED | ERR_INFORMATION_LEAK | WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT, d: DATA_NORMAL, _) => Some(EmitLocalChannelDown(d))
case _ => None
}
emitEvent_opt.foreach {

View file

@ -238,8 +238,8 @@ trait ErrorHandlers extends CommonHandlers {
val redeemableHtlcTxs = htlcTxs.values.flatten.map(tx => PublishFinalTx(tx, tx.fee, Some(commitTx.txid)))
List(PublishFinalTx(commitTx, commitment.commitInput.outPoint, commitment.capacity, "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 =>
val redeemableHtlcTxs = htlcTxs.values.flatten.map(tx => PublishReplaceableTx(tx, commitment))
val claimLocalAnchor = claimAnchorTxs.collect { case tx: Transactions.ClaimLocalAnchorOutputTx if !localCommitPublished.isConfirmed => PublishReplaceableTx(tx, commitment) }
val redeemableHtlcTxs = htlcTxs.values.flatten.map(tx => PublishReplaceableTx(tx, commitment, commitTx))
val claimLocalAnchor = claimAnchorTxs.collect { case tx: Transactions.ClaimLocalAnchorOutputTx if !localCommitPublished.isConfirmed => PublishReplaceableTx(tx, commitment, commitTx) }
List(PublishFinalTx(commitTx, commitment.commitInput.outPoint, commitment.capacity, "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))
}
publishIfNeeded(publishQueue, irrevocablySpent)
@ -298,8 +298,8 @@ trait ErrorHandlers extends CommonHandlers {
def doPublish(remoteCommitPublished: RemoteCommitPublished, commitment: FullCommitment): Unit = {
import remoteCommitPublished._
val claimLocalAnchor = claimAnchorTxs.collect { case tx: Transactions.ClaimLocalAnchorOutputTx if !remoteCommitPublished.isConfirmed => PublishReplaceableTx(tx, commitment) }
val redeemableHtlcTxs = claimHtlcTxs.values.flatten.map(tx => PublishReplaceableTx(tx, commitment))
val claimLocalAnchor = claimAnchorTxs.collect { case tx: Transactions.ClaimLocalAnchorOutputTx if !remoteCommitPublished.isConfirmed => PublishReplaceableTx(tx, commitment, commitTx) }
val redeemableHtlcTxs = claimHtlcTxs.values.flatten.map(tx => PublishReplaceableTx(tx, commitment, commitTx))
val publishQueue = claimLocalAnchor ++ claimMainOutputTx.map(tx => PublishFinalTx(tx, tx.fee, None)).toSeq ++ redeemableHtlcTxs
publishIfNeeded(publishQueue, irrevocablySpent)

View file

@ -48,11 +48,10 @@ object ReplaceableTxPrePublisher {
private case object ParentTxOk extends Command
private case object FundingTxNotFound extends RuntimeException with Command
private case object CommitTxAlreadyConfirmed extends RuntimeException with Command
private case object RemoteCommitTxNotInMempool extends RuntimeException with Command
private case object LocalCommitTxConfirmed extends Command
private case object LocalCommitTxPublished extends Command
private case object RemoteCommitTxConfirmed extends Command
private case object RemoteCommitTxPublished extends Command
private case object RemoteCommitTxPublished extends RuntimeException with Command
private case object HtlcOutputAlreadySpent extends Command
private case class UnknownFailure(reason: Throwable) extends Command
// @formatter:on
@ -138,21 +137,19 @@ private class ReplaceableTxPrePublisher(nodeParams: NodeParams,
// The funding transaction was found, let's see if we can still spend it.
bitcoinClient.isTransactionOutputSpendable(fundingOutpoint.txid, fundingOutpoint.index.toInt, includeMempool = false).flatMap {
case false => Future.failed(CommitTxAlreadyConfirmed)
case true =>
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.
bitcoinClient.getMempoolTx(localAnchorTx.input.outPoint.txid).map(_.txid).transformWith {
// We could improve this: we've seen the remote commit in our mempool at least once, so we could try to republish it ourselves.
case Failure(_) => Future.failed(RemoteCommitTxNotInMempool)
case Success(remoteCommitTxId) => Future.successful(remoteCommitTxId)
}
} else {
// We must ensure our local commit tx is in the mempool before publishing the anchor transaction.
case true if cmd.isLocalCommitAnchor =>
// We are trying to bump our local commitment. Let's check if the remote commitment is published: if it is,
// we will skip publishing our local commitment, because the remote commitment is more interesting (we don't
// have any CSV delays and don't need 2nd-stage HTLC transactions).
getRemoteCommitConfirmations(cmd.commitment).flatMap {
case Some(_) => Future.failed(RemoteCommitTxPublished)
// Otherwise, 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 => bitcoinClient.publishTransaction(cmd.commitTx)
}
case true =>
// We're trying to bump a remote commitment: we must make sure it is in our mempool first.
bitcoinClient.publishTransaction(cmd.commitTx)
}
case None =>
// If the funding transaction cannot be found (e.g. when using 0-conf), we should retry later.
@ -161,7 +158,7 @@ private class ReplaceableTxPrePublisher(nodeParams: NodeParams,
case Success(_) => ParentTxOk
case Failure(FundingTxNotFound) => FundingTxNotFound
case Failure(CommitTxAlreadyConfirmed) => CommitTxAlreadyConfirmed
case Failure(RemoteCommitTxNotInMempool) => RemoteCommitTxNotInMempool
case Failure(RemoteCommitTxPublished) => RemoteCommitTxPublished
case Failure(reason) if reason.getMessage.contains("rejecting replacement") => RemoteCommitTxPublished
case Failure(reason) => UnknownFailure(reason)
}
@ -177,14 +174,9 @@ private class ReplaceableTxPrePublisher(nodeParams: NodeParams,
log.debug("commit tx is already confirmed, no need to claim our anchor")
replyTo ! PreconditionsFailed(TxPublisher.TxRejectedReason.TxSkipped(retryNextBlock = false))
Behaviors.stopped
case RemoteCommitTxNotInMempool =>
log.debug("remote commit tx cannot be found in our mempool: we can't spend our anchor")
replyTo ! PreconditionsFailed(TxPublisher.TxRejectedReason.TxSkipped(retryNextBlock = true))
Behaviors.stopped
case RemoteCommitTxPublished =>
log.warn("cannot publish commit tx: there is a conflicting tx in the mempool")
// We retry until that conflicting commit tx is confirmed or we're able to publish our local commit tx.
replyTo ! PreconditionsFailed(TxPublisher.TxRejectedReason.TxSkipped(retryNextBlock = true))
log.warn("not publishing local commit tx: we're using the remote commit tx instead")
replyTo ! PreconditionsFailed(TxPublisher.TxRejectedReason.TxSkipped(retryNextBlock = false))
Behaviors.stopped
case UnknownFailure(reason) =>
log.error(s"could not check ${cmd.desc} preconditions, proceeding anyway: ", reason)

View file

@ -25,7 +25,7 @@ import fr.acinq.eclair.blockchain.CurrentBlockHeight
import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient
import fr.acinq.eclair.blockchain.fee.ConfirmationTarget
import fr.acinq.eclair.channel.FullCommitment
import fr.acinq.eclair.transactions.Transactions.{ReplaceableTransactionWithInputInfo, TransactionWithInputInfo}
import fr.acinq.eclair.transactions.Transactions.{ClaimLocalAnchorOutputTx, ReplaceableTransactionWithInputInfo, TransactionWithInputInfo}
import fr.acinq.eclair.{BlockHeight, Logs, NodeParams}
import java.util.UUID
@ -88,10 +88,21 @@ object TxPublisher {
object PublishFinalTx {
def apply(txInfo: TransactionWithInputInfo, fee: Satoshi, parentTx_opt: Option[TxId]): PublishFinalTx = PublishFinalTx(txInfo.tx, txInfo.input.outPoint, txInfo.amountIn, txInfo.desc, fee, parentTx_opt)
}
/** Publish an unsigned transaction that can be RBF-ed. */
case class PublishReplaceableTx(txInfo: ReplaceableTransactionWithInputInfo, commitment: FullCommitment) extends PublishTx {
/**
* Publish an unsigned transaction that can be RBF-ed.
*
* @param commitTx commitment transaction that this transaction is spending.
*/
case class PublishReplaceableTx(txInfo: ReplaceableTransactionWithInputInfo, commitment: FullCommitment, commitTx: Transaction) extends PublishTx {
override def input: OutPoint = txInfo.input.outPoint
override def desc: String = txInfo.desc
/** True if we're trying to bump our local commit with an anchor transaction. */
lazy val isLocalCommitAnchor = txInfo match {
case txInfo: ClaimLocalAnchorOutputTx => txInfo.input.outPoint.txid == commitment.localCommit.commitTxAndRemoteSig.commitTx.tx.txid
case _ => false
}
}
sealed trait PublishTxResult extends Command { def cmd: PublishTx }

View file

@ -341,9 +341,7 @@ class ReplaceableTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike w
publisher ! Publish(probe.ref, anchorTx)
val result = probe.expectMsgType[TxRejected]
assert(result.cmd == anchorTx)
// When the remote commit tx is still unconfirmed, we want to retry in case it is evicted from the mempool and our
// commit is then published.
assert(result.reason == TxSkipped(retryNextBlock = true))
assert(result.reason == TxSkipped(retryNextBlock = false))
}
}
@ -572,7 +570,7 @@ class ReplaceableTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike w
}
}
test("remote commit tx not published, not spending remote anchor output") {
test("remote commit tx not published, publishing it and spending anchor output") {
withFixture(Seq(500 millibtc), ChannelTypes.AnchorOutputsZeroFeeHtlcTx()) { f =>
import f._
@ -580,14 +578,28 @@ class ReplaceableTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike w
// Note that we don't publish the remote commit, to simulate the case where the watch triggers but the remote commit is then evicted from our mempool.
probe.send(alice, WatchFundingSpentTriggered(commitTx))
val publishAnchor = alice2blockchain.expectMsgType[PublishReplaceableTx]
assert(publishAnchor.commitTx == commitTx)
assert(publishAnchor.txInfo.input.outPoint.txid == commitTx.txid)
assert(publishAnchor.txInfo.isInstanceOf[ClaimLocalAnchorOutputTx])
val targetFeerate = FeeratePerKw(3000 sat)
setFeerate(targetFeerate)
val anchorTx = publishAnchor.copy(txInfo = publishAnchor.txInfo.asInstanceOf[ClaimLocalAnchorOutputTx].copy(confirmationTarget = ConfirmationTarget.Absolute(aliceBlockHeight() + 6)))
publisher ! Publish(probe.ref, anchorTx)
// wait for the commit tx and anchor tx to be published
val mempoolTxs = getMempoolTxs(2)
assert(mempoolTxs.map(_.txid).contains(commitTx.txid))
val result = probe.expectMsgType[TxRejected]
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(6)
system.eventStream.publish(CurrentBlockHeight(currentBlockHeight(probe)))
val result = probe.expectMsgType[TxConfirmed]
assert(result.cmd == anchorTx)
assert(result.reason == TxSkipped(retryNextBlock = true))
assert(result.tx.txIn.map(_.outPoint.txid).contains(commitTx.txid))
assert(mempoolTxs.map(_.txid).contains(result.tx.txid))
}
}

View file

@ -105,7 +105,7 @@ class TxPublisherSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val confirmBefore = ConfirmationTarget.Absolute(nodeParams.currentBlockHeight + 12)
val input = OutPoint(randomTxId(), 3)
val cmd = PublishReplaceableTx(ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), confirmBefore), null)
val cmd = PublishReplaceableTx(ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), confirmBefore), null, null)
txPublisher ! cmd
val child = factory.expectMsgType[ReplaceableTxPublisherSpawned].actor
val p = child.expectMsgType[ReplaceableTxPublisher.Publish]
@ -118,13 +118,13 @@ class TxPublisherSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val confirmBefore = nodeParams.currentBlockHeight + 12
val input = OutPoint(randomTxId(), 3)
val anchorTx = ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), ConfirmationTarget.Priority(ConfirmationPriority.Medium))
val cmd = PublishReplaceableTx(anchorTx, null)
val cmd = PublishReplaceableTx(anchorTx, null, 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 priority:
txPublisher ! PublishReplaceableTx(anchorTx.copy(confirmationTarget = ConfirmationTarget.Priority(ConfirmationPriority.Slow)), null)
txPublisher ! PublishReplaceableTx(anchorTx.copy(confirmationTarget = ConfirmationTarget.Priority(ConfirmationPriority.Slow)), null, null)
child.expectNoMessage(100 millis)
factory.expectNoMessage(100 millis)
@ -175,7 +175,7 @@ class TxPublisherSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val attempt2 = factory.expectMsgType[FinalTxPublisherSpawned].actor
attempt2.expectMsgType[FinalTxPublisher.Publish]
val cmd3 = PublishReplaceableTx(ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, TxOut(20_000 sat, Nil) :: Nil, 0), ConfirmationTarget.Absolute(nodeParams.currentBlockHeight)), null)
val cmd3 = PublishReplaceableTx(ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, TxOut(20_000 sat, Nil) :: Nil, 0), ConfirmationTarget.Absolute(nodeParams.currentBlockHeight)), null, null)
txPublisher ! cmd3
val attempt3 = factory.expectMsgType[ReplaceableTxPublisherSpawned].actor
attempt3.expectMsgType[ReplaceableTxPublisher.Publish]
@ -197,7 +197,7 @@ class TxPublisherSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val attempt1 = factory.expectMsgType[FinalTxPublisherSpawned]
attempt1.actor.expectMsgType[FinalTxPublisher.Publish]
val cmd2 = PublishReplaceableTx(ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, TxOut(20_000 sat, Nil) :: Nil, 0), ConfirmationTarget.Absolute(nodeParams.currentBlockHeight)), null)
val cmd2 = PublishReplaceableTx(ClaimLocalAnchorOutputTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, TxOut(20_000 sat, Nil) :: Nil, 0), ConfirmationTarget.Absolute(nodeParams.currentBlockHeight)), null, null)
txPublisher ! cmd2
val attempt2 = factory.expectMsgType[ReplaceableTxPublisherSpawned]
attempt2.actor.expectMsgType[ReplaceableTxPublisher.Publish]
@ -237,7 +237,7 @@ class TxPublisherSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val target = nodeParams.currentBlockHeight + 12
val input = OutPoint(randomTxId(), 7)
val paymentHash = randomBytes32()
val cmd = PublishReplaceableTx(HtlcSuccessTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), paymentHash, 3, ConfirmationTarget.Absolute(target)), null)
val cmd = PublishReplaceableTx(HtlcSuccessTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), paymentHash, 3, ConfirmationTarget.Absolute(target)), null, null)
txPublisher ! cmd
val attempt1 = factory.expectMsgType[ReplaceableTxPublisherSpawned]
attempt1.actor.expectMsgType[ReplaceableTxPublisher.Publish]
@ -301,7 +301,7 @@ class TxPublisherSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
val input = OutPoint(randomTxId(), 7)
val paymentHash = randomBytes32()
val cmd = PublishReplaceableTx(HtlcSuccessTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), paymentHash, 3, ConfirmationTarget.Absolute(nodeParams.currentBlockHeight)), null)
val cmd = PublishReplaceableTx(HtlcSuccessTx(InputInfo(input, TxOut(25_000 sat, Nil), Nil), Transaction(2, TxIn(input, Nil, 0) :: Nil, Nil, 0), paymentHash, 3, ConfirmationTarget.Absolute(nodeParams.currentBlockHeight)), null, null)
txPublisher ! cmd
val attempt1 = factory.expectMsgType[ReplaceableTxPublisherSpawned]
attempt1.actor.expectMsgType[ReplaceableTxPublisher.Publish]

View file

@ -41,7 +41,6 @@ import fr.acinq.eclair.payment.relay.Relayer
import fr.acinq.eclair.testutils.PimpTestProbe.convert
import fr.acinq.eclair.transactions.DirectedHtlc.{incoming, outgoing}
import fr.acinq.eclair.transactions.Transactions
import fr.acinq.eclair.transactions.Transactions.ClaimLocalAnchorOutputTx
import fr.acinq.eclair.wire.protocol._
import org.scalatest.Inside.inside
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
@ -2719,7 +2718,10 @@ 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])
inside(alice2blockchain.expectMsgType[PublishReplaceableTx]) { tx =>
assert(tx.txInfo.isInstanceOf[Transactions.ClaimLocalAnchorOutputTx])
assert(tx.commitTx == bobCommitTx1)
}
val claimMain = alice2blockchain.expectMsgType[PublishFinalTx].tx
val claimHtlcsTxsOut = htlcs.aliceToBob.map(_ => assertPublished(alice2blockchain, "claim-htlc-timeout"))
claimHtlcsTxsOut.foreach(tx => Transaction.correctlySpends(tx, Seq(bobCommitTx1), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS))

View file

@ -3123,8 +3123,9 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(amountClaimed == 823680.sat)
// alice sets the confirmation targets to the HTLC expiry
assert(claimHtlcTxs.collect { case PublishReplaceableTx(tx: ClaimHtlcSuccessTx, _) => (tx.htlcId, tx.confirmationTarget.confirmBefore) }.toMap == Map(htlcb1.id -> htlcb1.cltvExpiry.blockHeight))
assert(claimHtlcTxs.collect { case PublishReplaceableTx(tx: ClaimHtlcTimeoutTx, _) => (tx.htlcId, tx.confirmationTarget.confirmBefore) }.toMap == Map(htlca1.id -> htlca1.cltvExpiry.blockHeight, htlca2.id -> htlca2.cltvExpiry.blockHeight))
assert(claimHtlcTxs.map(_.commitTx.txid).toSet == Set(bobCommitTx.txid))
assert(claimHtlcTxs.collect { case PublishReplaceableTx(tx: ClaimHtlcSuccessTx, _, _) => (tx.htlcId, tx.confirmationTarget.confirmBefore) }.toMap == Map(htlcb1.id -> htlcb1.cltvExpiry.blockHeight))
assert(claimHtlcTxs.collect { case PublishReplaceableTx(tx: ClaimHtlcTimeoutTx, _, _) => (tx.htlcId, tx.confirmationTarget.confirmBefore) }.toMap == Map(htlca1.id -> htlca1.cltvExpiry.blockHeight, htlca2.id -> htlca2.cltvExpiry.blockHeight))
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == bobCommitTx.txid)
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMain.txid)
@ -3215,7 +3216,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(amountClaimed == 829850.sat)
// alice sets the confirmation targets to the HTLC expiry
assert(claimHtlcTxs.collect { case PublishReplaceableTx(tx: ClaimHtlcTimeoutTx, _) => (tx.htlcId, tx.confirmationTarget.confirmBefore) }.toMap == Map(htlca1.id -> htlca1.cltvExpiry.blockHeight, htlca2.id -> htlca2.cltvExpiry.blockHeight))
assert(claimHtlcTxs.map(_.commitTx.txid).toSet == Set(bobCommitTx.txid))
assert(claimHtlcTxs.collect { case PublishReplaceableTx(tx: ClaimHtlcTimeoutTx, _, _) => (tx.htlcId, tx.confirmationTarget.confirmBefore) }.toMap == Map(htlca1.id -> htlca1.cltvExpiry.blockHeight, htlca2.id -> htlca2.cltvExpiry.blockHeight))
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == bobCommitTx.txid)
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMain.txid) // claim-main

View file

@ -426,7 +426,11 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
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]
val localAnchor2 = inside(alice2blockchain.expectMsgType[PublishReplaceableTx]) { tx =>
assert(tx.txInfo.isInstanceOf[ClaimLocalAnchorOutputTx])
assert(tx.commitTx == localCommitPublished1.commitTx)
tx.txInfo.asInstanceOf[ClaimLocalAnchorOutputTx]
}
assert(localAnchor2.confirmationTarget == ConfirmationTarget.Priority(ConfirmationPriority.Fast))
val localCommitPublished2 = alice.stateData.asInstanceOf[DATA_CLOSING].localCommitPublished.get
assert(localCommitPublished2.claimAnchorTxs.contains(localAnchor2))
@ -434,7 +438,11 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
// 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]
val localAnchor3 = inside(alice2blockchain.expectMsgType[PublishReplaceableTx]) { tx =>
assert(tx.txInfo.isInstanceOf[ClaimLocalAnchorOutputTx])
assert(tx.commitTx == localCommitPublished1.commitTx)
tx.txInfo.asInstanceOf[ClaimLocalAnchorOutputTx]
}
assert(localAnchor3.confirmationTarget == ConfirmationTarget.Priority(ConfirmationPriority.Fast))
assert(alice.stateData.asInstanceOf[DATA_CLOSING].localCommitPublished.contains(localCommitPublished2))
}
@ -888,7 +896,11 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
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]
val localAnchor2 = inside(alice2blockchain.expectMsgType[PublishReplaceableTx]) { tx =>
assert(tx.txInfo.isInstanceOf[ClaimLocalAnchorOutputTx])
assert(tx.commitTx == bobCommitTx)
tx.txInfo.asInstanceOf[ClaimLocalAnchorOutputTx]
}
assert(localAnchor2.confirmationTarget == ConfirmationTarget.Priority(ConfirmationPriority.Fast))
val closingState2 = alice.stateData.asInstanceOf[DATA_CLOSING].remoteCommitPublished.get
assert(closingState2.claimAnchorTxs.contains(localAnchor2))
@ -896,7 +908,11 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
// 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]
val localAnchor3 = inside(alice2blockchain.expectMsgType[PublishReplaceableTx]) { tx =>
assert(tx.txInfo.isInstanceOf[ClaimLocalAnchorOutputTx])
assert(tx.commitTx == bobCommitTx)
tx.txInfo.asInstanceOf[ClaimLocalAnchorOutputTx]
}
assert(localAnchor3.confirmationTarget == ConfirmationTarget.Priority(ConfirmationPriority.Fast))
assert(alice.stateData.asInstanceOf[DATA_CLOSING].remoteCommitPublished.contains(closingState2))
}
@ -1254,7 +1270,10 @@ 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])
inside(alice2blockchain.expectMsgType[PublishReplaceableTx]) { tx =>
assert(tx.txInfo.isInstanceOf[ClaimLocalAnchorOutputTx])
assert(tx.commitTx == bobCommitTx)
}
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)