Skip to content

Commit d43d06f

Browse files
authored
Rework TxPublisher (#1844)
Splt the TxPublisher in many smaller actors with clear responsibilities. Each tx publishing attempt is its own actor and watches the tx until it either confirms or becomes evicted, and reports the result to its parent. The TxPublisher (one per channel) orchestrates publishing attempts and will in the future decide to RBF txs based on deadline information.
1 parent afb1b41 commit d43d06f

38 files changed

+3066
-1271
lines changed

eclair-core/src/main/resources/reference.conf

+1
Original file line numberDiff line numberDiff line change
@@ -86,6 +86,7 @@ eclair {
8686
fulfill-safety-before-timeout-blocks = 24
8787
min-final-expiry-delta-blocks = 30 // Bolt 11 invoice's min_final_cltv_expiry; must be strictly greater than fulfill-safety-before-timeout-blocks
8888
max-block-processing-delay = 30 seconds // we add a random delay before processing blocks, capped at this value, to prevent herd effect
89+
max-tx-publish-retry-delay = 60 seconds // we add a random delay before retrying failed transaction publication
8990

9091
fee-base-msat = 1000
9192
fee-proportional-millionths = 100 // fee charged per transferred satoshi in millionths of a satoshi (100 = 0.01%)

eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala

+2
Original file line numberDiff line numberDiff line change
@@ -64,6 +64,7 @@ case class NodeParams(nodeKeyManager: NodeKeyManager,
6464
fulfillSafetyBeforeTimeout: CltvExpiryDelta,
6565
minFinalExpiryDelta: CltvExpiryDelta,
6666
maxBlockProcessingDelay: FiniteDuration,
67+
maxTxPublishRetryDelay: FiniteDuration,
6768
htlcMinimum: MilliSatoshi,
6869
toRemoteDelay: CltvExpiryDelta,
6970
maxToLocalDelay: CltvExpiryDelta,
@@ -339,6 +340,7 @@ object NodeParams extends Logging {
339340
fulfillSafetyBeforeTimeout = fulfillSafetyBeforeTimeout,
340341
minFinalExpiryDelta = minFinalExpiryDelta,
341342
maxBlockProcessingDelay = FiniteDuration(config.getDuration("max-block-processing-delay").getSeconds, TimeUnit.SECONDS),
343+
maxTxPublishRetryDelay = FiniteDuration(config.getDuration("max-tx-publish-retry-delay").getSeconds, TimeUnit.SECONDS),
342344
htlcMinimum = htlcMinimum,
343345
toRemoteDelay = CltvExpiryDelta(config.getInt("to-remote-delay-blocks")),
344346
maxToLocalDelay = CltvExpiryDelta(config.getInt("max-to-local-delay-blocks")),

eclair-core/src/main/scala/fr/acinq/eclair/blockchain/bitcoind/BitcoinCoreWallet.scala

+3-32
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ import fr.acinq.bitcoin.Crypto.PublicKey
2020
import fr.acinq.bitcoin._
2121
import fr.acinq.eclair.blockchain._
2222
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient.{FundTransactionOptions, FundTransactionResponse, SignTransactionResponse, toSatoshi}
23-
import fr.acinq.eclair.blockchain.bitcoind.rpc.{BitcoinJsonRPCClient, ExtendedBitcoinClient, JsonRPCError}
23+
import fr.acinq.eclair.blockchain.bitcoind.rpc.{BitcoinJsonRPCClient, ExtendedBitcoinClient}
2424
import fr.acinq.eclair.blockchain.fee.{FeeratePerKB, FeeratePerKw}
2525
import fr.acinq.eclair.transactions.Transactions
2626
import grizzled.slf4j.Logging
@@ -61,7 +61,7 @@ class BitcoinCoreWallet(rpcClient: BitcoinJsonRPCClient)(implicit ec: ExecutionC
6161
val f = signTransaction(tx)
6262
// if signature fails (e.g. because wallet is encrypted) we need to unlock the utxos
6363
f.recoverWith { case _ =>
64-
unlockOutpoints(tx.txIn.map(_.outPoint))
64+
bitcoinClient.unlockOutpoints(tx.txIn.map(_.outPoint))
6565
.recover { case t: Throwable => // no-op, just add a log in case of failure
6666
logger.warn(s"Cannot unlock failed transaction's UTXOs txid=${tx.txid}", t)
6767
t
@@ -152,41 +152,12 @@ class BitcoinCoreWallet(rpcClient: BitcoinJsonRPCClient)(implicit ec: ExecutionC
152152
}
153153
}
154154

155-
override def rollback(tx: Transaction): Future[Boolean] = unlockOutpoints(tx.txIn.map(_.outPoint)) // we unlock all utxos used by the tx
155+
override def rollback(tx: Transaction): Future[Boolean] = bitcoinClient.unlockOutpoints(tx.txIn.map(_.outPoint)) // we unlock all utxos used by the tx
156156

157157
override def doubleSpent(tx: Transaction): Future[Boolean] = bitcoinClient.doubleSpent(tx)
158158

159-
/**
160-
* @param outPoints outpoints to unlock
161-
* @return true if all outpoints were successfully unlocked, false otherwise
162-
*/
163-
private def unlockOutpoints(outPoints: Seq[OutPoint])(implicit ec: ExecutionContext): Future[Boolean] = {
164-
// we unlock utxos one by one and not as a list as it would fail at the first utxo that is not actually locked and the rest would not be processed
165-
val futures = outPoints
166-
.map(outPoint => Utxo(outPoint.txid, outPoint.index))
167-
.map(utxo => rpcClient
168-
.invoke("lockunspent", true, List(utxo))
169-
.mapTo[JBool]
170-
.transformWith {
171-
case Success(JBool(result)) => Future.successful(result)
172-
case Failure(JsonRPCError(error)) if error.message.contains("expected locked output") =>
173-
Future.successful(true) // we consider that the outpoint was successfully unlocked (since it was not locked to begin with)
174-
case Failure(t) =>
175-
logger.warn(s"Cannot unlock utxo=$utxo", t)
176-
Future.successful(false)
177-
})
178-
val future = Future.sequence(futures)
179-
// return true if all outpoints were unlocked false otherwise
180-
future.map(_.forall(b => b))
181-
}
182-
183159
}
184160

185161
object BitcoinCoreWallet {
186-
187-
// @formatter:off
188-
case class Utxo(txid: ByteVector32, vout: Long)
189162
case class WalletTransaction(address: String, amount: Satoshi, fees: Satoshi, blockHash: ByteVector32, confirmations: Long, txid: ByteVector32, timestamp: Long)
190-
// @formatter:on
191-
192163
}

eclair-core/src/main/scala/fr/acinq/eclair/blockchain/bitcoind/ZmqWatcher.scala

+3-4
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,6 @@ import fr.acinq.eclair.blockchain.Monitoring.Metrics
2424
import fr.acinq.eclair.blockchain._
2525
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient
2626
import fr.acinq.eclair.blockchain.watchdogs.BlockchainWatchdog
27-
import fr.acinq.eclair.channel.TxPublisher.PublishTx
2827
import fr.acinq.eclair.wire.protocol.ChannelAnnouncement
2928
import fr.acinq.eclair.{KamonExt, ShortChannelId}
3029
import org.json4s.JsonAST._
@@ -149,8 +148,8 @@ object ZmqWatcher {
149148
case class WatchTxConfirmed(replyTo: ActorRef[WatchTxConfirmedTriggered], txId: ByteVector32, minDepth: Long) extends WatchConfirmed[WatchTxConfirmedTriggered]
150149
case class WatchTxConfirmedTriggered(blockHeight: Int, txIndex: Int, tx: Transaction) extends WatchConfirmedTriggered
151150

152-
case class WatchParentTxConfirmed(replyTo: ActorRef[WatchParentTxConfirmedTriggered], txId: ByteVector32, minDepth: Long, childTx: PublishTx) extends WatchConfirmed[WatchParentTxConfirmedTriggered]
153-
case class WatchParentTxConfirmedTriggered(blockHeight: Int, txIndex: Int, tx: Transaction, childTx: PublishTx) extends WatchConfirmedTriggered
151+
case class WatchParentTxConfirmed(replyTo: ActorRef[WatchParentTxConfirmedTriggered], txId: ByteVector32, minDepth: Long) extends WatchConfirmed[WatchParentTxConfirmedTriggered]
152+
case class WatchParentTxConfirmedTriggered(blockHeight: Int, txIndex: Int, tx: Transaction) extends WatchConfirmedTriggered
154153

155154
// TODO: not implemented yet: notify me if confirmation number gets below minDepth?
156155
case class WatchFundingLost(replyTo: ActorRef[WatchFundingLostTriggered], txId: ByteVector32, minDepth: Long) extends Watch[WatchFundingLostTriggered]
@@ -378,7 +377,7 @@ private class ZmqWatcher(chainHash: ByteVector32, blockCount: AtomicLong, client
378377
case w: WatchFundingConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchFundingConfirmedTriggered(height, index, tx))
379378
case w: WatchFundingDeeplyBuried => context.self ! TriggerEvent(w.replyTo, w, WatchFundingDeeplyBuriedTriggered(height, index, tx))
380379
case w: WatchTxConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchTxConfirmedTriggered(height, index, tx))
381-
case w: WatchParentTxConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchParentTxConfirmedTriggered(height, index, tx, w.childTx))
380+
case w: WatchParentTxConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchParentTxConfirmedTriggered(height, index, tx))
382381
}
383382
}
384383
}

