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:
parent
cae22d71be
commit
f6b051cf73
9 changed files with 104 additions and 58 deletions
|
@ -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 {
|
||||
|
|
|
@ -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)
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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 }
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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]
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
Loading…
Add table
Reference in a new issue