mirror of
https://github.com/ACINQ/eclair.git
synced 2025-02-22 14:22:39 +01:00
Extract tx publishing from watchers (#1749)
Introduce a `TxPublisher` actor to publish channel txs. Move logic from watcher to this new actor. Remove the `TxSigningKit` abstraction that was introduced a bit too early. The `TxPublisher` will hold all the logic so we'll start by providing the full commitments, and we'll extract more compact objects later. We also now publish the commit-tx and its anchor-tx independently.
This commit is contained in:
parent
3da0b80cb2
commit
48c0c4c98b
27 changed files with 1561 additions and 1308 deletions
|
@ -17,7 +17,7 @@
|
|||
package fr.acinq.eclair
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.{ActorRef, ActorSystem, Props, SupervisorStrategy}
|
||||
import akka.actor.{ActorContext, ActorRef, ActorSystem, Props, SupervisorStrategy}
|
||||
import akka.pattern.after
|
||||
import akka.util.Timeout
|
||||
import com.softwaremill.sttp.okhttp.OkHttpFutureBackend
|
||||
|
@ -28,7 +28,7 @@ import fr.acinq.eclair.blockchain.bitcoind.rpc.{BasicBitcoinJsonRPCClient, Batch
|
|||
import fr.acinq.eclair.blockchain.bitcoind.zmq.ZMQActor
|
||||
import fr.acinq.eclair.blockchain.bitcoind.{BitcoinCoreWallet, ZmqWatcher}
|
||||
import fr.acinq.eclair.blockchain.fee._
|
||||
import fr.acinq.eclair.channel.Register
|
||||
import fr.acinq.eclair.channel.{Channel, Register, TxPublisher}
|
||||
import fr.acinq.eclair.crypto.keymanager.{LocalChannelKeyManager, LocalNodeKeyManager}
|
||||
import fr.acinq.eclair.db.Databases.FileBackup
|
||||
import fr.acinq.eclair.db.{Databases, DbEventHandler, FileBackupHandler}
|
||||
|
@ -230,10 +230,11 @@ class Setup(datadir: File,
|
|||
})
|
||||
_ <- feeratesRetrieved.future
|
||||
|
||||
extendedBitcoinClient = new ExtendedBitcoinClient(new BatchingBitcoinJsonRPCClient(bitcoin))
|
||||
watcher = {
|
||||
system.actorOf(SimpleSupervisor.props(Props(new ZMQActor(config.getString("bitcoind.zmqblock"), Some(zmqBlockConnected))), "zmqblock", SupervisorStrategy.Restart))
|
||||
system.actorOf(SimpleSupervisor.props(Props(new ZMQActor(config.getString("bitcoind.zmqtx"), Some(zmqTxConnected))), "zmqtx", SupervisorStrategy.Restart))
|
||||
system.actorOf(SimpleSupervisor.props(ZmqWatcher.props(nodeParams.chainHash, blockCount, new ExtendedBitcoinClient(new BatchingBitcoinJsonRPCClient(bitcoin))), "watcher", SupervisorStrategy.Resume))
|
||||
system.actorOf(SimpleSupervisor.props(ZmqWatcher.props(nodeParams.chainHash, blockCount, extendedBitcoinClient), "watcher", SupervisorStrategy.Resume))
|
||||
}
|
||||
|
||||
router = system.actorOf(SimpleSupervisor.props(Router.props(nodeParams, watcher, Some(routerInitialized)), "router", SupervisorStrategy.Resume))
|
||||
|
@ -267,7 +268,8 @@ class Setup(datadir: File,
|
|||
// we want to make sure the handler for post-restart broken HTLCs has finished initializing.
|
||||
_ <- postRestartCleanUpInitialized.future
|
||||
|
||||
channelFactory = Peer.SimpleChannelFactory(nodeParams, watcher, relayer, wallet)
|
||||
txPublisherFactory = Channel.SimpleTxPublisherFactory(nodeParams, watcher, extendedBitcoinClient)
|
||||
channelFactory = Peer.SimpleChannelFactory(nodeParams, watcher, relayer, wallet, txPublisherFactory)
|
||||
peerFactory = Switchboard.SimplePeerFactory(nodeParams, wallet, channelFactory)
|
||||
|
||||
switchboard = system.actorOf(SimpleSupervisor.props(Switchboard.props(nodeParams, peerFactory), "switchboard", SupervisorStrategy.Resume))
|
||||
|
|
|
@ -17,10 +17,8 @@
|
|||
package fr.acinq.eclair.blockchain
|
||||
|
||||
import akka.actor.ActorRef
|
||||
import fr.acinq.bitcoin.{ByteVector32, Satoshi, Transaction}
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.bitcoin.{ByteVector32, Transaction}
|
||||
import fr.acinq.eclair.channel.BitcoinEvent
|
||||
import fr.acinq.eclair.transactions.Transactions.TransactionSigningKit
|
||||
import fr.acinq.eclair.wire.protocol.ChannelAnnouncement
|
||||
|
||||
/**
|
||||
|
@ -110,19 +108,6 @@ final case class WatchEventSpentBasic(event: BitcoinEvent) extends WatchEvent
|
|||
// TODO: not implemented yet.
|
||||
final case class WatchEventLost(event: BitcoinEvent) extends WatchEvent
|
||||
|
||||
// @formatter:off
|
||||
sealed trait PublishStrategy
|
||||
object PublishStrategy {
|
||||
case object JustPublish extends PublishStrategy
|
||||
case class SetFeerate(currentFeerate: FeeratePerKw, targetFeerate: FeeratePerKw, dustLimit: Satoshi, signingKit: TransactionSigningKit) extends PublishStrategy {
|
||||
override def toString = s"SetFeerate(target=$targetFeerate)"
|
||||
}
|
||||
}
|
||||
// @formatter:on
|
||||
|
||||
/** Publish the provided tx as soon as possible depending on lock time, csv and publishing strategy. */
|
||||
final case class PublishAsap(tx: Transaction, strategy: PublishStrategy)
|
||||
|
||||
// @formatter:off
|
||||
sealed trait UtxoStatus
|
||||
object UtxoStatus {
|
||||
|
|
|
@ -16,31 +16,21 @@
|
|||
|
||||
package fr.acinq.eclair.blockchain.bitcoind
|
||||
|
||||
import java.util.concurrent.Executors
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
import akka.actor.typed.SupervisorStrategy
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.actor.typed.scaladsl.adapter.ClassicActorContextOps
|
||||
import akka.actor.{Actor, ActorLogging, Cancellable, Props, Terminated}
|
||||
import akka.pattern.pipe
|
||||
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
|
||||
import fr.acinq.bitcoin.DeterministicWallet.ExtendedPublicKey
|
||||
import fr.acinq.bitcoin._
|
||||
import fr.acinq.eclair.KamonExt
|
||||
import fr.acinq.eclair.blockchain.Monitoring.Metrics
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient
|
||||
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient.{FundTransactionOptions, FundTransactionResponse}
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.eclair.blockchain.watchdogs.BlockchainWatchdog
|
||||
import fr.acinq.eclair.channel.{BITCOIN_PARENT_TX_CONFIRMED, Commitments}
|
||||
import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
|
||||
import fr.acinq.eclair.transactions.Transactions.{HtlcSuccessTx, HtlcTimeoutTx, TransactionSigningKit, TransactionWithInputInfo, weight2fee}
|
||||
import fr.acinq.eclair.transactions.{Scripts, Transactions}
|
||||
import org.json4s.JsonAST.{JArray, JBool, JDecimal, JInt, JString}
|
||||
import org.json4s.JsonAST._
|
||||
import scodec.bits.ByteVector
|
||||
|
||||
import scala.collection.immutable.SortedMap
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.{ExecutionContext, Future}
|
||||
import scala.util.Try
|
||||
|
@ -60,25 +50,23 @@ class ZmqWatcher(chainHash: ByteVector32, blockCount: AtomicLong, client: Extend
|
|||
|
||||
context.system.eventStream.subscribe(self, classOf[NewBlock])
|
||||
context.system.eventStream.subscribe(self, classOf[NewTransaction])
|
||||
context.system.eventStream.subscribe(self, classOf[CurrentBlockCount])
|
||||
|
||||
private val watchdog = context.spawn(Behaviors.supervise(BlockchainWatchdog(chainHash, 150 seconds)).onFailure(SupervisorStrategy.resume), "blockchain-watchdog")
|
||||
|
||||
// this is to initialize block count
|
||||
self ! TickNewBlock
|
||||
|
||||
// @formatter:off
|
||||
private case class PublishNextBlock(p: PublishAsap)
|
||||
private case class TriggerEvent(w: Watch, e: WatchEvent)
|
||||
|
||||
// @formatter:off
|
||||
private sealed trait AddWatchResult
|
||||
private case object Keep extends AddWatchResult
|
||||
private case object Ignore extends AddWatchResult
|
||||
// @formatter:on
|
||||
|
||||
def receive: Receive = watching(Set(), Map(), SortedMap(), None)
|
||||
def receive: Receive = watching(Set(), Map(), None)
|
||||
|
||||
def watching(watches: Set[Watch], watchedUtxos: Map[OutPoint, Set[Watch]], block2tx: SortedMap[Long, Seq[PublishAsap]], nextTick: Option[Cancellable]): Receive = {
|
||||
def watching(watches: Set[Watch], watchedUtxos: Map[OutPoint, Set[Watch]], nextTick: Option[Cancellable]): Receive = {
|
||||
|
||||
case NewTransaction(tx) =>
|
||||
log.debug("analyzing txid={} tx={}", tx.txid, tx)
|
||||
|
@ -100,7 +88,7 @@ class ZmqWatcher(chainHash: ByteVector32, blockCount: AtomicLong, client: Extend
|
|||
log.debug("scheduling a new task to check on tx confirmations")
|
||||
// we do this to avoid herd effects in testing when generating a lots of blocks in a row
|
||||
val task = context.system.scheduler.scheduleOnce(2 seconds, self, TickNewBlock)
|
||||
context become watching(watches, watchedUtxos, block2tx, Some(task))
|
||||
context become watching(watches, watchedUtxos, Some(task))
|
||||
|
||||
case TickNewBlock =>
|
||||
client.getBlockCount.map {
|
||||
|
@ -114,7 +102,7 @@ class ZmqWatcher(chainHash: ByteVector32, blockCount: AtomicLong, client: Extend
|
|||
KamonExt.timeFuture(Metrics.NewBlockCheckConfirmedDuration.withoutTags()) {
|
||||
Future.sequence(watches.collect { case w: WatchConfirmed => checkConfirmed(w) })
|
||||
}
|
||||
context become watching(watches, watchedUtxos, block2tx, None)
|
||||
context become watching(watches, watchedUtxos, None)
|
||||
|
||||
case TriggerEvent(w, e) if watches.contains(w) =>
|
||||
log.info("triggering {}", w)
|
||||
|
@ -125,14 +113,9 @@ class ZmqWatcher(chainHash: ByteVector32, blockCount: AtomicLong, client: Extend
|
|||
// They are never cleaned up but it is not a big deal for now (1 channel == 1 watch)
|
||||
()
|
||||
case _ =>
|
||||
context become watching(watches - w, removeWatchedUtxos(watchedUtxos, w), block2tx, nextTick)
|
||||
context become watching(watches - w, removeWatchedUtxos(watchedUtxos, w), nextTick)
|
||||
}
|
||||
|
||||
case CurrentBlockCount(count) =>
|
||||
val toPublish = block2tx.filterKeys(_ <= count)
|
||||
toPublish.values.flatten.foreach(tx => publish(tx))
|
||||
context become watching(watches, watchedUtxos, block2tx -- toPublish.keys, nextTick)
|
||||
|
||||
case w: Watch =>
|
||||
|
||||
val result = w match {
|
||||
|
@ -196,42 +179,10 @@ class ZmqWatcher(chainHash: ByteVector32, blockCount: AtomicLong, client: Extend
|
|||
case Keep =>
|
||||
log.debug("adding watch {} for {}", w, sender)
|
||||
context.watch(w.replyTo)
|
||||
context become watching(watches + w, addWatchedUtxos(watchedUtxos, w), block2tx, nextTick)
|
||||
context become watching(watches + w, addWatchedUtxos(watchedUtxos, w), nextTick)
|
||||
case Ignore => ()
|
||||
}
|
||||
|
||||
case p@PublishAsap(tx, _) =>
|
||||
val blockCount = this.blockCount.get()
|
||||
val cltvTimeout = Scripts.cltvTimeout(tx)
|
||||
val csvTimeouts = Scripts.csvTimeouts(tx)
|
||||
if (csvTimeouts.nonEmpty) {
|
||||
// watcher supports txs with multiple csv-delayed inputs: we watch all delayed parents and try to publish every
|
||||
// time a parent's relative delays are satisfied, so we will eventually succeed.
|
||||
csvTimeouts.foreach { case (parentTxId, csvTimeout) =>
|
||||
log.info(s"txid=${tx.txid} has a relative timeout of $csvTimeout blocks, watching parentTxId=$parentTxId tx={}", tx)
|
||||
self ! WatchConfirmed(self, parentTxId, minDepth = csvTimeout, BITCOIN_PARENT_TX_CONFIRMED(p))
|
||||
}
|
||||
} else if (cltvTimeout > blockCount) {
|
||||
log.info(s"delaying publication of txid=${tx.txid} until block=$cltvTimeout (curblock=$blockCount)")
|
||||
val block2tx1 = block2tx.updated(cltvTimeout, block2tx.getOrElse(cltvTimeout, Seq.empty[PublishAsap]) :+ p)
|
||||
context become watching(watches, watchedUtxos, block2tx1, nextTick)
|
||||
} else publish(p)
|
||||
|
||||
case WatchEventConfirmed(BITCOIN_PARENT_TX_CONFIRMED(p@PublishAsap(tx, _)), _, _, _) =>
|
||||
log.info(s"parent tx of txid=${tx.txid} has been confirmed")
|
||||
val blockCount = this.blockCount.get()
|
||||
val cltvTimeout = Scripts.cltvTimeout(tx)
|
||||
if (cltvTimeout > blockCount) {
|
||||
log.info(s"delaying publication of txid=${tx.txid} until block=$cltvTimeout (curblock=$blockCount)")
|
||||
val block2tx1 = block2tx.updated(cltvTimeout, block2tx.getOrElse(cltvTimeout, Seq.empty[PublishAsap]) :+ p)
|
||||
context become watching(watches, watchedUtxos, block2tx1, nextTick)
|
||||
} else publish(p)
|
||||
|
||||
case PublishNextBlock(p) =>
|
||||
val nextBlockCount = this.blockCount.get() + 1
|
||||
val block2tx1 = block2tx.updated(nextBlockCount, block2tx.getOrElse(nextBlockCount, Seq.empty[PublishAsap]) :+ p)
|
||||
context become watching(watches, watchedUtxos, block2tx1, nextTick)
|
||||
|
||||
case ValidateRequest(ann) => client.validate(ann).pipeTo(sender)
|
||||
|
||||
case GetTxWithMeta(txid) => client.getTransactionMeta(txid).pipeTo(sender)
|
||||
|
@ -240,165 +191,12 @@ class ZmqWatcher(chainHash: ByteVector32, blockCount: AtomicLong, client: Extend
|
|||
// we remove watches associated to dead actor
|
||||
val deprecatedWatches = watches.filter(_.replyTo == actor)
|
||||
val watchedUtxos1 = deprecatedWatches.foldLeft(watchedUtxos) { case (m, w) => removeWatchedUtxos(m, w) }
|
||||
context.become(watching(watches -- deprecatedWatches, watchedUtxos1, block2tx, nextTick))
|
||||
context.become(watching(watches -- deprecatedWatches, watchedUtxos1, nextTick))
|
||||
|
||||
case Symbol("watches") => sender ! watches
|
||||
|
||||
}
|
||||
|
||||
// NOTE: we use a single thread to publish transactions so that it preserves order.
|
||||
// CHANGING THIS WILL RESULT IN CONCURRENCY ISSUES WHILE PUBLISHING PARENT AND CHILD TXS
|
||||
val singleThreadExecutionContext = ExecutionContext.fromExecutor(Executors.newSingleThreadExecutor())
|
||||
|
||||
def publish(p: PublishAsap): Future[ByteVector32] = {
|
||||
p.strategy match {
|
||||
case PublishStrategy.SetFeerate(currentFeerate, targetFeerate, dustLimit, signingKit) =>
|
||||
log.info("publishing tx: input={}:{} txid={} tx={}", signingKit.spentOutpoint.txid, signingKit.spentOutpoint.index, p.tx.txid, p.tx)
|
||||
val publishF = signingKit match {
|
||||
case signingKit: TransactionSigningKit.ClaimAnchorOutputSigningKit => publishCommitWithAnchor(p.tx, currentFeerate, targetFeerate, dustLimit, signingKit)
|
||||
case signingKit: TransactionSigningKit.HtlcTxSigningKit => publishHtlcTx(currentFeerate, targetFeerate, dustLimit, signingKit)
|
||||
}
|
||||
publishF.recoverWith {
|
||||
case t: Throwable if t.getMessage.contains("(code: -4)") || t.getMessage.contains("(code: -6)") =>
|
||||
log.warning("not enough funds to publish tx, will retry next block: reason={} input={}:{} txid={}", t.getMessage, signingKit.spentOutpoint.txid, signingKit.spentOutpoint.index, p.tx.txid)
|
||||
self ! PublishNextBlock(p)
|
||||
Future.failed(t)
|
||||
case t: Throwable =>
|
||||
log.error("cannot publish tx: reason={} input={}:{} txid={}", t.getMessage, signingKit.spentOutpoint.txid, signingKit.spentOutpoint.index, p.tx.txid)
|
||||
Future.failed(t)
|
||||
}
|
||||
case PublishStrategy.JustPublish =>
|
||||
log.info("publishing tx: txid={} tx={}", p.tx.txid, p.tx)
|
||||
publish(p.tx, isRetry = false)
|
||||
}
|
||||
}
|
||||
|
||||
def publish(tx: Transaction, isRetry: Boolean): Future[ByteVector32] = {
|
||||
client.publishTransaction(tx)(singleThreadExecutionContext).recoverWith {
|
||||
case t: Throwable if t.getMessage.contains("(code: -25)") && !isRetry => // we retry only once
|
||||
import akka.pattern.after
|
||||
after(3 seconds, context.system.scheduler)(Future.successful({})).flatMap(_ => publish(tx, isRetry = true))
|
||||
case t: Throwable =>
|
||||
log.error("cannot publish tx: reason={} txid={}", t.getMessage, tx.txid)
|
||||
Future.failed(t)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Publish the commit tx, and optionally an anchor tx that spends from the commit tx and helps get it confirmed with CPFP.
|
||||
*/
|
||||
def publishCommitWithAnchor(commitTx: Transaction, currentFeerate: FeeratePerKw, targetFeerate: FeeratePerKw, dustLimit: Satoshi, signingKit: TransactionSigningKit.ClaimAnchorOutputSigningKit): Future[ByteVector32] = {
|
||||
import signingKit._
|
||||
if (targetFeerate <= currentFeerate) {
|
||||
log.info(s"publishing commit tx without the anchor (current feerate=$currentFeerate): txid=${commitTx.txid}")
|
||||
publish(commitTx, isRetry = false)
|
||||
} else {
|
||||
log.info(s"publishing commit tx with the anchor (target feerate=$targetFeerate): txid=${commitTx.txid}")
|
||||
// We want the feerate of the package (commit tx + tx spending anchor) to equal targetFeerate.
|
||||
// Thus we have: anchorFeerate = targetFeerate + (weight-commit-tx / weight-anchor-tx) * (targetFeerate - commitTxFeerate)
|
||||
// If we use the smallest weight possible for the anchor tx, the feerate we use will thus be greater than what we want,
|
||||
// and we can adjust it afterwards by raising the change output amount.
|
||||
val anchorFeerate = targetFeerate + FeeratePerKw(targetFeerate.feerate - currentFeerate.feerate) * commitTx.weight() / Transactions.claimAnchorOutputMinWeight
|
||||
// NB: fundrawtransaction requires at least one output, and may add at most one additional change output.
|
||||
// Since the purpose of this transaction is just to do a CPFP, the resulting tx should have a single change output
|
||||
// (note that bitcoind doesn't let us publish a transaction with no outputs).
|
||||
// To work around these limitations, we start with a dummy output and later merge that dummy output with the optional
|
||||
// change output added by bitcoind.
|
||||
// NB: fundrawtransaction doesn't support non-wallet inputs, so we have to remove our anchor input and re-add it later.
|
||||
// That means bitcoind will not take our anchor input's weight into account when adding inputs to set the fee.
|
||||
// That's ok, we can increase the fee later by decreasing the output amount. But we need to ensure we'll have enough
|
||||
// to cover the weight of our anchor input, which is why we set it to the following value.
|
||||
val dummyChangeAmount = Transactions.weight2fee(anchorFeerate, Transactions.claimAnchorOutputMinWeight) + dustLimit
|
||||
publish(commitTx, isRetry = false).flatMap(commitTxId => {
|
||||
val txNotFunded = Transaction(2, Nil, TxOut(dummyChangeAmount, Script.pay2wpkh(Transactions.PlaceHolderPubKey)) :: Nil, 0)
|
||||
client.fundTransaction(txNotFunded, FundTransactionOptions(anchorFeerate, lockUtxos = true))(singleThreadExecutionContext)
|
||||
}).flatMap(fundTxResponse => {
|
||||
// We merge the outputs if there's more than one.
|
||||
fundTxResponse.changePosition match {
|
||||
case Some(changePos) =>
|
||||
val changeOutput = fundTxResponse.tx.txOut(changePos.toInt)
|
||||
val txSingleOutput = fundTxResponse.tx.copy(txOut = Seq(changeOutput.copy(amount = changeOutput.amount + dummyChangeAmount)))
|
||||
Future.successful(fundTxResponse.copy(tx = txSingleOutput))
|
||||
case None =>
|
||||
client.getChangeAddress()(singleThreadExecutionContext).map(pubkeyHash => {
|
||||
val txSingleOutput = fundTxResponse.tx.copy(txOut = Seq(TxOut(dummyChangeAmount, Script.pay2wpkh(pubkeyHash))))
|
||||
fundTxResponse.copy(tx = txSingleOutput)
|
||||
})
|
||||
}
|
||||
}).map(fundTxResponse => {
|
||||
require(fundTxResponse.tx.txOut.size == 1, "funded transaction should have a single change output")
|
||||
// NB: we insert the anchor input in the *first* position because our signing helpers only sign input #0.
|
||||
val unsignedTx = txWithInput.copy(tx = fundTxResponse.tx.copy(txIn = txWithInput.tx.txIn.head +: fundTxResponse.tx.txIn))
|
||||
adjustAnchorOutputChange(unsignedTx, commitTx, fundTxResponse.amountIn + Transactions.AnchorOutputsCommitmentFormat.anchorAmount, currentFeerate, targetFeerate, dustLimit)
|
||||
}).flatMap(claimAnchorTx => {
|
||||
val claimAnchorSig = keyManager.sign(claimAnchorTx, localFundingPubKey, Transactions.TxOwner.Local, commitmentFormat)
|
||||
val signedClaimAnchorTx = Transactions.addSigs(claimAnchorTx, claimAnchorSig)
|
||||
val commitInfo = ExtendedBitcoinClient.PreviousTx(signedClaimAnchorTx.input, signedClaimAnchorTx.tx.txIn.head.witness)
|
||||
client.signTransaction(signedClaimAnchorTx.tx, Seq(commitInfo))(singleThreadExecutionContext)
|
||||
}).flatMap(signTxResponse => {
|
||||
client.publishTransaction(signTxResponse.tx)(singleThreadExecutionContext)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Publish an htlc tx, and optionally RBF it before by adding new inputs/outputs to help get it confirmed.
|
||||
*/
|
||||
def publishHtlcTx(currentFeerate: FeeratePerKw, targetFeerate: FeeratePerKw, dustLimit: Satoshi, signingKit: TransactionSigningKit.HtlcTxSigningKit): Future[ByteVector32] = {
|
||||
import signingKit._
|
||||
if (targetFeerate <= currentFeerate) {
|
||||
val localSig = keyManager.sign(txWithInput, localHtlcBasepoint, localPerCommitmentPoint, Transactions.TxOwner.Local, commitmentFormat)
|
||||
val signedHtlcTx = addHtlcTxSigs(txWithInput, localSig, signingKit)
|
||||
log.info("publishing htlc tx without adding inputs: txid={}", signedHtlcTx.tx.txid)
|
||||
client.publishTransaction(signedHtlcTx.tx)(singleThreadExecutionContext)
|
||||
} else {
|
||||
log.info("publishing htlc tx with additional inputs: commit input={}:{} target feerate={}", txWithInput.input.outPoint.txid, txWithInput.input.outPoint.index, targetFeerate)
|
||||
// NB: fundrawtransaction doesn't support non-wallet inputs, so we clear the input and re-add it later.
|
||||
val txNotFunded = txWithInput.tx.copy(txIn = Nil, txOut = txWithInput.tx.txOut.head.copy(amount = dustLimit) :: Nil)
|
||||
val htlcTxWeight = signingKit match {
|
||||
case _: TransactionSigningKit.HtlcSuccessSigningKit => commitmentFormat.htlcSuccessWeight
|
||||
case _: TransactionSigningKit.HtlcTimeoutSigningKit => commitmentFormat.htlcTimeoutWeight
|
||||
}
|
||||
// We want the feerate of our final HTLC tx to equal targetFeerate. However, we removed the HTLC input from what we
|
||||
// send to fundrawtransaction, so bitcoind will not know the total weight of the final tx. In order to make up for
|
||||
// this difference, we need to tell bitcoind to target a higher feerate that takes into account the weight of the
|
||||
// input we removed.
|
||||
// That feerate will satisfy the following equality:
|
||||
// feerate * weight_seen_by_bitcoind = target_feerate * (weight_seen_by_bitcoind + htlc_input_weight)
|
||||
// So: feerate = target_feerate * (1 + htlc_input_weight / weight_seen_by_bitcoind)
|
||||
// Because bitcoind will add at least one P2WPKH input, weight_seen_by_bitcoind >= htlc_tx_weight + p2wpkh_weight
|
||||
// Thus: feerate <= target_feerate * (1 + htlc_input_weight / (htlc_tx_weight + p2wpkh_weight))
|
||||
// NB: we don't take into account the fee paid by our HTLC input: we will take it into account when we adjust the
|
||||
// change output amount (unless bitcoind didn't add any change output, in that case we will overpay the fee slightly).
|
||||
val weightRatio = 1.0 + (Transactions.htlcInputMaxWeight.toDouble / (htlcTxWeight + Transactions.claimP2WPKHOutputWeight))
|
||||
client.fundTransaction(txNotFunded, FundTransactionOptions(targetFeerate * weightRatio, lockUtxos = true, changePosition = Some(1)))(singleThreadExecutionContext).map(fundTxResponse => {
|
||||
log.info(s"added ${fundTxResponse.tx.txIn.length} wallet input(s) and ${fundTxResponse.tx.txOut.length - 1} wallet output(s) to htlc tx spending commit input=${txWithInput.input.outPoint.txid}:${txWithInput.input.outPoint.index}")
|
||||
// We add the HTLC input (from the commit tx) and restore the HTLC output.
|
||||
// NB: we can't modify them because they are signed by our peer (with SIGHASH_SINGLE | SIGHASH_ANYONECANPAY).
|
||||
val txWithHtlcInput = fundTxResponse.tx.copy(
|
||||
txIn = txWithInput.tx.txIn ++ fundTxResponse.tx.txIn,
|
||||
txOut = txWithInput.tx.txOut ++ fundTxResponse.tx.txOut.tail
|
||||
)
|
||||
val unsignedTx = signingKit match {
|
||||
case htlcSuccess: TransactionSigningKit.HtlcSuccessSigningKit => htlcSuccess.txWithInput.copy(tx = txWithHtlcInput)
|
||||
case htlcTimeout: TransactionSigningKit.HtlcTimeoutSigningKit => htlcTimeout.txWithInput.copy(tx = txWithHtlcInput)
|
||||
}
|
||||
adjustHtlcTxChange(unsignedTx, fundTxResponse.amountIn + unsignedTx.input.txOut.amount, targetFeerate, dustLimit, signingKit)
|
||||
}).flatMap(unsignedTx => {
|
||||
val localSig = keyManager.sign(unsignedTx, localHtlcBasepoint, localPerCommitmentPoint, Transactions.TxOwner.Local, commitmentFormat)
|
||||
val signedHtlcTx = addHtlcTxSigs(unsignedTx, localSig, signingKit)
|
||||
val inputInfo = ExtendedBitcoinClient.PreviousTx(signedHtlcTx.input, signedHtlcTx.tx.txIn.head.witness)
|
||||
client.signTransaction(signedHtlcTx.tx, Seq(inputInfo), allowIncomplete = true)(singleThreadExecutionContext).flatMap(signTxResponse => {
|
||||
// NB: bitcoind messes up the witness stack for our htlc input, so we need to restore it.
|
||||
// See https://github.com/bitcoin/bitcoin/issues/21151
|
||||
val completeTx = signedHtlcTx.tx.copy(txIn = signedHtlcTx.tx.txIn.head +: signTxResponse.tx.txIn.tail)
|
||||
log.info("publishing bumped htlc tx: commit input={}:{} txid={} tx={}", txWithInput.input.outPoint.txid, txWithInput.input.outPoint.index, completeTx.txid, completeTx)
|
||||
client.publishTransaction(completeTx)(singleThreadExecutionContext)
|
||||
})
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
def checkConfirmed(w: WatchConfirmed): Future[Unit] = {
|
||||
log.debug("checking confirmations of txid={}", w.txId)
|
||||
// NB: this is very inefficient since internally we call `getrawtransaction` three times, but it doesn't really
|
||||
|
@ -498,54 +296,4 @@ object ZmqWatcher {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Adjust the amount of the change output of an anchor tx to match our target feerate.
|
||||
* We need this because fundrawtransaction doesn't allow us to leave non-wallet inputs, so we have to add them
|
||||
* afterwards which may bring the resulting feerate below our target.
|
||||
*/
|
||||
def adjustAnchorOutputChange(unsignedTx: Transactions.ClaimLocalAnchorOutputTx, commitTx: Transaction, amountIn: Satoshi, currentFeerate: FeeratePerKw, targetFeerate: FeeratePerKw, dustLimit: Satoshi): Transactions.ClaimLocalAnchorOutputTx = {
|
||||
require(unsignedTx.tx.txOut.size == 1, "funded transaction should have a single change output")
|
||||
// We take into account witness weight and adjust the fee to match our desired feerate.
|
||||
val dummySignedClaimAnchorTx = Transactions.addSigs(unsignedTx, Transactions.PlaceHolderSig)
|
||||
// NB: we assume that our bitcoind wallet uses only P2WPKH inputs when funding txs.
|
||||
val estimatedWeight = commitTx.weight() + dummySignedClaimAnchorTx.tx.weight() + Transactions.claimP2WPKHOutputWitnessWeight * (dummySignedClaimAnchorTx.tx.txIn.size - 1)
|
||||
val targetFee = Transactions.weight2fee(targetFeerate, estimatedWeight) - Transactions.weight2fee(currentFeerate, commitTx.weight())
|
||||
val amountOut = dustLimit.max(amountIn - targetFee)
|
||||
unsignedTx.copy(tx = unsignedTx.tx.copy(txOut = unsignedTx.tx.txOut.head.copy(amount = amountOut) :: Nil))
|
||||
}
|
||||
|
||||
def addHtlcTxSigs(unsignedHtlcTx: Transactions.HtlcTx, localSig: ByteVector64, signingKit: TransactionSigningKit.HtlcTxSigningKit): Transactions.HtlcTx = {
|
||||
signingKit match {
|
||||
case htlcSuccess: TransactionSigningKit.HtlcSuccessSigningKit =>
|
||||
Transactions.addSigs(unsignedHtlcTx.asInstanceOf[HtlcSuccessTx], localSig, signingKit.remoteSig, htlcSuccess.preimage, signingKit.commitmentFormat)
|
||||
case htlcTimeout: TransactionSigningKit.HtlcTimeoutSigningKit =>
|
||||
Transactions.addSigs(unsignedHtlcTx.asInstanceOf[HtlcTimeoutTx], localSig, signingKit.remoteSig, signingKit.commitmentFormat)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Adjust the change output of an htlc tx to match our target feerate.
|
||||
* We need this because fundrawtransaction doesn't allow us to leave non-wallet inputs, so we have to add them
|
||||
* afterwards which may bring the resulting feerate below our target.
|
||||
*/
|
||||
def adjustHtlcTxChange(unsignedTx: Transactions.HtlcTx, amountIn: Satoshi, targetFeerate: FeeratePerKw, dustLimit: Satoshi, signingKit: TransactionSigningKit.HtlcTxSigningKit): Transactions.HtlcTx = {
|
||||
require(unsignedTx.tx.txOut.size <= 2, "funded transaction should have at most one change output")
|
||||
val dummySignedTx = addHtlcTxSigs(unsignedTx, Transactions.PlaceHolderSig, signingKit)
|
||||
// We adjust the change output to obtain the targeted feerate.
|
||||
val estimatedWeight = dummySignedTx.tx.weight() + Transactions.claimP2WPKHOutputWitnessWeight * (dummySignedTx.tx.txIn.size - 1)
|
||||
val targetFee = Transactions.weight2fee(targetFeerate, estimatedWeight)
|
||||
val changeAmount = amountIn - dummySignedTx.tx.txOut.head.amount - targetFee
|
||||
if (dummySignedTx.tx.txOut.length == 2 && changeAmount >= dustLimit) {
|
||||
unsignedTx match {
|
||||
case htlcSuccess: HtlcSuccessTx => htlcSuccess.copy(tx = htlcSuccess.tx.copy(txOut = Seq(htlcSuccess.tx.txOut.head, htlcSuccess.tx.txOut(1).copy(amount = changeAmount))))
|
||||
case htlcTimeout: HtlcTimeoutTx => htlcTimeout.copy(tx = htlcTimeout.tx.copy(txOut = Seq(htlcTimeout.tx.txOut.head, htlcTimeout.tx.txOut(1).copy(amount = changeAmount))))
|
||||
}
|
||||
} else {
|
||||
unsignedTx match {
|
||||
case htlcSuccess: HtlcSuccessTx => htlcSuccess.copy(tx = htlcSuccess.tx.copy(txOut = Seq(htlcSuccess.tx.txOut.head)))
|
||||
case htlcTimeout: HtlcTimeoutTx => htlcTimeout.copy(tx = htlcTimeout.tx.copy(txOut = Seq(htlcTimeout.tx.txOut.head)))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -16,7 +16,9 @@
|
|||
|
||||
package fr.acinq.eclair.channel
|
||||
|
||||
import akka.actor.{ActorRef, FSM, OneForOneStrategy, Props, Status, SupervisorStrategy}
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.actor.typed.scaladsl.adapter.ClassicActorContextOps
|
||||
import akka.actor.{ActorContext, ActorRef, FSM, OneForOneStrategy, Props, Status, SupervisorStrategy}
|
||||
import akka.event.Logging.MDC
|
||||
import akka.pattern.pipe
|
||||
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
|
||||
|
@ -24,8 +26,10 @@ import fr.acinq.bitcoin.{ByteVector32, OutPoint, Satoshi, SatoshiLong, Script, S
|
|||
import fr.acinq.eclair.Logs.LogCategory
|
||||
import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient
|
||||
import fr.acinq.eclair.channel.Helpers.{Closing, Funding}
|
||||
import fr.acinq.eclair.channel.Monitoring.{Metrics, Tags}
|
||||
import fr.acinq.eclair.channel.TxPublisher.{PublishRawTx, PublishTx, SetChannelId, SignAndPublishTx}
|
||||
import fr.acinq.eclair.crypto.ShaChain
|
||||
import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
|
||||
import fr.acinq.eclair.db.PendingRelayDb
|
||||
|
@ -47,7 +51,19 @@ import scala.util.{Failure, Success, Try}
|
|||
*/
|
||||
|
||||
object Channel {
|
||||
def props(nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: PublicKey, blockchain: ActorRef, relayer: ActorRef, origin_opt: Option[ActorRef]): Props = Props(new Channel(nodeParams, wallet, remoteNodeId, blockchain, relayer, origin_opt))
|
||||
|
||||
trait TxPublisherFactory {
|
||||
def spawnTxPublisher(context: ActorContext, remoteNodeId: PublicKey): akka.actor.typed.ActorRef[TxPublisher.Command]
|
||||
}
|
||||
|
||||
case class SimpleTxPublisherFactory(nodeParams: NodeParams, watcher: ActorRef, bitcoinClient: ExtendedBitcoinClient) extends TxPublisherFactory {
|
||||
override def spawnTxPublisher(context: ActorContext, remoteNodeId: PublicKey): akka.actor.typed.ActorRef[TxPublisher.Command] = {
|
||||
context.spawn(Behaviors.supervise(TxPublisher(nodeParams, remoteNodeId, watcher, bitcoinClient)).onFailure(akka.actor.typed.SupervisorStrategy.restart), "tx-publisher")
|
||||
}
|
||||
}
|
||||
|
||||
def props(nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: PublicKey, blockchain: ActorRef, relayer: ActorRef, txPublisherFactory: TxPublisherFactory, origin_opt: Option[ActorRef]): Props =
|
||||
Props(new Channel(nodeParams, wallet, remoteNodeId, blockchain, relayer, txPublisherFactory, origin_opt))
|
||||
|
||||
// see https://github.com/lightningnetwork/lightning-rfc/blob/master/07-routing-gossip.md#requirements
|
||||
val ANNOUNCEMENTS_MINCONF = 6
|
||||
|
@ -100,7 +116,7 @@ object Channel {
|
|||
|
||||
}
|
||||
|
||||
class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId: PublicKey, blockchain: ActorRef, relayer: ActorRef, origin_opt: Option[ActorRef] = None)(implicit ec: ExecutionContext = ExecutionContext.Implicits.global) extends FSM[State, Data] with FSMDiagnosticActorLogging[State, Data] {
|
||||
class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId: PublicKey, blockchain: ActorRef, relayer: ActorRef, txPublisherFactory: Channel.TxPublisherFactory, origin_opt: Option[ActorRef] = None)(implicit ec: ExecutionContext = ExecutionContext.Implicits.global) extends FSM[State, Data] with FSMDiagnosticActorLogging[State, Data] {
|
||||
|
||||
import Channel._
|
||||
|
||||
|
@ -111,7 +127,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
|
||||
// we assume that the peer is the channel's parent
|
||||
private val peer = context.parent
|
||||
//noinspection ActorMutableStateInspection
|
||||
// noinspection ActorMutableStateInspection
|
||||
// the last active connection we are aware of; note that the peer manages connections and asynchronously notifies
|
||||
// the channel, which means that if we get disconnected, the previous active connection will die and some messages will
|
||||
// be sent to dead letters, before the channel gets notified of the disconnection; knowing that this will happen, we
|
||||
|
@ -119,6 +135,8 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
// that the active connection may point to dead letters at all time
|
||||
private var activeConnection = context.system.deadLetters
|
||||
|
||||
private val txPublisher = txPublisherFactory.spawnTxPublisher(context, remoteNodeId)
|
||||
|
||||
// this will be used to detect htlc timeouts
|
||||
context.system.eventStream.subscribe(self, classOf[CurrentBlockCount])
|
||||
// this will be used to make sure the current commitment fee is up-to-date
|
||||
|
@ -165,6 +183,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
case Event(initFunder@INPUT_INIT_FUNDER(temporaryChannelId, fundingSatoshis, pushMsat, initialFeeratePerKw, fundingTxFeeratePerKw, _, localParams, remote, _, channelFlags, channelVersion), Nothing) =>
|
||||
context.system.eventStream.publish(ChannelCreated(self, peer, remoteNodeId, isFunder = true, temporaryChannelId, initialFeeratePerKw, Some(fundingTxFeeratePerKw)))
|
||||
activeConnection = remote
|
||||
txPublisher ! SetChannelId(remoteNodeId, temporaryChannelId)
|
||||
val fundingPubKey = keyManager.fundingPublicKey(localParams.fundingKeyPath).publicKey
|
||||
val channelKeyPath = keyManager.keyPath(localParams, channelVersion)
|
||||
val open = OpenChannel(nodeParams.chainHash,
|
||||
|
@ -192,11 +211,13 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
|
||||
case Event(inputFundee@INPUT_INIT_FUNDEE(_, localParams, remote, _, _), Nothing) if !localParams.isFunder =>
|
||||
activeConnection = remote
|
||||
txPublisher ! SetChannelId(remoteNodeId, inputFundee.temporaryChannelId)
|
||||
goto(WAIT_FOR_OPEN_CHANNEL) using DATA_WAIT_FOR_OPEN_CHANNEL(inputFundee)
|
||||
|
||||
case Event(INPUT_RESTORED(data), _) =>
|
||||
log.info("restoring channel")
|
||||
context.system.eventStream.publish(ChannelRestored(self, data.channelId, peer, remoteNodeId, data.commitments.localParams.isFunder, data.commitments))
|
||||
txPublisher ! SetChannelId(remoteNodeId, data.channelId)
|
||||
data match {
|
||||
// NB: order matters!
|
||||
case closing: DATA_CLOSING if Closing.nothingAtStake(closing) =>
|
||||
|
@ -413,6 +434,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
)
|
||||
val channelId = toLongId(fundingTx.hash, fundingTxOutputIndex)
|
||||
peer ! ChannelIdAssigned(self, remoteNodeId, temporaryChannelId, channelId) // we notify the peer asap so it knows how to route messages
|
||||
txPublisher ! SetChannelId(remoteNodeId, channelId)
|
||||
context.system.eventStream.publish(ChannelIdAssigned(self, remoteNodeId, temporaryChannelId, channelId))
|
||||
// NB: we don't send a ChannelSignatureSent for the first commit
|
||||
goto(WAIT_FOR_FUNDING_SIGNED) using DATA_WAIT_FOR_FUNDING_SIGNED(channelId, localParams, remoteParams, fundingTx, fundingTxFee, initialRelayFees_opt, localSpec, localCommitTx, RemoteCommit(0, remoteSpec, remoteCommitTx.tx.txid, remoteFirstPerCommitmentPoint), open.channelFlags, channelVersion, fundingCreated) sending fundingCreated
|
||||
|
@ -469,6 +491,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
remoteNextCommitInfo = Right(randomKey.publicKey), // we will receive their next per-commitment point in the next message, so we temporarily put a random byte array,
|
||||
commitInput, ShaChain.init, channelId = channelId)
|
||||
peer ! ChannelIdAssigned(self, remoteNodeId, temporaryChannelId, channelId) // we notify the peer asap so it knows how to route messages
|
||||
txPublisher ! SetChannelId(remoteNodeId, channelId)
|
||||
context.system.eventStream.publish(ChannelIdAssigned(self, remoteNodeId, temporaryChannelId, channelId))
|
||||
context.system.eventStream.publish(ChannelSignatureReceived(self, commitments))
|
||||
// NB: we don't send a ChannelSignatureSent for the first commit
|
||||
|
@ -1341,7 +1364,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
}
|
||||
val revokedCommitPublished1 = d.revokedCommitPublished.map { rev =>
|
||||
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 => txPublisher ! PublishRawTx(claimTx))
|
||||
penaltyTxs.foreach(claimTx => blockchain ! WatchSpent(self, tx.txid, claimTx.input.outPoint.index.toInt, BITCOIN_OUTPUT_SPENT, hints = Set(claimTx.tx.txid)))
|
||||
rev1
|
||||
}
|
||||
|
@ -1355,7 +1378,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
// If the tx is one of our HTLC txs, we now publish a 3rd-stage claim-htlc-tx that claims its output.
|
||||
val (localCommitPublished1, claimHtlcTx_opt) = Closing.claimLocalCommitHtlcTxOutput(localCommitPublished, keyManager, d.commitments, tx, nodeParams.onChainFeeConf.feeEstimator, nodeParams.onChainFeeConf.feeTargets)
|
||||
claimHtlcTx_opt.foreach(claimHtlcTx => {
|
||||
blockchain ! PublishAsap(claimHtlcTx.tx, PublishStrategy.JustPublish)
|
||||
txPublisher ! PublishRawTx(claimHtlcTx)
|
||||
blockchain ! WatchConfirmed(self, claimHtlcTx.tx.txid, nodeParams.minDepthBlocks, BITCOIN_TX_CONFIRMED(claimHtlcTx.tx))
|
||||
})
|
||||
Closing.updateLocalCommitPublished(localCommitPublished1, tx)
|
||||
|
@ -1990,7 +2013,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
case Some(fundingTx) =>
|
||||
// if we are funder, we never give up
|
||||
log.info(s"republishing the funding tx...")
|
||||
blockchain ! PublishAsap(fundingTx, PublishStrategy.JustPublish)
|
||||
txPublisher ! PublishRawTx(fundingTx, "funding-tx")
|
||||
// we also check if the funding tx has been double-spent
|
||||
checkDoubleSpent(fundingTx)
|
||||
context.system.scheduler.scheduleOnce(1 day, blockchain, GetTxWithMeta(txid))
|
||||
|
@ -2142,7 +2165,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
}
|
||||
|
||||
private def doPublish(closingTx: ClosingTx): Unit = {
|
||||
blockchain ! PublishAsap(closingTx.tx, PublishStrategy.JustPublish)
|
||||
txPublisher ! PublishRawTx(closingTx)
|
||||
blockchain ! WatchConfirmed(self, closingTx.tx.txid, nodeParams.minDepthBlocks, BITCOIN_TX_CONFIRMED(closingTx.tx))
|
||||
}
|
||||
|
||||
|
@ -2171,11 +2194,11 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
/**
|
||||
* This helper method will publish txs only if they haven't yet reached minDepth
|
||||
*/
|
||||
private def publishIfNeeded(txs: Iterable[PublishAsap], irrevocablySpent: Map[OutPoint, Transaction]): Unit = {
|
||||
private def publishIfNeeded(txs: Iterable[PublishTx], irrevocablySpent: Map[OutPoint, Transaction]): Unit = {
|
||||
val (skip, process) = txs.partition(publishTx => Closing.inputsAlreadySpent(publishTx.tx, irrevocablySpent))
|
||||
process.foreach { publishTx =>
|
||||
log.info(s"publishing txid=${publishTx.tx.txid}")
|
||||
blockchain ! publishTx
|
||||
txPublisher ! publishTx
|
||||
}
|
||||
skip.foreach(publishTx => log.info(s"no need to republish txid=${publishTx.tx.txid}, it has already been confirmed"))
|
||||
}
|
||||
|
@ -2209,11 +2232,11 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
|
||||
val publishQueue = commitments.commitmentFormat match {
|
||||
case Transactions.DefaultCommitmentFormat =>
|
||||
val txs = List(commitTx) ++ claimMainDelayedOutputTx.map(_.tx) ++ htlcTxs.values.flatten.map(_.tx) ++ claimHtlcDelayedTxs.map(_.tx)
|
||||
txs.map(tx => PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
List(PublishRawTx(commitTx, "commit-tx")) ++ (claimMainDelayedOutputTx ++ htlcTxs.values.flatten ++ claimHtlcDelayedTxs).map(tx => PublishRawTx(tx))
|
||||
case Transactions.AnchorOutputsCommitmentFormat =>
|
||||
val (publishCommitTx, htlcTxs) = Helpers.Closing.createLocalCommitAnchorPublishStrategy(keyManager, commitments, nodeParams.onChainFeeConf.feeEstimator, nodeParams.onChainFeeConf.feeTargets)
|
||||
List(publishCommitTx) ++ claimMainDelayedOutputTx.map(tx => PublishAsap(tx.tx, PublishStrategy.JustPublish)) ++ htlcTxs ++ claimHtlcDelayedTxs.map(tx => PublishAsap(tx.tx, PublishStrategy.JustPublish))
|
||||
val claimLocalAnchor = claimAnchorTxs.collect { case tx: Transactions.ClaimLocalAnchorOutputTx => SignAndPublishTx(tx, commitments) }
|
||||
val redeemableHtlcTxs = htlcTxs.values.collect { case Some(tx) => SignAndPublishTx(tx, commitments) }
|
||||
List(PublishRawTx(commitTx, "commit-tx")) ++ claimLocalAnchor ++ claimMainDelayedOutputTx.map(tx => PublishRawTx(tx)) ++ redeemableHtlcTxs ++ claimHtlcDelayedTxs.map(tx => PublishRawTx(tx))
|
||||
}
|
||||
publishIfNeeded(publishQueue, irrevocablySpent)
|
||||
|
||||
|
@ -2276,7 +2299,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
private def doPublish(remoteCommitPublished: RemoteCommitPublished): Unit = {
|
||||
import remoteCommitPublished._
|
||||
|
||||
val publishQueue = (claimMainOutputTx ++ claimHtlcTxs.values.flatten).map(tx => PublishAsap(tx.tx, PublishStrategy.JustPublish))
|
||||
val publishQueue = (claimMainOutputTx ++ claimHtlcTxs.values.flatten).map(tx => PublishRawTx(tx))
|
||||
publishIfNeeded(publishQueue, irrevocablySpent)
|
||||
|
||||
// we watch:
|
||||
|
@ -2315,7 +2338,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
private def doPublish(revokedCommitPublished: RevokedCommitPublished): Unit = {
|
||||
import revokedCommitPublished._
|
||||
|
||||
val publishQueue = (claimMainOutputTx ++ mainPenaltyTx ++ htlcPenaltyTxs ++ claimHtlcDelayedPenaltyTxs).map(tx => PublishAsap(tx.tx, PublishStrategy.JustPublish))
|
||||
val publishQueue = (claimMainOutputTx ++ mainPenaltyTx ++ htlcPenaltyTxs ++ claimHtlcDelayedPenaltyTxs).map(tx => PublishRawTx(tx))
|
||||
publishIfNeeded(publishQueue, irrevocablySpent)
|
||||
|
||||
// we watch:
|
||||
|
|
|
@ -19,8 +19,8 @@ package fr.acinq.eclair.channel
|
|||
import akka.actor.{ActorRef, PossiblyHarmful}
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.{ByteVector32, DeterministicWallet, OutPoint, Satoshi, Transaction}
|
||||
import fr.acinq.eclair.blockchain.PublishAsap
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.eclair.channel.TxPublisher.PublishTx
|
||||
import fr.acinq.eclair.payment.OutgoingPacket.Upstream
|
||||
import fr.acinq.eclair.router.Announcements
|
||||
import fr.acinq.eclair.transactions.CommitmentSpec
|
||||
|
@ -105,7 +105,7 @@ case object BITCOIN_FUNDING_SPENT extends BitcoinEvent
|
|||
case object BITCOIN_OUTPUT_SPENT extends BitcoinEvent
|
||||
case class BITCOIN_TX_CONFIRMED(tx: Transaction) extends BitcoinEvent
|
||||
case class BITCOIN_FUNDING_EXTERNAL_CHANNEL_SPENT(shortChannelId: ShortChannelId) extends BitcoinEvent
|
||||
case class BITCOIN_PARENT_TX_CONFIRMED(publishChildTx: PublishAsap) extends BitcoinEvent
|
||||
case class BITCOIN_PARENT_TX_CONFIRMED(childTx: PublishTx) extends BitcoinEvent
|
||||
|
||||
/*
|
||||
.d8888b. .d88888b. 888b d888 888b d888 d8888 888b 888 8888888b. .d8888b.
|
||||
|
@ -293,7 +293,7 @@ sealed trait CommitPublished {
|
|||
* @param claimHtlcDelayedTxs 3rd-stage txs (spending the output of HTLC txs).
|
||||
* @param claimAnchorTxs txs spending anchor outputs to bump the feerate of the commitment tx (if applicable).
|
||||
*/
|
||||
case class LocalCommitPublished(commitTx: Transaction, claimMainDelayedOutputTx: Option[ClaimLocalDelayedOutputTx], htlcTxs: Map[OutPoint, Option[HtlcTx]], claimHtlcDelayedTxs: List[ClaimLocalDelayedOutputTx], claimAnchorTxs: List[ClaimAnchorOutputTx], irrevocablySpent: Map[OutPoint, Transaction]) extends CommitPublished {
|
||||
case class LocalCommitPublished(commitTx: Transaction, claimMainDelayedOutputTx: Option[ClaimLocalDelayedOutputTx], htlcTxs: Map[OutPoint, Option[HtlcTx]], claimHtlcDelayedTxs: List[HtlcDelayedTx], claimAnchorTxs: List[ClaimAnchorOutputTx], irrevocablySpent: Map[OutPoint, Transaction]) extends CommitPublished {
|
||||
/**
|
||||
* A local commit is considered done when:
|
||||
* - all commitment tx outputs that we can spend have been spent and confirmed (even if the spending tx was not ours)
|
||||
|
|
|
@ -21,8 +21,8 @@ import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey, sha256}
|
|||
import fr.acinq.bitcoin.Script._
|
||||
import fr.acinq.bitcoin._
|
||||
import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.blockchain.EclairWallet
|
||||
import fr.acinq.eclair.blockchain.fee.{FeeEstimator, FeeTargets, FeeratePerKw}
|
||||
import fr.acinq.eclair.blockchain.{EclairWallet, PublishAsap, PublishStrategy}
|
||||
import fr.acinq.eclair.channel.Channel.REFRESH_CHANNEL_UPDATE_INTERVAL
|
||||
import fr.acinq.eclair.crypto.Generators
|
||||
import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
|
||||
|
@ -511,7 +511,7 @@ object Helpers {
|
|||
val feeratePerKwDelayed = feeEstimator.getFeeratePerKw(feeTargets.claimMainBlockTarget)
|
||||
|
||||
// first we will claim our main output as soon as the delay is over
|
||||
val mainDelayedTx = generateTx("main-delayed-output") {
|
||||
val mainDelayedTx = generateTx("local-main-delayed") {
|
||||
Transactions.makeClaimLocalDelayedOutputTx(tx, localParams.dustLimit, localRevocationPubkey, remoteParams.toSelfDelay, localDelayedPubkey, localParams.defaultFinalScriptPubKey, feeratePerKwDelayed).map(claimDelayed => {
|
||||
val sig = keyManager.sign(claimDelayed, keyManager.delayedPaymentPoint(channelKeyPath), localPerCommitmentPoint, TxOwner.Local, commitmentFormat)
|
||||
Transactions.addSigs(claimDelayed, sig)
|
||||
|
@ -572,13 +572,12 @@ object Helpers {
|
|||
val localPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, commitments.localCommit.index.toInt)
|
||||
val localRevocationPubkey = Generators.revocationPubKey(remoteParams.revocationBasepoint, localPerCommitmentPoint)
|
||||
val localDelayedPubkey = Generators.derivePubKey(keyManager.delayedPaymentPoint(channelKeyPath).publicKey, localPerCommitmentPoint)
|
||||
val htlcDelayedTx = generateTx("claim-htlc-delayed") {
|
||||
Transactions.makeClaimLocalDelayedOutputTx(tx, localParams.dustLimit, localRevocationPubkey, remoteParams.toSelfDelay, localDelayedPubkey, localParams.defaultFinalScriptPubKey, feeratePerKwDelayed).map(claimDelayed => {
|
||||
val htlcDelayedTx = generateTx("htlc-delayed") {
|
||||
Transactions.makeHtlcDelayedTx(tx, localParams.dustLimit, localRevocationPubkey, remoteParams.toSelfDelay, localDelayedPubkey, localParams.defaultFinalScriptPubKey, feeratePerKwDelayed).map(claimDelayed => {
|
||||
val sig = keyManager.sign(claimDelayed, keyManager.delayedPaymentPoint(channelKeyPath), localPerCommitmentPoint, TxOwner.Local, commitmentFormat)
|
||||
Transactions.addSigs(claimDelayed, sig)
|
||||
})
|
||||
}
|
||||
|
||||
val localCommitPublished1 = localCommitPublished.copy(claimHtlcDelayedTxs = localCommitPublished.claimHtlcDelayedTxs ++ htlcDelayedTx.toSeq)
|
||||
(localCommitPublished1, htlcDelayedTx)
|
||||
} else {
|
||||
|
@ -586,43 +585,6 @@ object Helpers {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create tx publishing strategy (target feerate) for our local commit tx and its HTLC txs. Only used for anchor outputs.
|
||||
*/
|
||||
def createLocalCommitAnchorPublishStrategy(keyManager: ChannelKeyManager, commitments: Commitments, feeEstimator: FeeEstimator, feeTargets: FeeTargets): (PublishAsap, List[PublishAsap]) = {
|
||||
val commitTx = commitments.localCommit.publishableTxs.commitTx.tx
|
||||
val currentFeerate = commitments.localCommit.spec.feeratePerKw
|
||||
val targetFeerate = feeEstimator.getFeeratePerKw(feeTargets.commitmentBlockTarget)
|
||||
val localFundingPubKey = keyManager.fundingPublicKey(commitments.localParams.fundingKeyPath)
|
||||
val channelKeyPath = keyManager.keyPath(commitments.localParams, commitments.channelVersion)
|
||||
val localPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, commitments.localCommit.index)
|
||||
val localHtlcBasepoint = keyManager.htlcPoint(channelKeyPath)
|
||||
|
||||
// If we have an anchor output available, we will use it to CPFP the commit tx.
|
||||
val publishCommitTx = Transactions.makeClaimLocalAnchorOutputTx(commitTx, localFundingPubKey.publicKey).map(claimAnchorOutputTx => {
|
||||
TransactionSigningKit.ClaimAnchorOutputSigningKit(keyManager, claimAnchorOutputTx, localFundingPubKey)
|
||||
}) match {
|
||||
case Left(_) => PublishAsap(commitTx, PublishStrategy.JustPublish)
|
||||
case Right(signingKit) => PublishAsap(commitTx, PublishStrategy.SetFeerate(currentFeerate, targetFeerate, commitments.localParams.dustLimit, signingKit))
|
||||
}
|
||||
|
||||
// HTLC txs will use RBF to add wallet inputs to reach the targeted feerate.
|
||||
val preimages = commitments.localChanges.all.collect { case u: UpdateFulfillHtlc => u.paymentPreimage }.map(r => Crypto.sha256(r) -> r).toMap
|
||||
val htlcTxs = commitments.localCommit.publishableTxs.htlcTxsAndSigs.collect {
|
||||
case HtlcTxAndSigs(htlcSuccess: Transactions.HtlcSuccessTx, localSig, remoteSig) if preimages.contains(htlcSuccess.paymentHash) =>
|
||||
val preimage = preimages(htlcSuccess.paymentHash)
|
||||
val signedTx = Transactions.addSigs(htlcSuccess, localSig, remoteSig, preimage, commitments.commitmentFormat)
|
||||
val signingKit = TransactionSigningKit.HtlcSuccessSigningKit(keyManager, commitments.commitmentFormat, signedTx, localHtlcBasepoint, localPerCommitmentPoint, remoteSig, preimage)
|
||||
PublishAsap(signedTx.tx, PublishStrategy.SetFeerate(currentFeerate, targetFeerate, commitments.localParams.dustLimit, signingKit))
|
||||
case HtlcTxAndSigs(htlcTimeout: Transactions.HtlcTimeoutTx, localSig, remoteSig) =>
|
||||
val signedTx = Transactions.addSigs(htlcTimeout, localSig, remoteSig, commitments.commitmentFormat)
|
||||
val signingKit = TransactionSigningKit.HtlcTimeoutSigningKit(keyManager, commitments.commitmentFormat, signedTx, localHtlcBasepoint, localPerCommitmentPoint, remoteSig)
|
||||
PublishAsap(signedTx.tx, PublishStrategy.SetFeerate(currentFeerate, targetFeerate, commitments.localParams.dustLimit, signingKit))
|
||||
}
|
||||
|
||||
(publishCommitTx, htlcTxs)
|
||||
}
|
||||
|
||||
/**
|
||||
* Claim all the HTLCs that we've received from their current commit tx, if the channel used option_static_remotekey
|
||||
* we don't need to claim our main output because it directly pays to one of our wallet's p2wpkh addresses.
|
||||
|
@ -725,13 +687,13 @@ object Helpers {
|
|||
val feeratePerKwMain = feeEstimator.getFeeratePerKw(feeTargets.claimMainBlockTarget)
|
||||
|
||||
val mainTx = commitments.commitmentFormat match {
|
||||
case DefaultCommitmentFormat => generateTx("claim-p2wpkh-output") {
|
||||
case DefaultCommitmentFormat => generateTx("remote-main") {
|
||||
Transactions.makeClaimP2WPKHOutputTx(tx, commitments.localParams.dustLimit, localPubkey, commitments.localParams.defaultFinalScriptPubKey, feeratePerKwMain).map(claimMain => {
|
||||
val sig = keyManager.sign(claimMain, keyManager.paymentPoint(channelKeyPath), remotePerCommitmentPoint, TxOwner.Local, commitments.commitmentFormat)
|
||||
Transactions.addSigs(claimMain, localPubkey, sig)
|
||||
})
|
||||
}
|
||||
case AnchorOutputsCommitmentFormat => generateTx("claim-remote-delayed-output") {
|
||||
case AnchorOutputsCommitmentFormat => generateTx("remote-main-delayed") {
|
||||
Transactions.makeClaimRemoteDelayedOutputTx(tx, commitments.localParams.dustLimit, localPaymentPoint, commitments.localParams.defaultFinalScriptPubKey, feeratePerKwMain).map(claimMain => {
|
||||
val sig = keyManager.sign(claimMain, keyManager.paymentPoint(channelKeyPath), TxOwner.Local, commitments.commitmentFormat)
|
||||
Transactions.addSigs(claimMain, sig)
|
||||
|
@ -787,7 +749,7 @@ object Helpers {
|
|||
case v if v.paysDirectlyToWallet =>
|
||||
log.info(s"channel uses option_static_remotekey to pay directly to our wallet, there is nothing to do")
|
||||
None
|
||||
case v if v.hasAnchorOutputs => generateTx("claim-remote-delayed-output") {
|
||||
case v if v.hasAnchorOutputs => generateTx("remote-main-delayed") {
|
||||
Transactions.makeClaimRemoteDelayedOutputTx(commitTx, localParams.dustLimit, localPaymentPoint, localParams.defaultFinalScriptPubKey, feeratePerKwMain).map(claimMain => {
|
||||
val sig = keyManager.sign(claimMain, keyManager.paymentPoint(channelKeyPath), TxOwner.Local, commitmentFormat)
|
||||
Transactions.addSigs(claimMain, sig)
|
||||
|
@ -879,7 +841,7 @@ object Helpers {
|
|||
val feeratePerKwPenalty = feeEstimator.getFeeratePerKw(target = 1)
|
||||
|
||||
val penaltyTxs = Transactions.makeClaimHtlcDelayedOutputPenaltyTxs(htlcTx, localParams.dustLimit, remoteRevocationPubkey, localParams.toSelfDelay, remoteDelayedPaymentPubkey, localParams.defaultFinalScriptPubKey, feeratePerKwPenalty).flatMap(claimHtlcDelayedOutputPenaltyTx => {
|
||||
generateTx("claim-htlc-delayed-penalty") {
|
||||
generateTx("htlc-delayed-penalty") {
|
||||
claimHtlcDelayedOutputPenaltyTx.map(htlcDelayedPenalty => {
|
||||
val sig = keyManager.sign(htlcDelayedPenalty, keyManager.revocationPoint(channelKeyPath), remotePerCommitmentSecret, TxOwner.Local, commitmentFormat)
|
||||
val signedTx = Transactions.addSigs(htlcDelayedPenalty, sig)
|
||||
|
|
|
@ -0,0 +1,431 @@
|
|||
/*
|
||||
* Copyright 2021 ACINQ SAS
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package fr.acinq.eclair.channel
|
||||
|
||||
import akka.actor.typed.eventstream.EventStream
|
||||
import akka.actor.typed.scaladsl.adapter.TypedActorRefOps
|
||||
import akka.actor.typed.scaladsl.{ActorContext, Behaviors}
|
||||
import akka.actor.typed.{ActorRef, Behavior}
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.{ByteVector32, ByteVector64, Crypto, Satoshi, Script, Transaction, TxOut}
|
||||
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient
|
||||
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient.FundTransactionOptions
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.eclair.blockchain.{CurrentBlockCount, WatchConfirmed, WatchEventConfirmed}
|
||||
import fr.acinq.eclair.transactions.Transactions._
|
||||
import fr.acinq.eclair.transactions.{Scripts, Transactions}
|
||||
import fr.acinq.eclair.wire.protocol.UpdateFulfillHtlc
|
||||
import fr.acinq.eclair.{Logs, NodeParams}
|
||||
|
||||
import java.util.concurrent.Executors
|
||||
import scala.collection.immutable.SortedMap
|
||||
import scala.concurrent.{ExecutionContext, ExecutionContextExecutor, Future}
|
||||
|
||||
/**
|
||||
* Created by t-bast on 25/03/2021.
|
||||
*/
|
||||
|
||||
/**
|
||||
* This actor ensures its parent channel's on-chain transactions confirm in a timely manner.
|
||||
* It sets the fees, tracks confirmation progress and bumps fees if necessary.
|
||||
*/
|
||||
object TxPublisher {
|
||||
|
||||
// @formatter:off
|
||||
sealed trait Command
|
||||
sealed trait PublishTx extends Command {
|
||||
def tx: Transaction
|
||||
def desc: String
|
||||
}
|
||||
/** Publish a fully signed transaction without modifying it. */
|
||||
case class PublishRawTx(tx: Transaction, desc: String) extends PublishTx
|
||||
object PublishRawTx {
|
||||
def apply(txInfo: TransactionWithInputInfo): PublishRawTx = PublishRawTx(txInfo.tx, txInfo.desc)
|
||||
}
|
||||
/**
|
||||
* Publish an unsigned transaction. Once (csv and cltv) delays have been satisfied, the tx publisher will set the fees,
|
||||
* sign the transaction and broadcast it.
|
||||
*/
|
||||
case class SignAndPublishTx(txInfo: TransactionWithInputInfo, commitments: Commitments) extends PublishTx {
|
||||
override def tx: Transaction = txInfo.tx
|
||||
override def desc: String = txInfo.desc
|
||||
}
|
||||
case class WrappedCurrentBlockCount(currentBlockCount: Long) extends Command
|
||||
case class ParentTxConfirmed(childTx: PublishTx, parentTxId: ByteVector32) extends Command
|
||||
private case class PublishNextBlock(p: PublishTx) extends Command
|
||||
case class SetChannelId(remoteNodeId: PublicKey, channelId: ByteVector32) extends Command
|
||||
// @formatter:on
|
||||
|
||||
// NOTE: we use a single thread to publish transactions so that it preserves order.
|
||||
// CHANGING THIS WILL RESULT IN CONCURRENCY ISSUES WHILE PUBLISHING PARENT AND CHILD TXS!
|
||||
val singleThreadExecutionContext: ExecutionContextExecutor = ExecutionContext.fromExecutor(Executors.newSingleThreadExecutor())
|
||||
|
||||
def apply(nodeParams: NodeParams, remoteNodeId: PublicKey, watcher: akka.actor.ActorRef, client: ExtendedBitcoinClient): Behavior[Command] =
|
||||
Behaviors.setup { context =>
|
||||
Behaviors.withMdc(Logs.mdc(remoteNodeId_opt = Some(remoteNodeId))) {
|
||||
context.system.eventStream ! EventStream.Subscribe(context.messageAdapter[CurrentBlockCount](cbc => WrappedCurrentBlockCount(cbc.blockCount)))
|
||||
new TxPublisher(nodeParams, watcher, client, context).run(SortedMap.empty, Map.empty)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Adjust the amount of the change output of an anchor tx to match our target feerate.
|
||||
* We need this because fundrawtransaction doesn't allow us to leave non-wallet inputs, so we have to add them
|
||||
* afterwards which may bring the resulting feerate below our target.
|
||||
*/
|
||||
def adjustAnchorOutputChange(unsignedTx: ClaimLocalAnchorOutputTx, commitTx: Transaction, amountIn: Satoshi, currentFeerate: FeeratePerKw, targetFeerate: FeeratePerKw, dustLimit: Satoshi): ClaimLocalAnchorOutputTx = {
|
||||
require(unsignedTx.tx.txOut.size == 1, "funded transaction should have a single change output")
|
||||
// We take into account witness weight and adjust the fee to match our desired feerate.
|
||||
val dummySignedClaimAnchorTx = addSigs(unsignedTx, PlaceHolderSig)
|
||||
// NB: we assume that our bitcoind wallet uses only P2WPKH inputs when funding txs.
|
||||
val estimatedWeight = commitTx.weight() + dummySignedClaimAnchorTx.tx.weight() + claimP2WPKHOutputWitnessWeight * (dummySignedClaimAnchorTx.tx.txIn.size - 1)
|
||||
val targetFee = weight2fee(targetFeerate, estimatedWeight) - weight2fee(currentFeerate, commitTx.weight())
|
||||
val amountOut = dustLimit.max(amountIn - targetFee)
|
||||
unsignedTx.copy(tx = unsignedTx.tx.copy(txOut = unsignedTx.tx.txOut.head.copy(amount = amountOut) :: Nil))
|
||||
}
|
||||
|
||||
/**
|
||||
* Adjust the change output of an htlc tx to match our target feerate.
|
||||
* We need this because fundrawtransaction doesn't allow us to leave non-wallet inputs, so we have to add them
|
||||
* afterwards which may bring the resulting feerate below our target.
|
||||
*/
|
||||
def adjustHtlcTxChange(unsignedTx: HtlcTx, amountIn: Satoshi, targetFeerate: FeeratePerKw, commitments: Commitments): HtlcTx = {
|
||||
require(unsignedTx.tx.txOut.size <= 2, "funded transaction should have at most one change output")
|
||||
val dummySignedTx = unsignedTx match {
|
||||
case tx: HtlcSuccessTx => addSigs(tx, PlaceHolderSig, PlaceHolderSig, ByteVector32.Zeroes, commitments.commitmentFormat)
|
||||
case tx: HtlcTimeoutTx => addSigs(tx, PlaceHolderSig, PlaceHolderSig, commitments.commitmentFormat)
|
||||
}
|
||||
// We adjust the change output to obtain the targeted feerate.
|
||||
val estimatedWeight = dummySignedTx.tx.weight() + claimP2WPKHOutputWitnessWeight * (dummySignedTx.tx.txIn.size - 1)
|
||||
val targetFee = weight2fee(targetFeerate, estimatedWeight)
|
||||
val changeAmount = amountIn - dummySignedTx.tx.txOut.head.amount - targetFee
|
||||
if (dummySignedTx.tx.txOut.length == 2 && changeAmount >= commitments.localParams.dustLimit) {
|
||||
unsignedTx match {
|
||||
case htlcSuccess: HtlcSuccessTx => htlcSuccess.copy(tx = htlcSuccess.tx.copy(txOut = Seq(htlcSuccess.tx.txOut.head, htlcSuccess.tx.txOut(1).copy(amount = changeAmount))))
|
||||
case htlcTimeout: HtlcTimeoutTx => htlcTimeout.copy(tx = htlcTimeout.tx.copy(txOut = Seq(htlcTimeout.tx.txOut.head, htlcTimeout.tx.txOut(1).copy(amount = changeAmount))))
|
||||
}
|
||||
} else {
|
||||
unsignedTx match {
|
||||
case htlcSuccess: HtlcSuccessTx => htlcSuccess.copy(tx = htlcSuccess.tx.copy(txOut = Seq(htlcSuccess.tx.txOut.head)))
|
||||
case htlcTimeout: HtlcTimeoutTx => htlcTimeout.copy(tx = htlcTimeout.tx.copy(txOut = Seq(htlcTimeout.tx.txOut.head)))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
sealed trait HtlcTxAndWitnessData {
|
||||
// @formatter:off
|
||||
def txInfo: HtlcTx
|
||||
def updateTx(tx: Transaction): HtlcTxAndWitnessData
|
||||
def addSigs(localSig: ByteVector64, commitmentFormat: CommitmentFormat): HtlcTx
|
||||
// @formatter:on
|
||||
}
|
||||
|
||||
object HtlcTxAndWitnessData {
|
||||
|
||||
case class HtlcSuccess(txInfo: HtlcSuccessTx, remoteSig: ByteVector64, preimage: ByteVector32) extends HtlcTxAndWitnessData {
|
||||
// @formatter:off
|
||||
override def updateTx(tx: Transaction): HtlcTxAndWitnessData = copy(txInfo = txInfo.copy(tx = tx))
|
||||
override def addSigs(localSig: ByteVector64, commitmentFormat: CommitmentFormat): HtlcTx = Transactions.addSigs(txInfo, localSig, remoteSig, preimage, commitmentFormat)
|
||||
// @formatter:on
|
||||
}
|
||||
|
||||
case class HtlcTimeout(txInfo: HtlcTimeoutTx, remoteSig: ByteVector64) extends HtlcTxAndWitnessData {
|
||||
// @formatter:off
|
||||
override def updateTx(tx: Transaction): HtlcTxAndWitnessData = copy(txInfo = txInfo.copy(tx = tx))
|
||||
override def addSigs(localSig: ByteVector64, commitmentFormat: CommitmentFormat): HtlcTx = Transactions.addSigs(txInfo, localSig, remoteSig, commitmentFormat)
|
||||
// @formatter:on
|
||||
}
|
||||
|
||||
def apply(txInfo: HtlcTx, commitments: Commitments): Option[HtlcTxAndWitnessData] = {
|
||||
txInfo match {
|
||||
case tx: HtlcSuccessTx =>
|
||||
commitments.localChanges.all.collectFirst {
|
||||
case u: UpdateFulfillHtlc if Crypto.sha256(u.paymentPreimage) == tx.paymentHash => u.paymentPreimage
|
||||
}.flatMap(preimage => {
|
||||
commitments.localCommit.publishableTxs.htlcTxsAndSigs.collectFirst {
|
||||
case HtlcTxAndSigs(HtlcSuccessTx(input, _, _, _), _, remoteSig) if input.outPoint == tx.input.outPoint => HtlcSuccess(tx, remoteSig, preimage)
|
||||
}
|
||||
})
|
||||
case tx: HtlcTimeoutTx =>
|
||||
commitments.localCommit.publishableTxs.htlcTxsAndSigs.collectFirst {
|
||||
case HtlcTxAndSigs(HtlcTimeoutTx(input, _, _), _, remoteSig) if input.outPoint == tx.input.outPoint => HtlcTimeout(tx, remoteSig)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private class TxPublisher(nodeParams: NodeParams, watcher: akka.actor.ActorRef, client: ExtendedBitcoinClient, context: ActorContext[TxPublisher.Command])(implicit ec: ExecutionContext = ExecutionContext.Implicits.global) {
|
||||
|
||||
import TxPublisher._
|
||||
import nodeParams.onChainFeeConf.{feeEstimator, feeTargets}
|
||||
import nodeParams.{channelKeyManager => keyManager}
|
||||
|
||||
private case class TxWithRelativeDelay(childTx: PublishTx, parentTxIds: Set[ByteVector32])
|
||||
|
||||
private val log = context.log
|
||||
|
||||
private val watchConfirmedResponseMapper: ActorRef[WatchEventConfirmed] = context.messageAdapter(w => w.event match {
|
||||
case BITCOIN_PARENT_TX_CONFIRMED(childTx) => ParentTxConfirmed(childTx, w.tx.txid)
|
||||
})
|
||||
|
||||
/**
|
||||
* @param cltvDelayedTxs when transactions are cltv-delayed, we wait until the target blockchain height is reached.
|
||||
* @param csvDelayedTxs when transactions are csv-delayed, we wait for all parent txs to have enough confirmations.
|
||||
*/
|
||||
private def run(cltvDelayedTxs: SortedMap[Long, Seq[PublishTx]], csvDelayedTxs: Map[ByteVector32, TxWithRelativeDelay]): Behavior[Command] =
|
||||
Behaviors.receiveMessage {
|
||||
case p: PublishTx =>
|
||||
val blockCount = nodeParams.currentBlockHeight
|
||||
val cltvTimeout = Scripts.cltvTimeout(p.tx)
|
||||
val csvTimeouts = Scripts.csvTimeouts(p.tx)
|
||||
if (csvTimeouts.nonEmpty) {
|
||||
csvTimeouts.foreach {
|
||||
case (parentTxId, csvTimeout) =>
|
||||
log.info(s"${p.desc} txid=${p.tx.txid} has a relative timeout of $csvTimeout blocks, watching parentTxId=$parentTxId tx={}", p.tx)
|
||||
watcher ! WatchConfirmed(watchConfirmedResponseMapper.toClassic, parentTxId, minDepth = csvTimeout, BITCOIN_PARENT_TX_CONFIRMED(p))
|
||||
}
|
||||
run(cltvDelayedTxs, csvDelayedTxs + (p.tx.txid -> TxWithRelativeDelay(p, csvTimeouts.keySet)))
|
||||
} else if (cltvTimeout > blockCount) {
|
||||
log.info(s"delaying publication of ${p.desc} txid=${p.tx.txid} until block=$cltvTimeout (current block=$blockCount)")
|
||||
val cltvDelayedTxs1 = cltvDelayedTxs + (cltvTimeout -> (cltvDelayedTxs.getOrElse(cltvTimeout, Seq.empty) :+ p))
|
||||
run(cltvDelayedTxs1, csvDelayedTxs)
|
||||
} else {
|
||||
publish(p)
|
||||
Behaviors.same
|
||||
}
|
||||
|
||||
case ParentTxConfirmed(p, parentTxId) =>
|
||||
log.info(s"parent tx of ${p.desc} txid=${p.tx.txid} has been confirmed (parent txid=$parentTxId)")
|
||||
val blockCount = nodeParams.currentBlockHeight
|
||||
csvDelayedTxs.get(p.tx.txid) match {
|
||||
case Some(TxWithRelativeDelay(_, parentTxIds)) =>
|
||||
val txWithRelativeDelay1 = TxWithRelativeDelay(p, parentTxIds - parentTxId)
|
||||
if (txWithRelativeDelay1.parentTxIds.isEmpty) {
|
||||
log.info(s"all parent txs of ${p.desc} txid=${p.tx.txid} have been confirmed")
|
||||
val csvDelayedTx1 = csvDelayedTxs - p.tx.txid
|
||||
val cltvTimeout = Scripts.cltvTimeout(p.tx)
|
||||
if (cltvTimeout > blockCount) {
|
||||
log.info(s"delaying publication of ${p.desc} txid=${p.tx.txid} until block=$cltvTimeout (current block=$blockCount)")
|
||||
val cltvDelayedTxs1 = cltvDelayedTxs + (cltvTimeout -> (cltvDelayedTxs.getOrElse(cltvTimeout, Seq.empty) :+ p))
|
||||
run(cltvDelayedTxs1, csvDelayedTx1)
|
||||
} else {
|
||||
publish(p)
|
||||
run(cltvDelayedTxs, csvDelayedTx1)
|
||||
}
|
||||
} else {
|
||||
log.info(s"some parent txs of ${p.desc} txid=${p.tx.txid} are still unconfirmed (parent txids=${txWithRelativeDelay1.parentTxIds.mkString(",")})")
|
||||
run(cltvDelayedTxs, csvDelayedTxs + (p.tx.txid -> txWithRelativeDelay1))
|
||||
}
|
||||
case None =>
|
||||
log.warn(s"${p.desc} txid=${p.tx.txid} not found for parent txid=$parentTxId")
|
||||
Behaviors.same
|
||||
}
|
||||
|
||||
case WrappedCurrentBlockCount(blockCount) =>
|
||||
val toPublish = cltvDelayedTxs.view.filterKeys(_ <= blockCount)
|
||||
toPublish.values.flatten.foreach(tx => publish(tx))
|
||||
run(cltvDelayedTxs -- toPublish.keys, csvDelayedTxs)
|
||||
|
||||
case PublishNextBlock(p) =>
|
||||
val nextBlockCount = nodeParams.currentBlockHeight + 1
|
||||
val cltvDelayedTxs1 = cltvDelayedTxs + (nextBlockCount -> (cltvDelayedTxs.getOrElse(nextBlockCount, Seq.empty) :+ p))
|
||||
run(cltvDelayedTxs1, csvDelayedTxs)
|
||||
|
||||
case SetChannelId(remoteNodeId, channelId) =>
|
||||
Behaviors.withMdc(Logs.mdc(remoteNodeId_opt = Some(remoteNodeId), channelId_opt = Some(channelId))) {
|
||||
run(cltvDelayedTxs, csvDelayedTxs)
|
||||
}
|
||||
}
|
||||
|
||||
private def publish(p: PublishTx): Future[ByteVector32] = {
|
||||
p match {
|
||||
case SignAndPublishTx(txInfo, commitments) =>
|
||||
log.info("publishing {}: input={}:{} txid={} tx={}", txInfo.desc, txInfo.input.outPoint.txid, txInfo.input.outPoint.index, p.tx.txid, p.tx)
|
||||
val publishF = txInfo match {
|
||||
case tx: ClaimLocalAnchorOutputTx => publishLocalAnchorTx(tx, commitments)
|
||||
case tx: HtlcTx => publishHtlcTx(tx, commitments)
|
||||
case _ =>
|
||||
log.error(s"ignoring unhandled transaction type ${txInfo.getClass.getSimpleName}")
|
||||
Future.successful(ByteVector32.Zeroes)
|
||||
}
|
||||
publishF.recoverWith {
|
||||
case t: Throwable if t.getMessage.contains("(code: -4)") || t.getMessage.contains("(code: -6)") =>
|
||||
log.warn("not enough funds to publish {}, will retry next block: reason={} input={}:{} txid={}", txInfo.desc, t.getMessage, txInfo.input.outPoint.txid, txInfo.input.outPoint.index, p.tx.txid)
|
||||
context.self ! PublishNextBlock(p)
|
||||
Future.failed(t)
|
||||
case t: Throwable =>
|
||||
log.error("cannot publish {}: reason={} input={}:{} txid={}", txInfo.desc, t.getMessage, txInfo.input.outPoint.txid, txInfo.input.outPoint.index, p.tx.txid)
|
||||
Future.failed(t)
|
||||
}
|
||||
case PublishRawTx(tx, desc) =>
|
||||
log.info("publishing {}: txid={} tx={}", desc, tx.txid, tx)
|
||||
publish(tx, desc)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This method uses a single thread to publish transactions so that it preserves the order of publication.
|
||||
* We need that to prevent concurrency issues while publishing parent and child transactions.
|
||||
*/
|
||||
private def publish(tx: Transaction, desc: String): Future[ByteVector32] = {
|
||||
client.publishTransaction(tx)(singleThreadExecutionContext).recoverWith {
|
||||
case t: Throwable =>
|
||||
log.error("cannot publish {}: reason={} txid={}", desc, t.getMessage, tx.txid)
|
||||
Future.failed(t)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Publish an anchor tx that spends from the commit tx and helps get it confirmed with CPFP (if the commit tx feerate
|
||||
* was too low).
|
||||
*/
|
||||
private def publishLocalAnchorTx(txInfo: ClaimLocalAnchorOutputTx, commitments: Commitments): Future[ByteVector32] = {
|
||||
val commitFeerate = commitments.localCommit.spec.feeratePerKw
|
||||
val commitTx = commitments.localCommit.publishableTxs.commitTx.tx
|
||||
val targetFeerate = feeEstimator.getFeeratePerKw(feeTargets.commitmentBlockTarget)
|
||||
if (targetFeerate <= commitFeerate) {
|
||||
log.info(s"publishing commit-tx without the anchor (current feerate=$commitFeerate): txid=${commitTx.txid}")
|
||||
Future.successful(commitTx.txid)
|
||||
} else {
|
||||
log.info(s"bumping commit-tx with the anchor (target feerate=$targetFeerate): txid=${commitTx.txid}")
|
||||
addInputs(txInfo, targetFeerate, commitments).flatMap(claimAnchorTx => {
|
||||
val claimAnchorSig = keyManager.sign(claimAnchorTx, keyManager.fundingPublicKey(commitments.localParams.fundingKeyPath), TxOwner.Local, commitments.commitmentFormat)
|
||||
val signedClaimAnchorTx = addSigs(claimAnchorTx, claimAnchorSig)
|
||||
val commitInfo = ExtendedBitcoinClient.PreviousTx(signedClaimAnchorTx.input, signedClaimAnchorTx.tx.txIn.head.witness)
|
||||
client.signTransaction(signedClaimAnchorTx.tx, Seq(commitInfo))
|
||||
}).flatMap(signTxResponse => {
|
||||
publish(signTxResponse.tx, txInfo.desc)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
private def addInputs(txInfo: ClaimLocalAnchorOutputTx, targetFeerate: FeeratePerKw, commitments: Commitments): Future[ClaimLocalAnchorOutputTx] = {
|
||||
val dustLimit = commitments.localParams.dustLimit
|
||||
val commitFeerate = commitments.localCommit.spec.feeratePerKw
|
||||
val commitTx = commitments.localCommit.publishableTxs.commitTx.tx
|
||||
// We want the feerate of the package (commit tx + tx spending anchor) to equal targetFeerate.
|
||||
// Thus we have: anchorFeerate = targetFeerate + (weight-commit-tx / weight-anchor-tx) * (targetFeerate - commitTxFeerate)
|
||||
// If we use the smallest weight possible for the anchor tx, the feerate we use will thus be greater than what we want,
|
||||
// and we can adjust it afterwards by raising the change output amount.
|
||||
val anchorFeerate = targetFeerate + FeeratePerKw(targetFeerate.feerate - commitFeerate.feerate) * commitTx.weight() / claimAnchorOutputMinWeight
|
||||
// NB: fundrawtransaction requires at least one output, and may add at most one additional change output.
|
||||
// Since the purpose of this transaction is just to do a CPFP, the resulting tx should have a single change output
|
||||
// (note that bitcoind doesn't let us publish a transaction with no outputs).
|
||||
// To work around these limitations, we start with a dummy output and later merge that dummy output with the optional
|
||||
// change output added by bitcoind.
|
||||
// NB: fundrawtransaction doesn't support non-wallet inputs, so we have to remove our anchor input and re-add it later.
|
||||
// That means bitcoind will not take our anchor input's weight into account when adding inputs to set the fee.
|
||||
// That's ok, we can increase the fee later by decreasing the output amount. But we need to ensure we'll have enough
|
||||
// to cover the weight of our anchor input, which is why we set it to the following value.
|
||||
val dummyChangeAmount = weight2fee(anchorFeerate, claimAnchorOutputMinWeight) + dustLimit
|
||||
val txNotFunded = Transaction(2, Nil, TxOut(dummyChangeAmount, Script.pay2wpkh(PlaceHolderPubKey)) :: Nil, 0)
|
||||
client.fundTransaction(txNotFunded, FundTransactionOptions(anchorFeerate, lockUtxos = true)).flatMap(fundTxResponse => {
|
||||
// We merge the outputs if there's more than one.
|
||||
fundTxResponse.changePosition match {
|
||||
case Some(changePos) =>
|
||||
val changeOutput = fundTxResponse.tx.txOut(changePos)
|
||||
val txSingleOutput = fundTxResponse.tx.copy(txOut = Seq(changeOutput.copy(amount = changeOutput.amount + dummyChangeAmount)))
|
||||
Future.successful(fundTxResponse.copy(tx = txSingleOutput))
|
||||
case None =>
|
||||
client.getChangeAddress().map(pubkeyHash => {
|
||||
val txSingleOutput = fundTxResponse.tx.copy(txOut = Seq(TxOut(dummyChangeAmount, Script.pay2wpkh(pubkeyHash))))
|
||||
fundTxResponse.copy(tx = txSingleOutput)
|
||||
})
|
||||
}
|
||||
}).map(fundTxResponse => {
|
||||
require(fundTxResponse.tx.txOut.size == 1, "funded transaction should have a single change output")
|
||||
// NB: we insert the anchor input in the *first* position because our signing helpers only sign input #0.
|
||||
val unsignedTx = txInfo.copy(tx = fundTxResponse.tx.copy(txIn = txInfo.tx.txIn.head +: fundTxResponse.tx.txIn))
|
||||
adjustAnchorOutputChange(unsignedTx, commitTx, fundTxResponse.amountIn + AnchorOutputsCommitmentFormat.anchorAmount, commitFeerate, targetFeerate, dustLimit)
|
||||
})
|
||||
}
|
||||
|
||||
/**
|
||||
* Publish an htlc tx, and optionally RBF it before by adding new inputs/outputs to help get it confirmed.
|
||||
*/
|
||||
private def publishHtlcTx(txInfo: HtlcTx, commitments: Commitments): Future[ByteVector32] = {
|
||||
val currentFeerate = commitments.localCommit.spec.feeratePerKw
|
||||
val targetFeerate = feeEstimator.getFeeratePerKw(feeTargets.commitmentBlockTarget)
|
||||
val channelKeyPath = keyManager.keyPath(commitments.localParams, commitments.channelVersion)
|
||||
val localPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, commitments.localCommit.index)
|
||||
val localHtlcBasepoint = keyManager.htlcPoint(channelKeyPath)
|
||||
HtlcTxAndWitnessData(txInfo, commitments) match {
|
||||
case Some(txWithWitnessData) =>
|
||||
if (targetFeerate <= currentFeerate) {
|
||||
val localSig = keyManager.sign(txInfo, localHtlcBasepoint, localPerCommitmentPoint, TxOwner.Local, commitments.commitmentFormat)
|
||||
val signedHtlcTx = txWithWitnessData.addSigs(localSig, commitments.commitmentFormat)
|
||||
log.info("publishing {} without adding inputs: txid={}", txInfo.desc, signedHtlcTx.tx.txid)
|
||||
publish(signedHtlcTx.tx, txInfo.desc)
|
||||
} else {
|
||||
log.info("publishing {} with additional inputs: commit input={}:{} target feerate={}", txInfo.desc, txInfo.input.outPoint.txid, txInfo.input.outPoint.index, targetFeerate)
|
||||
addInputs(txInfo, targetFeerate, commitments).flatMap(unsignedTx => {
|
||||
val localSig = keyManager.sign(unsignedTx, localHtlcBasepoint, localPerCommitmentPoint, TxOwner.Local, commitments.commitmentFormat)
|
||||
val signedHtlcTx = txWithWitnessData.updateTx(unsignedTx.tx).addSigs(localSig, commitments.commitmentFormat)
|
||||
val inputInfo = ExtendedBitcoinClient.PreviousTx(signedHtlcTx.input, signedHtlcTx.tx.txIn.head.witness)
|
||||
client.signTransaction(signedHtlcTx.tx, Seq(inputInfo), allowIncomplete = true).flatMap(signTxResponse => {
|
||||
// NB: bitcoind messes up the witness stack for our htlc input, so we need to restore it.
|
||||
// See https://github.com/bitcoin/bitcoin/issues/21151
|
||||
val completeTx = signedHtlcTx.tx.copy(txIn = signedHtlcTx.tx.txIn.head +: signTxResponse.tx.txIn.tail)
|
||||
log.info("publishing bumped {}: commit input={}:{} txid={} tx={}", txInfo.desc, txInfo.input.outPoint.txid, txInfo.input.outPoint.index, completeTx.txid, completeTx)
|
||||
publish(completeTx, txInfo.desc)
|
||||
})
|
||||
})
|
||||
}
|
||||
case None =>
|
||||
Future.failed(new IllegalArgumentException(s"witness data not found for htlcId=${txInfo.htlcId}, skipping..."))
|
||||
}
|
||||
}
|
||||
|
||||
private def addInputs(txInfo: HtlcTx, targetFeerate: FeeratePerKw, commitments: Commitments): Future[HtlcTx] = {
|
||||
// NB: fundrawtransaction doesn't support non-wallet inputs, so we clear the input and re-add it later.
|
||||
val txNotFunded = txInfo.tx.copy(txIn = Nil, txOut = txInfo.tx.txOut.head.copy(amount = commitments.localParams.dustLimit) :: Nil)
|
||||
val htlcTxWeight = txInfo match {
|
||||
case _: HtlcSuccessTx => commitments.commitmentFormat.htlcSuccessWeight
|
||||
case _: HtlcTimeoutTx => commitments.commitmentFormat.htlcTimeoutWeight
|
||||
}
|
||||
// We want the feerate of our final HTLC tx to equal targetFeerate. However, we removed the HTLC input from what we
|
||||
// send to fundrawtransaction, so bitcoind will not know the total weight of the final tx. In order to make up for
|
||||
// this difference, we need to tell bitcoind to target a higher feerate that takes into account the weight of the
|
||||
// input we removed.
|
||||
// That feerate will satisfy the following equality:
|
||||
// feerate * weight_seen_by_bitcoind = target_feerate * (weight_seen_by_bitcoind + htlc_input_weight)
|
||||
// So: feerate = target_feerate * (1 + htlc_input_weight / weight_seen_by_bitcoind)
|
||||
// Because bitcoind will add at least one P2WPKH input, weight_seen_by_bitcoind >= htlc_tx_weight + p2wpkh_weight
|
||||
// Thus: feerate <= target_feerate * (1 + htlc_input_weight / (htlc_tx_weight + p2wpkh_weight))
|
||||
// NB: we don't take into account the fee paid by our HTLC input: we will take it into account when we adjust the
|
||||
// change output amount (unless bitcoind didn't add any change output, in that case we will overpay the fee slightly).
|
||||
val weightRatio = 1.0 + (htlcInputMaxWeight.toDouble / (htlcTxWeight + claimP2WPKHOutputWeight))
|
||||
client.fundTransaction(txNotFunded, FundTransactionOptions(targetFeerate * weightRatio, lockUtxos = true, changePosition = Some(1))).map(fundTxResponse => {
|
||||
log.info(s"added ${fundTxResponse.tx.txIn.length} wallet input(s) and ${fundTxResponse.tx.txOut.length - 1} wallet output(s) to ${txInfo.desc} spending commit input=${txInfo.input.outPoint.txid}:${txInfo.input.outPoint.index}")
|
||||
// We add the HTLC input (from the commit tx) and restore the HTLC output.
|
||||
// NB: we can't modify them because they are signed by our peer (with SIGHASH_SINGLE | SIGHASH_ANYONECANPAY).
|
||||
val txWithHtlcInput = fundTxResponse.tx.copy(
|
||||
txIn = txInfo.tx.txIn ++ fundTxResponse.tx.txIn,
|
||||
txOut = txInfo.tx.txOut ++ fundTxResponse.tx.txOut.tail
|
||||
)
|
||||
val unsignedTx = txInfo match {
|
||||
case htlcSuccess: HtlcSuccessTx => htlcSuccess.copy(tx = txWithHtlcInput)
|
||||
case htlcTimeout: HtlcTimeoutTx => htlcTimeout.copy(tx = txWithHtlcInput)
|
||||
}
|
||||
adjustHtlcTxChange(unsignedTx, fundTxResponse.amountIn + unsignedTx.input.txOut.amount, targetFeerate, commitments)
|
||||
})
|
||||
}
|
||||
|
||||
}
|
|
@ -357,9 +357,9 @@ object Peer {
|
|||
def spawn(context: ActorContext, remoteNodeId: PublicKey, origin_opt: Option[ActorRef]): ActorRef
|
||||
}
|
||||
|
||||
case class SimpleChannelFactory(nodeParams: NodeParams, watcher: ActorRef, relayer: ActorRef, wallet: EclairWallet) extends ChannelFactory {
|
||||
case class SimpleChannelFactory(nodeParams: NodeParams, watcher: ActorRef, relayer: ActorRef, wallet: EclairWallet, txPublisherFactory: Channel.TxPublisherFactory) extends ChannelFactory {
|
||||
override def spawn(context: ActorContext, remoteNodeId: PublicKey, origin_opt: Option[ActorRef]): ActorRef =
|
||||
context.actorOf(Channel.props(nodeParams, wallet, remoteNodeId, watcher, relayer, origin_opt))
|
||||
context.actorOf(Channel.props(nodeParams, wallet, remoteNodeId, watcher, relayer, txPublisherFactory, origin_opt))
|
||||
}
|
||||
|
||||
def props(nodeParams: NodeParams, remoteNodeId: PublicKey, wallet: EclairWallet, channelFactory: ChannelFactory): Props = Props(new Peer(nodeParams, remoteNodeId, wallet, channelFactory))
|
||||
|
|
|
@ -17,13 +17,11 @@
|
|||
package fr.acinq.eclair.transactions
|
||||
|
||||
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey, ripemd160}
|
||||
import fr.acinq.bitcoin.DeterministicWallet.ExtendedPublicKey
|
||||
import fr.acinq.bitcoin.Script._
|
||||
import fr.acinq.bitcoin.SigVersion._
|
||||
import fr.acinq.bitcoin._
|
||||
import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
|
||||
import fr.acinq.eclair.transactions.CommitmentOutput._
|
||||
import fr.acinq.eclair.transactions.Scripts._
|
||||
import fr.acinq.eclair.wire.protocol.UpdateAddHtlc
|
||||
|
@ -84,6 +82,7 @@ object Transactions {
|
|||
|
||||
sealed trait TransactionWithInputInfo {
|
||||
def input: InputInfo
|
||||
def desc: String
|
||||
def tx: Transaction
|
||||
def fee: Satoshi = input.txOut.amount - tx.txOut.map(_.amount).sum
|
||||
def minRelayFee: Satoshi = {
|
||||
|
@ -94,7 +93,7 @@ object Transactions {
|
|||
def sighash(txOwner: TxOwner, commitmentFormat: CommitmentFormat): Int = SIGHASH_ALL
|
||||
}
|
||||
|
||||
case class CommitTx(input: InputInfo, tx: Transaction) extends TransactionWithInputInfo
|
||||
case class CommitTx(input: InputInfo, tx: Transaction) extends TransactionWithInputInfo { override def desc: String = "commit-tx" }
|
||||
sealed trait HtlcTx extends TransactionWithInputInfo {
|
||||
def htlcId: Long
|
||||
override def sighash(txOwner: TxOwner, commitmentFormat: CommitmentFormat): Int = commitmentFormat match {
|
||||
|
@ -105,44 +104,23 @@ object Transactions {
|
|||
}
|
||||
}
|
||||
}
|
||||
case class HtlcSuccessTx(input: InputInfo, tx: Transaction, paymentHash: ByteVector32, htlcId: Long) extends HtlcTx
|
||||
case class HtlcTimeoutTx(input: InputInfo, tx: Transaction, htlcId: Long) extends HtlcTx
|
||||
case class HtlcSuccessTx(input: InputInfo, tx: Transaction, paymentHash: ByteVector32, htlcId: Long) extends HtlcTx { override def desc: String = "htlc-success" }
|
||||
case class HtlcTimeoutTx(input: InputInfo, tx: Transaction, htlcId: Long) extends HtlcTx { override def desc: String = "htlc-timeout" }
|
||||
case class HtlcDelayedTx(input: InputInfo, tx: Transaction) extends TransactionWithInputInfo { override def desc: String = "htlc-delayed" }
|
||||
sealed trait ClaimHtlcTx extends TransactionWithInputInfo { def htlcId: Long }
|
||||
case class ClaimHtlcSuccessTx(input: InputInfo, tx: Transaction, htlcId: Long) extends ClaimHtlcTx
|
||||
case class ClaimHtlcTimeoutTx(input: InputInfo, tx: Transaction, htlcId: Long) extends ClaimHtlcTx
|
||||
case class ClaimHtlcSuccessTx(input: InputInfo, tx: Transaction, htlcId: Long) extends ClaimHtlcTx { override def desc: String = "claim-htlc-success" }
|
||||
case class ClaimHtlcTimeoutTx(input: InputInfo, tx: Transaction, htlcId: Long) extends ClaimHtlcTx { override def desc: String = "claim-htlc-timeout" }
|
||||
sealed trait ClaimAnchorOutputTx extends TransactionWithInputInfo
|
||||
case class ClaimLocalAnchorOutputTx(input: InputInfo, tx: Transaction) extends ClaimAnchorOutputTx
|
||||
case class ClaimRemoteAnchorOutputTx(input: InputInfo, tx: Transaction) extends ClaimAnchorOutputTx
|
||||
case class ClaimLocalAnchorOutputTx(input: InputInfo, tx: Transaction) extends ClaimAnchorOutputTx { override def desc: String = "local-anchor" }
|
||||
case class ClaimRemoteAnchorOutputTx(input: InputInfo, tx: Transaction) extends ClaimAnchorOutputTx { override def desc: String = "remote-anchor" }
|
||||
sealed trait ClaimRemoteCommitMainOutputTx extends TransactionWithInputInfo
|
||||
case class ClaimP2WPKHOutputTx(input: InputInfo, tx: Transaction) extends ClaimRemoteCommitMainOutputTx
|
||||
case class ClaimRemoteDelayedOutputTx(input: InputInfo, tx: Transaction) extends ClaimRemoteCommitMainOutputTx
|
||||
case class ClaimLocalDelayedOutputTx(input: InputInfo, tx: Transaction) extends TransactionWithInputInfo
|
||||
case class ClaimHtlcDelayedOutputPenaltyTx(input: InputInfo, tx: Transaction) extends TransactionWithInputInfo
|
||||
case class MainPenaltyTx(input: InputInfo, tx: Transaction) extends TransactionWithInputInfo
|
||||
case class HtlcPenaltyTx(input: InputInfo, tx: Transaction) extends TransactionWithInputInfo
|
||||
case class ClosingTx(input: InputInfo, tx: Transaction, toLocalOutput: Option[OutputInfo]) extends TransactionWithInputInfo
|
||||
|
||||
trait TransactionSigningKit {
|
||||
def keyManager: ChannelKeyManager
|
||||
def commitmentFormat: CommitmentFormat
|
||||
def spentOutpoint: OutPoint
|
||||
}
|
||||
object TransactionSigningKit {
|
||||
case class ClaimAnchorOutputSigningKit(keyManager: ChannelKeyManager, txWithInput: ClaimLocalAnchorOutputTx, localFundingPubKey: ExtendedPublicKey) extends TransactionSigningKit {
|
||||
override val commitmentFormat: CommitmentFormat = AnchorOutputsCommitmentFormat
|
||||
override val spentOutpoint = txWithInput.input.outPoint
|
||||
}
|
||||
|
||||
sealed trait HtlcTxSigningKit extends TransactionSigningKit {
|
||||
def txWithInput: HtlcTx
|
||||
override def spentOutpoint = txWithInput.input.outPoint
|
||||
def localHtlcBasepoint: ExtendedPublicKey
|
||||
def localPerCommitmentPoint: PublicKey
|
||||
def remoteSig: ByteVector64
|
||||
}
|
||||
case class HtlcSuccessSigningKit(keyManager: ChannelKeyManager, commitmentFormat: CommitmentFormat, txWithInput: HtlcSuccessTx, localHtlcBasepoint: ExtendedPublicKey, localPerCommitmentPoint: PublicKey, remoteSig: ByteVector64, preimage: ByteVector32) extends HtlcTxSigningKit
|
||||
case class HtlcTimeoutSigningKit(keyManager: ChannelKeyManager, commitmentFormat: CommitmentFormat, txWithInput: HtlcTimeoutTx, localHtlcBasepoint: ExtendedPublicKey, localPerCommitmentPoint: PublicKey, remoteSig: ByteVector64) extends HtlcTxSigningKit
|
||||
}
|
||||
case class ClaimP2WPKHOutputTx(input: InputInfo, tx: Transaction) extends ClaimRemoteCommitMainOutputTx { override def desc: String = "remote-main" }
|
||||
case class ClaimRemoteDelayedOutputTx(input: InputInfo, tx: Transaction) extends ClaimRemoteCommitMainOutputTx { override def desc: String = "remote-main-delayed" }
|
||||
case class ClaimLocalDelayedOutputTx(input: InputInfo, tx: Transaction) extends TransactionWithInputInfo { override def desc: String = "local-main-delayed" }
|
||||
case class MainPenaltyTx(input: InputInfo, tx: Transaction) extends TransactionWithInputInfo { override def desc: String = "main-penalty" }
|
||||
case class HtlcPenaltyTx(input: InputInfo, tx: Transaction) extends TransactionWithInputInfo { override def desc: String = "htlc-penalty" }
|
||||
case class ClaimHtlcDelayedOutputPenaltyTx(input: InputInfo, tx: Transaction) extends TransactionWithInputInfo { override def desc: String = "htlc-delayed-penalty" }
|
||||
case class ClosingTx(input: InputInfo, tx: Transaction, toLocalOutput: Option[OutputInfo]) extends TransactionWithInputInfo { override def desc: String = "closing" }
|
||||
|
||||
sealed trait TxGenerationSkipped
|
||||
case object OutputNotFound extends TxGenerationSkipped { override def toString = "output not found (probably trimmed)" }
|
||||
|
@ -154,9 +132,9 @@ object Transactions {
|
|||
* - [[ClaimLocalDelayedOutputTx]] spends to-local output of [[CommitTx]] after a delay
|
||||
* - When using anchor outputs, [[ClaimLocalAnchorOutputTx]] spends to-local anchor of [[CommitTx]]
|
||||
* - [[HtlcSuccessTx]] spends htlc-received outputs of [[CommitTx]] for which we have the preimage
|
||||
* - [[ClaimLocalDelayedOutputTx]] spends [[HtlcSuccessTx]] after a delay
|
||||
* - [[HtlcDelayedTx]] spends [[HtlcSuccessTx]] after a delay
|
||||
* - [[HtlcTimeoutTx]] spends htlc-sent outputs of [[CommitTx]] after a timeout
|
||||
* - [[ClaimLocalDelayedOutputTx]] spends [[HtlcTimeoutTx]] after a delay
|
||||
* - [[HtlcDelayedTx]] spends [[HtlcTimeoutTx]] after a delay
|
||||
*
|
||||
* When *remote* *current* [[CommitTx]] is published:
|
||||
* - When using the default commitment format, [[ClaimP2WPKHOutputTx]] spends to-local output of [[CommitTx]]
|
||||
|
@ -190,7 +168,7 @@ object Transactions {
|
|||
// 143 bytes (accepted_htlc_script) + 327 bytes (success_witness) + 41 bytes (commitment_input) = 511 bytes
|
||||
// See https://github.com/lightningnetwork/lightning-rfc/blob/master/03-transactions.md#expected-weight-of-htlc-timeout-and-htlc-success-transactions
|
||||
val htlcInputMaxWeight = 511
|
||||
val claimHtlcDelayedWeight = 483
|
||||
val htlcDelayedWeight = 483
|
||||
val claimHtlcSuccessWeight = 571
|
||||
val claimHtlcTimeoutWeight = 545
|
||||
val mainPenaltyWeight = 484
|
||||
|
@ -617,13 +595,25 @@ object Transactions {
|
|||
}
|
||||
}
|
||||
|
||||
def makeHtlcDelayedTx(htlcTx: Transaction, localDustLimit: Satoshi, localRevocationPubkey: PublicKey, toLocalDelay: CltvExpiryDelta, localDelayedPaymentPubkey: PublicKey, localFinalScriptPubKey: ByteVector, feeratePerKw: FeeratePerKw): Either[TxGenerationSkipped, HtlcDelayedTx] = {
|
||||
makeLocalDelayedOutputTx(htlcTx, localDustLimit, localRevocationPubkey, toLocalDelay, localDelayedPaymentPubkey, localFinalScriptPubKey, feeratePerKw).map {
|
||||
case (input, tx) => HtlcDelayedTx(input, tx)
|
||||
}
|
||||
}
|
||||
|
||||
def makeClaimLocalDelayedOutputTx(commitTx: Transaction, localDustLimit: Satoshi, localRevocationPubkey: PublicKey, toLocalDelay: CltvExpiryDelta, localDelayedPaymentPubkey: PublicKey, localFinalScriptPubKey: ByteVector, feeratePerKw: FeeratePerKw): Either[TxGenerationSkipped, ClaimLocalDelayedOutputTx] = {
|
||||
makeLocalDelayedOutputTx(commitTx, localDustLimit, localRevocationPubkey, toLocalDelay, localDelayedPaymentPubkey, localFinalScriptPubKey, feeratePerKw).map {
|
||||
case (input, tx) => ClaimLocalDelayedOutputTx(input, tx)
|
||||
}
|
||||
}
|
||||
|
||||
private def makeLocalDelayedOutputTx(parentTx: Transaction, localDustLimit: Satoshi, localRevocationPubkey: PublicKey, toLocalDelay: CltvExpiryDelta, localDelayedPaymentPubkey: PublicKey, localFinalScriptPubKey: ByteVector, feeratePerKw: FeeratePerKw): Either[TxGenerationSkipped, (InputInfo, Transaction)] = {
|
||||
val redeemScript = toLocalDelayed(localRevocationPubkey, toLocalDelay, localDelayedPaymentPubkey)
|
||||
val pubkeyScript = write(pay2wsh(redeemScript))
|
||||
findPubKeyScriptIndex(commitTx, pubkeyScript) match {
|
||||
findPubKeyScriptIndex(parentTx, pubkeyScript) match {
|
||||
case Left(skip) => Left(skip)
|
||||
case Right(outputIndex) =>
|
||||
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
|
||||
val input = InputInfo(OutPoint(parentTx, outputIndex), parentTx.txOut(outputIndex), write(redeemScript))
|
||||
// unsigned transaction
|
||||
val tx = Transaction(
|
||||
version = 2,
|
||||
|
@ -638,7 +628,7 @@ object Transactions {
|
|||
Left(AmountBelowDustLimit)
|
||||
} else {
|
||||
val tx1 = tx.copy(txOut = tx.txOut.head.copy(amount = amount) :: Nil)
|
||||
Right(ClaimLocalDelayedOutputTx(input, tx1))
|
||||
Right(input, tx1)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -861,6 +851,11 @@ object Transactions {
|
|||
claimDelayedOutputTx.copy(tx = claimDelayedOutputTx.tx.updateWitness(0, witness))
|
||||
}
|
||||
|
||||
def addSigs(htlcDelayedTx: HtlcDelayedTx, localSig: ByteVector64): HtlcDelayedTx = {
|
||||
val witness = witnessToLocalDelayedAfterDelay(localSig, htlcDelayedTx.input.redeemScript)
|
||||
htlcDelayedTx.copy(tx = htlcDelayedTx.tx.updateWitness(0, witness))
|
||||
}
|
||||
|
||||
def addSigs(claimAnchorOutputTx: ClaimLocalAnchorOutputTx, localSig: ByteVector64): ClaimLocalAnchorOutputTx = {
|
||||
val witness = witnessAnchor(localSig, claimAnchorOutputTx.input.redeemScript)
|
||||
claimAnchorOutputTx.copy(tx = claimAnchorOutputTx.tx.updateWitness(0, witness))
|
||||
|
|
|
@ -57,7 +57,7 @@ private[channel] object ChannelTypes0 {
|
|||
val claimHtlcDelayedTxsNew = claimHtlcDelayedTxs.map(tx => {
|
||||
val htlcTx = htlcTxs.find(_.txid == tx.txIn.head.outPoint.txid)
|
||||
require(htlcTx.nonEmpty, s"3rd-stage htlc tx doesn't spend one of our htlc txs: claim-htlc-tx=$tx, htlc-txs=${htlcTxs.mkString(",")}")
|
||||
ClaimLocalDelayedOutputTx(getPartialInputInfo(htlcTx.get, tx), tx)
|
||||
HtlcDelayedTx(getPartialInputInfo(htlcTx.get, tx), tx)
|
||||
})
|
||||
channel.LocalCommitPublished(commitTx, claimMainDelayedOutputTxNew, htlcTxsNew, claimHtlcDelayedTxsNew, Nil, irrevocablySpentNew)
|
||||
}
|
||||
|
|
|
@ -104,6 +104,7 @@ private[channel] object ChannelCodecs2 {
|
|||
val commitTxCodec: Codec[CommitTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[CommitTx]
|
||||
val htlcSuccessTxCodec: Codec[HtlcSuccessTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("paymentHash" | bytes32) :: ("htlcId" | uint64overflow)).as[HtlcSuccessTx]
|
||||
val htlcTimeoutTxCodec: Codec[HtlcTimeoutTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("htlcId" | uint64overflow)).as[HtlcTimeoutTx]
|
||||
val htlcDelayedTxCodec: Codec[HtlcDelayedTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[HtlcDelayedTx]
|
||||
val claimHtlcSuccessTxCodec: Codec[ClaimHtlcSuccessTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("htlcId" | uint64overflow)).as[ClaimHtlcSuccessTx]
|
||||
val claimHtlcTimeoutTxCodec: Codec[ClaimHtlcTimeoutTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("htlcId" | uint64overflow)).as[ClaimHtlcTimeoutTx]
|
||||
val claimLocalDelayedOutputTxCodec: Codec[ClaimLocalDelayedOutputTx] = (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[ClaimLocalDelayedOutputTx]
|
||||
|
@ -131,6 +132,7 @@ private[channel] object ChannelCodecs2 {
|
|||
.typecase(0x12, claimRemoteAnchorOutputTxCodec)
|
||||
.typecase(0x13, claimRemoteDelayedOutputTxCodec)
|
||||
.typecase(0x14, claimHtlcDelayedOutputPenaltyTxCodec)
|
||||
.typecase(0x15, htlcDelayedTxCodec)
|
||||
|
||||
val claimRemoteCommitMainOutputTxCodec: Codec[ClaimRemoteCommitMainOutputTx] = discriminated[ClaimRemoteCommitMainOutputTx].by(uint8)
|
||||
.typecase(0x01, claimP2WPKHOutputTxCodec)
|
||||
|
@ -239,7 +241,7 @@ private[channel] object ChannelCodecs2 {
|
|||
("commitTx" | txCodec) ::
|
||||
("claimMainDelayedOutputTx" | optional(bool8, claimLocalDelayedOutputTxCodec)) ::
|
||||
("htlcTxs" | mapCodec(outPointCodec, optional(bool8, htlcTxCodec))) ::
|
||||
("claimHtlcDelayedTx" | listOfN(uint16, claimLocalDelayedOutputTxCodec)) ::
|
||||
("claimHtlcDelayedTx" | listOfN(uint16, htlcDelayedTxCodec)) ::
|
||||
("claimAnchorTxs" | listOfN(uint16, claimAnchorOutputTxCodec)) ::
|
||||
("spent" | spentMapCodec)).as[LocalCommitPublished]
|
||||
|
||||
|
|
|
@ -19,32 +19,27 @@ package fr.acinq.eclair.blockchain.bitcoind
|
|||
import akka.Done
|
||||
import akka.actor.{ActorRef, Props}
|
||||
import akka.pattern.pipe
|
||||
import akka.testkit.{TestActorRef, TestFSMRef, TestProbe}
|
||||
import fr.acinq.bitcoin.{Block, Btc, BtcAmount, MilliBtcDouble, OutPoint, SatoshiLong, Script, ScriptWitness, Transaction, TxIn, TxOut}
|
||||
import akka.testkit.{TestActorRef, TestProbe}
|
||||
import fr.acinq.bitcoin.{Block, Btc, OutPoint, SatoshiLong, Script, Transaction, TxOut}
|
||||
import fr.acinq.eclair.blockchain.WatcherSpec._
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
|
||||
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient
|
||||
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient.{FundTransactionResponse, MempoolTx, SignTransactionResponse}
|
||||
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient.{FundTransactionResponse, SignTransactionResponse}
|
||||
import fr.acinq.eclair.blockchain.bitcoind.zmq.ZMQActor
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.{StateTestsHelperMethods, StateTestsTags}
|
||||
import fr.acinq.eclair.transactions.Transactions.TransactionSigningKit.{ClaimAnchorOutputSigningKit, HtlcSuccessSigningKit, HtlcTimeoutSigningKit}
|
||||
import fr.acinq.eclair.transactions.{Scripts, Transactions}
|
||||
import fr.acinq.eclair.{MilliSatoshiLong, TestConstants, TestKitBaseClass, randomBytes32, randomKey}
|
||||
import fr.acinq.eclair.{TestKitBaseClass, randomBytes32}
|
||||
import grizzled.slf4j.Logging
|
||||
import org.scalatest.BeforeAndAfterAll
|
||||
import org.scalatest.funsuite.AnyFunSuiteLike
|
||||
import org.scalatest.{BeforeAndAfterAll, Tag}
|
||||
|
||||
import java.util.UUID
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
import scala.concurrent.Promise
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.Random
|
||||
|
||||
class ZmqWatcherSpec extends TestKitBaseClass with AnyFunSuiteLike with BitcoindService with StateTestsHelperMethods with BeforeAndAfterAll with Logging {
|
||||
class ZmqWatcherSpec extends TestKitBaseClass with AnyFunSuiteLike with BitcoindService with BeforeAndAfterAll with Logging {
|
||||
|
||||
var zmqBlock: ActorRef = _
|
||||
var zmqTx: ActorRef = _
|
||||
|
@ -70,46 +65,17 @@ class ZmqWatcherSpec extends TestKitBaseClass with AnyFunSuiteLike with Bitcoind
|
|||
super.afterAll()
|
||||
}
|
||||
|
||||
case class Fixture(alice: TestFSMRef[State, Data, Channel],
|
||||
bob: TestFSMRef[State, Data, Channel],
|
||||
alice2bob: TestProbe,
|
||||
bob2alice: TestProbe,
|
||||
alice2watcher: TestProbe,
|
||||
bob2watcher: TestProbe,
|
||||
blockCount: AtomicLong,
|
||||
bitcoinClient: ExtendedBitcoinClient,
|
||||
bitcoinWallet: BitcoinCoreWallet,
|
||||
watcher: TestActorRef[ZmqWatcher],
|
||||
probe: TestProbe)
|
||||
case class Fixture(blockCount: AtomicLong, bitcoinClient: ExtendedBitcoinClient, bitcoinWallet: BitcoinCoreWallet, watcher: TestActorRef[ZmqWatcher], probe: TestProbe)
|
||||
|
||||
// NB: we can't use ScalaTest's fixtures, they would see uninitialized bitcoind fields because they sandbox each test.
|
||||
private def withWatcher(utxos: Seq[BtcAmount], testFun: Fixture => Any): Unit = {
|
||||
val probe = TestProbe()
|
||||
|
||||
// Create a unique wallet for this test and ensure it has some btc.
|
||||
val walletRpcClient = createWallet(s"lightning-${UUID.randomUUID()}")
|
||||
val bitcoinClient = new ExtendedBitcoinClient(walletRpcClient)
|
||||
val bitcoinWallet = new BitcoinCoreWallet(walletRpcClient)
|
||||
utxos.foreach(amount => {
|
||||
bitcoinWallet.getReceiveAddress.pipeTo(probe.ref)
|
||||
val walletAddress = probe.expectMsgType[String]
|
||||
sendToAddress(walletAddress, amount, probe)
|
||||
})
|
||||
generateBlocks(1)
|
||||
|
||||
private def withWatcher(testFun: Fixture => Any): Unit = {
|
||||
val blockCount = new AtomicLong()
|
||||
val probe = TestProbe()
|
||||
val bitcoinClient = new ExtendedBitcoinClient(bitcoinrpcclient)
|
||||
val bitcoinWallet = new BitcoinCoreWallet(bitcoinrpcclient)
|
||||
val watcher = TestActorRef[ZmqWatcher](ZmqWatcher.props(Block.RegtestGenesisBlock.hash, blockCount, bitcoinClient))
|
||||
// Setup a valid channel between alice and bob.
|
||||
val setup = init(TestConstants.Alice.nodeParams.copy(blockCount = blockCount), TestConstants.Bob.nodeParams.copy(blockCount = blockCount), bitcoinWallet)
|
||||
reachNormal(setup, Set(StateTestsTags.AnchorOutputs))
|
||||
import setup._
|
||||
awaitCond(alice.stateName == NORMAL)
|
||||
awaitCond(bob.stateName == NORMAL)
|
||||
// Generate blocks to ensure the funding tx is confirmed.
|
||||
generateBlocks(1)
|
||||
// Execute our test.
|
||||
try {
|
||||
testFun(Fixture(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain, blockCount, bitcoinClient, bitcoinWallet, watcher, probe))
|
||||
testFun(Fixture(blockCount, bitcoinClient, bitcoinWallet, watcher, probe))
|
||||
} finally {
|
||||
system.stop(watcher)
|
||||
}
|
||||
|
@ -151,28 +117,43 @@ class ZmqWatcherSpec extends TestKitBaseClass with AnyFunSuiteLike with Bitcoind
|
|||
}
|
||||
|
||||
test("watch for confirmed transactions") {
|
||||
withWatcher(Seq(500 millibtc), f => {
|
||||
withWatcher(f => {
|
||||
import f._
|
||||
|
||||
val address = getNewAddress(probe)
|
||||
val tx = sendToAddress(address, Btc(1), probe)
|
||||
|
||||
val listener = TestProbe()
|
||||
probe.send(watcher, WatchConfirmed(listener.ref, tx.txid, 4, BITCOIN_FUNDING_DEPTHOK))
|
||||
probe.send(watcher, WatchConfirmed(listener.ref, tx.txid, 4, BITCOIN_FUNDING_DEPTHOK)) // setting the watch multiple times should be a no-op
|
||||
generateBlocks(5)
|
||||
assert(listener.expectMsgType[WatchEventConfirmed].tx.txid === tx.txid)
|
||||
probe.send(watcher, WatchConfirmed(listener.ref, tx.txid, 1, BITCOIN_FUNDING_DEPTHOK))
|
||||
probe.send(watcher, WatchConfirmed(listener.ref, tx.txid, 4, BITCOIN_FUNDING_DEEPLYBURIED))
|
||||
probe.send(watcher, WatchConfirmed(listener.ref, tx.txid, 4, BITCOIN_FUNDING_DEEPLYBURIED)) // setting the watch multiple times should be a no-op
|
||||
listener.expectNoMsg(1 second)
|
||||
|
||||
generateBlocks(1)
|
||||
val w1 = listener.expectMsgType[WatchEventConfirmed]
|
||||
assert(w1.tx.txid === tx.txid)
|
||||
assert(w1.event === BITCOIN_FUNDING_DEPTHOK)
|
||||
listener.expectNoMsg(1 second)
|
||||
|
||||
generateBlocks(3)
|
||||
val w2 = listener.expectMsgType[WatchEventConfirmed]
|
||||
assert(w2.tx.txid === tx.txid)
|
||||
assert(w2.event === BITCOIN_FUNDING_DEEPLYBURIED)
|
||||
listener.expectNoMsg(1 second)
|
||||
|
||||
// If we try to watch a transaction that has already been confirmed, we should immediately receive a WatchEventConfirmed.
|
||||
probe.send(watcher, WatchConfirmed(listener.ref, tx.txid, 4, BITCOIN_FUNDING_DEPTHOK))
|
||||
probe.send(watcher, WatchConfirmed(listener.ref, tx.txid, 1, BITCOIN_FUNDING_DEPTHOK))
|
||||
assert(listener.expectMsgType[WatchEventConfirmed].tx.txid === tx.txid)
|
||||
probe.send(watcher, WatchConfirmed(listener.ref, tx.txid, 2, BITCOIN_FUNDING_DEPTHOK))
|
||||
assert(listener.expectMsgType[WatchEventConfirmed].tx.txid === tx.txid)
|
||||
probe.send(watcher, WatchConfirmed(listener.ref, tx.txid, 4, BITCOIN_FUNDING_DEEPLYBURIED))
|
||||
assert(listener.expectMsgType[WatchEventConfirmed].tx.txid === tx.txid)
|
||||
listener.expectNoMsg(1 second)
|
||||
})
|
||||
}
|
||||
|
||||
test("watch for spent transactions") {
|
||||
withWatcher(Seq(500 millibtc), f => {
|
||||
withWatcher(f => {
|
||||
import f._
|
||||
|
||||
val address = getNewAddress(probe)
|
||||
|
@ -220,7 +201,7 @@ class ZmqWatcherSpec extends TestKitBaseClass with AnyFunSuiteLike with Bitcoind
|
|||
}
|
||||
|
||||
test("watch for unknown spent transactions") {
|
||||
withWatcher(Seq(500 millibtc), f => {
|
||||
withWatcher(f => {
|
||||
import f._
|
||||
|
||||
// create a chain of transactions that we don't broadcast yet
|
||||
|
@ -251,461 +232,4 @@ class ZmqWatcherSpec extends TestKitBaseClass with AnyFunSuiteLike with Bitcoind
|
|||
})
|
||||
}
|
||||
|
||||
test("publish transactions with relative and absolute delays") {
|
||||
withWatcher(Seq(500 millibtc), f => {
|
||||
import f._
|
||||
|
||||
// Ensure watcher is synchronized with the latest block height.
|
||||
bitcoinClient.getBlockCount.pipeTo(probe.ref)
|
||||
val initialBlockCount = probe.expectMsgType[Long]
|
||||
awaitCond(blockCount.get === initialBlockCount)
|
||||
|
||||
// tx1 has an absolute delay but no relative delay
|
||||
val priv = dumpPrivateKey(getNewAddress(probe), probe)
|
||||
val tx1 = {
|
||||
bitcoinWallet.fundTransaction(Transaction(2, Nil, TxOut(150000 sat, Script.pay2wpkh(priv.publicKey)) :: Nil, initialBlockCount + 5), lockUtxos = true, FeeratePerKw(250 sat)).pipeTo(probe.ref)
|
||||
val funded = probe.expectMsgType[FundTransactionResponse].tx
|
||||
bitcoinWallet.signTransaction(funded).pipeTo(probe.ref)
|
||||
probe.expectMsgType[SignTransactionResponse].tx
|
||||
}
|
||||
probe.send(watcher, PublishAsap(tx1, PublishStrategy.JustPublish))
|
||||
generateBlocks(4)
|
||||
awaitCond(blockCount.get === initialBlockCount + 4)
|
||||
bitcoinClient.getMempool().pipeTo(probe.ref)
|
||||
assert(!probe.expectMsgType[Seq[Transaction]].exists(_.txid === tx1.txid)) // tx should not be broadcast yet
|
||||
generateBlocks(1)
|
||||
awaitCond({
|
||||
bitcoinClient.getMempool().pipeTo(probe.ref)
|
||||
probe.expectMsgType[Seq[Transaction]].exists(_.txid === tx1.txid)
|
||||
}, max = 20 seconds, interval = 1 second)
|
||||
|
||||
// tx2 has a relative delay but no absolute delay
|
||||
val tx2 = createSpendP2WPKH(tx1, priv, priv.publicKey, 10000 sat, sequence = 2, lockTime = 0)
|
||||
probe.send(watcher, WatchConfirmed(probe.ref, tx1.txid, 1, BITCOIN_FUNDING_DEPTHOK))
|
||||
probe.send(watcher, PublishAsap(tx2, PublishStrategy.JustPublish))
|
||||
generateBlocks(1)
|
||||
assert(probe.expectMsgType[WatchEventConfirmed].tx === tx1)
|
||||
generateBlocks(2)
|
||||
awaitCond({
|
||||
bitcoinClient.getMempool().pipeTo(probe.ref)
|
||||
probe.expectMsgType[Seq[Transaction]].exists(_.txid === tx2.txid)
|
||||
}, max = 20 seconds, interval = 1 second)
|
||||
|
||||
// tx3 has both relative and absolute delays
|
||||
val tx3 = createSpendP2WPKH(tx2, priv, priv.publicKey, 10000 sat, sequence = 1, lockTime = blockCount.get + 5)
|
||||
probe.send(watcher, WatchConfirmed(probe.ref, tx2.txid, 1, BITCOIN_FUNDING_DEPTHOK))
|
||||
probe.send(watcher, WatchSpent(probe.ref, tx2.txid, 0, BITCOIN_FUNDING_SPENT, hints = Set.empty))
|
||||
probe.send(watcher, PublishAsap(tx3, PublishStrategy.JustPublish))
|
||||
generateBlocks(1)
|
||||
assert(probe.expectMsgType[WatchEventConfirmed].tx === tx2)
|
||||
val currentBlockCount = blockCount.get
|
||||
// after 1 block, the relative delay is elapsed, but not the absolute delay
|
||||
generateBlocks(1)
|
||||
awaitCond(blockCount.get == currentBlockCount + 1)
|
||||
probe.expectNoMsg(1 second)
|
||||
generateBlocks(3)
|
||||
probe.expectMsg(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx3))
|
||||
bitcoinClient.getMempool().pipeTo(probe.ref)
|
||||
probe.expectMsgType[Seq[Transaction]].exists(_.txid === tx3.txid)
|
||||
})
|
||||
}
|
||||
|
||||
private def getMempoolTxs(bitcoinClient: ExtendedBitcoinClient, expectedTxCount: Int, probe: TestProbe = TestProbe()): Seq[MempoolTx] = {
|
||||
awaitCond({
|
||||
bitcoinClient.getMempool().pipeTo(probe.ref)
|
||||
probe.expectMsgType[Seq[Transaction]].size == expectedTxCount
|
||||
}, interval = 250 milliseconds)
|
||||
|
||||
bitcoinClient.getMempool().pipeTo(probe.ref)
|
||||
probe.expectMsgType[Seq[Transaction]].map(tx => {
|
||||
bitcoinClient.getMempoolTx(tx.txid).pipeTo(probe.ref)
|
||||
probe.expectMsgType[MempoolTx]
|
||||
})
|
||||
}
|
||||
|
||||
def closeChannelWithoutHtlcs(f: Fixture): PublishAsap = {
|
||||
import f._
|
||||
|
||||
val commitTx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
val currentFeerate = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.spec.feeratePerKw
|
||||
probe.send(alice, CMD_FORCECLOSE(probe.ref))
|
||||
probe.expectMsgType[CommandSuccess[CMD_FORCECLOSE]]
|
||||
|
||||
val publishCommitTx = alice2watcher.expectMsgType[PublishAsap]
|
||||
assert(publishCommitTx.tx.txid === commitTx.txid)
|
||||
assert(publishCommitTx.strategy.isInstanceOf[PublishStrategy.SetFeerate])
|
||||
val publishStrategy = publishCommitTx.strategy.asInstanceOf[PublishStrategy.SetFeerate]
|
||||
assert(publishStrategy.currentFeerate < publishStrategy.targetFeerate)
|
||||
assert(publishStrategy.currentFeerate === currentFeerate)
|
||||
assert(publishStrategy.targetFeerate === TestConstants.feeratePerKw)
|
||||
publishCommitTx
|
||||
}
|
||||
|
||||
test("commit tx feerate high enough, not spending anchor output") {
|
||||
withWatcher(Seq(500 millibtc), f => {
|
||||
import f._
|
||||
|
||||
val publishCommitTx = closeChannelWithoutHtlcs(f)
|
||||
val publishStrategy = publishCommitTx.strategy.asInstanceOf[PublishStrategy.SetFeerate]
|
||||
alice2watcher.forward(watcher, publishCommitTx.copy(strategy = publishStrategy.copy(targetFeerate = publishStrategy.currentFeerate)))
|
||||
|
||||
// wait for the commit tx and anchor tx to be published
|
||||
val mempoolTx = getMempoolTxs(bitcoinClient, 1, probe).head
|
||||
assert(mempoolTx.txid === publishCommitTx.tx.txid)
|
||||
|
||||
val targetFee = Transactions.weight2fee(publishStrategy.currentFeerate, mempoolTx.weight.toInt)
|
||||
val actualFee = mempoolTx.fees
|
||||
assert(targetFee * 0.9 <= actualFee && actualFee <= targetFee * 1.1, s"actualFee=$actualFee targetFee=$targetFee")
|
||||
})
|
||||
}
|
||||
|
||||
test("commit tx feerate too low, not enough wallet inputs to increase feerate") {
|
||||
withWatcher(Seq(10.1 millibtc), f => {
|
||||
import f._
|
||||
|
||||
val publishCommitTx = closeChannelWithoutHtlcs(f)
|
||||
alice2watcher.forward(watcher, publishCommitTx)
|
||||
|
||||
// wait for the commit tx to be published, anchor will not be published because we don't have enough funds
|
||||
val mempoolTx1 = getMempoolTxs(bitcoinClient, 1, probe).head
|
||||
assert(mempoolTx1.txid === publishCommitTx.tx.txid)
|
||||
|
||||
// add more funds to our wallet
|
||||
bitcoinWallet.getReceiveAddress.pipeTo(probe.ref)
|
||||
val walletAddress = probe.expectMsgType[String]
|
||||
sendToAddress(walletAddress, 1 millibtc, probe)
|
||||
generateBlocks(1)
|
||||
|
||||
// wait for the anchor tx to be published
|
||||
val mempoolTx2 = getMempoolTxs(bitcoinClient, 1, probe).head
|
||||
bitcoinClient.getTransaction(mempoolTx2.txid).pipeTo(probe.ref)
|
||||
val anchorTx = probe.expectMsgType[Transaction]
|
||||
assert(anchorTx.txIn.exists(_.outPoint.txid == mempoolTx1.txid))
|
||||
val targetFee = Transactions.weight2fee(TestConstants.feeratePerKw, (mempoolTx1.weight + mempoolTx2.weight).toInt)
|
||||
val actualFee = mempoolTx1.fees + mempoolTx2.fees
|
||||
assert(targetFee * 0.9 <= actualFee && actualFee <= targetFee * 1.1, s"actualFee=$actualFee targetFee=$targetFee")
|
||||
})
|
||||
}
|
||||
|
||||
test("commit tx feerate too low, spending anchor output") {
|
||||
withWatcher(Seq(500 millibtc), f => {
|
||||
import f._
|
||||
|
||||
val publishCommitTx = closeChannelWithoutHtlcs(f)
|
||||
alice2watcher.forward(watcher, publishCommitTx)
|
||||
|
||||
// wait for the commit tx and anchor tx to be published
|
||||
val mempoolTxs = getMempoolTxs(bitcoinClient, 2, probe)
|
||||
assert(mempoolTxs.map(_.txid).contains(publishCommitTx.tx.txid))
|
||||
|
||||
val targetFee = Transactions.weight2fee(TestConstants.feeratePerKw, 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")
|
||||
})
|
||||
}
|
||||
|
||||
test("commit tx feerate too low, spending anchor outputs with multiple wallet inputs") {
|
||||
val utxos = Seq(
|
||||
// channel funding
|
||||
10 millibtc,
|
||||
// bumping utxos
|
||||
25000 sat,
|
||||
22000 sat,
|
||||
15000 sat
|
||||
)
|
||||
withWatcher(utxos, f => {
|
||||
import f._
|
||||
|
||||
val publishCommitTx = closeChannelWithoutHtlcs(f)
|
||||
alice2watcher.forward(watcher, publishCommitTx)
|
||||
|
||||
// wait for the commit tx and anchor tx to be published
|
||||
val mempoolTxs = getMempoolTxs(bitcoinClient, 2, probe)
|
||||
assert(mempoolTxs.map(_.txid).contains(publishCommitTx.tx.txid))
|
||||
val claimAnchorTx = mempoolTxs.find(_.txid != publishCommitTx.tx.txid).map(tx => {
|
||||
bitcoinClient.getTransaction(tx.txid).pipeTo(probe.ref)
|
||||
probe.expectMsgType[Transaction]
|
||||
})
|
||||
assert(claimAnchorTx.nonEmpty)
|
||||
assert(claimAnchorTx.get.txIn.length > 2) // we added more than 1 wallet input
|
||||
|
||||
val targetFee = Transactions.weight2fee(TestConstants.feeratePerKw, 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")
|
||||
})
|
||||
}
|
||||
|
||||
test("adjust anchor tx change amount", Tag("fuzzy")) {
|
||||
withWatcher(Seq(500 millibtc), f => {
|
||||
val PublishAsap(commitTx, PublishStrategy.SetFeerate(currentFeerate, targetFeerate, dustLimit, signingKit: ClaimAnchorOutputSigningKit)) = closeChannelWithoutHtlcs(f)
|
||||
for (_ <- 1 to 100) {
|
||||
val walletInputsCount = 1 + Random.nextInt(5)
|
||||
val walletInputs = (1 to walletInputsCount).map(_ => TxIn(OutPoint(randomBytes32, 0), Nil, 0))
|
||||
val amountIn = dustLimit * walletInputsCount + Random.nextInt(25_000_000).sat
|
||||
val amountOut = dustLimit + Random.nextLong(amountIn.toLong).sat
|
||||
val unsignedTx = signingKit.txWithInput.copy(tx = signingKit.txWithInput.tx.copy(
|
||||
txIn = signingKit.txWithInput.tx.txIn ++ walletInputs,
|
||||
txOut = TxOut(amountOut, Script.pay2wpkh(randomKey.publicKey)) :: Nil,
|
||||
))
|
||||
val adjustedTx = adjustAnchorOutputChange(unsignedTx, commitTx, amountIn, currentFeerate, targetFeerate, dustLimit)
|
||||
assert(adjustedTx.tx.txIn.size === unsignedTx.tx.txIn.size)
|
||||
assert(adjustedTx.tx.txOut.size === 1)
|
||||
assert(adjustedTx.tx.txOut.head.amount >= dustLimit)
|
||||
if (adjustedTx.tx.txOut.head.amount > dustLimit) {
|
||||
// Simulate tx signing to check final feerate.
|
||||
val signedTx = {
|
||||
val anchorSigned = Transactions.addSigs(adjustedTx, Transactions.PlaceHolderSig)
|
||||
val signedWalletInputs = anchorSigned.tx.txIn.tail.map(txIn => txIn.copy(witness = ScriptWitness(Seq(Scripts.der(Transactions.PlaceHolderSig), Transactions.PlaceHolderPubKey.value))))
|
||||
anchorSigned.tx.copy(txIn = anchorSigned.tx.txIn.head +: signedWalletInputs)
|
||||
}
|
||||
// We want the package anchor tx + commit tx to reach our target feerate, but the commit tx already pays a (smaller) fee
|
||||
val targetFee = Transactions.weight2fee(targetFeerate, signedTx.weight() + commitTx.weight()) - Transactions.weight2fee(currentFeerate, commitTx.weight())
|
||||
val actualFee = amountIn - signedTx.txOut.map(_.amount).sum
|
||||
assert(targetFee * 0.9 <= actualFee && actualFee <= targetFee * 1.1, s"actualFee=$actualFee targetFee=$targetFee amountIn=$amountIn tx=$signedTx")
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
def closeChannelWithHtlcs(f: Fixture): (PublishAsap, PublishAsap, PublishAsap) = {
|
||||
import f._
|
||||
|
||||
// Add htlcs in both directions and ensure that preimages are available.
|
||||
addHtlc(5_000_000 msat, alice, bob, alice2bob, bob2alice)
|
||||
crossSign(alice, bob, alice2bob, bob2alice)
|
||||
val (r, htlc) = addHtlc(4_000_000 msat, bob, alice, bob2alice, alice2bob)
|
||||
crossSign(bob, alice, bob2alice, alice2bob)
|
||||
probe.send(alice, CMD_FULFILL_HTLC(htlc.id, r, replyTo_opt = Some(probe.ref)))
|
||||
probe.expectMsgType[CommandSuccess[CMD_FULFILL_HTLC]]
|
||||
|
||||
// Force-close channel and verify txs sent to watcher.
|
||||
val commitTx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
val currentFeerate = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.spec.feeratePerKw
|
||||
assert(commitTx.txOut.size === 6)
|
||||
probe.send(alice, CMD_FORCECLOSE(probe.ref))
|
||||
probe.expectMsgType[CommandSuccess[CMD_FORCECLOSE]]
|
||||
val publishCommitTx = alice2watcher.expectMsgType[PublishAsap]
|
||||
assert(alice2watcher.expectMsgType[PublishAsap].strategy === PublishStrategy.JustPublish) // claim main output
|
||||
val publishHtlcSuccess = alice2watcher.expectMsgType[PublishAsap]
|
||||
val publishHtlcTimeout = alice2watcher.expectMsgType[PublishAsap]
|
||||
Seq(publishCommitTx, publishHtlcSuccess, publishHtlcTimeout).foreach(publishTx => {
|
||||
assert(publishTx.strategy.isInstanceOf[PublishStrategy.SetFeerate])
|
||||
val publishStrategy = publishTx.strategy.asInstanceOf[PublishStrategy.SetFeerate]
|
||||
assert(publishStrategy.currentFeerate === currentFeerate)
|
||||
assert(publishStrategy.currentFeerate < publishStrategy.targetFeerate)
|
||||
assert(publishStrategy.targetFeerate === TestConstants.feeratePerKw)
|
||||
})
|
||||
|
||||
(publishCommitTx, publishHtlcSuccess, publishHtlcTimeout)
|
||||
}
|
||||
|
||||
test("htlc tx feerate high enough, not adding wallet inputs") {
|
||||
withWatcher(Seq(500 millibtc), f => {
|
||||
import f._
|
||||
|
||||
val currentFeerate = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.spec.feeratePerKw
|
||||
val (publishCommitTx, publishHtlcSuccess, publishHtlcTimeout) = closeChannelWithHtlcs(f)
|
||||
|
||||
// Publish the commit tx.
|
||||
alice2watcher.forward(watcher, publishCommitTx)
|
||||
alice2watcher.forward(watcher, publishHtlcSuccess.copy(strategy = publishHtlcSuccess.strategy.asInstanceOf[PublishStrategy.SetFeerate].copy(targetFeerate = currentFeerate)))
|
||||
alice2watcher.forward(watcher, publishHtlcTimeout.copy(strategy = publishHtlcTimeout.strategy.asInstanceOf[PublishStrategy.SetFeerate].copy(targetFeerate = currentFeerate)))
|
||||
// HTLC txs will only be published once the commit tx is confirmed (csv delay)
|
||||
getMempoolTxs(bitcoinClient, 2, probe)
|
||||
generateBlocks(2)
|
||||
|
||||
// The HTLC-success tx will be immediately published.
|
||||
val htlcSuccessTx = getMempoolTxs(bitcoinClient, 1, probe).head
|
||||
val htlcSuccessTargetFee = Transactions.weight2fee(currentFeerate, htlcSuccessTx.weight.toInt)
|
||||
assert(htlcSuccessTargetFee * 0.9 <= htlcSuccessTx.fees && htlcSuccessTx.fees <= htlcSuccessTargetFee * 1.1, s"actualFee=${htlcSuccessTx.fees} targetFee=$htlcSuccessTargetFee")
|
||||
|
||||
// The HTLC-timeout tx will be published once its absolute timeout is satisfied.
|
||||
generateBlocks(144)
|
||||
val htlcTimeoutTx = getMempoolTxs(bitcoinClient, 1, probe).head
|
||||
val htlcTimeoutTargetFee = Transactions.weight2fee(currentFeerate, htlcTimeoutTx.weight.toInt)
|
||||
assert(htlcTimeoutTargetFee * 0.9 <= htlcTimeoutTx.fees && htlcTimeoutTx.fees <= htlcTimeoutTargetFee * 1.1, s"actualFee=${htlcSuccessTx.fees} targetFee=$htlcTimeoutTargetFee")
|
||||
})
|
||||
}
|
||||
|
||||
test("htlc tx feerate too low, not enough wallet inputs to increase feerate") {
|
||||
withWatcher(Seq(10.1 millibtc), f => {
|
||||
import f._
|
||||
|
||||
val initialBlockCount = blockCount.get()
|
||||
val (publishCommitTx, publishHtlcSuccess, _) = closeChannelWithHtlcs(f)
|
||||
val publishCommitStrategy = publishCommitTx.strategy.asInstanceOf[PublishStrategy.SetFeerate]
|
||||
|
||||
// Publish the commit tx without the anchor.
|
||||
alice2watcher.forward(watcher, publishCommitTx.copy(strategy = publishCommitStrategy.copy(targetFeerate = publishCommitStrategy.currentFeerate)))
|
||||
alice2watcher.forward(watcher, publishHtlcSuccess)
|
||||
// HTLC txs will only be published once the commit tx is confirmed (csv delay)
|
||||
getMempoolTxs(bitcoinClient, 1, probe)
|
||||
generateBlocks(2)
|
||||
awaitCond(blockCount.get() > initialBlockCount)
|
||||
|
||||
// Add more funds to our wallet to allow bumping HTLC txs.
|
||||
bitcoinWallet.getReceiveAddress.pipeTo(probe.ref)
|
||||
val walletAddress = probe.expectMsgType[String]
|
||||
sendToAddress(walletAddress, 1 millibtc, probe)
|
||||
generateBlocks(1)
|
||||
|
||||
// The HTLC-success tx will be immediately published.
|
||||
val htlcSuccessTx = getMempoolTxs(bitcoinClient, 1, probe).head
|
||||
val htlcSuccessTargetFee = Transactions.weight2fee(TestConstants.feeratePerKw, htlcSuccessTx.weight.toInt)
|
||||
assert(htlcSuccessTargetFee * 0.9 <= htlcSuccessTx.fees && htlcSuccessTx.fees <= htlcSuccessTargetFee * 1.1, s"actualFee=${htlcSuccessTx.fees} targetFee=$htlcSuccessTargetFee")
|
||||
})
|
||||
}
|
||||
|
||||
test("htlc tx feerate too low, adding wallet inputs") {
|
||||
withWatcher(Seq(500 millibtc), f => {
|
||||
import f._
|
||||
|
||||
val (publishCommitTx, publishHtlcSuccess, publishHtlcTimeout) = closeChannelWithHtlcs(f)
|
||||
|
||||
// Publish the commit tx.
|
||||
alice2watcher.forward(watcher, publishCommitTx)
|
||||
alice2watcher.forward(watcher, publishHtlcSuccess)
|
||||
alice2watcher.forward(watcher, publishHtlcTimeout)
|
||||
// HTLC txs will only be published once the commit tx is confirmed (csv delay)
|
||||
getMempoolTxs(bitcoinClient, 2, probe)
|
||||
generateBlocks(2)
|
||||
|
||||
// The HTLC-success tx will be immediately published.
|
||||
val htlcSuccessTx = getMempoolTxs(bitcoinClient, 1, probe).head
|
||||
val htlcSuccessTargetFee = Transactions.weight2fee(TestConstants.feeratePerKw, htlcSuccessTx.weight.toInt)
|
||||
assert(htlcSuccessTargetFee * 0.9 <= htlcSuccessTx.fees && htlcSuccessTx.fees <= htlcSuccessTargetFee * 1.1, s"actualFee=${htlcSuccessTx.fees} targetFee=$htlcSuccessTargetFee")
|
||||
|
||||
// The HTLC-timeout tx will be published once its absolute timeout is satisfied.
|
||||
generateBlocks(144)
|
||||
val htlcTimeoutTx = getMempoolTxs(bitcoinClient, 1, probe).head
|
||||
val htlcTimeoutTargetFee = Transactions.weight2fee(TestConstants.feeratePerKw, htlcTimeoutTx.weight.toInt)
|
||||
assert(htlcTimeoutTargetFee * 0.9 <= htlcTimeoutTx.fees && htlcTimeoutTx.fees <= htlcTimeoutTargetFee * 1.1, s"actualFee=${htlcSuccessTx.fees} targetFee=$htlcTimeoutTargetFee")
|
||||
})
|
||||
}
|
||||
|
||||
test("htlc tx feerate too low, adding multiple wallet inputs") {
|
||||
val utxos = Seq(
|
||||
// channel funding
|
||||
10 millibtc,
|
||||
// bumping utxos
|
||||
6000 sat,
|
||||
5900 sat,
|
||||
5800 sat,
|
||||
5700 sat,
|
||||
5600 sat,
|
||||
5500 sat,
|
||||
5400 sat,
|
||||
5300 sat,
|
||||
5200 sat,
|
||||
5100 sat
|
||||
)
|
||||
withWatcher(utxos, f => {
|
||||
import f._
|
||||
|
||||
val (publishCommitTx, publishHtlcSuccess, publishHtlcTimeout) = closeChannelWithHtlcs(f)
|
||||
val publishCommitStrategy = publishCommitTx.strategy.asInstanceOf[PublishStrategy.SetFeerate]
|
||||
|
||||
// Publish the commit tx without the anchor.
|
||||
alice2watcher.forward(watcher, publishCommitTx.copy(strategy = publishCommitStrategy.copy(targetFeerate = publishCommitStrategy.currentFeerate)))
|
||||
alice2watcher.forward(watcher, publishHtlcSuccess)
|
||||
alice2watcher.forward(watcher, publishHtlcTimeout)
|
||||
// HTLC txs will only be published once the commit tx is confirmed (csv delay)
|
||||
getMempoolTxs(bitcoinClient, 1, probe)
|
||||
generateBlocks(2)
|
||||
|
||||
// The HTLC-success tx will be immediately published.
|
||||
val htlcSuccessTx = getMempoolTxs(bitcoinClient, 1, probe).head
|
||||
bitcoinClient.getTransaction(htlcSuccessTx.txid).pipeTo(probe.ref)
|
||||
assert(probe.expectMsgType[Transaction].txIn.length > 2) // we added more than 1 wallet input
|
||||
val htlcSuccessTargetFee = Transactions.weight2fee(TestConstants.feeratePerKw, htlcSuccessTx.weight.toInt)
|
||||
assert(htlcSuccessTargetFee * 0.9 <= htlcSuccessTx.fees && htlcSuccessTx.fees <= htlcSuccessTargetFee * 1.4, s"actualFee=${htlcSuccessTx.fees} targetFee=$htlcSuccessTargetFee")
|
||||
|
||||
// The HTLC-timeout tx will be published once its absolute timeout is satisfied.
|
||||
generateBlocks(144)
|
||||
val htlcTimeoutTx = getMempoolTxs(bitcoinClient, 1, probe).head
|
||||
bitcoinClient.getTransaction(htlcTimeoutTx.txid).pipeTo(probe.ref)
|
||||
assert(probe.expectMsgType[Transaction].txIn.length > 2) // we added more than 1 wallet input
|
||||
val htlcTimeoutTargetFee = Transactions.weight2fee(TestConstants.feeratePerKw, htlcTimeoutTx.weight.toInt)
|
||||
assert(htlcTimeoutTargetFee * 0.9 <= htlcTimeoutTx.fees && htlcTimeoutTx.fees <= htlcTimeoutTargetFee * 1.4, s"actualFee=${htlcSuccessTx.fees} targetFee=$htlcTimeoutTargetFee")
|
||||
})
|
||||
}
|
||||
|
||||
test("htlc tx sent after commit tx confirmed") {
|
||||
withWatcher(Seq(500 millibtc), f => {
|
||||
import f._
|
||||
|
||||
// Add incoming htlc.
|
||||
val (r, htlc) = addHtlc(5_000_000 msat, bob, alice, bob2alice, alice2bob)
|
||||
crossSign(bob, alice, bob2alice, alice2bob)
|
||||
|
||||
// Force-close channel and verify txs sent to watcher.
|
||||
val commitTx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
assert(commitTx.txOut.size === 5)
|
||||
probe.send(alice, CMD_FORCECLOSE(probe.ref))
|
||||
probe.expectMsgType[CommandSuccess[CMD_FORCECLOSE]]
|
||||
val publishCommitTx = alice2watcher.expectMsgType[PublishAsap]
|
||||
assert(alice2watcher.expectMsgType[PublishAsap].strategy === PublishStrategy.JustPublish) // claim main output
|
||||
alice2watcher.expectMsgType[WatchConfirmed] // commit tx
|
||||
alice2watcher.expectMsgType[WatchConfirmed] // claim main output
|
||||
alice2watcher.expectMsgType[WatchSpent] // alice doesn't have the preimage yet to redeem the htlc but she watches the output
|
||||
alice2watcher.expectNoMsg(100 millis)
|
||||
|
||||
// Publish and confirm the commit tx.
|
||||
alice2watcher.forward(watcher, publishCommitTx)
|
||||
getMempoolTxs(bitcoinClient, 2, probe)
|
||||
generateBlocks(2)
|
||||
|
||||
probe.send(alice, CMD_FULFILL_HTLC(htlc.id, r, replyTo_opt = Some(probe.ref)))
|
||||
probe.expectMsgType[CommandSuccess[CMD_FULFILL_HTLC]]
|
||||
alice2watcher.expectMsg(publishCommitTx)
|
||||
assert(alice2watcher.expectMsgType[PublishAsap].strategy === PublishStrategy.JustPublish) // claim main output
|
||||
val publishHtlcSuccess = alice2watcher.expectMsgType[PublishAsap]
|
||||
alice2watcher.forward(watcher, publishHtlcSuccess)
|
||||
|
||||
// The HTLC-success tx will be immediately published.
|
||||
val htlcSuccessTx = getMempoolTxs(bitcoinClient, 1, probe).head
|
||||
val htlcSuccessTargetFee = Transactions.weight2fee(TestConstants.feeratePerKw, htlcSuccessTx.weight.toInt)
|
||||
assert(htlcSuccessTargetFee * 0.9 <= htlcSuccessTx.fees && htlcSuccessTx.fees <= htlcSuccessTargetFee * 1.1, s"actualFee=${htlcSuccessTx.fees} targetFee=$htlcSuccessTargetFee")
|
||||
})
|
||||
}
|
||||
|
||||
test("adjust htlc tx change amount", Tag("fuzzy")) {
|
||||
withWatcher(Seq(500 millibtc), f => {
|
||||
val (_, publishHtlcSuccess, publishHtlcTimeout) = closeChannelWithHtlcs(f)
|
||||
val PublishAsap(htlcSuccessTx, PublishStrategy.SetFeerate(_, targetFeerate, dustLimit, successSigningKit: HtlcSuccessSigningKit)) = publishHtlcSuccess
|
||||
val PublishAsap(htlcTimeoutTx, PublishStrategy.SetFeerate(_, _, _, timeoutSigningKit: HtlcTimeoutSigningKit)) = publishHtlcTimeout
|
||||
for (_ <- 1 to 100) {
|
||||
val walletInputsCount = 1 + Random.nextInt(5)
|
||||
val walletInputs = (1 to walletInputsCount).map(_ => TxIn(OutPoint(randomBytes32, 0), Nil, 0))
|
||||
val walletAmountIn = dustLimit * walletInputsCount + Random.nextInt(25_000_000).sat
|
||||
val changeOutput = TxOut(Random.nextLong(walletAmountIn.toLong).sat, Script.pay2wpkh(randomKey.publicKey))
|
||||
val unsignedHtlcSuccessTx = successSigningKit.txWithInput.copy(tx = htlcSuccessTx.copy(
|
||||
txIn = htlcSuccessTx.txIn ++ walletInputs,
|
||||
txOut = htlcSuccessTx.txOut ++ Seq(changeOutput)
|
||||
))
|
||||
val unsignedHtlcTimeoutTx = timeoutSigningKit.txWithInput.copy(tx = htlcTimeoutTx.copy(
|
||||
txIn = htlcTimeoutTx.txIn ++ walletInputs,
|
||||
txOut = htlcTimeoutTx.txOut ++ Seq(changeOutput)
|
||||
))
|
||||
for ((unsignedTx, signingKit) <- Seq((unsignedHtlcSuccessTx, successSigningKit), (unsignedHtlcTimeoutTx, timeoutSigningKit))) {
|
||||
val totalAmountIn = unsignedTx.input.txOut.amount + walletAmountIn
|
||||
val adjustedTx = adjustHtlcTxChange(unsignedTx, totalAmountIn, targetFeerate, dustLimit, signingKit)
|
||||
assert(adjustedTx.tx.txIn.size === unsignedTx.tx.txIn.size)
|
||||
assert(adjustedTx.tx.txOut.size === 1 || adjustedTx.tx.txOut.size === 2)
|
||||
if (adjustedTx.tx.txOut.size == 2) {
|
||||
// Simulate tx signing to check final feerate.
|
||||
val signedTx = {
|
||||
val htlcSigned = addHtlcTxSigs(adjustedTx, Transactions.PlaceHolderSig, signingKit)
|
||||
val signedWalletInputs = htlcSigned.tx.txIn.tail.map(txIn => txIn.copy(witness = ScriptWitness(Seq(Scripts.der(Transactions.PlaceHolderSig), Transactions.PlaceHolderPubKey.value))))
|
||||
htlcSigned.tx.copy(txIn = htlcSigned.tx.txIn.head +: signedWalletInputs)
|
||||
}
|
||||
val targetFee = Transactions.weight2fee(targetFeerate, signedTx.weight())
|
||||
val actualFee = totalAmountIn - signedTx.txOut.map(_.amount).sum
|
||||
assert(targetFee * 0.9 <= actualFee && actualFee <= targetFee * 1.1, s"actualFee=$actualFee targetFee=$targetFee amountIn=$walletAmountIn tx=$signedTx")
|
||||
}
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
}
|
|
@ -24,6 +24,7 @@ import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
|||
import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.channel.states.StateTestsBase
|
||||
import fr.acinq.eclair.channel.states.StateTestsHelperMethods.FakeTxPublisherFactory
|
||||
import fr.acinq.eclair.payment.OutgoingPacket.Upstream
|
||||
import fr.acinq.eclair.payment._
|
||||
import fr.acinq.eclair.payment.receive.MultiPartHandler.ReceivePayment
|
||||
|
@ -68,8 +69,8 @@ class FuzzySpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with StateT
|
|||
val relayerA = system.actorOf(Relayer.props(aliceParams, TestProbe().ref, registerA, paymentHandlerA))
|
||||
val relayerB = system.actorOf(Relayer.props(bobParams, TestProbe().ref, registerB, paymentHandlerB))
|
||||
val wallet = new TestWallet
|
||||
val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(aliceParams, wallet, bobParams.nodeId, alice2blockchain.ref, relayerA), alicePeer.ref)
|
||||
val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bobParams, wallet, aliceParams.nodeId, bob2blockchain.ref, relayerB), bobPeer.ref)
|
||||
val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(aliceParams, wallet, bobParams.nodeId, alice2blockchain.ref, relayerA, FakeTxPublisherFactory(alice2blockchain)), alicePeer.ref)
|
||||
val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bobParams, wallet, aliceParams.nodeId, bob2blockchain.ref, relayerB, FakeTxPublisherFactory(bob2blockchain)), bobPeer.ref)
|
||||
within(30 seconds) {
|
||||
val aliceInit = Init(Alice.channelParams.features)
|
||||
val bobInit = Init(Bob.channelParams.features)
|
||||
|
@ -77,10 +78,14 @@ class FuzzySpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with StateT
|
|||
registerB ! bob
|
||||
// no announcements
|
||||
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, Alice.channelParams, pipe, bobInit, channelFlags = 0x00.toByte, ChannelVersion.STANDARD)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, pipe, aliceInit, ChannelVersion.STANDARD)
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
pipe ! (alice, bob)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
alice2blockchain.expectMsgType[WatchSpent]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob2blockchain.expectMsgType[WatchSpent]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
awaitCond(alice.stateName == WAIT_FOR_FUNDING_CONFIRMED)
|
||||
|
|
|
@ -1,104 +0,0 @@
|
|||
/*
|
||||
* Copyright 2019 ACINQ SAS
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package fr.acinq.eclair.channel
|
||||
|
||||
import java.util.UUID
|
||||
import java.util.concurrent.CountDownLatch
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
|
||||
import akka.actor.{Actor, ActorRef, ActorSystem, Props}
|
||||
import akka.testkit.TestProbe
|
||||
import fr.acinq.bitcoin.{ByteVector32, Crypto}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher
|
||||
import fr.acinq.eclair.payment.relay.Relayer
|
||||
import fr.acinq.eclair.wire.protocol.{Init, UpdateAddHtlc}
|
||||
import org.scalatest.funsuite.AnyFunSuite
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.Random
|
||||
|
||||
class ThroughputSpec extends AnyFunSuite {
|
||||
ignore("throughput") {
|
||||
implicit val system = ActorSystem("test")
|
||||
val pipe = system.actorOf(Props[Pipe], "pipe")
|
||||
val blockCount = new AtomicLong()
|
||||
val blockchain = system.actorOf(ZmqWatcher.props(randomBytes32, blockCount, new TestBitcoinClient()), "blockchain")
|
||||
val paymentHandler = system.actorOf(Props(new Actor() {
|
||||
|
||||
context.become(run(Map()))
|
||||
|
||||
override def receive: Receive = ???
|
||||
|
||||
def run(h2r: Map[ByteVector32, ByteVector32]): Receive = {
|
||||
case ('add, tgt: ActorRef) =>
|
||||
val r = randomBytes32
|
||||
val h = Crypto.sha256(r)
|
||||
tgt ! CMD_ADD_HTLC(self, 1 msat, h, CltvExpiry(1), TestConstants.emptyOnionPacket, Origin.LocalHot(self, UUID.randomUUID()))
|
||||
context.become(run(h2r + (h -> r)))
|
||||
|
||||
case ('sig, tgt: ActorRef) => tgt ! CMD_SIGN()
|
||||
|
||||
case htlc: UpdateAddHtlc if h2r.contains(htlc.paymentHash) =>
|
||||
val r = h2r(htlc.paymentHash)
|
||||
sender ! CMD_FULFILL_HTLC(htlc.id, r)
|
||||
context.become(run(h2r - htlc.paymentHash))
|
||||
}
|
||||
}), "payment-handler")
|
||||
val registerA = TestProbe()
|
||||
val registerB = TestProbe()
|
||||
val relayerA = system.actorOf(Relayer.props(Alice.nodeParams, TestProbe().ref, registerA.ref, paymentHandler))
|
||||
val relayerB = system.actorOf(Relayer.props(Bob.nodeParams, TestProbe().ref, registerB.ref, paymentHandler))
|
||||
val wallet = new TestWallet
|
||||
val alice = system.actorOf(Channel.props(Alice.nodeParams, wallet, Bob.nodeParams.nodeId, blockchain, relayerA, None), "a")
|
||||
val bob = system.actorOf(Channel.props(Bob.nodeParams, wallet, Alice.nodeParams.nodeId, blockchain, relayerB, None), "b")
|
||||
val aliceInit = Init(Alice.channelParams.features)
|
||||
val bobInit = Init(Bob.channelParams.features)
|
||||
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, Alice.channelParams, pipe, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
|
||||
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, pipe, aliceInit, ChannelVersion.STANDARD)
|
||||
|
||||
val latch = new CountDownLatch(2)
|
||||
val listener = system.actorOf(Props(new Actor {
|
||||
override def receive: Receive = {
|
||||
case ChannelStateChanged(_, _, _, _, _, NORMAL, _) => latch.countDown()
|
||||
}
|
||||
}), "listener")
|
||||
system.eventStream.subscribe(listener, classOf[ChannelEvent])
|
||||
|
||||
pipe ! (alice, bob)
|
||||
latch.await()
|
||||
|
||||
val random = new Random()
|
||||
|
||||
def msg = random.nextInt(100) % 5 match {
|
||||
case 0 | 1 | 2 | 3 => 'add
|
||||
case 4 => 'sig
|
||||
}
|
||||
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
system.scheduler.schedule(0 seconds, 50 milliseconds, new Runnable() {
|
||||
override def run(): Unit = paymentHandler ! (msg, alice)
|
||||
})
|
||||
system.scheduler.schedule(5 seconds, 70 milliseconds, new Runnable() {
|
||||
override def run(): Unit = paymentHandler ! (msg, bob)
|
||||
})
|
||||
|
||||
Thread.sleep(Long.MaxValue)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,637 @@
|
|||
/*
|
||||
* Copyright 2021 ACINQ SAS
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package fr.acinq.eclair.channel
|
||||
|
||||
import akka.actor.typed.scaladsl.adapter.{ClassicActorSystemOps, TypedActorRefOps}
|
||||
import akka.pattern.pipe
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.bitcoin.{BtcAmount, ByteVector32, MilliBtcDouble, OutPoint, SIGHASH_ALL, SatoshiLong, Script, ScriptFlags, ScriptWitness, SigVersion, Transaction, TxIn, TxOut}
|
||||
import fr.acinq.eclair.TestConstants.TestFeeEstimator
|
||||
import fr.acinq.eclair.blockchain.WatcherSpec.createSpendP2WPKH
|
||||
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient
|
||||
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient.{FundTransactionResponse, MempoolTx, SignTransactionResponse}
|
||||
import fr.acinq.eclair.blockchain.bitcoind.{BitcoinCoreWallet, BitcoindService}
|
||||
import fr.acinq.eclair.blockchain.fee.{FeeratePerKw, FeeratesPerKw}
|
||||
import fr.acinq.eclair.blockchain.{WatchConfirmed, WatchSpent}
|
||||
import fr.acinq.eclair.channel.TxPublisher._
|
||||
import fr.acinq.eclair.channel.states.{StateTestsHelperMethods, StateTestsTags}
|
||||
import fr.acinq.eclair.transactions.Transactions.{ClaimLocalAnchorOutputTx, HtlcSuccessTx, HtlcTimeoutTx, addSigs}
|
||||
import fr.acinq.eclair.transactions.{Scripts, Transactions}
|
||||
import fr.acinq.eclair.{MilliSatoshiLong, TestConstants, TestKitBaseClass, randomBytes32, randomKey}
|
||||
import grizzled.slf4j.Logging
|
||||
import org.scalatest.funsuite.AnyFunSuiteLike
|
||||
import org.scalatest.{BeforeAndAfterAll, Tag}
|
||||
|
||||
import java.util.UUID
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
import scala.concurrent.duration.DurationInt
|
||||
import scala.util.Random
|
||||
|
||||
class TxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike with BitcoindService with StateTestsHelperMethods with BeforeAndAfterAll with Logging {
|
||||
|
||||
override def beforeAll(): Unit = {
|
||||
logger.info("starting bitcoind")
|
||||
startBitcoind()
|
||||
waitForBitcoindReady()
|
||||
super.beforeAll()
|
||||
}
|
||||
|
||||
override def afterAll(): Unit = {
|
||||
logger.info("stopping bitcoind")
|
||||
stopBitcoind()
|
||||
super.afterAll()
|
||||
}
|
||||
|
||||
case class Fixture(alice: TestFSMRef[State, Data, Channel],
|
||||
bob: TestFSMRef[State, Data, Channel],
|
||||
alice2bob: TestProbe,
|
||||
bob2alice: TestProbe,
|
||||
alice2blockchain: TestProbe,
|
||||
bob2blockchain: TestProbe,
|
||||
blockCount: AtomicLong,
|
||||
bitcoinClient: ExtendedBitcoinClient,
|
||||
bitcoinWallet: BitcoinCoreWallet,
|
||||
txPublisher: akka.actor.typed.ActorRef[TxPublisher.Command],
|
||||
probe: TestProbe) {
|
||||
|
||||
def createBlocks(count: Int): Unit = {
|
||||
val current = blockCount.get()
|
||||
generateBlocks(count)
|
||||
blockCount.set(current + count)
|
||||
txPublisher ! WrappedCurrentBlockCount(current + count)
|
||||
}
|
||||
|
||||
def getMempool: Seq[Transaction] = {
|
||||
bitcoinClient.getMempool().pipeTo(probe.ref)
|
||||
probe.expectMsgType[Seq[Transaction]]
|
||||
}
|
||||
|
||||
def getMempoolTxs(expectedTxCount: Int): Seq[MempoolTx] = {
|
||||
awaitCond(getMempool.size == expectedTxCount, interval = 250 milliseconds)
|
||||
getMempool.map(tx => {
|
||||
bitcoinClient.getMempoolTx(tx.txid).pipeTo(probe.ref)
|
||||
probe.expectMsgType[MempoolTx]
|
||||
})
|
||||
}
|
||||
|
||||
def setOnChainFeerate(feerate: FeeratePerKw): Unit = {
|
||||
alice.underlyingActor.nodeParams.onChainFeeConf.feeEstimator.asInstanceOf[TestFeeEstimator].setFeerate(FeeratesPerKw.single(feerate))
|
||||
bob.underlyingActor.nodeParams.onChainFeeConf.feeEstimator.asInstanceOf[TestFeeEstimator].setFeerate(FeeratesPerKw.single(feerate))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// NB: we can't use ScalaTest's fixtures, they would see uninitialized bitcoind fields because they sandbox each test.
|
||||
private def withFixture(utxos: Seq[BtcAmount], testFun: Fixture => Any): Unit = {
|
||||
// Create a unique wallet for this test and ensure it has some btc.
|
||||
val testId = UUID.randomUUID()
|
||||
val walletRpcClient = createWallet(s"lightning-$testId")
|
||||
val bitcoinClient = new ExtendedBitcoinClient(walletRpcClient)
|
||||
val bitcoinWallet = new BitcoinCoreWallet(walletRpcClient)
|
||||
val probe = TestProbe()
|
||||
utxos.foreach(amount => {
|
||||
bitcoinWallet.getReceiveAddress.pipeTo(probe.ref)
|
||||
val walletAddress = probe.expectMsgType[String]
|
||||
sendToAddress(walletAddress, amount, probe)
|
||||
})
|
||||
generateBlocks(1)
|
||||
|
||||
val blockCount = new AtomicLong()
|
||||
val aliceNodeParams = TestConstants.Alice.nodeParams.copy(blockCount = blockCount)
|
||||
// Setup a valid channel between alice and bob.
|
||||
val setup = init(aliceNodeParams, TestConstants.Bob.nodeParams.copy(blockCount = blockCount), bitcoinWallet)
|
||||
reachNormal(setup, Set(StateTestsTags.AnchorOutputs))
|
||||
import setup._
|
||||
awaitCond(alice.stateName == NORMAL)
|
||||
awaitCond(bob.stateName == NORMAL)
|
||||
|
||||
// Generate blocks to ensure the funding tx is confirmed and set initial block count.
|
||||
generateBlocks(1)
|
||||
bitcoinClient.getBlockCount.pipeTo(probe.ref)
|
||||
blockCount.set(probe.expectMsgType[Long])
|
||||
|
||||
// Execute our test.
|
||||
val txPublisher = system.spawn(TxPublisher(aliceNodeParams, TestConstants.Bob.nodeParams.nodeId, alice2blockchain.ref, bitcoinClient), testId.toString)
|
||||
try {
|
||||
testFun(Fixture(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain, blockCount, bitcoinClient, bitcoinWallet, txPublisher, probe))
|
||||
} finally {
|
||||
system.stop(txPublisher.ref.toClassic)
|
||||
}
|
||||
}
|
||||
|
||||
test("publish transactions with relative and absolute delays") {
|
||||
withFixture(Seq(500 millibtc), f => {
|
||||
import f._
|
||||
|
||||
// tx1 has an absolute delay but no relative delay
|
||||
val priv = dumpPrivateKey(getNewAddress(probe), probe)
|
||||
val tx1 = {
|
||||
bitcoinWallet.fundTransaction(Transaction(2, Nil, TxOut(150000 sat, Script.pay2wpkh(priv.publicKey)) :: Nil, blockCount.get() + 5), lockUtxos = true, FeeratePerKw(250 sat)).pipeTo(probe.ref)
|
||||
val funded = probe.expectMsgType[FundTransactionResponse].tx
|
||||
bitcoinWallet.signTransaction(funded).pipeTo(probe.ref)
|
||||
probe.expectMsgType[SignTransactionResponse].tx
|
||||
}
|
||||
txPublisher ! PublishRawTx(tx1, "funding-tx")
|
||||
createBlocks(4)
|
||||
assert(!getMempool.exists(_.txid === tx1.txid)) // tx should not be broadcast yet
|
||||
createBlocks(1)
|
||||
awaitCond(getMempool.exists(_.txid === tx1.txid), max = 20 seconds, interval = 1 second)
|
||||
|
||||
// tx2 has a relative delay but no absolute delay
|
||||
val tx2 = createSpendP2WPKH(tx1, priv, priv.publicKey, 10000 sat, sequence = 2, lockTime = 0)
|
||||
txPublisher ! PublishRawTx(tx2, "child-tx")
|
||||
val watchParentTx2 = alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
assert(watchParentTx2.txId === tx1.txid)
|
||||
assert(watchParentTx2.minDepth === 2)
|
||||
createBlocks(2)
|
||||
txPublisher ! ParentTxConfirmed(watchParentTx2.event.asInstanceOf[BITCOIN_PARENT_TX_CONFIRMED].childTx, tx1.txid)
|
||||
awaitCond(getMempool.exists(_.txid === tx2.txid), max = 20 seconds, interval = 1 second)
|
||||
|
||||
// tx3 has both relative and absolute delays
|
||||
val tx3 = createSpendP2WPKH(tx2, priv, priv.publicKey, 10000 sat, sequence = 1, lockTime = blockCount.get + 5)
|
||||
txPublisher ! PublishRawTx(tx3, "grand-child-tx")
|
||||
val watchParentTx3 = alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
assert(watchParentTx3.txId === tx2.txid)
|
||||
assert(watchParentTx3.minDepth === 1)
|
||||
// after 1 block, the relative delay is elapsed, but not the absolute delay
|
||||
createBlocks(1)
|
||||
txPublisher ! ParentTxConfirmed(watchParentTx3.event.asInstanceOf[BITCOIN_PARENT_TX_CONFIRMED].childTx, tx2.txid)
|
||||
assert(!getMempool.exists(_.txid === tx3.txid))
|
||||
// after 4 more blocks, the absolute delay is elapsed
|
||||
createBlocks(4)
|
||||
awaitCond(getMempool.exists(_.txid === tx3.txid), max = 20 seconds, interval = 1 second)
|
||||
})
|
||||
}
|
||||
|
||||
test("publish transaction spending parent multiple times with different relative delays") {
|
||||
withFixture(Seq(500 millibtc, 500 millibtc), f => {
|
||||
import f._
|
||||
|
||||
val priv = dumpPrivateKey(getNewAddress(probe), probe)
|
||||
val outputAmount = 125000 sat
|
||||
val Seq(parentTx1, parentTx2) = (1 to 2).map(_ => {
|
||||
val outputs = Seq(TxOut(outputAmount, Script.pay2wpkh(priv.publicKey)), TxOut(outputAmount, Script.pay2wpkh(priv.publicKey)))
|
||||
bitcoinWallet.fundTransaction(Transaction(2, Nil, outputs, 0), lockUtxos = true, FeeratePerKw(250 sat)).pipeTo(probe.ref)
|
||||
val funded = probe.expectMsgType[FundTransactionResponse].tx
|
||||
bitcoinWallet.signTransaction(funded).pipeTo(probe.ref)
|
||||
probe.expectMsgType[SignTransactionResponse].tx
|
||||
})
|
||||
txPublisher ! PublishRawTx(parentTx1, "parent-tx-1")
|
||||
txPublisher ! PublishRawTx(parentTx2, "parent-tx-2")
|
||||
assert(getMempoolTxs(2).map(_.txid).toSet === Set(parentTx1.txid, parentTx2.txid))
|
||||
|
||||
val tx = {
|
||||
val Right(outputIndexes1) = Transactions.findPubKeyScriptIndexes(parentTx1, Script.write(Script.pay2wpkh(priv.publicKey)))
|
||||
val Right(outputIndexes2) = Transactions.findPubKeyScriptIndexes(parentTx2, Script.write(Script.pay2wpkh(priv.publicKey)))
|
||||
val inputs = Seq(
|
||||
TxIn(OutPoint(parentTx1, outputIndexes1.head), Nil, 1),
|
||||
TxIn(OutPoint(parentTx1, outputIndexes1.last), Nil, 2),
|
||||
TxIn(OutPoint(parentTx2, outputIndexes2.head), Nil, 3),
|
||||
TxIn(OutPoint(parentTx2, outputIndexes2.last), Nil, 4),
|
||||
)
|
||||
val unsigned = Transaction(2, inputs, TxOut(450000 sat, Script.pay2wpkh(priv.publicKey)) :: Nil, 0)
|
||||
(0 to 3).foldLeft(unsigned) {
|
||||
case (current, i) =>
|
||||
val sig = Transaction.signInput(current, i, Script.pay2pkh(priv.publicKey), SIGHASH_ALL, outputAmount, SigVersion.SIGVERSION_WITNESS_V0, priv)
|
||||
current.updateWitness(i, ScriptWitness(sig :: priv.publicKey.value :: Nil))
|
||||
}
|
||||
}
|
||||
|
||||
Transaction.correctlySpends(tx, parentTx1 :: parentTx2 :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
txPublisher ! PublishRawTx(tx, "child-tx")
|
||||
val watches = Seq(
|
||||
alice2blockchain.expectMsgType[WatchConfirmed],
|
||||
alice2blockchain.expectMsgType[WatchConfirmed],
|
||||
)
|
||||
watches.foreach(w => assert(w.event.isInstanceOf[BITCOIN_PARENT_TX_CONFIRMED]))
|
||||
val w1 = watches.find(_.txId == parentTx1.txid).get
|
||||
assert(w1.minDepth === 2)
|
||||
val w2 = watches.find(_.txId == parentTx2.txid).get
|
||||
assert(w2.minDepth === 4)
|
||||
alice2blockchain.expectNoMsg(1 second)
|
||||
|
||||
createBlocks(2)
|
||||
txPublisher ! ParentTxConfirmed(w1.event.asInstanceOf[BITCOIN_PARENT_TX_CONFIRMED].childTx, w1.txId)
|
||||
assert(!getMempool.exists(_.txid === tx.txid))
|
||||
createBlocks(2)
|
||||
txPublisher ! ParentTxConfirmed(w2.event.asInstanceOf[BITCOIN_PARENT_TX_CONFIRMED].childTx, w2.txId)
|
||||
awaitCond(getMempool.exists(_.txid === tx.txid))
|
||||
})
|
||||
}
|
||||
|
||||
def closeChannelWithoutHtlcs(f: Fixture): (Transaction, SignAndPublishTx) = {
|
||||
import f._
|
||||
|
||||
val commitTx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx
|
||||
probe.send(alice, CMD_FORCECLOSE(probe.ref))
|
||||
probe.expectMsgType[CommandSuccess[CMD_FORCECLOSE]]
|
||||
|
||||
// Forward the commit tx to the publisher.
|
||||
val commit = alice2blockchain.expectMsg(PublishRawTx(commitTx))
|
||||
txPublisher ! commit
|
||||
// Forward the anchor tx to the publisher.
|
||||
val anchor = alice2blockchain.expectMsgType[SignAndPublishTx]
|
||||
assert(anchor.txInfo.input.outPoint.txid === commitTx.tx.txid)
|
||||
assert(anchor.txInfo.isInstanceOf[ClaimLocalAnchorOutputTx])
|
||||
txPublisher ! anchor
|
||||
|
||||
(commitTx.tx, anchor)
|
||||
}
|
||||
|
||||
test("commit tx feerate high enough, not spending anchor output") {
|
||||
withFixture(Seq(500 millibtc), f => {
|
||||
import f._
|
||||
|
||||
val commitFeerate = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.spec.feeratePerKw
|
||||
assert(commitFeerate < TestConstants.feeratePerKw)
|
||||
setOnChainFeerate(commitFeerate)
|
||||
val (commitTx, _) = closeChannelWithoutHtlcs(f)
|
||||
|
||||
// wait for the commit tx to be published
|
||||
val mempoolTx = getMempoolTxs(1).head
|
||||
assert(mempoolTx.txid === commitTx.txid)
|
||||
})
|
||||
}
|
||||
|
||||
test("commit tx feerate too low, not enough wallet inputs to increase feerate") {
|
||||
withFixture(Seq(10.1 millibtc), f => {
|
||||
import f._
|
||||
|
||||
val (commitTx, anchorTx) = closeChannelWithoutHtlcs(f)
|
||||
|
||||
// wait for the commit tx to be published, anchor will not be published because we don't have enough funds
|
||||
val mempoolTx1 = getMempoolTxs(1).head
|
||||
assert(mempoolTx1.txid === commitTx.txid)
|
||||
|
||||
// add more funds to our wallet
|
||||
bitcoinWallet.getReceiveAddress.pipeTo(probe.ref)
|
||||
val walletAddress = probe.expectMsgType[String]
|
||||
sendToAddress(walletAddress, 1 millibtc, probe)
|
||||
createBlocks(1)
|
||||
|
||||
// wait for the anchor tx to be published
|
||||
val mempoolTx2 = getMempoolTxs(1).head
|
||||
bitcoinClient.getTransaction(mempoolTx2.txid).pipeTo(probe.ref)
|
||||
val publishedAnchorTx = probe.expectMsgType[Transaction]
|
||||
assert(publishedAnchorTx.txid !== anchorTx.tx.txid)
|
||||
assert(publishedAnchorTx.txIn.exists(_.outPoint.txid == mempoolTx1.txid))
|
||||
val targetFee = Transactions.weight2fee(TestConstants.feeratePerKw, (mempoolTx1.weight + mempoolTx2.weight).toInt)
|
||||
val actualFee = mempoolTx1.fees + mempoolTx2.fees
|
||||
assert(targetFee * 0.9 <= actualFee && actualFee <= targetFee * 1.1, s"actualFee=$actualFee targetFee=$targetFee")
|
||||
})
|
||||
}
|
||||
|
||||
test("commit tx feerate too low, spending anchor output") {
|
||||
withFixture(Seq(500 millibtc), f => {
|
||||
import f._
|
||||
|
||||
val (commitTx, _) = closeChannelWithoutHtlcs(f)
|
||||
|
||||
// wait for the commit tx and anchor tx to be published
|
||||
val mempoolTxs = getMempoolTxs(2)
|
||||
assert(mempoolTxs.map(_.txid).contains(commitTx.txid))
|
||||
|
||||
val targetFee = Transactions.weight2fee(TestConstants.feeratePerKw, 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")
|
||||
})
|
||||
}
|
||||
|
||||
test("commit tx feerate too low, spending anchor outputs with multiple wallet inputs") {
|
||||
val utxos = Seq(
|
||||
// channel funding
|
||||
10 millibtc,
|
||||
// bumping utxos
|
||||
25000 sat,
|
||||
22000 sat,
|
||||
15000 sat
|
||||
)
|
||||
withFixture(utxos, f => {
|
||||
import f._
|
||||
|
||||
val (commitTx, _) = closeChannelWithoutHtlcs(f)
|
||||
|
||||
// wait for the commit tx and anchor tx to be published
|
||||
val mempoolTxs = getMempoolTxs(2)
|
||||
assert(mempoolTxs.map(_.txid).contains(commitTx.txid))
|
||||
val claimAnchorTx = mempoolTxs.find(_.txid != commitTx.txid).map(tx => {
|
||||
bitcoinClient.getTransaction(tx.txid).pipeTo(probe.ref)
|
||||
probe.expectMsgType[Transaction]
|
||||
})
|
||||
assert(claimAnchorTx.nonEmpty)
|
||||
assert(claimAnchorTx.get.txIn.exists(_.outPoint.txid == commitTx.txid))
|
||||
assert(claimAnchorTx.get.txIn.length > 2) // we added more than 1 wallet input
|
||||
|
||||
val targetFee = Transactions.weight2fee(TestConstants.feeratePerKw, 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")
|
||||
})
|
||||
}
|
||||
|
||||
test("adjust anchor tx change amount", Tag("fuzzy")) {
|
||||
withFixture(Seq(500 millibtc), f => {
|
||||
val commitFeerate = f.alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.spec.feeratePerKw
|
||||
assert(commitFeerate < TestConstants.feeratePerKw)
|
||||
val (commitTx, anchorTx) = closeChannelWithoutHtlcs(f)
|
||||
val anchorTxInfo = anchorTx.txInfo.asInstanceOf[ClaimLocalAnchorOutputTx]
|
||||
val dustLimit = anchorTx.commitments.localParams.dustLimit
|
||||
for (_ <- 1 to 100) {
|
||||
val walletInputsCount = 1 + Random.nextInt(5)
|
||||
val walletInputs = (1 to walletInputsCount).map(_ => TxIn(OutPoint(randomBytes32, 0), Nil, 0))
|
||||
val amountIn = dustLimit * walletInputsCount + Random.nextInt(25_000_000).sat
|
||||
val amountOut = dustLimit + Random.nextLong(amountIn.toLong).sat
|
||||
val unsignedTx = anchorTxInfo.copy(tx = anchorTxInfo.tx.copy(
|
||||
txIn = anchorTxInfo.tx.txIn ++ walletInputs,
|
||||
txOut = TxOut(amountOut, Script.pay2wpkh(randomKey.publicKey)) :: Nil,
|
||||
))
|
||||
val adjustedTx = adjustAnchorOutputChange(unsignedTx, commitTx, amountIn, commitFeerate, TestConstants.feeratePerKw, dustLimit)
|
||||
assert(adjustedTx.tx.txIn.size === unsignedTx.tx.txIn.size)
|
||||
assert(adjustedTx.tx.txOut.size === 1)
|
||||
assert(adjustedTx.tx.txOut.head.amount >= dustLimit)
|
||||
if (adjustedTx.tx.txOut.head.amount > dustLimit) {
|
||||
// Simulate tx signing to check final feerate.
|
||||
val signedTx = {
|
||||
val anchorSigned = Transactions.addSigs(adjustedTx, Transactions.PlaceHolderSig)
|
||||
val signedWalletInputs = anchorSigned.tx.txIn.tail.map(txIn => txIn.copy(witness = ScriptWitness(Seq(Scripts.der(Transactions.PlaceHolderSig), Transactions.PlaceHolderPubKey.value))))
|
||||
anchorSigned.tx.copy(txIn = anchorSigned.tx.txIn.head +: signedWalletInputs)
|
||||
}
|
||||
// We want the package anchor tx + commit tx to reach our target feerate, but the commit tx already pays a (smaller) fee
|
||||
val targetFee = Transactions.weight2fee(TestConstants.feeratePerKw, signedTx.weight() + commitTx.weight()) - Transactions.weight2fee(commitFeerate, commitTx.weight())
|
||||
val actualFee = amountIn - signedTx.txOut.map(_.amount).sum
|
||||
assert(targetFee * 0.9 <= actualFee && actualFee <= targetFee * 1.1, s"actualFee=$actualFee targetFee=$targetFee amountIn=$amountIn tx=$signedTx")
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
def closeChannelWithHtlcs(f: Fixture): (Transaction, SignAndPublishTx, SignAndPublishTx) = {
|
||||
import f._
|
||||
|
||||
// Add htlcs in both directions and ensure that preimages are available.
|
||||
addHtlc(5_000_000 msat, alice, bob, alice2bob, bob2alice)
|
||||
crossSign(alice, bob, alice2bob, bob2alice)
|
||||
val (r, htlc) = addHtlc(4_000_000 msat, bob, alice, bob2alice, alice2bob)
|
||||
crossSign(bob, alice, bob2alice, alice2bob)
|
||||
probe.send(alice, CMD_FULFILL_HTLC(htlc.id, r, replyTo_opt = Some(probe.ref)))
|
||||
probe.expectMsgType[CommandSuccess[CMD_FULFILL_HTLC]]
|
||||
|
||||
// Force-close channel and verify txs sent to watcher.
|
||||
val commitTx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx
|
||||
val currentFeerate = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.spec.feeratePerKw
|
||||
assert(currentFeerate < TestConstants.feeratePerKw)
|
||||
assert(commitTx.tx.txOut.size === 6)
|
||||
probe.send(alice, CMD_FORCECLOSE(probe.ref))
|
||||
probe.expectMsgType[CommandSuccess[CMD_FORCECLOSE]]
|
||||
|
||||
alice2blockchain.expectMsg(PublishRawTx(commitTx))
|
||||
txPublisher ! PublishRawTx(commitTx)
|
||||
assert(alice2blockchain.expectMsgType[SignAndPublishTx].txInfo.isInstanceOf[ClaimLocalAnchorOutputTx])
|
||||
alice2blockchain.expectMsgType[PublishRawTx] // claim main output
|
||||
val htlcSuccess = alice2blockchain.expectMsgType[SignAndPublishTx]
|
||||
assert(htlcSuccess.txInfo.isInstanceOf[HtlcSuccessTx])
|
||||
val htlcTimeout = alice2blockchain.expectMsgType[SignAndPublishTx]
|
||||
assert(htlcTimeout.txInfo.isInstanceOf[HtlcTimeoutTx])
|
||||
|
||||
alice2blockchain.expectMsgType[WatchConfirmed] // commit tx
|
||||
alice2blockchain.expectMsgType[WatchConfirmed] // claim main output
|
||||
alice2blockchain.expectMsgType[WatchSpent] // htlc-success tx
|
||||
alice2blockchain.expectMsgType[WatchSpent] // htlc-timeout tx
|
||||
alice2blockchain.expectNoMessage(100 millis)
|
||||
|
||||
(commitTx.tx, htlcSuccess, htlcTimeout)
|
||||
}
|
||||
|
||||
test("htlc tx feerate high enough, not adding wallet inputs") {
|
||||
withFixture(Seq(500 millibtc), f => {
|
||||
import f._
|
||||
|
||||
val currentFeerate = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.spec.feeratePerKw
|
||||
setOnChainFeerate(currentFeerate)
|
||||
val (commitTx, htlcSuccess, htlcTimeout) = closeChannelWithHtlcs(f)
|
||||
txPublisher ! htlcSuccess
|
||||
txPublisher ! htlcTimeout
|
||||
// HTLC txs will only be published once the commit tx is confirmed (csv delay)
|
||||
getMempoolTxs(1)
|
||||
createBlocks(2)
|
||||
txPublisher ! ParentTxConfirmed(htlcSuccess, commitTx.txid)
|
||||
txPublisher ! ParentTxConfirmed(htlcTimeout, commitTx.txid)
|
||||
|
||||
// The HTLC-success tx will be immediately published.
|
||||
val htlcSuccessTx = getMempoolTxs(1).head
|
||||
val htlcSuccessTargetFee = Transactions.weight2fee(currentFeerate, htlcSuccessTx.weight.toInt)
|
||||
assert(htlcSuccessTargetFee * 0.9 <= htlcSuccessTx.fees && htlcSuccessTx.fees <= htlcSuccessTargetFee * 1.1, s"actualFee=${htlcSuccessTx.fees} targetFee=$htlcSuccessTargetFee")
|
||||
|
||||
// The HTLC-timeout tx will be published once its absolute timeout is satisfied.
|
||||
createBlocks(144)
|
||||
val htlcTimeoutTx = getMempoolTxs(1).head
|
||||
val htlcTimeoutTargetFee = Transactions.weight2fee(currentFeerate, htlcTimeoutTx.weight.toInt)
|
||||
assert(htlcTimeoutTargetFee * 0.9 <= htlcTimeoutTx.fees && htlcTimeoutTx.fees <= htlcTimeoutTargetFee * 1.1, s"actualFee=${htlcTimeoutTx.fees} targetFee=$htlcTimeoutTargetFee")
|
||||
})
|
||||
}
|
||||
|
||||
test("htlc tx feerate too low, not enough wallet inputs to increase feerate") {
|
||||
withFixture(Seq(10.1 millibtc), f => {
|
||||
import f._
|
||||
|
||||
val (commitTx, htlcSuccess, _) = closeChannelWithHtlcs(f)
|
||||
txPublisher ! htlcSuccess
|
||||
// HTLC txs will only be published once the commit tx is confirmed (csv delay)
|
||||
getMempoolTxs(1)
|
||||
createBlocks(2)
|
||||
txPublisher ! ParentTxConfirmed(htlcSuccess, commitTx.txid)
|
||||
|
||||
// Add more funds to our wallet to allow bumping HTLC txs.
|
||||
bitcoinWallet.getReceiveAddress.pipeTo(probe.ref)
|
||||
val walletAddress = probe.expectMsgType[String]
|
||||
sendToAddress(walletAddress, 1 millibtc, probe)
|
||||
createBlocks(1)
|
||||
|
||||
// The HTLC-success tx will be immediately published.
|
||||
val htlcSuccessTx = getMempoolTxs(1).head
|
||||
val htlcSuccessTargetFee = Transactions.weight2fee(TestConstants.feeratePerKw, htlcSuccessTx.weight.toInt)
|
||||
assert(htlcSuccessTargetFee * 0.9 <= htlcSuccessTx.fees && htlcSuccessTx.fees <= htlcSuccessTargetFee * 1.1, s"actualFee=${htlcSuccessTx.fees} targetFee=$htlcSuccessTargetFee")
|
||||
})
|
||||
}
|
||||
|
||||
test("htlc tx feerate too low, adding wallet inputs") {
|
||||
withFixture(Seq(500 millibtc), f => {
|
||||
import f._
|
||||
|
||||
val (commitTx, htlcSuccess, htlcTimeout) = closeChannelWithHtlcs(f)
|
||||
txPublisher ! htlcSuccess
|
||||
txPublisher ! htlcTimeout
|
||||
// HTLC txs will only be published once the commit tx is confirmed (csv delay)
|
||||
getMempoolTxs(1)
|
||||
createBlocks(2)
|
||||
txPublisher ! ParentTxConfirmed(htlcSuccess, commitTx.txid)
|
||||
txPublisher ! ParentTxConfirmed(htlcTimeout, commitTx.txid)
|
||||
|
||||
// The HTLC-success tx will be immediately published.
|
||||
val htlcSuccessTx = getMempoolTxs(1).head
|
||||
val htlcSuccessTargetFee = Transactions.weight2fee(TestConstants.feeratePerKw, htlcSuccessTx.weight.toInt)
|
||||
assert(htlcSuccessTargetFee * 0.9 <= htlcSuccessTx.fees && htlcSuccessTx.fees <= htlcSuccessTargetFee * 1.1, s"actualFee=${htlcSuccessTx.fees} targetFee=$htlcSuccessTargetFee")
|
||||
|
||||
// The HTLC-timeout tx will be published once its absolute timeout is satisfied.
|
||||
createBlocks(144)
|
||||
val htlcTimeoutTx = getMempoolTxs(1).head
|
||||
val htlcTimeoutTargetFee = Transactions.weight2fee(TestConstants.feeratePerKw, htlcTimeoutTx.weight.toInt)
|
||||
assert(htlcTimeoutTargetFee * 0.9 <= htlcTimeoutTx.fees && htlcTimeoutTx.fees <= htlcTimeoutTargetFee * 1.1, s"actualFee=${htlcTimeoutTx.fees} targetFee=$htlcTimeoutTargetFee")
|
||||
})
|
||||
}
|
||||
|
||||
test("htlc tx feerate too low, adding multiple wallet inputs") {
|
||||
val utxos = Seq(
|
||||
// channel funding
|
||||
10 millibtc,
|
||||
// bumping utxos
|
||||
6000 sat,
|
||||
5900 sat,
|
||||
5800 sat,
|
||||
5700 sat,
|
||||
5600 sat,
|
||||
5500 sat,
|
||||
5400 sat,
|
||||
5300 sat,
|
||||
5200 sat,
|
||||
5100 sat
|
||||
)
|
||||
withFixture(utxos, f => {
|
||||
import f._
|
||||
|
||||
val (commitTx, htlcSuccess, htlcTimeout) = closeChannelWithHtlcs(f)
|
||||
txPublisher ! htlcSuccess
|
||||
txPublisher ! htlcTimeout
|
||||
// HTLC txs will only be published once the commit tx is confirmed (csv delay)
|
||||
getMempoolTxs(1)
|
||||
createBlocks(2)
|
||||
txPublisher ! ParentTxConfirmed(htlcSuccess, commitTx.txid)
|
||||
txPublisher ! ParentTxConfirmed(htlcTimeout, commitTx.txid)
|
||||
|
||||
// The HTLC-success tx will be immediately published.
|
||||
val htlcSuccessTx = getMempoolTxs(1).head
|
||||
bitcoinClient.getTransaction(htlcSuccessTx.txid).pipeTo(probe.ref)
|
||||
assert(probe.expectMsgType[Transaction].txIn.length > 2) // we added more than 1 wallet input
|
||||
val htlcSuccessTargetFee = Transactions.weight2fee(TestConstants.feeratePerKw, htlcSuccessTx.weight.toInt)
|
||||
assert(htlcSuccessTargetFee * 0.9 <= htlcSuccessTx.fees && htlcSuccessTx.fees <= htlcSuccessTargetFee * 1.4, s"actualFee=${htlcSuccessTx.fees} targetFee=$htlcSuccessTargetFee")
|
||||
|
||||
// The HTLC-timeout tx will be published once its absolute timeout is satisfied.
|
||||
createBlocks(144)
|
||||
val htlcTimeoutTx = getMempoolTxs(1).head
|
||||
bitcoinClient.getTransaction(htlcTimeoutTx.txid).pipeTo(probe.ref)
|
||||
assert(probe.expectMsgType[Transaction].txIn.length > 2) // we added more than 1 wallet input
|
||||
val htlcTimeoutTargetFee = Transactions.weight2fee(TestConstants.feeratePerKw, htlcTimeoutTx.weight.toInt)
|
||||
assert(htlcTimeoutTargetFee * 0.9 <= htlcTimeoutTx.fees && htlcTimeoutTx.fees <= htlcTimeoutTargetFee * 1.4, s"actualFee=${htlcTimeoutTx.fees} targetFee=$htlcTimeoutTargetFee")
|
||||
})
|
||||
}
|
||||
|
||||
test("htlc tx sent after commit tx confirmed") {
|
||||
withFixture(Seq(500 millibtc), f => {
|
||||
import f._
|
||||
|
||||
// Add incoming htlc.
|
||||
val (r, htlc) = addHtlc(5_000_000 msat, bob, alice, bob2alice, alice2bob)
|
||||
crossSign(bob, alice, bob2alice, alice2bob)
|
||||
|
||||
// Force-close channel and verify txs sent to watcher.
|
||||
val commitTx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx
|
||||
assert(commitTx.tx.txOut.size === 5)
|
||||
probe.send(alice, CMD_FORCECLOSE(probe.ref))
|
||||
probe.expectMsgType[CommandSuccess[CMD_FORCECLOSE]]
|
||||
alice2blockchain.expectMsg(PublishRawTx(commitTx))
|
||||
val anchorTx = alice2blockchain.expectMsgType[SignAndPublishTx]
|
||||
alice2blockchain.expectMsgType[PublishRawTx] // claim main output
|
||||
alice2blockchain.expectMsgType[WatchConfirmed] // commit tx
|
||||
alice2blockchain.expectMsgType[WatchConfirmed] // claim main output
|
||||
alice2blockchain.expectMsgType[WatchSpent] // alice doesn't have the preimage yet to redeem the htlc but she watches the output
|
||||
alice2blockchain.expectNoMessage(100 millis)
|
||||
|
||||
// Publish and confirm the commit tx.
|
||||
txPublisher ! PublishRawTx(commitTx)
|
||||
txPublisher ! anchorTx
|
||||
getMempoolTxs(2)
|
||||
createBlocks(2)
|
||||
|
||||
probe.send(alice, CMD_FULFILL_HTLC(htlc.id, r, replyTo_opt = Some(probe.ref)))
|
||||
probe.expectMsgType[CommandSuccess[CMD_FULFILL_HTLC]]
|
||||
alice2blockchain.expectMsg(PublishRawTx(commitTx))
|
||||
val anchorTx2 = alice2blockchain.expectMsgType[SignAndPublishTx]
|
||||
assert(anchorTx2.txInfo === anchorTx.txInfo)
|
||||
alice2blockchain.expectMsgType[PublishRawTx] // claim main output
|
||||
val htlcSuccess = alice2blockchain.expectMsgType[SignAndPublishTx]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed] // commit tx
|
||||
alice2blockchain.expectMsgType[WatchConfirmed] // claim main output
|
||||
alice2blockchain.expectMsgType[WatchSpent] // htlc output
|
||||
alice2blockchain.expectNoMessage(100 millis)
|
||||
|
||||
txPublisher ! htlcSuccess
|
||||
val w = alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
assert(w.txId === commitTx.tx.txid)
|
||||
assert(w.minDepth === 1)
|
||||
txPublisher ! ParentTxConfirmed(htlcSuccess, commitTx.tx.txid)
|
||||
|
||||
// The HTLC-success tx will be immediately published.
|
||||
val htlcSuccessTx = getMempoolTxs(1).head
|
||||
val htlcSuccessTargetFee = Transactions.weight2fee(TestConstants.feeratePerKw, htlcSuccessTx.weight.toInt)
|
||||
assert(htlcSuccessTargetFee * 0.9 <= htlcSuccessTx.fees && htlcSuccessTx.fees <= htlcSuccessTargetFee * 1.1, s"actualFee=${htlcSuccessTx.fees} targetFee=$htlcSuccessTargetFee")
|
||||
})
|
||||
}
|
||||
|
||||
test("adjust htlc tx change amount", Tag("fuzzy")) {
|
||||
withFixture(Seq(500 millibtc), f => {
|
||||
val (_, htlcSuccess, htlcTimeout) = closeChannelWithHtlcs(f)
|
||||
val commitments = htlcSuccess.commitments
|
||||
val dustLimit = commitments.localParams.dustLimit
|
||||
val targetFeerate = TestConstants.feeratePerKw
|
||||
for (_ <- 1 to 100) {
|
||||
val walletInputsCount = 1 + Random.nextInt(5)
|
||||
val walletInputs = (1 to walletInputsCount).map(_ => TxIn(OutPoint(randomBytes32, 0), Nil, 0))
|
||||
val walletAmountIn = dustLimit * walletInputsCount + Random.nextInt(25_000_000).sat
|
||||
val changeOutput = TxOut(Random.nextLong(walletAmountIn.toLong).sat, Script.pay2wpkh(randomKey.publicKey))
|
||||
val unsignedHtlcSuccessTx = htlcSuccess.txInfo.asInstanceOf[HtlcSuccessTx].copy(tx = htlcSuccess.tx.copy(
|
||||
txIn = htlcSuccess.tx.txIn ++ walletInputs,
|
||||
txOut = htlcSuccess.tx.txOut ++ Seq(changeOutput)
|
||||
))
|
||||
val unsignedHtlcTimeoutTx = htlcTimeout.txInfo.asInstanceOf[HtlcTimeoutTx].copy(tx = htlcTimeout.tx.copy(
|
||||
txIn = htlcTimeout.tx.txIn ++ walletInputs,
|
||||
txOut = htlcTimeout.tx.txOut ++ Seq(changeOutput)
|
||||
))
|
||||
for (unsignedTx <- Seq(unsignedHtlcSuccessTx, unsignedHtlcTimeoutTx)) {
|
||||
val totalAmountIn = unsignedTx.input.txOut.amount + walletAmountIn
|
||||
val adjustedTx = adjustHtlcTxChange(unsignedTx, totalAmountIn, targetFeerate, commitments)
|
||||
assert(adjustedTx.tx.txIn.size === unsignedTx.tx.txIn.size)
|
||||
assert(adjustedTx.tx.txOut.size === 1 || adjustedTx.tx.txOut.size === 2)
|
||||
if (adjustedTx.tx.txOut.size == 2) {
|
||||
// Simulate tx signing to check final feerate.
|
||||
val signedTx = {
|
||||
val htlcSigned = adjustedTx match {
|
||||
case tx: HtlcSuccessTx => addSigs(tx, Transactions.PlaceHolderSig, Transactions.PlaceHolderSig, ByteVector32.Zeroes, commitments.commitmentFormat)
|
||||
case tx: HtlcTimeoutTx => addSigs(tx, Transactions.PlaceHolderSig, Transactions.PlaceHolderSig, commitments.commitmentFormat)
|
||||
}
|
||||
val signedWalletInputs = htlcSigned.tx.txIn.tail.map(txIn => txIn.copy(witness = ScriptWitness(Seq(Scripts.der(Transactions.PlaceHolderSig), Transactions.PlaceHolderPubKey.value))))
|
||||
htlcSigned.tx.copy(txIn = htlcSigned.tx.txIn.head +: signedWalletInputs)
|
||||
}
|
||||
val targetFee = Transactions.weight2fee(targetFeerate, signedTx.weight())
|
||||
val actualFee = totalAmountIn - signedTx.txOut.map(_.amount).sum
|
||||
assert(targetFee * 0.9 <= actualFee && actualFee <= targetFee * 1.1, s"actualFee=$actualFee targetFee=$targetFee amountIn=$walletAmountIn tx=$signedTx")
|
||||
}
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
}
|
|
@ -16,14 +16,17 @@
|
|||
|
||||
package fr.acinq.eclair.channel.states
|
||||
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.typed.scaladsl.adapter.actorRefAdapter
|
||||
import akka.actor.{ActorContext, ActorRef}
|
||||
import akka.testkit.{TestFSMRef, TestKitBase, TestProbe}
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.{ByteVector32, Crypto, SatoshiLong, ScriptFlags, Transaction}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob, TestFeeEstimator}
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.blockchain.fee.FeeTargets
|
||||
import fr.acinq.eclair.channel.TxPublisher.{PublishRawTx, PublishTx, SignAndPublishTx}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.StateTestsHelperMethods.FakeTxPublisherFactory
|
||||
import fr.acinq.eclair.payment.OutgoingPacket
|
||||
import fr.acinq.eclair.payment.OutgoingPacket.Upstream
|
||||
import fr.acinq.eclair.router.Router.ChannelHop
|
||||
|
@ -95,8 +98,8 @@ trait StateTestsHelperMethods extends TestKitBase {
|
|||
system.eventStream.subscribe(channelUpdateListener.ref, classOf[LocalChannelUpdate])
|
||||
system.eventStream.subscribe(channelUpdateListener.ref, classOf[LocalChannelDown])
|
||||
val router = TestProbe()
|
||||
val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(nodeParamsA, wallet, Bob.nodeParams.nodeId, alice2blockchain.ref, relayerA.ref), alicePeer.ref)
|
||||
val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(nodeParamsB, wallet, Alice.nodeParams.nodeId, bob2blockchain.ref, relayerB.ref), bobPeer.ref)
|
||||
val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(nodeParamsA, wallet, Bob.nodeParams.nodeId, alice2blockchain.ref, relayerA.ref, FakeTxPublisherFactory(alice2blockchain)), alicePeer.ref)
|
||||
val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(nodeParamsB, wallet, Alice.nodeParams.nodeId, bob2blockchain.ref, relayerB.ref, FakeTxPublisherFactory(bob2blockchain)), bobPeer.ref)
|
||||
SetupFixture(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain, router, relayerA, relayerB, channelUpdateListener, wallet)
|
||||
}
|
||||
|
||||
|
@ -132,7 +135,9 @@ trait StateTestsHelperMethods extends TestKitBase {
|
|||
val aliceInit = Init(aliceParams.features)
|
||||
val bobInit = Init(bobParams.features)
|
||||
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, fundingSatoshis, pushMsat, initialFeeratePerKw, TestConstants.feeratePerKw, None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelVersion)
|
||||
assert(alice2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId === ByteVector32.Zeroes)
|
||||
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, bobParams, bob2alice.ref, aliceInit, channelVersion)
|
||||
assert(bob2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId === ByteVector32.Zeroes)
|
||||
alice2bob.expectMsgType[OpenChannel]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[AcceptChannel]
|
||||
|
@ -141,8 +146,10 @@ trait StateTestsHelperMethods extends TestKitBase {
|
|||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[FundingSigned]
|
||||
bob2alice.forward(alice)
|
||||
assert(alice2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId != ByteVector32.Zeroes)
|
||||
alice2blockchain.expectMsgType[WatchSpent]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
assert(bob2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId != ByteVector32.Zeroes)
|
||||
bob2blockchain.expectMsgType[WatchSpent]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
awaitCond(alice.stateName == WAIT_FOR_FUNDING_CONFIRMED)
|
||||
|
@ -251,9 +258,9 @@ trait StateTestsHelperMethods extends TestKitBase {
|
|||
rCloseFee = r2s.expectMsgType[ClosingSigned].feeSatoshis
|
||||
r2s.forward(s)
|
||||
} while (sCloseFee != rCloseFee)
|
||||
s2blockchain.expectMsgType[PublishAsap]
|
||||
s2blockchain.expectMsgType[PublishTx]
|
||||
s2blockchain.expectMsgType[WatchConfirmed]
|
||||
r2blockchain.expectMsgType[PublishAsap]
|
||||
r2blockchain.expectMsgType[PublishTx]
|
||||
r2blockchain.expectMsgType[WatchConfirmed]
|
||||
awaitCond(s.stateName == CLOSING)
|
||||
awaitCond(r.stateName == CLOSING)
|
||||
|
@ -264,25 +271,28 @@ trait StateTestsHelperMethods extends TestKitBase {
|
|||
// an error occurs and s publishes its commit tx
|
||||
val commitTx = s.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
s ! Error(ByteVector32.Zeroes, "oops")
|
||||
assert(s2blockchain.expectMsgType[PublishAsap].tx == commitTx)
|
||||
awaitCond(s.stateName == CLOSING)
|
||||
val closingState = s.stateData.asInstanceOf[DATA_CLOSING]
|
||||
assert(closingState.localCommitPublished.isDefined)
|
||||
val localCommitPublished = closingState.localCommitPublished.get
|
||||
|
||||
assert(s2blockchain.expectMsgType[PublishTx].tx == commitTx)
|
||||
if (closingState.commitments.commitmentFormat == Transactions.AnchorOutputsCommitmentFormat) {
|
||||
assert(s2blockchain.expectMsgType[SignAndPublishTx].txInfo.isInstanceOf[ClaimLocalAnchorOutputTx])
|
||||
}
|
||||
// if s has a main output in the commit tx (when it has a non-dust balance), it should be claimed
|
||||
localCommitPublished.claimMainDelayedOutputTx.foreach(tx => s2blockchain.expectMsg(PublishAsap(tx.tx, PublishStrategy.JustPublish)))
|
||||
s.stateData.asInstanceOf[DATA_CLOSING].commitments.commitmentFormat match {
|
||||
localCommitPublished.claimMainDelayedOutputTx.foreach(tx => s2blockchain.expectMsg(PublishRawTx(tx)))
|
||||
closingState.commitments.commitmentFormat match {
|
||||
case Transactions.DefaultCommitmentFormat =>
|
||||
// all htlcs success/timeout should be published as-is, without claiming their outputs
|
||||
s2blockchain.expectMsgAllOf(localCommitPublished.htlcTxs.values.toSeq.collect { case Some(tx) => PublishAsap(tx.tx, PublishStrategy.JustPublish) }: _*)
|
||||
s2blockchain.expectMsgAllOf(localCommitPublished.htlcTxs.values.toSeq.collect { case Some(tx) => PublishRawTx(tx) }: _*)
|
||||
assert(localCommitPublished.claimHtlcDelayedTxs.isEmpty)
|
||||
case Transactions.AnchorOutputsCommitmentFormat =>
|
||||
// all htlcs success/timeout should be published with a fee bumping strategy, without claiming their outputs
|
||||
val htlcTxs = localCommitPublished.htlcTxs.values.collect { case Some(tx: HtlcTx) => tx.tx }
|
||||
val publishedTxs = htlcTxs.map(_ => s2blockchain.expectMsgType[PublishAsap])
|
||||
val publishedTxs = htlcTxs.map(_ => s2blockchain.expectMsgType[PublishTx])
|
||||
assert(publishedTxs.map(_.tx).toSet == htlcTxs.toSet)
|
||||
publishedTxs.foreach(p => p.strategy.isInstanceOf[PublishStrategy.SetFeerate])
|
||||
publishedTxs.foreach(p => p.isInstanceOf[SignAndPublishTx])
|
||||
assert(localCommitPublished.claimHtlcDelayedTxs.isEmpty)
|
||||
}
|
||||
|
||||
|
@ -299,7 +309,7 @@ trait StateTestsHelperMethods extends TestKitBase {
|
|||
s2blockchain.expectNoMsg(1 second)
|
||||
|
||||
// s is now in CLOSING state with txs pending for confirmation before going in CLOSED state
|
||||
s.stateData.asInstanceOf[DATA_CLOSING].localCommitPublished.get
|
||||
closingState.localCommitPublished.get
|
||||
}
|
||||
|
||||
def remoteClose(rCommitTx: Transaction, s: TestFSMRef[State, Data, Channel], s2blockchain: TestProbe): RemoteCommitPublished = {
|
||||
|
@ -307,22 +317,20 @@ trait StateTestsHelperMethods extends TestKitBase {
|
|||
s ! WatchEventSpent(BITCOIN_FUNDING_SPENT, rCommitTx)
|
||||
awaitCond(s.stateName == CLOSING)
|
||||
val closingData = s.stateData.asInstanceOf[DATA_CLOSING]
|
||||
|
||||
def getRemoteCommitPublished(d: DATA_CLOSING): Option[RemoteCommitPublished] = d.remoteCommitPublished.orElse(d.nextRemoteCommitPublished).orElse(d.futureRemoteCommitPublished)
|
||||
|
||||
assert(getRemoteCommitPublished(closingData).isDefined)
|
||||
val remoteCommitPublished_opt = closingData.remoteCommitPublished.orElse(closingData.nextRemoteCommitPublished).orElse(closingData.futureRemoteCommitPublished)
|
||||
assert(remoteCommitPublished_opt.isDefined)
|
||||
assert(closingData.localCommitPublished.isEmpty)
|
||||
val remoteCommitPublished = getRemoteCommitPublished(closingData).get
|
||||
val remoteCommitPublished = remoteCommitPublished_opt.get
|
||||
|
||||
// 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)
|
||||
s2blockchain.expectMsg(PublishAsap(claimMain.tx, PublishStrategy.JustPublish))
|
||||
s2blockchain.expectMsg(PublishRawTx(claimMain))
|
||||
})
|
||||
// all htlcs success/timeout should be claimed
|
||||
val claimHtlcTxs = remoteCommitPublished.claimHtlcTxs.values.collect { case Some(tx: ClaimHtlcTx) => tx }.toSeq
|
||||
claimHtlcTxs.foreach(claimHtlc => Transaction.correctlySpends(claimHtlc.tx, rCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS))
|
||||
s2blockchain.expectMsgAllOf(claimHtlcTxs.map(claimHtlc => PublishAsap(claimHtlc.tx, PublishStrategy.JustPublish)): _*)
|
||||
s2blockchain.expectMsgAllOf(claimHtlcTxs.map(claimHtlc => PublishRawTx(claimHtlc)): _*)
|
||||
|
||||
// we watch the confirmation of the "final" transactions that send funds to our wallets (main delayed output and 2nd stage htlc transactions)
|
||||
assert(s2blockchain.expectMsgType[WatchConfirmed].event == BITCOIN_TX_CONFIRMED(rCommitTx))
|
||||
|
@ -337,7 +345,7 @@ trait StateTestsHelperMethods extends TestKitBase {
|
|||
s2blockchain.expectNoMsg(1 second)
|
||||
|
||||
// s is now in CLOSING state with txs pending for confirmation before going in CLOSED state
|
||||
getRemoteCommitPublished(s.stateData.asInstanceOf[DATA_CLOSING]).get
|
||||
remoteCommitPublished
|
||||
}
|
||||
|
||||
def channelId(a: TestFSMRef[State, Data, Channel]): ByteVector32 = a.stateData.channelId
|
||||
|
@ -350,4 +358,12 @@ trait StateTestsHelperMethods extends TestKitBase {
|
|||
|
||||
def getClaimHtlcTimeoutTxs(rcp: RemoteCommitPublished): Seq[ClaimHtlcTimeoutTx] = rcp.claimHtlcTxs.values.collect { case Some(tx: ClaimHtlcTimeoutTx) => tx }.toSeq
|
||||
|
||||
}
|
||||
|
||||
object StateTestsHelperMethods {
|
||||
|
||||
case class FakeTxPublisherFactory(txPublisher: TestProbe) extends Channel.TxPublisherFactory {
|
||||
override def spawnTxPublisher(context: ActorContext, remoteNodeId: PublicKey): akka.actor.typed.ActorRef[TxPublisher.Command] = txPublisher.ref
|
||||
}
|
||||
|
||||
}
|
|
@ -61,7 +61,9 @@ class WaitForFundingCreatedStateSpec extends TestKitBaseClass with FixtureAnyFun
|
|||
val bobInit = Init(bobParams.features)
|
||||
within(30 seconds) {
|
||||
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, fundingSatoshis, pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, aliceParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, bobParams, bob2alice.ref, aliceInit, ChannelVersion.STANDARD)
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
alice2bob.expectMsgType[OpenChannel]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[AcceptChannel]
|
||||
|
@ -77,6 +79,7 @@ class WaitForFundingCreatedStateSpec extends TestKitBaseClass with FixtureAnyFun
|
|||
alice2bob.forward(bob)
|
||||
awaitCond(bob.stateName == WAIT_FOR_FUNDING_CONFIRMED)
|
||||
bob2alice.expectMsgType[FundingSigned]
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob2blockchain.expectMsgType[WatchSpent]
|
||||
val watchConfirmed = bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
assert(watchConfirmed.minDepth === Alice.nodeParams.minDepthBlocks)
|
||||
|
@ -88,6 +91,7 @@ class WaitForFundingCreatedStateSpec extends TestKitBaseClass with FixtureAnyFun
|
|||
alice2bob.forward(bob)
|
||||
awaitCond(bob.stateName == WAIT_FOR_FUNDING_CONFIRMED)
|
||||
bob2alice.expectMsgType[FundingSigned]
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob2blockchain.expectMsgType[WatchSpent]
|
||||
val watchConfirmed = bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
// when we are fundee, we use a higher min depth for wumbo channels
|
||||
|
|
|
@ -58,13 +58,16 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
val bobInit = Init(bobParams.features)
|
||||
within(30 seconds) {
|
||||
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, fundingSatoshis, pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, aliceParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, bobParams, bob2alice.ref, aliceInit, ChannelVersion.STANDARD)
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
alice2bob.expectMsgType[OpenChannel]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[AcceptChannel]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[FundingCreated]
|
||||
alice2bob.forward(bob)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
awaitCond(alice.stateName == WAIT_FOR_FUNDING_SIGNED)
|
||||
withFixture(test.toNoArgTest(FixtureParam(alice, alice2bob, bob2alice, alice2blockchain)))
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@ import akka.testkit.{TestFSMRef, TestProbe}
|
|||
import fr.acinq.bitcoin.{ByteVector32, SatoshiLong, Script, Transaction}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.channel.TxPublisher.{PublishRawTx, PublishTx}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.StateTestsBase
|
||||
import fr.acinq.eclair.transactions.Scripts.multiSig2of2
|
||||
|
@ -45,7 +46,9 @@ class WaitForFundingConfirmedStateSpec extends TestKitBaseClass with FixtureAnyF
|
|||
val bobInit = Init(Bob.channelParams.features)
|
||||
within(30 seconds) {
|
||||
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, bob2alice.ref, aliceInit, ChannelVersion.STANDARD)
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
alice2bob.expectMsgType[OpenChannel]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[AcceptChannel]
|
||||
|
@ -54,6 +57,7 @@ class WaitForFundingConfirmedStateSpec extends TestKitBaseClass with FixtureAnyF
|
|||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[FundingSigned]
|
||||
bob2alice.forward(alice)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
alice2blockchain.expectMsgType[WatchSpent]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
awaitCond(alice.stateName == WAIT_FOR_FUNDING_CONFIRMED)
|
||||
|
@ -159,7 +163,7 @@ class WaitForFundingConfirmedStateSpec extends TestKitBaseClass with FixtureAnyF
|
|||
// bob publishes his commitment tx
|
||||
val tx = bob.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, tx)
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
}
|
||||
|
@ -169,7 +173,7 @@ class WaitForFundingConfirmedStateSpec extends TestKitBaseClass with FixtureAnyF
|
|||
val tx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, Transaction(0, Nil, Nil, 0))
|
||||
alice2bob.expectMsgType[Error]
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
awaitCond(alice.stateName == ERR_INFORMATION_LEAK)
|
||||
}
|
||||
|
||||
|
@ -178,8 +182,8 @@ class WaitForFundingConfirmedStateSpec extends TestKitBaseClass with FixtureAnyF
|
|||
val tx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
alice ! Error(ByteVector32.Zeroes, "oops")
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsgType[PublishAsap] // claim-main-delayed
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
alice2blockchain.expectMsgType[PublishTx] // claim-main-delayed
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(tx))
|
||||
}
|
||||
|
||||
|
@ -197,8 +201,8 @@ class WaitForFundingConfirmedStateSpec extends TestKitBaseClass with FixtureAnyF
|
|||
val tx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
alice ! CMD_FORCECLOSE(sender.ref)
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsgType[PublishAsap] // claim-main-delayed
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
alice2blockchain.expectMsgType[PublishTx] // claim-main-delayed
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(tx))
|
||||
}
|
||||
|
||||
|
|
|
@ -20,6 +20,7 @@ import akka.testkit.{TestFSMRef, TestProbe}
|
|||
import fr.acinq.bitcoin.{ByteVector32, Transaction}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.channel.TxPublisher.{PublishRawTx, PublishTx}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.StateTestsBase
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
|
@ -46,7 +47,9 @@ class WaitForFundingLockedStateSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
val bobInit = Init(Bob.channelParams.features)
|
||||
within(30 seconds) {
|
||||
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Some(initialRelayFees), Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, bob2alice.ref, aliceInit, ChannelVersion.STANDARD)
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
alice2bob.expectMsgType[OpenChannel]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[AcceptChannel]
|
||||
|
@ -55,8 +58,10 @@ class WaitForFundingLockedStateSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[FundingSigned]
|
||||
bob2alice.forward(alice)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
alice2blockchain.expectMsgType[WatchSpent]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob2blockchain.expectMsgType[WatchSpent]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
awaitCond(alice.stateName == WAIT_FOR_FUNDING_CONFIRMED)
|
||||
|
@ -88,7 +93,7 @@ class WaitForFundingLockedStateSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
// bob publishes his commitment tx
|
||||
val tx = bob.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_LOCKED].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, tx)
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
}
|
||||
|
@ -98,8 +103,8 @@ class WaitForFundingLockedStateSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
val tx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_LOCKED].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, Transaction(0, Nil, Nil, 0))
|
||||
alice2bob.expectMsgType[Error]
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
awaitCond(alice.stateName == ERR_INFORMATION_LEAK)
|
||||
}
|
||||
|
||||
|
@ -108,8 +113,8 @@ class WaitForFundingLockedStateSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
val tx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_LOCKED].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
alice ! Error(ByteVector32.Zeroes, "oops")
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(tx))
|
||||
}
|
||||
|
||||
|
@ -127,8 +132,8 @@ class WaitForFundingLockedStateSpec extends TestKitBaseClass with FixtureAnyFunS
|
|||
val tx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_FUNDING_LOCKED].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
alice ! CMD_FORCECLOSE(sender.ref)
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(tx))
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,6 +27,7 @@ import fr.acinq.eclair._
|
|||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.blockchain.fee.{FeeratePerByte, FeeratePerKw, FeeratesPerKw}
|
||||
import fr.acinq.eclair.channel.Channel._
|
||||
import fr.acinq.eclair.channel.TxPublisher.{PublishRawTx, PublishTx}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.{StateTestsBase, StateTestsTags}
|
||||
import fr.acinq.eclair.crypto.Sphinx
|
||||
|
@ -443,8 +444,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray) === UnexpectedHtlcId(channelId(bob), expected = 4, actual = 42).getMessage)
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -458,8 +459,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(bob.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === bob.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -473,8 +474,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(bob.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === bob.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -489,9 +490,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(bob.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === bob.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
assert(bob2blockchain.expectMsgType[PublishAsap].tx === tx)
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
assert(bob2blockchain.expectMsgType[PublishTx].tx === tx)
|
||||
}
|
||||
|
||||
test("recv UpdateAddHtlc (insufficient funds w/ pending htlcs 1/2)") { f =>
|
||||
|
@ -506,8 +505,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(bob.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === bob.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -522,8 +521,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(bob.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === bob.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -536,8 +535,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(alice.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === alice.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -554,8 +553,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(bob.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === bob.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -868,8 +867,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(bob.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === bob.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -883,8 +882,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray).startsWith("invalid commitment signature"))
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -902,8 +901,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
bob ! badCommitSig
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray) === HtlcSigCountMismatch(channelId(bob), expected = 1, actual = 2).getMessage)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -921,8 +920,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
bob ! badCommitSig
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray).startsWith("invalid htlc signature"))
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -1033,8 +1032,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(alice.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === alice.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -1047,8 +1046,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(alice.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === alice.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -1287,8 +1286,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(alice.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === alice.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -1300,8 +1299,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(alice.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === alice.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -1318,9 +1317,9 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(alice.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === alice.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
alice2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -1488,9 +1487,9 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
val error = alice2bob.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray) === InvalidFailureCode(ByteVector32.Zeroes).getMessage)
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish)) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -1507,8 +1506,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(alice.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === alice.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -1520,8 +1519,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(alice.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === alice.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -1612,8 +1611,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(alice.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === alice.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -1629,8 +1628,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(bob.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === bob.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish)) // commit tx
|
||||
//bob2blockchain.expectMsgType[PublishAsap] // main delayed (removed because of the high fees)
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx) // commit tx
|
||||
//bob2blockchain.expectMsgType[PublishTx] // main delayed (removed because of the high fees)
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -1644,9 +1643,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(bob.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === bob.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
assert(bob2blockchain.expectMsgType[PublishAsap].tx === tx) // commit tx
|
||||
bob2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
assert(bob2blockchain.expectMsgType[PublishTx].tx === tx) // commit tx
|
||||
}
|
||||
|
||||
test("recv UpdateFee (local/remote feerates are too different)") { f =>
|
||||
|
@ -1665,8 +1662,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(bob.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === bob.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
bob2blockchain.expectMsg(PublishAsap(commitTx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === commitTx)
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -1686,9 +1683,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(bob.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === bob.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
assert(bob2blockchain.expectMsgType[PublishAsap].tx === commitTx)
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
assert(bob2blockchain.expectMsgType[PublishTx].tx === commitTx)
|
||||
}
|
||||
|
||||
test("recv UpdateFee (remote feerate is too small, anchor outputs)", Tag(StateTestsTags.AnchorOutputs)) { f =>
|
||||
|
@ -1707,9 +1702,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(bob.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === bob.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
assert(bob2blockchain.expectMsgType[PublishAsap].tx === commitTx)
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
assert(bob2blockchain.expectMsgType[PublishTx].tx === commitTx)
|
||||
}
|
||||
|
||||
test("recv UpdateFee (remote feerate is too small)") { f =>
|
||||
|
@ -1724,8 +1717,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(bob.stateName == CLOSING)
|
||||
// channel should be advertised as down
|
||||
assert(channelUpdateListener.expectMsgType[LocalChannelDown].channelId === bob.stateData.asInstanceOf[DATA_CLOSING].channelId)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -1900,8 +1893,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
// actual test begins
|
||||
bob ! Shutdown(ByteVector32.Zeroes, TestConstants.Alice.channelParams.defaultFinalScriptPubKey)
|
||||
bob2alice.expectMsgType[Error]
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
}
|
||||
|
@ -1938,8 +1931,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
import f._
|
||||
bob ! Shutdown(ByteVector32.Zeroes, hex"00112233445566778899")
|
||||
bob2alice.expectMsgType[Error]
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
}
|
||||
|
@ -1954,8 +1947,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
// actual test begins
|
||||
bob ! Shutdown(ByteVector32.Zeroes, hex"00112233445566778899")
|
||||
bob2alice.expectMsgType[Error]
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
}
|
||||
|
@ -2054,10 +2047,10 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
val aliceCommitTx = initialState.commitments.localCommit.publishableTxs.commitTx.tx
|
||||
alice ! CurrentBlockCount(400145)
|
||||
alice2blockchain.expectMsg(PublishAsap(aliceCommitTx, PublishStrategy.JustPublish))
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === aliceCommitTx)
|
||||
|
||||
alice2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout
|
||||
alice2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout
|
||||
val watch = alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
assert(watch.event === BITCOIN_TX_CONFIRMED(aliceCommitTx))
|
||||
}
|
||||
|
@ -2088,9 +2081,9 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
assert(isFatal)
|
||||
assert(err.isInstanceOf[HtlcsWillTimeoutUpstream])
|
||||
|
||||
bob2blockchain.expectMsg(PublishAsap(initialCommitTx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
assert(bob2blockchain.expectMsgType[PublishAsap].tx.txOut === htlcSuccessTx.txOut)
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === initialCommitTx)
|
||||
bob2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
assert(bob2blockchain.expectMsgType[PublishTx].tx.txOut === htlcSuccessTx.txOut)
|
||||
assert(bob2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(initialCommitTx))
|
||||
alice2blockchain.expectNoMsg(500 millis)
|
||||
}
|
||||
|
@ -2121,9 +2114,9 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
assert(isFatal)
|
||||
assert(err.isInstanceOf[HtlcsWillTimeoutUpstream])
|
||||
|
||||
bob2blockchain.expectMsg(PublishAsap(initialCommitTx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
assert(bob2blockchain.expectMsgType[PublishAsap].tx.txOut === htlcSuccessTx.txOut)
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === initialCommitTx)
|
||||
bob2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
assert(bob2blockchain.expectMsgType[PublishTx].tx.txOut === htlcSuccessTx.txOut)
|
||||
assert(bob2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(initialCommitTx))
|
||||
alice2blockchain.expectNoMsg(500 millis)
|
||||
}
|
||||
|
@ -2158,9 +2151,9 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
assert(isFatal)
|
||||
assert(err.isInstanceOf[HtlcsWillTimeoutUpstream])
|
||||
|
||||
bob2blockchain.expectMsg(PublishAsap(initialCommitTx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
assert(bob2blockchain.expectMsgType[PublishAsap].tx.txOut === htlcSuccessTx.txOut)
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === initialCommitTx)
|
||||
bob2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
assert(bob2blockchain.expectMsgType[PublishTx].tx.txOut === htlcSuccessTx.txOut)
|
||||
assert(bob2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(initialCommitTx))
|
||||
alice2blockchain.expectNoMsg(500 millis)
|
||||
}
|
||||
|
@ -2213,8 +2206,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
val event = CurrentFeerates(FeeratesPerKw.single(FeeratePerKw(14000 sat)))
|
||||
bob ! event
|
||||
bob2alice.expectMsgType[Error]
|
||||
bob2blockchain.expectMsgType[PublishAsap] // commit tx
|
||||
bob2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
bob2blockchain.expectMsgType[PublishTx] // commit tx
|
||||
bob2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
}
|
||||
|
@ -2237,9 +2230,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
val event = CurrentFeerates(FeeratesPerKw.single(TestConstants.anchorOutputsFeeratePerKw * 2))
|
||||
bob ! event
|
||||
bob2alice.expectMsgType[Error]
|
||||
bob2blockchain.expectMsgType[PublishAsap] // commit tx
|
||||
bob2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
bob2blockchain.expectMsgType[PublishTx] // commit tx
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
}
|
||||
|
||||
|
@ -2254,8 +2245,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
// when we try to add an HTLC, we still disagree on the feerate so we close
|
||||
alice2bob.send(bob, UpdateAddHtlc(ByteVector32.Zeroes, 0, 2500000 msat, randomBytes32, CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket))
|
||||
bob2alice.expectMsgType[Error]
|
||||
bob2blockchain.expectMsgType[PublishAsap] // commit tx
|
||||
bob2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
bob2blockchain.expectMsgType[PublishTx] // commit tx
|
||||
bob2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
}
|
||||
|
@ -2289,7 +2280,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, bobCommitTx)
|
||||
|
||||
// in response to that, alice publishes its claim txs
|
||||
val claimTxs = for (_ <- 0 until 4) yield alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val claimTxs = for (_ <- 0 until 4) yield alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val claimMain = claimTxs(0)
|
||||
// in addition to its main output, alice can only claim 3 out of 4 htlcs, she can't do anything regarding the htlc sent by bob for which she does not have the preimage
|
||||
val amountClaimed = (for (claimHtlcTx <- claimTxs) yield {
|
||||
|
@ -2359,7 +2350,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, bobCommitTx)
|
||||
|
||||
// in response to that, alice publishes its claim txs
|
||||
val claimTxs = for (i <- 0 until 3) yield alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val claimTxs = for (i <- 0 until 3) yield alice2blockchain.expectMsgType[PublishTx].tx
|
||||
// in addition to its main output, alice can only claim 2 out of 3 htlcs, she can't do anything regarding the htlc sent by bob for which she does not have the preimage
|
||||
val amountClaimed = (for (claimHtlcTx <- claimTxs) yield {
|
||||
assert(claimHtlcTx.txIn.size == 1)
|
||||
|
@ -2414,9 +2405,9 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, revokedTx)
|
||||
alice2bob.expectMsgType[Error]
|
||||
|
||||
val mainTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val mainPenaltyTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val htlcPenaltyTxs = for (_ <- 0 until 4) yield alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val mainTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val mainPenaltyTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val htlcPenaltyTxs = for (_ <- 0 until 4) yield alice2blockchain.expectMsgType[PublishTx].tx
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event == BITCOIN_TX_CONFIRMED(revokedTx))
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event == BITCOIN_TX_CONFIRMED(mainTx))
|
||||
assert(alice2blockchain.expectMsgType[WatchSpent].event === BITCOIN_OUTPUT_SPENT) // main-penalty
|
||||
|
@ -2478,9 +2469,9 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, revokedTx)
|
||||
alice2bob.expectMsgType[Error]
|
||||
|
||||
val mainTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val mainPenaltyTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val htlcPenaltyTxs = for (_ <- 0 until 2) yield alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val mainTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val mainPenaltyTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val htlcPenaltyTxs = for (_ <- 0 until 2) yield alice2blockchain.expectMsgType[PublishTx].tx
|
||||
// let's make sure that htlc-penalty txs each spend a different output
|
||||
assert(htlcPenaltyTxs.map(_.txIn.head.outPoint.index).toSet.size === htlcPenaltyTxs.size)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event == BITCOIN_TX_CONFIRMED(revokedTx))
|
||||
|
@ -2519,7 +2510,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
// an error occurs and alice publishes her commit tx
|
||||
val aliceCommitTx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
alice ! Error(ByteVector32.Zeroes, "oops")
|
||||
alice2blockchain.expectMsg(PublishAsap(aliceCommitTx, PublishStrategy.JustPublish))
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === aliceCommitTx)
|
||||
assert(aliceCommitTx.txOut.size == 6) // two main outputs and 4 pending htlcs
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
assert(alice.stateData.asInstanceOf[DATA_CLOSING].localCommitPublished.isDefined)
|
||||
|
@ -2535,10 +2526,10 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
// - 1 tx to claim the main delayed output
|
||||
// - 3 txs for each htlc
|
||||
// NB: 3rd-stage txs will only be published once the htlc txs confirm
|
||||
val claimMain = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val htlcTx1 = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val htlcTx2 = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val htlcTx3 = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val claimMain = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val htlcTx1 = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val htlcTx2 = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val htlcTx3 = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
// the main delayed output and htlc txs spend the commitment transaction
|
||||
Seq(claimMain, htlcTx1, htlcTx2, htlcTx3).foreach(tx => Transaction.correctlySpends(tx, aliceCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS))
|
||||
|
||||
|
@ -2555,7 +2546,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
alice ! WatchEventSpent(BITCOIN_OUTPUT_SPENT, htlcTimeoutTx)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(htlcTimeoutTx))
|
||||
alice ! WatchEventConfirmed(BITCOIN_TX_CONFIRMED(htlcTimeoutTx), 2701, 3, htlcTimeoutTx)
|
||||
val claimHtlcDelayedTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val claimHtlcDelayedTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
Transaction.correctlySpends(claimHtlcDelayedTx, htlcTimeoutTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(claimHtlcDelayedTx))
|
||||
})
|
||||
|
@ -2572,7 +2563,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
// an error occurs and alice publishes her commit tx
|
||||
val bobCommitTx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
bob ! Error(ByteVector32.Zeroes, "oops")
|
||||
bob2blockchain.expectMsg(PublishAsap(bobCommitTx, PublishStrategy.JustPublish))
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === bobCommitTx)
|
||||
assert(bobCommitTx.txOut.size == 1) // only one main output
|
||||
alice2blockchain.expectNoMsg(1 second)
|
||||
|
||||
|
|
|
@ -23,6 +23,7 @@ import fr.acinq.bitcoin.{ByteVector32, ScriptFlags, Transaction}
|
|||
import fr.acinq.eclair.TestConstants.{Alice, Bob, TestFeeEstimator}
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratesPerKw
|
||||
import fr.acinq.eclair.channel.TxPublisher.{PublishRawTx, PublishTx}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.StateTestsBase
|
||||
import fr.acinq.eclair.router.Announcements
|
||||
|
@ -273,7 +274,7 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, bobCommitTx)
|
||||
|
||||
// alice is able to claim its main output
|
||||
val claimMainOutput = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val claimMainOutput = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
Transaction.correctlySpends(claimMainOutput, bobCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
}
|
||||
|
||||
|
@ -318,7 +319,7 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, bobCommitTx)
|
||||
|
||||
// alice is able to claim its main output
|
||||
val claimMainOutput = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val claimMainOutput = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
Transaction.correctlySpends(claimMainOutput, bobCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
}
|
||||
|
||||
|
@ -338,8 +339,8 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
// alice then finds out bob is lying
|
||||
bob2alice.send(alice, invalidReestablish)
|
||||
val error = alice2bob.expectMsgType[Error]
|
||||
assert(alice2blockchain.expectMsgType[PublishAsap].tx === aliceCommitTx)
|
||||
val claimMainOutput = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
assert(alice2blockchain.expectMsgType[PublishTx].tx === aliceCommitTx)
|
||||
val claimMainOutput = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
Transaction.correctlySpends(claimMainOutput, aliceCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
assert(error === Error(channelId(alice), InvalidRevokedCommitProof(channelId(alice), 0, 42, invalidReestablish.yourLastPerCommitmentSecret).getMessage))
|
||||
}
|
||||
|
@ -482,15 +483,15 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
assert(isFatal)
|
||||
assert(err.isInstanceOf[HtlcsWillTimeoutUpstream])
|
||||
|
||||
bob2blockchain.expectMsg(PublishAsap(initialCommitTx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === initialCommitTx)
|
||||
bob2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
assert(bob2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(initialCommitTx))
|
||||
bob2blockchain.expectMsgType[WatchConfirmed] // main delayed
|
||||
bob2blockchain.expectMsgType[WatchSpent] // htlc
|
||||
|
||||
bob2blockchain.expectMsg(PublishAsap(initialCommitTx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
assert(bob2blockchain.expectMsgType[PublishAsap].tx.txOut === htlcSuccessTx.txOut)
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === initialCommitTx)
|
||||
bob2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
assert(bob2blockchain.expectMsgType[PublishTx].tx.txOut === htlcSuccessTx.txOut)
|
||||
assert(bob2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(initialCommitTx))
|
||||
bob2blockchain.expectMsgType[WatchConfirmed] // main delayed
|
||||
bob2blockchain.expectMsgType[WatchSpent] // htlc
|
||||
|
@ -545,7 +546,7 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
// alice is funder
|
||||
alice ! CurrentFeerates(networkFeerate)
|
||||
if (shouldClose) {
|
||||
alice2blockchain.expectMsg(PublishAsap(aliceCommitTx, PublishStrategy.JustPublish))
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === aliceCommitTx)
|
||||
} else {
|
||||
alice2blockchain.expectNoMsg()
|
||||
}
|
||||
|
@ -654,7 +655,7 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
// bob is fundee
|
||||
bob ! CurrentFeerates(networkFeerate)
|
||||
if (shouldClose) {
|
||||
bob2blockchain.expectMsg(PublishAsap(bobCommitTx, PublishStrategy.JustPublish))
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === bobCommitTx)
|
||||
} else {
|
||||
bob2blockchain.expectNoMsg()
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ import fr.acinq.bitcoin.Crypto.PrivateKey
|
|||
import fr.acinq.bitcoin.{ByteVector32, ByteVector64, Crypto, SatoshiLong, ScriptFlags, Transaction}
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.blockchain.fee.{FeeratePerKw, FeeratesPerKw}
|
||||
import fr.acinq.eclair.channel.TxPublisher.{PublishRawTx, PublishTx}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.{StateTestsBase, StateTestsTags}
|
||||
import fr.acinq.eclair.payment.OutgoingPacket.Upstream
|
||||
|
@ -183,10 +184,10 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
alice ! fulfill
|
||||
alice2bob.expectMsgType[Error]
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish)) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout 1
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout 2
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout 1
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout 2
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -196,10 +197,10 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
alice ! UpdateFulfillHtlc(ByteVector32.Zeroes, 42, ByteVector32.Zeroes)
|
||||
alice2bob.expectMsgType[Error]
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish)) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout 1
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout 2
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout 1
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout 2
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -287,10 +288,10 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
alice ! UpdateFailHtlc(ByteVector32.Zeroes, 42, ByteVector.fill(152)(0))
|
||||
alice2bob.expectMsgType[Error]
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish)) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout 1
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout 2
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout 1
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout 2
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -310,10 +311,10 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
val error = alice2bob.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray) === InvalidFailureCode(ByteVector32.Zeroes).getMessage)
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish)) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout 1
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout 2
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout 1
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout 2
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -378,8 +379,8 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
bob ! CommitSig(ByteVector32.Zeroes, ByteVector64.Zeroes, Nil)
|
||||
bob2alice.expectMsgType[Error]
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish)) // commit tx
|
||||
bob2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx) // commit tx
|
||||
bob2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -389,8 +390,8 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
bob ! CommitSig(ByteVector32.Zeroes, ByteVector64.Zeroes, Nil)
|
||||
bob2alice.expectMsgType[Error]
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish)) // commit tx
|
||||
bob2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx) // commit tx
|
||||
bob2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -445,9 +446,9 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
bob ! RevokeAndAck(ByteVector32.Zeroes, PrivateKey(randomBytes32), PrivateKey(randomBytes32).publicKey)
|
||||
bob2alice.expectMsgType[Error]
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish)) // commit tx
|
||||
bob2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
bob2blockchain.expectMsgType[PublishAsap] // htlc success
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx) // commit tx
|
||||
bob2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
bob2blockchain.expectMsgType[PublishTx] // htlc success
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -458,10 +459,10 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
alice ! RevokeAndAck(ByteVector32.Zeroes, PrivateKey(randomBytes32), PrivateKey(randomBytes32).publicKey)
|
||||
alice2bob.expectMsgType[Error]
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish)) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout 1
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout 2
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout 1
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout 2
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -546,10 +547,10 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
alice ! UpdateFee(ByteVector32.Zeroes, FeeratePerKw(12000 sat))
|
||||
alice2bob.expectMsgType[Error]
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish)) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout 1
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout 2
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout 1
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout 2
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -563,8 +564,8 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray) === CannotAffordFees(channelId(bob), missing = 72120000L sat, reserve = 20000L sat, fees = 72400000L sat).getMessage)
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish)) // commit tx
|
||||
//bob2blockchain.expectMsgType[PublishAsap] // main delayed (removed because of the high fees)
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx) // commit tx
|
||||
//bob2blockchain.expectMsgType[PublishTx] // main delayed (removed because of the high fees)
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -575,8 +576,8 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray) === "local/remote feerates are too different: remoteFeeratePerKw=65000 localFeeratePerKw=10000")
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish)) // commit tx
|
||||
bob2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx) // commit tx
|
||||
bob2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -587,8 +588,8 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray) === "remote fee rate is too small: remoteFeeratePerKw=252")
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish)) // commit tx
|
||||
bob2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx) // commit tx
|
||||
bob2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -614,10 +615,10 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
val initialState = alice.stateData.asInstanceOf[DATA_SHUTDOWN]
|
||||
val aliceCommitTx = initialState.commitments.localCommit.publishableTxs.commitTx.tx
|
||||
alice ! CurrentBlockCount(400145)
|
||||
alice2blockchain.expectMsg(PublishAsap(aliceCommitTx, PublishStrategy.JustPublish)) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout 1
|
||||
alice2blockchain.expectMsgType[PublishAsap] // htlc timeout 2
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === aliceCommitTx) // commit tx
|
||||
alice2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout 1
|
||||
alice2blockchain.expectMsgType[PublishTx] // htlc timeout 2
|
||||
val watch = alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
assert(watch.event === BITCOIN_TX_CONFIRMED(aliceCommitTx))
|
||||
}
|
||||
|
@ -665,8 +666,8 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
val event = CurrentFeerates(FeeratesPerKw.single(FeeratePerKw(1000 sat)))
|
||||
bob ! event
|
||||
bob2alice.expectMsgType[Error]
|
||||
bob2blockchain.expectMsgType[PublishAsap] // commit tx
|
||||
bob2blockchain.expectMsgType[PublishAsap] // main delayed
|
||||
bob2blockchain.expectMsgType[PublishTx] // commit tx
|
||||
bob2blockchain.expectMsgType[PublishTx] // main delayed
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
}
|
||||
|
@ -679,7 +680,7 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, bobCommitTx)
|
||||
|
||||
// in response to that, alice publishes its claim txs
|
||||
val claimTxs = for (_ <- 0 until 3) yield alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val claimTxs = for (_ <- 0 until 3) yield alice2blockchain.expectMsgType[PublishTx].tx
|
||||
// in addition to its main output, alice can only claim 2 out of 3 htlcs, she can't do anything regarding the htlc sent by bob for which she does not have the preimage
|
||||
val amountClaimed = (for (claimHtlcTx <- claimTxs) yield {
|
||||
assert(claimHtlcTx.txIn.size == 1)
|
||||
|
@ -726,7 +727,7 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, bobCommitTx)
|
||||
|
||||
// in response to that, alice publishes its claim txs
|
||||
val claimTxs = for (_ <- 0 until 2) yield alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val claimTxs = for (_ <- 0 until 2) yield alice2blockchain.expectMsgType[PublishTx].tx
|
||||
// in addition to its main output, alice can only claim 2 out of 3 htlcs, she can't do anything regarding the htlc sent by bob for which she does not have the preimage
|
||||
val amountClaimed = (for (claimHtlcTx <- claimTxs) yield {
|
||||
assert(claimHtlcTx.txIn.size == 1)
|
||||
|
@ -766,10 +767,10 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, revokedTx)
|
||||
alice2bob.expectMsgType[Error]
|
||||
|
||||
val mainTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val mainPenaltyTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val htlc1PenaltyTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val htlc2PenaltyTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val mainTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val mainPenaltyTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val htlc1PenaltyTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val htlc2PenaltyTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event == BITCOIN_TX_CONFIRMED(revokedTx))
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event == BITCOIN_TX_CONFIRMED(mainTx))
|
||||
assert(alice2blockchain.expectMsgType[WatchSpent].event === BITCOIN_OUTPUT_SPENT) // main-penalty
|
||||
|
@ -812,9 +813,9 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, revokedTx)
|
||||
alice2bob.expectMsgType[Error]
|
||||
|
||||
val mainTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val mainPenaltyTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val htlcPenaltyTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val mainTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val mainPenaltyTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val htlcPenaltyTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event == BITCOIN_TX_CONFIRMED(revokedTx))
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event == BITCOIN_TX_CONFIRMED(mainTx))
|
||||
assert(alice2blockchain.expectMsgType[WatchSpent].event === BITCOIN_OUTPUT_SPENT) // main-penalty
|
||||
|
@ -850,16 +851,16 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
val sender = TestProbe()
|
||||
alice ! CMD_FORCECLOSE(sender.ref)
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_FORCECLOSE]]
|
||||
alice2blockchain.expectMsg(PublishAsap(aliceCommitTx, PublishStrategy.JustPublish))
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === aliceCommitTx)
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
assert(alice.stateData.asInstanceOf[DATA_CLOSING].localCommitPublished.isDefined)
|
||||
val lcp = alice.stateData.asInstanceOf[DATA_CLOSING].localCommitPublished.get
|
||||
assert(lcp.htlcTxs.size === 2)
|
||||
assert(lcp.claimHtlcDelayedTxs.isEmpty) // 3rd-stage txs will be published once htlc txs confirm
|
||||
|
||||
val claimMain = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val htlc1 = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val htlc2 = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val claimMain = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val htlc1 = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val htlc2 = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
Seq(claimMain, htlc1, htlc2).foreach(tx => Transaction.correctlySpends(tx, aliceCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS))
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(aliceCommitTx))
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(claimMain))
|
||||
|
@ -872,7 +873,7 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
alice ! WatchEventSpent(BITCOIN_OUTPUT_SPENT, htlcTimeoutTx)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(htlcTimeoutTx))
|
||||
alice ! WatchEventConfirmed(BITCOIN_TX_CONFIRMED(htlcTimeoutTx), 2701, 3, htlcTimeoutTx)
|
||||
val claimHtlcDelayedTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val claimHtlcDelayedTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
Transaction.correctlySpends(claimHtlcDelayedTx, htlcTimeoutTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(claimHtlcDelayedTx))
|
||||
})
|
||||
|
@ -884,7 +885,7 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
import f._
|
||||
val aliceCommitTx = alice.stateData.asInstanceOf[DATA_SHUTDOWN].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
alice ! Error(ByteVector32.Zeroes, "oops")
|
||||
alice2blockchain.expectMsg(PublishAsap(aliceCommitTx, PublishStrategy.JustPublish))
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === aliceCommitTx)
|
||||
assert(aliceCommitTx.txOut.size == 4) // two main outputs and two htlcs
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
assert(alice.stateData.asInstanceOf[DATA_CLOSING].localCommitPublished.isDefined)
|
||||
|
@ -893,7 +894,7 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
|
|||
// - 1 tx to claim the main delayed output
|
||||
// - 2 txs for each htlc
|
||||
// NB: 3rd-stage txs will only be published once the htlc txs confirm
|
||||
val claimTxs = for (_ <- 0 until 3) yield alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val claimTxs = for (_ <- 0 until 3) yield alice2blockchain.expectMsgType[PublishTx].tx
|
||||
// the main delayed output and htlc txs spend the commitment transaction
|
||||
claimTxs.foreach(tx => Transaction.correctlySpends(tx, aliceCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS))
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(aliceCommitTx))
|
||||
|
|
|
@ -23,6 +23,7 @@ import fr.acinq.eclair.TestConstants.Bob
|
|||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.blockchain.fee.{FeeratePerKw, FeeratesPerKw}
|
||||
import fr.acinq.eclair.channel.Helpers.Closing
|
||||
import fr.acinq.eclair.channel.TxPublisher.{PublishRawTx, PublishTx}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.{StateTestsBase, StateTestsTags}
|
||||
import fr.acinq.eclair.transactions.Transactions
|
||||
|
@ -145,8 +146,8 @@ class NegotiatingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
val bobCloseFee = bob2alice.expectMsgType[ClosingSigned].feeSatoshis
|
||||
assert(aliceCloseFee === bobCloseFee)
|
||||
bob2alice.forward(alice)
|
||||
val mutualCloseTxAlice = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val mutualCloseTxBob = bob2blockchain.expectMsgType[PublishAsap].tx
|
||||
val mutualCloseTxAlice = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val mutualCloseTxBob = bob2blockchain.expectMsgType[PublishTx].tx
|
||||
assert(mutualCloseTxAlice === mutualCloseTxBob)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(mutualCloseTxAlice))
|
||||
assert(bob2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(mutualCloseTxBob))
|
||||
|
@ -162,8 +163,8 @@ class NegotiatingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
sender.send(bob, aliceCloseSig.copy(feeSatoshis = 99000 sat)) // sig doesn't matter, it is checked later
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray).startsWith("invalid close fee: fee_satoshis=Satoshi(99000)"))
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -174,8 +175,8 @@ class NegotiatingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
bob ! aliceCloseSig.copy(signature = ByteVector64.Zeroes)
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(new String(error.data.toArray).startsWith("invalid close signature"))
|
||||
bob2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
assert(bob2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
}
|
||||
|
||||
|
@ -196,10 +197,10 @@ class NegotiatingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
// bob publishes the mutual close and alice is notified that the funding tx has been spent
|
||||
// actual test starts here
|
||||
assert(alice.stateName == NEGOTIATING)
|
||||
val mutualCloseTx = bob2blockchain.expectMsgType[PublishAsap].tx
|
||||
val mutualCloseTx = bob2blockchain.expectMsgType[PublishTx].tx
|
||||
assert(bob2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(mutualCloseTx))
|
||||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, mutualCloseTx)
|
||||
alice2blockchain.expectMsg(PublishAsap(mutualCloseTx, PublishStrategy.JustPublish))
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === mutualCloseTx)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === mutualCloseTx.txid)
|
||||
alice2blockchain.expectNoMsg(100 millis)
|
||||
assert(alice.stateName == CLOSING)
|
||||
|
@ -219,7 +220,7 @@ class NegotiatingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
val Right(bobClosingTx) = Closing.checkClosingSignature(Bob.channelKeyManager, d.commitments, d.localShutdown.scriptPubKey, d.remoteShutdown.scriptPubKey, aliceClose1.feeSatoshis, aliceClose1.signature)
|
||||
|
||||
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, bobClosingTx.tx)
|
||||
alice2blockchain.expectMsg(PublishAsap(bobClosingTx.tx, PublishStrategy.JustPublish))
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === bobClosingTx.tx)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === bobClosingTx.tx.txid)
|
||||
alice2blockchain.expectNoMsg(100 millis)
|
||||
assert(alice.stateName == CLOSING)
|
||||
|
@ -237,8 +238,8 @@ class NegotiatingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
|
|||
val tx = alice.stateData.asInstanceOf[DATA_NEGOTIATING].commitments.localCommit.publishableTxs.commitTx.tx
|
||||
alice ! Error(ByteVector32.Zeroes, "oops")
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2blockchain.expectMsg(PublishAsap(tx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === tx)
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(tx))
|
||||
}
|
||||
|
||||
|
|
|
@ -23,6 +23,7 @@ import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
|||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.blockchain.fee.{FeeratePerKw, FeeratesPerKw}
|
||||
import fr.acinq.eclair.channel.Helpers.Closing
|
||||
import fr.acinq.eclair.channel.TxPublisher.{PublishRawTx, PublishTx}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.{StateTestsBase, StateTestsTags}
|
||||
import fr.acinq.eclair.payment._
|
||||
|
@ -65,7 +66,9 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
val aliceInit = Init(Alice.channelParams.features)
|
||||
val bobInit = Init(Bob.channelParams.features)
|
||||
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, bob2alice.ref, aliceInit, ChannelVersion.STANDARD)
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
alice2bob.expectMsgType[OpenChannel]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[AcceptChannel]
|
||||
|
@ -74,8 +77,10 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[FundingSigned]
|
||||
bob2alice.forward(alice)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
alice2blockchain.expectMsgType[WatchSpent]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob2blockchain.expectMsgType[WatchSpent]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
awaitCond(alice.stateName == WAIT_FOR_FUNDING_CONFIRMED)
|
||||
|
@ -132,8 +137,8 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
val sender = TestProbe()
|
||||
alice ! CMD_FORCECLOSE(sender.ref)
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
alice2blockchain.expectMsgType[PublishAsap] // claim-main-delayed
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
alice2blockchain.expectMsgType[PublishTx] // claim-main-delayed
|
||||
|
||||
// test starts here
|
||||
alice ! BITCOIN_FUNDING_PUBLISH_FAILED
|
||||
|
@ -146,8 +151,8 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
val sender = TestProbe()
|
||||
alice ! CMD_FORCECLOSE(sender.ref)
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
alice2blockchain.expectMsgType[PublishAsap] // claim-main-delayed
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
alice2blockchain.expectMsgType[PublishTx] // claim-main-delayed
|
||||
|
||||
// test starts here
|
||||
alice ! BITCOIN_FUNDING_TIMEOUT
|
||||
|
@ -162,8 +167,8 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
alice ! CMD_FORCECLOSE(sender.ref)
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2bob.expectMsgType[Error]
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
alice2blockchain.expectMsgType[PublishAsap] // claim-main-delayed
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
alice2blockchain.expectMsgType[PublishTx] // claim-main-delayed
|
||||
alice2blockchain.expectMsgType[WatchConfirmed] // commitment
|
||||
alice2blockchain.expectMsgType[WatchConfirmed] // claim-main-delayed
|
||||
|
||||
|
@ -181,15 +186,15 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
alice ! CMD_FORCECLOSE(sender.ref)
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2bob.expectMsgType[Error]
|
||||
alice2blockchain.expectMsgType[PublishAsap]
|
||||
alice2blockchain.expectMsgType[PublishAsap] // claim-main-delayed
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
alice2blockchain.expectMsgType[PublishTx] // claim-main-delayed
|
||||
alice2blockchain.expectMsgType[WatchConfirmed] // commitment
|
||||
alice2blockchain.expectMsgType[WatchConfirmed] // claim-main-delayed
|
||||
|
||||
// test starts here
|
||||
alice ! GetTxWithMetaResponse(fundingTx.txid, None, System.currentTimeMillis.milliseconds.toSeconds)
|
||||
alice2bob.expectNoMsg(200 millis)
|
||||
alice2blockchain.expectMsg(PublishAsap(fundingTx, PublishStrategy.JustPublish)) // we republish the funding tx
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === fundingTx) // we republish the funding tx
|
||||
assert(alice.stateName == CLOSING) // the above expectNoMsg will make us wait, so this checks that we are still in CLOSING
|
||||
}
|
||||
|
||||
|
@ -200,8 +205,8 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
bob ! CMD_FORCECLOSE(sender.ref)
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
bob2alice.expectMsgType[Error]
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
bob2blockchain.expectMsgType[PublishAsap] // claim-main-delayed
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[PublishTx] // claim-main-delayed
|
||||
bob2blockchain.expectMsgType[WatchConfirmed] // commitment
|
||||
bob2blockchain.expectMsgType[WatchConfirmed] // claim-main-delayed
|
||||
|
||||
|
@ -219,8 +224,8 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
bob ! CMD_FORCECLOSE(sender.ref)
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
bob2alice.expectMsgType[Error]
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
bob2blockchain.expectMsgType[PublishAsap] // claim-main-delayed
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[PublishTx] // claim-main-delayed
|
||||
bob2blockchain.expectMsgType[WatchConfirmed] // commitment
|
||||
bob2blockchain.expectMsgType[WatchConfirmed] // claim-main-delayed
|
||||
|
||||
|
@ -238,8 +243,8 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
bob ! CMD_FORCECLOSE(sender.ref)
|
||||
awaitCond(bob.stateName == CLOSING)
|
||||
bob2alice.expectMsgType[Error]
|
||||
bob2blockchain.expectMsgType[PublishAsap]
|
||||
bob2blockchain.expectMsgType[PublishAsap] // claim-main-delayed
|
||||
bob2blockchain.expectMsgType[PublishTx]
|
||||
bob2blockchain.expectMsgType[PublishTx] // claim-main-delayed
|
||||
bob2blockchain.expectMsgType[WatchConfirmed] // commitment
|
||||
bob2blockchain.expectMsgType[WatchConfirmed] // claim-main-delayed
|
||||
|
||||
|
@ -299,7 +304,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
// let's make alice publish this closing tx
|
||||
alice ! Error(ByteVector32.Zeroes, "")
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
alice2blockchain.expectMsg(PublishAsap(mutualCloseTx.tx, PublishStrategy.JustPublish))
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === mutualCloseTx.tx)
|
||||
assert(mutualCloseTx === alice.stateData.asInstanceOf[DATA_CLOSING].mutualClosePublished.last)
|
||||
|
||||
// actual test starts here
|
||||
|
@ -407,8 +412,8 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
relayerA.expectNoMsg(100 millis)
|
||||
|
||||
// We claim the htlc-delayed output now that the HTLC tx has been confirmed.
|
||||
val claimHtlcDelayedTx = alice2blockchain.expectMsgType[PublishAsap]
|
||||
assert(claimHtlcDelayedTx.strategy === PublishStrategy.JustPublish)
|
||||
val claimHtlcDelayedTx = alice2blockchain.expectMsgType[PublishTx]
|
||||
assert(claimHtlcDelayedTx.isInstanceOf[PublishRawTx])
|
||||
Transaction.correctlySpends(claimHtlcDelayedTx.tx, Seq(htlcTimeoutTx), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
awaitCond(alice.stateData.asInstanceOf[DATA_CLOSING].localCommitPublished.get.claimHtlcDelayedTxs.length === 1)
|
||||
alice ! WatchEventConfirmed(BITCOIN_TX_CONFIRMED(claimHtlcDelayedTx.tx), 202, 0, claimHtlcDelayedTx.tx)
|
||||
|
@ -573,14 +578,15 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
val beforeRestart = alice.stateData.asInstanceOf[DATA_CLOSING]
|
||||
alice.setState(WAIT_FOR_INIT_INTERNAL, Nothing)
|
||||
alice ! INPUT_RESTORED(beforeRestart)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
|
||||
// the commit tx hasn't been confirmed yet, so we watch the funding output first
|
||||
assert(alice2blockchain.expectMsgType[WatchSpent].event === BITCOIN_FUNDING_SPENT)
|
||||
// then we should re-publish unconfirmed transactions
|
||||
assert(alice2blockchain.expectMsgType[PublishAsap].tx === closingState.commitTx)
|
||||
closingState.claimMainDelayedOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[PublishAsap].tx === claimMain.tx))
|
||||
assert(alice2blockchain.expectMsgType[PublishAsap].tx === htlcTimeoutTx.tx)
|
||||
assert(alice2blockchain.expectMsgType[PublishTx].tx === closingState.commitTx)
|
||||
closingState.claimMainDelayedOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[PublishTx].tx === claimMain.tx))
|
||||
assert(alice2blockchain.expectMsgType[PublishTx].tx === htlcTimeoutTx.tx)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === closingState.commitTx.txid)
|
||||
closingState.claimMainDelayedOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === claimMain.tx.txid))
|
||||
assert(alice2blockchain.expectMsgType[WatchSpent].outputIndex === htlcTimeoutTx.input.outPoint.index)
|
||||
|
@ -591,17 +597,18 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(alice.stateData.asInstanceOf[DATA_CLOSING].localCommitPublished.get.claimHtlcDelayedTxs.nonEmpty)
|
||||
val beforeSecondRestart = alice.stateData.asInstanceOf[DATA_CLOSING]
|
||||
val claimHtlcTimeoutTx = beforeSecondRestart.localCommitPublished.get.claimHtlcDelayedTxs.head
|
||||
assert(alice2blockchain.expectMsgType[PublishAsap].tx === claimHtlcTimeoutTx.tx)
|
||||
assert(alice2blockchain.expectMsgType[PublishTx].tx === claimHtlcTimeoutTx.tx)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === claimHtlcTimeoutTx.tx.txid)
|
||||
|
||||
// simulate another node restart
|
||||
alice.setState(WAIT_FOR_INIT_INTERNAL, Nothing)
|
||||
alice ! INPUT_RESTORED(beforeSecondRestart)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
|
||||
// we should re-publish unconfirmed transactions
|
||||
closingState.claimMainDelayedOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[PublishAsap].tx === claimMain.tx))
|
||||
assert(alice2blockchain.expectMsgType[PublishAsap].tx === claimHtlcTimeoutTx.tx)
|
||||
closingState.claimMainDelayedOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[PublishTx].tx === claimMain.tx))
|
||||
assert(alice2blockchain.expectMsgType[PublishTx].tx === claimHtlcTimeoutTx.tx)
|
||||
closingState.claimMainDelayedOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === claimMain.tx.txid))
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === claimHtlcTimeoutTx.tx.txid)
|
||||
}
|
||||
|
@ -772,10 +779,10 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
|
||||
// Alice receives the preimage for the first HTLC from downstream; she can now claim the corresponding HTLC output.
|
||||
alice ! CMD_FULFILL_HTLC(htlc1.id, r1, commit = true)
|
||||
alice2blockchain.expectMsg(PublishAsap(closingState.claimMainOutputTx.get.tx, PublishStrategy.JustPublish))
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === closingState.claimMainOutputTx.get.tx)
|
||||
val claimHtlcSuccessTx = getClaimHtlcSuccessTxs(alice.stateData.asInstanceOf[DATA_CLOSING].remoteCommitPublished.get).head.tx
|
||||
Transaction.correctlySpends(claimHtlcSuccessTx, bobCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
alice2blockchain.expectMsg(PublishAsap(claimHtlcSuccessTx, PublishStrategy.JustPublish))
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === claimHtlcSuccessTx)
|
||||
|
||||
// Alice resets watches on all relevant transactions.
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(bobCommitTx))
|
||||
|
@ -812,11 +819,12 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
val beforeRestart = alice.stateData.asInstanceOf[DATA_CLOSING]
|
||||
alice.setState(WAIT_FOR_INIT_INTERNAL, Nothing)
|
||||
alice ! INPUT_RESTORED(beforeRestart)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
|
||||
// we should re-publish unconfirmed transactions
|
||||
closingState.claimMainOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[PublishAsap].tx === claimMain.tx))
|
||||
assert(alice2blockchain.expectMsgType[PublishAsap].tx === htlcTimeoutTx.tx)
|
||||
closingState.claimMainOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[PublishTx].tx === claimMain.tx))
|
||||
assert(alice2blockchain.expectMsgType[PublishTx].tx === htlcTimeoutTx.tx)
|
||||
closingState.claimMainOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === claimMain.tx.txid))
|
||||
assert(alice2blockchain.expectMsgType[WatchSpent].outputIndex === htlcTimeoutTx.input.outPoint.index)
|
||||
}
|
||||
|
@ -940,11 +948,11 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
|
||||
// Alice receives the preimage for the first HTLC from downstream; she can now claim the corresponding HTLC output.
|
||||
alice ! CMD_FULFILL_HTLC(htlc1.id, r1, commit = true)
|
||||
alice2blockchain.expectMsg(PublishAsap(closingState.claimMainOutputTx.get.tx, PublishStrategy.JustPublish))
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === closingState.claimMainOutputTx.get.tx)
|
||||
val claimHtlcSuccessTx = getClaimHtlcSuccessTxs(alice.stateData.asInstanceOf[DATA_CLOSING].nextRemoteCommitPublished.get).head.tx
|
||||
Transaction.correctlySpends(claimHtlcSuccessTx, bobCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
alice2blockchain.expectMsg(PublishAsap(claimHtlcSuccessTx, PublishStrategy.JustPublish))
|
||||
alice2blockchain.expectMsg(PublishAsap(claimHtlcTimeoutTx, PublishStrategy.JustPublish))
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === claimHtlcSuccessTx)
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === claimHtlcTimeoutTx)
|
||||
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(bobCommitTx))
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(closingState.claimMainOutputTx.get.tx))
|
||||
|
@ -974,13 +982,14 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
val beforeRestart = alice.stateData.asInstanceOf[DATA_CLOSING]
|
||||
alice.setState(WAIT_FOR_INIT_INTERNAL, Nothing)
|
||||
alice ! INPUT_RESTORED(beforeRestart)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
|
||||
// the commit tx hasn't been confirmed yet, so we watch the funding output first
|
||||
assert(alice2blockchain.expectMsgType[WatchSpent].event === BITCOIN_FUNDING_SPENT)
|
||||
// then we should re-publish unconfirmed transactions
|
||||
closingState.claimMainOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[PublishAsap].tx === claimMain.tx))
|
||||
claimHtlcTimeoutTxs.foreach(claimHtlcTimeout => assert(alice2blockchain.expectMsgType[PublishAsap].tx === claimHtlcTimeout.tx))
|
||||
closingState.claimMainOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[PublishTx].tx === claimMain.tx))
|
||||
claimHtlcTimeoutTxs.foreach(claimHtlcTimeout => assert(alice2blockchain.expectMsgType[PublishTx].tx === claimHtlcTimeout.tx))
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === bobCommitTx.txid)
|
||||
closingState.claimMainOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === claimMain.tx.txid))
|
||||
claimHtlcTimeoutTxs.foreach(claimHtlcTimeout => assert(alice2blockchain.expectMsgType[WatchSpent].outputIndex === claimHtlcTimeout.input.outPoint.index))
|
||||
|
@ -1036,7 +1045,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
import f._
|
||||
val bobCommitTx = testFutureRemoteCommitTxConfirmed(f, ChannelVersion.STANDARD)
|
||||
// alice is able to claim its main output
|
||||
val claimMainTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val claimMainTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
Transaction.correctlySpends(claimMainTx, bobCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === bobCommitTx.txid)
|
||||
awaitCond(alice.stateData.asInstanceOf[DATA_CLOSING].futureRemoteCommitPublished.isDefined)
|
||||
|
@ -1063,7 +1072,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
import f._
|
||||
val bobCommitTx = testFutureRemoteCommitTxConfirmed(f, ChannelVersion.ANCHOR_OUTPUTS)
|
||||
// alice is able to claim its main output
|
||||
val claimMainTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val claimMainTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
Transaction.correctlySpends(claimMainTx, bobCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === bobCommitTx.txid)
|
||||
awaitCond(alice.stateData.asInstanceOf[DATA_CLOSING].futureRemoteCommitPublished.isDefined)
|
||||
|
@ -1088,7 +1097,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
awaitCond(alice.stateName == CLOSING)
|
||||
|
||||
// then we should claim our main output
|
||||
val claimMainTx = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val claimMainTx = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
Transaction.correctlySpends(claimMainTx, bobCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === bobCommitTx.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === claimMainTx.txid)
|
||||
|
@ -1183,10 +1192,10 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
|
||||
// alice publishes the penalty txs
|
||||
if (!channelVersion.paysDirectlyToWallet) {
|
||||
alice2blockchain.expectMsg(PublishAsap(rvk.claimMainOutputTx.get.tx, PublishStrategy.JustPublish))
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === rvk.claimMainOutputTx.get.tx)
|
||||
}
|
||||
alice2blockchain.expectMsg(PublishAsap(rvk.mainPenaltyTx.get.tx, PublishStrategy.JustPublish))
|
||||
assert(Set(alice2blockchain.expectMsgType[PublishAsap].tx, alice2blockchain.expectMsgType[PublishAsap].tx) === rvk.htlcPenaltyTxs.map(_.tx).toSet)
|
||||
assert(alice2blockchain.expectMsgType[PublishRawTx].tx === rvk.mainPenaltyTx.get.tx)
|
||||
assert(Set(alice2blockchain.expectMsgType[PublishTx].tx, alice2blockchain.expectMsgType[PublishTx].tx) === rvk.htlcPenaltyTxs.map(_.tx).toSet)
|
||||
for (penaltyTx <- penaltyTxs) {
|
||||
Transaction.correctlySpends(penaltyTx.tx, bobRevokedTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
}
|
||||
|
@ -1258,9 +1267,9 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
assert(alice.stateData.asInstanceOf[DATA_CLOSING].revokedCommitPublished.last.commitTx === revokedTx)
|
||||
|
||||
// alice publishes penalty txs
|
||||
val claimMain = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val mainPenalty = alice2blockchain.expectMsgType[PublishAsap].tx
|
||||
val htlcPenaltyTxs = (1 to htlcCount).map(_ => alice2blockchain.expectMsgType[PublishAsap].tx)
|
||||
val claimMain = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val mainPenalty = alice2blockchain.expectMsgType[PublishTx].tx
|
||||
val htlcPenaltyTxs = (1 to htlcCount).map(_ => alice2blockchain.expectMsgType[PublishTx].tx)
|
||||
(claimMain +: mainPenalty +: htlcPenaltyTxs).foreach(tx => Transaction.correctlySpends(tx, revokedTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS))
|
||||
|
||||
// alice watches confirmation for the outputs only her can claim
|
||||
|
@ -1303,14 +1312,15 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
val beforeRestart = alice.stateData.asInstanceOf[DATA_CLOSING]
|
||||
alice.setState(WAIT_FOR_INIT_INTERNAL, Nothing)
|
||||
alice ! INPUT_RESTORED(beforeRestart)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
|
||||
// the commit tx hasn't been confirmed yet, so we watch the funding output first
|
||||
assert(alice2blockchain.expectMsgType[WatchSpent].event === BITCOIN_FUNDING_SPENT)
|
||||
// then we should re-publish unconfirmed transactions
|
||||
rvk.claimMainOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[PublishAsap].tx === claimMain.tx))
|
||||
assert(alice2blockchain.expectMsgType[PublishAsap].tx === rvk.mainPenaltyTx.get.tx)
|
||||
rvk.htlcPenaltyTxs.foreach(htlcPenalty => assert(alice2blockchain.expectMsgType[PublishAsap].tx === htlcPenalty.tx))
|
||||
rvk.claimMainOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[PublishTx].tx === claimMain.tx))
|
||||
assert(alice2blockchain.expectMsgType[PublishTx].tx === rvk.mainPenaltyTx.get.tx)
|
||||
rvk.htlcPenaltyTxs.foreach(htlcPenalty => assert(alice2blockchain.expectMsgType[PublishTx].tx === htlcPenalty.tx))
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === bobRevokedTx.txid)
|
||||
rvk.claimMainOutputTx.foreach(claimMain => assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === claimMain.tx.txid))
|
||||
assert(alice2blockchain.expectMsgType[WatchSpent].outputIndex === rvk.mainPenaltyTx.get.input.outPoint.index)
|
||||
|
@ -1350,7 +1360,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
|
||||
// alice publishes the penalty txs and watches outputs
|
||||
val claimTxsCount = if (channelVersion.paysDirectlyToWallet) 5 else 6 // 2 main outputs and 4 htlcs
|
||||
(1 to claimTxsCount).foreach(_ => alice2blockchain.expectMsgType[PublishAsap])
|
||||
(1 to claimTxsCount).foreach(_ => alice2blockchain.expectMsgType[PublishTx])
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === rvk.commitTx.txid)
|
||||
if (!channelVersion.paysDirectlyToWallet) {
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === rvk.claimMainOutputTx.get.tx.txid)
|
||||
|
@ -1379,7 +1389,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
val claimHtlcSuccessPenalty1 = alice.stateData.asInstanceOf[DATA_CLOSING].revokedCommitPublished.head.claimHtlcDelayedPenaltyTxs.last
|
||||
Transaction.correctlySpends(claimHtlcSuccessPenalty1.tx, bobHtlcSuccessTx1.tx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === bobHtlcSuccessTx1.tx.txid)
|
||||
assert(alice2blockchain.expectMsgType[PublishAsap].tx === claimHtlcSuccessPenalty1.tx)
|
||||
assert(alice2blockchain.expectMsgType[PublishTx].tx === claimHtlcSuccessPenalty1.tx)
|
||||
val watchSpent1 = alice2blockchain.expectMsgType[WatchSpent]
|
||||
assert(watchSpent1.txId === bobHtlcSuccessTx1.tx.txid)
|
||||
assert(watchSpent1.outputIndex === claimHtlcSuccessPenalty1.input.outPoint.index)
|
||||
|
@ -1390,7 +1400,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
val claimHtlcTimeoutPenalty = alice.stateData.asInstanceOf[DATA_CLOSING].revokedCommitPublished.head.claimHtlcDelayedPenaltyTxs.last
|
||||
Transaction.correctlySpends(claimHtlcTimeoutPenalty.tx, bobHtlcTimeoutTx.tx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === bobHtlcTimeoutTx.tx.txid)
|
||||
assert(alice2blockchain.expectMsgType[PublishAsap].tx === claimHtlcTimeoutPenalty.tx)
|
||||
assert(alice2blockchain.expectMsgType[PublishTx].tx === claimHtlcTimeoutPenalty.tx)
|
||||
val watchSpent2 = alice2blockchain.expectMsgType[WatchSpent]
|
||||
assert(watchSpent2.txId === bobHtlcTimeoutTx.tx.txid)
|
||||
assert(watchSpent2.outputIndex === claimHtlcTimeoutPenalty.input.outPoint.index)
|
||||
|
@ -1405,7 +1415,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
assert(claimHtlcSuccessPenalty1.tx.txid != claimHtlcSuccessPenalty2.tx.txid)
|
||||
Transaction.correctlySpends(claimHtlcSuccessPenalty2.tx, bobHtlcSuccessTx2 :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
assert(alice2blockchain.expectMsgType[WatchConfirmed].txId === bobHtlcSuccessTx2.txid)
|
||||
assert(alice2blockchain.expectMsgType[PublishAsap].tx === claimHtlcSuccessPenalty2.tx)
|
||||
assert(alice2blockchain.expectMsgType[PublishTx].tx === claimHtlcSuccessPenalty2.tx)
|
||||
val watchSpent3 = alice2blockchain.expectMsgType[WatchSpent]
|
||||
assert(watchSpent3.txId === bobHtlcSuccessTx2.txid)
|
||||
assert(watchSpent3.outputIndex === claimHtlcSuccessPenalty2.input.outPoint.index)
|
||||
|
@ -1458,7 +1468,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
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
|
||||
(1 to 6).foreach(_ => alice2blockchain.expectMsgType[PublishTx]) // 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
|
||||
|
@ -1503,10 +1513,10 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
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]
|
||||
alice2blockchain.expectMsgType[PublishTx],
|
||||
alice2blockchain.expectMsgType[PublishTx],
|
||||
alice2blockchain.expectMsgType[PublishTx],
|
||||
alice2blockchain.expectMsgType[PublishTx]
|
||||
)
|
||||
assert(publishedPenaltyTxs.map(_.tx) === claimHtlcDelayedPenaltyTxs.map(_.tx).toSet)
|
||||
val watchedOutpoints = Seq(
|
||||
|
|
|
@ -23,6 +23,7 @@ import fr.acinq.eclair.TestConstants.{Alice, Bob, TestFeeEstimator}
|
|||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.blockchain.fee.{FeeratePerKw, FeeratesPerKw}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.StateTestsHelperMethods.FakeTxPublisherFactory
|
||||
import fr.acinq.eclair.payment.receive.{ForwardHandler, PaymentHandler}
|
||||
import fr.acinq.eclair.wire.protocol.Init
|
||||
import fr.acinq.eclair.{MilliSatoshiLong, TestKitBaseClass, TestUtils}
|
||||
|
@ -60,18 +61,24 @@ class RustyTestsSpec extends TestKitBaseClass with Matchers with FixtureAnyFunSu
|
|||
val relayer = paymentHandler
|
||||
val wallet = new TestWallet
|
||||
val feeEstimator = new TestFeeEstimator
|
||||
val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(Alice.nodeParams.copy(blockCount = blockCount, onChainFeeConf = Alice.nodeParams.onChainFeeConf.copy(feeEstimator = feeEstimator)), wallet, Bob.nodeParams.nodeId, alice2blockchain.ref, relayer), alicePeer.ref)
|
||||
val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(Bob.nodeParams.copy(blockCount = blockCount, onChainFeeConf = Bob.nodeParams.onChainFeeConf.copy(feeEstimator = feeEstimator)), wallet, Alice.nodeParams.nodeId, bob2blockchain.ref, relayer), bobPeer.ref)
|
||||
val aliceNodeParams = Alice.nodeParams.copy(blockCount = blockCount, onChainFeeConf = Alice.nodeParams.onChainFeeConf.copy(feeEstimator = feeEstimator))
|
||||
val bobNodeParams = Bob.nodeParams.copy(blockCount = blockCount, onChainFeeConf = Bob.nodeParams.onChainFeeConf.copy(feeEstimator = feeEstimator))
|
||||
val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(aliceNodeParams, wallet, Bob.nodeParams.nodeId, alice2blockchain.ref, relayer, FakeTxPublisherFactory(alice2blockchain)), alicePeer.ref)
|
||||
val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bobNodeParams, wallet, Alice.nodeParams.nodeId, bob2blockchain.ref, relayer, FakeTxPublisherFactory(bob2blockchain)), bobPeer.ref)
|
||||
val aliceInit = Init(Alice.channelParams.features)
|
||||
val bobInit = Init(Bob.channelParams.features)
|
||||
// alice and bob will both have 1 000 000 sat
|
||||
feeEstimator.setFeerate(FeeratesPerKw.single(FeeratePerKw(10000 sat)))
|
||||
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, 2000000 sat, 1000000000 msat, feeEstimator.getFeeratePerKw(target = 2), feeEstimator.getFeeratePerKw(target = 6), None, Alice.channelParams, pipe, bobInit, ChannelFlags.Empty, ChannelVersion.STANDARD)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, pipe, aliceInit, ChannelVersion.STANDARD)
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
pipe ! (alice, bob)
|
||||
within(30 seconds) {
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
alice2blockchain.expectMsgType[WatchSpent]
|
||||
alice2blockchain.expectMsgType[WatchConfirmed]
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
|
||||
bob2blockchain.expectMsgType[WatchSpent]
|
||||
bob2blockchain.expectMsgType[WatchConfirmed]
|
||||
awaitCond(alice.stateName == WAIT_FOR_FUNDING_CONFIRMED)
|
||||
|
|
|
@ -127,15 +127,15 @@ class TransactionsSpec extends AnyFunSuite with Logging {
|
|||
assert(claimP2WPKHOutputTx.fee >= claimP2WPKHOutputTx.minRelayFee)
|
||||
}
|
||||
{
|
||||
// ClaimHtlcDelayedTx
|
||||
// first we create a fake htlcSuccessOrTimeoutTx tx, containing only the output that will be spent by the ClaimDelayedOutputTx
|
||||
// HtlcDelayedTx
|
||||
// first we create a fake htlcSuccessOrTimeoutTx tx, containing only the output that will be spent by the 3rd-stage tx
|
||||
val pubKeyScript = write(pay2wsh(toLocalDelayed(localRevocationPriv.publicKey, toLocalDelay, localPaymentPriv.publicKey)))
|
||||
val htlcSuccessOrTimeoutTx = Transaction(version = 0, txIn = Nil, txOut = TxOut(20000 sat, pubKeyScript) :: Nil, lockTime = 0)
|
||||
val Right(claimHtlcDelayedTx) = makeClaimLocalDelayedOutputTx(htlcSuccessOrTimeoutTx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
val Right(htlcDelayedTx) = makeHtlcDelayedTx(htlcSuccessOrTimeoutTx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
// we use dummy signatures to compute the weight
|
||||
val weight = Transaction.weight(addSigs(claimHtlcDelayedTx, PlaceHolderSig).tx)
|
||||
assert(claimHtlcDelayedWeight == weight)
|
||||
assert(claimHtlcDelayedTx.fee >= claimHtlcDelayedTx.minRelayFee)
|
||||
val weight = Transaction.weight(addSigs(htlcDelayedTx, PlaceHolderSig).tx)
|
||||
assert(htlcDelayedWeight == weight)
|
||||
assert(htlcDelayedTx.fee >= htlcDelayedTx.minRelayFee)
|
||||
}
|
||||
{
|
||||
// MainPenaltyTx
|
||||
|
@ -317,13 +317,13 @@ class TransactionsSpec extends AnyFunSuite with Logging {
|
|||
}
|
||||
{
|
||||
// local spends delayed output of htlc1 timeout tx
|
||||
val Right(claimHtlcDelayed) = makeClaimLocalDelayedOutputTx(htlcTimeoutTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
val localSig = sign(claimHtlcDelayed, localDelayedPaymentPriv, TxOwner.Local, DefaultCommitmentFormat)
|
||||
val signedTx = addSigs(claimHtlcDelayed, localSig)
|
||||
val Right(htlcDelayed) = makeHtlcDelayedTx(htlcTimeoutTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
val localSig = sign(htlcDelayed, localDelayedPaymentPriv, TxOwner.Local, DefaultCommitmentFormat)
|
||||
val signedTx = addSigs(htlcDelayed, localSig)
|
||||
assert(checkSpendable(signedTx).isSuccess)
|
||||
// local can't claim delayed output of htlc3 timeout tx because it is below the dust limit
|
||||
val claimHtlcDelayed1 = makeClaimLocalDelayedOutputTx(htlcTimeoutTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
assert(claimHtlcDelayed1 === Left(OutputNotFound))
|
||||
val htlcDelayed1 = makeHtlcDelayedTx(htlcTimeoutTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
assert(htlcDelayed1 === Left(OutputNotFound))
|
||||
}
|
||||
{
|
||||
// remote spends local->remote htlc1/htlc3 output directly in case of success
|
||||
|
@ -347,13 +347,13 @@ class TransactionsSpec extends AnyFunSuite with Logging {
|
|||
}
|
||||
{
|
||||
// local spends delayed output of htlc2 success tx
|
||||
val Right(claimHtlcDelayed) = makeClaimLocalDelayedOutputTx(htlcSuccessTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
val localSig = sign(claimHtlcDelayed, localDelayedPaymentPriv, TxOwner.Local, DefaultCommitmentFormat)
|
||||
val signedTx = addSigs(claimHtlcDelayed, localSig)
|
||||
val Right(htlcDelayed) = makeHtlcDelayedTx(htlcSuccessTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
val localSig = sign(htlcDelayed, localDelayedPaymentPriv, TxOwner.Local, DefaultCommitmentFormat)
|
||||
val signedTx = addSigs(htlcDelayed, localSig)
|
||||
assert(checkSpendable(signedTx).isSuccess)
|
||||
// local can't claim delayed output of htlc4 success tx because it is below the dust limit
|
||||
val claimHtlcDelayed1 = makeClaimLocalDelayedOutputTx(htlcSuccessTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
assert(claimHtlcDelayed1 === Left(AmountBelowDustLimit))
|
||||
val htlcDelayed1 = makeHtlcDelayedTx(htlcSuccessTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
assert(htlcDelayed1 === Left(AmountBelowDustLimit))
|
||||
}
|
||||
{
|
||||
// local spends main delayed output
|
||||
|
@ -588,13 +588,13 @@ class TransactionsSpec extends AnyFunSuite with Logging {
|
|||
}
|
||||
{
|
||||
// local spends delayed output of htlc1 timeout tx
|
||||
val Right(claimHtlcDelayed) = makeClaimLocalDelayedOutputTx(htlcTimeoutTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
val localSig = sign(claimHtlcDelayed, localDelayedPaymentPriv, TxOwner.Local, AnchorOutputsCommitmentFormat)
|
||||
val signedTx = addSigs(claimHtlcDelayed, localSig)
|
||||
val Right(htlcDelayed) = makeHtlcDelayedTx(htlcTimeoutTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
val localSig = sign(htlcDelayed, localDelayedPaymentPriv, TxOwner.Local, AnchorOutputsCommitmentFormat)
|
||||
val signedTx = addSigs(htlcDelayed, localSig)
|
||||
assert(checkSpendable(signedTx).isSuccess)
|
||||
// local can't claim delayed output of htlc3 timeout tx because it is below the dust limit
|
||||
val claimHtlcDelayed1 = makeClaimLocalDelayedOutputTx(htlcTimeoutTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
assert(claimHtlcDelayed1 === Left(OutputNotFound))
|
||||
val htlcDelayed1 = makeHtlcDelayedTx(htlcTimeoutTxs(0).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
assert(htlcDelayed1 === Left(OutputNotFound))
|
||||
}
|
||||
{
|
||||
// local spends offered htlc with HTLC-success tx
|
||||
|
@ -617,11 +617,11 @@ class TransactionsSpec extends AnyFunSuite with Logging {
|
|||
}
|
||||
{
|
||||
// local spends delayed output of htlc2a and htlc2b success txs
|
||||
val Right(claimHtlcDelayedA) = makeClaimLocalDelayedOutputTx(htlcSuccessTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
val Right(claimHtlcDelayedB) = makeClaimLocalDelayedOutputTx(htlcSuccessTxs(2).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
for (claimHtlcDelayed <- Seq(claimHtlcDelayedA, claimHtlcDelayedB)) {
|
||||
val localSig = sign(claimHtlcDelayed, localDelayedPaymentPriv, TxOwner.Local, AnchorOutputsCommitmentFormat)
|
||||
val signedTx = addSigs(claimHtlcDelayed, localSig)
|
||||
val Right(htlcDelayedA) = makeHtlcDelayedTx(htlcSuccessTxs(1).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
val Right(htlcDelayedB) = makeHtlcDelayedTx(htlcSuccessTxs(2).tx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw)
|
||||
for (htlcDelayed <- Seq(htlcDelayedA, htlcDelayedB)) {
|
||||
val localSig = sign(htlcDelayed, localDelayedPaymentPriv, TxOwner.Local, AnchorOutputsCommitmentFormat)
|
||||
val signedTx = addSigs(htlcDelayed, localSig)
|
||||
assert(checkSpendable(signedTx).isSuccess)
|
||||
}
|
||||
// local can't claim delayed output of htlc4 success tx because it is below the dust limit
|
||||
|
|
Loading…
Add table
Reference in a new issue