eclair-core/src/main/scala/fr/acinq/eclair/blockchain/bitcoind/rpc/ExtendedBitcoinClient.scala

+32-4
Original file line numberDiff line numberDiff line change
@@ -23,12 +23,13 @@ import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{GetTxWithMetaResponse, Ut
2323
import fr.acinq.eclair.blockchain.fee.{FeeratePerKB, FeeratePerKw}
2424
import fr.acinq.eclair.transactions.Transactions
2525
import fr.acinq.eclair.wire.protocol.ChannelAnnouncement
26+
import grizzled.slf4j.Logging
2627
import org.json4s.Formats
2728
import org.json4s.JsonAST._
2829
import scodec.bits.ByteVector
2930

3031
import scala.concurrent.{ExecutionContext, Future}
31-
import scala.util.Try
32+
import scala.util.{Failure, Success, Try}
3233

3334
/**
3435
* Created by PM on 26/04/2016.
@@ -39,7 +40,7 @@ import scala.util.Try
3940
* Note that all wallet utilities (signing transactions, setting fees, locking outputs, etc) can be found in
4041
* [[fr.acinq.eclair.blockchain.bitcoind.BitcoinCoreWallet]].
4142
*/
42-
class ExtendedBitcoinClient(val rpcClient: BitcoinJsonRPCClient) {
43+
class ExtendedBitcoinClient(val rpcClient: BitcoinJsonRPCClient) extends Logging {
4344

4445
import ExtendedBitcoinClient._
4546

@@ -151,6 +152,30 @@ class ExtendedBitcoinClient(val rpcClient: BitcoinJsonRPCClient) {
151152
getRawTransaction(tx.txid).map(_ => tx.txid).recoverWith { case _ => Future.failed(e) }
152153
}
153154

155+
/**
156+
* @param outPoints outpoints to unlock.
157+
* @return true if all outpoints were successfully unlocked, false otherwise.
158+
*/
159+
def unlockOutpoints(outPoints: Seq[OutPoint])(implicit ec: ExecutionContext): Future[Boolean] = {
160+
// we unlock utxos one by one and not as a list as it would fail at the first utxo that is not actually locked and the rest would not be processed
161+
val futures = outPoints
162+
.map(outPoint => Utxo(outPoint.txid, outPoint.index))
163+
.map(utxo => rpcClient
164+
.invoke("lockunspent", true, List(utxo))
165+
.mapTo[JBool]
166+
.transformWith {
167+
case Success(JBool(result)) => Future.successful(result)
168+
case Failure(JsonRPCError(error)) if error.message.contains("expected locked output") =>
169+
Future.successful(true) // we consider that the outpoint was successfully unlocked (since it was not locked to begin with)
170+
case Failure(t) =>
171+
logger.warn(s"cannot unlock utxo=$utxo:", t)
172+
Future.successful(false)
173+
})
174+
val future = Future.sequence(futures)
175+
// return true if all outpoints were unlocked false otherwise
176+
future.map(_.forall(b => b))
177+
}
178+
154179
def isTransactionOutputSpendable(txid: ByteVector32, outputIndex: Int, includeMempool: Boolean)(implicit ec: ExecutionContext): Future[Boolean] =
155180
for {
156181
json <- rpcClient.invoke("gettxout", txid, outputIndex, includeMempool)
@@ -206,8 +231,9 @@ class ExtendedBitcoinClient(val rpcClient: BitcoinJsonRPCClient) {
206231
def getMempool()(implicit ec: ExecutionContext): Future[Seq[Transaction]] =
207232
for {
208233
txids <- rpcClient.invoke("getrawmempool").map(json => json.extract[List[String]].map(ByteVector32.fromValidHex))
209-
txs <- Future.sequence(txids.map(getTransaction(_)))
210-
} yield txs
234+
// NB: if a transaction is evicted before we've called getTransaction, we need to ignore it instead of failing.
235+
txs <- Future.sequence(txids.map(getTransaction(_).map(Some(_)).recover { case _ => None }))
236+
} yield txs.flatten
211237

212238
def getMempoolTx(txid: ByteVector32)(implicit ec: ExecutionContext): Future[MempoolTx] = {
213239
rpcClient.invoke("getmempoolentry", txid).map(json => {
@@ -296,6 +322,8 @@ object ExtendedBitcoinClient {
296322
*/
297323
case class MempoolTx(txid: ByteVector32, vsize: Long, weight: Long, replaceable: Boolean, fees: Satoshi, ancestorCount: Int, ancestorFees: Satoshi, descendantCount: Int, descendantFees: Satoshi)
298324

325+
case class Utxo(txid: ByteVector32, vout: Long)
326+
299327
def toSatoshi(btcAmount: BigDecimal): Satoshi = Satoshi(btcAmount.bigDecimal.scaleByPowerOfTen(8).longValue)
300328

301329
}

eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala

+18-18
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,8 @@ import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient
3232
import fr.acinq.eclair.channel.Helpers.{Closing, Funding}
3333
import fr.acinq.eclair.channel.Monitoring.Metrics.ProcessMessage
3434
import fr.acinq.eclair.channel.Monitoring.{Metrics, Tags}
35-
import fr.acinq.eclair.channel.TxPublisher.{PublishRawTx, PublishTx, SetChannelId, SignAndPublishTx}
35+
import fr.acinq.eclair.channel.publish.TxPublisher
36+
import fr.acinq.eclair.channel.publish.TxPublisher.{PublishRawTx, PublishReplaceableTx, PublishTx, SetChannelId}
3637
import fr.acinq.eclair.crypto.ShaChain
3738
import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
3839
import fr.acinq.eclair.db.DbEventHandler.ChannelEvent.EventType
@@ -63,7 +64,7 @@ object Channel {
6364

6465
case class SimpleTxPublisherFactory(nodeParams: NodeParams, watcher: typed.ActorRef[ZmqWatcher.Command], bitcoinClient: ExtendedBitcoinClient) extends TxPublisherFactory {
6566
override def spawnTxPublisher(context: ActorContext, remoteNodeId: PublicKey): typed.ActorRef[TxPublisher.Command] = {
66-
context.spawn(Behaviors.supervise(TxPublisher(nodeParams, remoteNodeId, watcher, bitcoinClient)).onFailure(typed.SupervisorStrategy.restart), "tx-publisher")
67+
context.spawn(Behaviors.supervise(TxPublisher(nodeParams, remoteNodeId, TxPublisher.SimpleChildFactory(nodeParams, bitcoinClient, watcher))).onFailure(typed.SupervisorStrategy.restart), "tx-publisher")
6768
}
6869
}
6970

@@ -1380,7 +1381,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
13801381
}
13811382
val revokedCommitPublished1 = d.revokedCommitPublished.map { rev =>
13821383
val (rev1, penaltyTxs) = Closing.claimRevokedHtlcTxOutputs(keyManager, d.commitments, rev, tx, nodeParams.onChainFeeConf.feeEstimator)
1383-
penaltyTxs.foreach(claimTx => txPublisher ! PublishRawTx(claimTx))
1384+
penaltyTxs.foreach(claimTx => txPublisher ! PublishRawTx(claimTx, None))
13841385
penaltyTxs.foreach(claimTx => blockchain ! WatchOutputSpent(self, tx.txid, claimTx.input.outPoint.index.toInt, hints = Set(claimTx.tx.txid)))
13851386
rev1
13861387
}
@@ -1394,7 +1395,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
13941395
// If the tx is one of our HTLC txs, we now publish a 3rd-stage claim-htlc-tx that claims its output.
13951396
val (localCommitPublished1, claimHtlcTx_opt) = Closing.claimLocalCommitHtlcTxOutput(localCommitPublished, keyManager, d.commitments, tx, nodeParams.onChainFeeConf.feeEstimator, nodeParams.onChainFeeConf.feeTargets)
13961397
claimHtlcTx_opt.foreach(claimHtlcTx => {
1397-
txPublisher ! PublishRawTx(claimHtlcTx)
1398+
txPublisher ! PublishRawTx(claimHtlcTx, None)
13981399
blockchain ! WatchTxConfirmed(self, claimHtlcTx.tx.txid, nodeParams.minDepthBlocks)
13991400
})
14001401
Closing.updateLocalCommitPublished(localCommitPublished1, tx)
@@ -2046,7 +2047,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
20462047
case Some(fundingTx) =>
20472048
// if we are funder, we never give up
20482049
log.info(s"republishing the funding tx...")
2049-
txPublisher ! PublishRawTx(fundingTx, "funding-tx")
2050+
txPublisher ! PublishRawTx(fundingTx, fundingTx.txIn.head.outPoint, "funding-tx", None)
20502051
// we also check if the funding tx has been double-spent
20512052
checkDoubleSpent(fundingTx)
20522053
context.system.scheduler.scheduleOnce(1 day, blockchain.toClassic, GetTxWithMeta(self, txid))
@@ -2199,7 +2200,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
21992200
}
22002201

22012202
private def doPublish(closingTx: ClosingTx): Unit = {
2202-
txPublisher ! PublishRawTx(closingTx)
2203+
txPublisher ! PublishRawTx(closingTx, None)
22032204
blockchain ! WatchTxConfirmed(self, closingTx.tx.txid, nodeParams.minDepthBlocks)
22042205
}
22052206

@@ -2229,12 +2230,9 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
22292230
* This helper method will publish txs only if they haven't yet reached minDepth
22302231
*/
22312232
private def publishIfNeeded(txs: Iterable[PublishTx], irrevocablySpent: Map[OutPoint, Transaction]): Unit = {
2232-
val (skip, process) = txs.partition(publishTx => Closing.inputsAlreadySpent(publishTx.tx, irrevocablySpent))
2233-
process.foreach { publishTx =>
2234-
log.info(s"publishing txid=${publishTx.tx.txid}")
2235-
txPublisher ! publishTx
2236-
}
2237-
skip.foreach(publishTx => log.info(s"no need to republish txid=${publishTx.tx.txid}, it has already been confirmed"))
2233+
val (skip, process) = txs.partition(publishTx => Closing.inputAlreadySpent(publishTx.input, irrevocablySpent))
2234+
process.foreach { publishTx => txPublisher ! publishTx }
2235+
skip.foreach(publishTx => log.info("no need to republish tx spending {}:{}, it has already been confirmed", publishTx.input.txid, publishTx.input.index))
22382236
}
22392237

22402238
/**
@@ -2264,13 +2262,15 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
22642262
private def doPublish(localCommitPublished: LocalCommitPublished, commitments: Commitments): Unit = {
22652263
import localCommitPublished._
22662264

2265+
val commitInput = commitments.commitInput.outPoint
22672266
val publishQueue = commitments.commitmentFormat match {
22682267
case Transactions.DefaultCommitmentFormat =>
2269-
List(PublishRawTx(commitTx, "commit-tx")) ++ (claimMainDelayedOutputTx ++ htlcTxs.values.flatten ++ claimHtlcDelayedTxs).map(tx => PublishRawTx(tx))
2268+
val redeemableHtlcTxs = htlcTxs.values.flatten.map(tx => PublishRawTx(tx, Some(commitTx.txid)))
2269+
List(PublishRawTx(commitTx, commitInput, "commit-tx", None)) ++ (claimMainDelayedOutputTx.map(tx => PublishRawTx(tx, None)) ++ redeemableHtlcTxs ++ claimHtlcDelayedTxs.map(tx => PublishRawTx(tx, None)))
22702270
case Transactions.AnchorOutputsCommitmentFormat =>
2271-
val claimLocalAnchor = claimAnchorTxs.collect { case tx: Transactions.ClaimLocalAnchorOutputTx => SignAndPublishTx(tx, commitments) }
2272-
val redeemableHtlcTxs = htlcTxs.values.collect { case Some(tx) => SignAndPublishTx(tx, commitments) }
2273-
List(PublishRawTx(commitTx, "commit-tx")) ++ claimLocalAnchor ++ claimMainDelayedOutputTx.map(tx => PublishRawTx(tx)) ++ redeemableHtlcTxs ++ claimHtlcDelayedTxs.map(tx => PublishRawTx(tx))
2271+
val claimLocalAnchor = claimAnchorTxs.collect { case tx: Transactions.ClaimLocalAnchorOutputTx => PublishReplaceableTx(tx, commitments) }
2272+
val redeemableHtlcTxs = htlcTxs.values.collect { case Some(tx) => PublishReplaceableTx(tx, commitments) }
2273+
List(PublishRawTx(commitTx, commitInput, "commit-tx", None)) ++ claimLocalAnchor ++ claimMainDelayedOutputTx.map(tx => PublishRawTx(tx, None)) ++ redeemableHtlcTxs ++ claimHtlcDelayedTxs.map(tx => PublishRawTx(tx, None))
22742274
}
22752275
publishIfNeeded(publishQueue, irrevocablySpent)
22762276

@@ -2333,7 +2333,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
23332333
private def doPublish(remoteCommitPublished: RemoteCommitPublished): Unit = {
23342334
import remoteCommitPublished._
23352335

2336-
val publishQueue = (claimMainOutputTx ++ claimHtlcTxs.values.flatten).map(tx => PublishRawTx(tx))
2336+
val publishQueue = (claimMainOutputTx ++ claimHtlcTxs.values.flatten).map(tx => PublishRawTx(tx, None))
23372337
publishIfNeeded(publishQueue, irrevocablySpent)
23382338

23392339
// we watch:
@@ -2372,7 +2372,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
23722372
private def doPublish(revokedCommitPublished: RevokedCommitPublished): Unit = {
23732373
import revokedCommitPublished._
23742374

2375-
val publishQueue = (claimMainOutputTx ++ mainPenaltyTx ++ htlcPenaltyTxs ++ claimHtlcDelayedPenaltyTxs).map(tx => PublishRawTx(tx))
2375+
val publishQueue = (claimMainOutputTx ++ mainPenaltyTx ++ htlcPenaltyTxs ++ claimHtlcDelayedPenaltyTxs).map(tx => PublishRawTx(tx, None))
23762376
publishIfNeeded(publishQueue, irrevocablySpent)
23772377

23782378
// we watch:

eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala

-1
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,6 @@ import akka.actor.{ActorRef, PossiblyHarmful}
2020
import fr.acinq.bitcoin.Crypto.PublicKey
2121
import fr.acinq.bitcoin.{ByteVector32, DeterministicWallet, OutPoint, Satoshi, Transaction}
2222
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
23-
import fr.acinq.eclair.channel.TxPublisher.PublishTx
2423
import fr.acinq.eclair.payment.OutgoingPacket.Upstream
2524
import fr.acinq.eclair.router.Announcements
2625
import fr.acinq.eclair.transactions.CommitmentSpec

0 commit comments

Comments
 (0)