From 66a111f938bf918f70a1326002ca32bf16d1441f Mon Sep 17 00:00:00 2001 From: t-bast Date: Wed, 29 Jun 2022 09:58:20 +0200 Subject: [PATCH 01/11] Dual funding channel confirmation Once we've exchanged signatures for the funding tx, we wait for it to confirm. If an error occurs, we need to wait for the funding tx to be double-spent before we can consider the channel closed and safely forget it. --- .../acinq/eclair/balance/CheckBalance.scala | 2 + .../fr/acinq/eclair/channel/ChannelData.scala | 30 +- .../fr/acinq/eclair/channel/Commitments.scala | 12 +- .../fr/acinq/eclair/channel/Helpers.scala | 19 +- .../fr/acinq/eclair/channel/fsm/Channel.scala | 94 ++++-- .../channel/fsm/ChannelOpenDualFunded.scala | 172 +++++++++- .../channel/fsm/ChannelOpenSingleFunded.scala | 1 - .../channel/fsm/DualFundingHandlers.scala | 49 ++- .../eclair/channel/fsm/ErrorHandlers.scala | 41 ++- .../fr/acinq/eclair/db/DbEventHandler.scala | 2 +- .../acinq/eclair/json/JsonSerializers.scala | 13 +- .../channel/version3/ChannelCodecs3.scala | 67 +++- .../eclair/wire/protocol/CommonCodecs.scala | 2 + .../scala/fr/acinq/eclair/TestDatabases.scala | 3 +- .../blockchain/DummyOnChainWallet.scala | 3 +- .../ChannelStateTestsHelperMethods.scala | 111 ++++-- .../WaitForDualFundingCreatedStateSpec.scala | 44 +-- ...WaitForDualFundingConfirmedStateSpec.scala | 315 ++++++++++++++++++ .../c/WaitForDualFundingReadyStateSpec.scala | 223 +++++++++++++ .../channel/states/e/NormalStateSpec.scala | 21 +- .../eclair/integration/IntegrationSpec.scala | 4 + .../integration/PaymentIntegrationSpec.scala | 4 +- 22 files changed, 1090 insertions(+), 142 deletions(-) create mode 100644 eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala create mode 100644 eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/balance/CheckBalance.scala b/eclair-core/src/main/scala/fr/acinq/eclair/balance/CheckBalance.scala index 7d08a54d09..d47ede4820 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/balance/CheckBalance.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/balance/CheckBalance.scala @@ -202,6 +202,8 @@ object CheckBalance { .foldLeft(OffChainBalance()) { case (r, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) => r.modify(_.waitForFundingConfirmed).using(updateMainBalance(d.commitments.localCommit)) case (r, d: DATA_WAIT_FOR_CHANNEL_READY) => r.modify(_.waitForChannelReady).using(updateMainBalance(d.commitments.localCommit)) + case (r, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => r.modify(_.waitForFundingConfirmed).using(updateMainBalance(d.commitments.localCommit)) + case (r, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) => r.modify(_.waitForChannelReady).using(updateMainBalance(d.commitments.localCommit)) case (r, d: DATA_NORMAL) => r.modify(_.normal).using(updateMainAndHtlcBalance(d.commitments, knownPreimages)) case (r, d: DATA_SHUTDOWN) => r.modify(_.shutdown).using(updateMainAndHtlcBalance(d.commitments, knownPreimages)) case (r, d: DATA_NEGOTIATING) => r.modify(_.negotiating).using(updateMainBalance(d.commitments.localCommit)) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala index 5b27b730bc..0823908eb5 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala @@ -16,12 +16,11 @@ package fr.acinq.eclair.channel -import akka.actor.typed -import akka.actor.{ActorRef, PossiblyHarmful} +import akka.actor.{ActorRef, PossiblyHarmful, typed} import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.bitcoin.scalacompat.{ByteVector32, DeterministicWallet, OutPoint, Satoshi, Transaction} import fr.acinq.eclair.blockchain.fee.FeeratePerKw -import fr.acinq.eclair.channel.InteractiveTxBuilder.{InteractiveTxParams, SignedSharedTransaction} +import fr.acinq.eclair.channel.InteractiveTxBuilder._ import fr.acinq.eclair.payment.OutgoingPaymentPacket.Upstream import fr.acinq.eclair.transactions.CommitmentSpec import fr.acinq.eclair.transactions.Transactions._ @@ -63,7 +62,8 @@ case object WAIT_FOR_INIT_DUAL_FUNDED_CHANNEL extends ChannelState case object WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL extends ChannelState case object WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL extends ChannelState case object WAIT_FOR_DUAL_FUNDING_CREATED extends ChannelState -case object WAIT_FOR_DUAL_FUNDING_PLACEHOLDER extends ChannelState +case object WAIT_FOR_DUAL_FUNDING_CONFIRMED extends ChannelState +case object WAIT_FOR_DUAL_FUNDING_READY extends ChannelState // Channel opened: case object NORMAL extends ChannelState case object SHUTDOWN extends ChannelState @@ -479,16 +479,18 @@ final case class DATA_WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL(init: INPUT_INIT_CHANN final case class DATA_WAIT_FOR_DUAL_FUNDING_CREATED(channelId: ByteVector32, txBuilder: typed.ActorRef[InteractiveTxBuilder.Command], deferred: Option[ChannelReady]) extends TransientChannelData -final case class DATA_WAIT_FOR_DUAL_FUNDING_PLACEHOLDER(commitments: Commitments, - fundingTx: SignedSharedTransaction, - fundingParams: InteractiveTxParams, - previousFundingTxs: Seq[DualFundingTx], - waitingSince: BlockHeight, // how long have we been waiting for a funding tx to confirm - lastChecked: BlockHeight, // last time we checked if the channel was double-spent - rbfAttempt: Option[typed.ActorRef[InteractiveTxBuilder.Command]], - deferred: Option[ChannelReady]) extends TransientChannelData { - val channelId: ByteVector32 = commitments.channelId -} +final case class DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments: Commitments, + fundingTx: SignedSharedTransaction, + fundingParams: InteractiveTxParams, + previousFundingTxs: Seq[DualFundingTx], + waitingSince: BlockHeight, // how long have we been waiting for a funding tx to confirm + lastChecked: BlockHeight, // last time we checked if the channel was double-spent + rbfAttempt: Option[typed.ActorRef[InteractiveTxBuilder.Command]], + deferred: Option[ChannelReady]) extends PersistentChannelData +final case class DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments: Commitments, + shortIds: ShortIds, + otherFundingTxs: Seq[DualFundingTx], + lastSent: ChannelReady) extends PersistentChannelData final case class DATA_NORMAL(commitments: Commitments, shortIds: ShortIds, diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala index ac2003d223..23091ac163 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala @@ -212,10 +212,18 @@ case class Commitments(channelId: ByteVector32, val capacity: Satoshi = commitInput.txOut.amount /** Channel reserve that applies to our funds. */ - val localChannelReserve: Satoshi = remoteParams.requestedChannelReserve_opt.getOrElse(0 sat) + val localChannelReserve: Satoshi = if (channelFeatures.hasFeature(Features.DualFunding)) { + (capacity / 100).max(remoteParams.dustLimit) + } else { + remoteParams.requestedChannelReserve_opt.getOrElse(0 sat) + } /** Channel reserve that applies to our peer's funds. */ - val remoteChannelReserve: Satoshi = localParams.requestedChannelReserve_opt.getOrElse(0 sat) + val remoteChannelReserve: Satoshi = if (channelFeatures.hasFeature(Features.DualFunding)) { + (capacity / 100).max(localParams.dustLimit) + } else { + localParams.requestedChannelReserve_opt.getOrElse(0 sat) + } // NB: when computing availableBalanceForSend and availableBalanceForReceive, the initiator keeps an extra buffer on // top of its usual channel reserve to avoid getting channels stuck in case the on-chain feerate increases (see diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Helpers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Helpers.scala index e84c2c4494..346a664512 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Helpers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Helpers.scala @@ -54,7 +54,9 @@ object Helpers { remoteParams = data.commitments.remoteParams.copy(initFeatures = remoteInit.features)) data match { case d: DATA_WAIT_FOR_FUNDING_CONFIRMED => d.copy(commitments = commitments1) + case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => d.copy(commitments = commitments1) case d: DATA_WAIT_FOR_CHANNEL_READY => d.copy(commitments = commitments1) + case d: DATA_WAIT_FOR_DUAL_FUNDING_READY => d.copy(commitments = commitments1) case d: DATA_NORMAL => d.copy(commitments = commitments1) case d: DATA_SHUTDOWN => d.copy(commitments = commitments1) case d: DATA_NEGOTIATING => d.copy(commitments = commitments1) @@ -560,12 +562,17 @@ object Helpers { * * @return true if channel was never open, or got closed immediately, had never any htlcs and local never had a positive balance */ - def nothingAtStake(data: PersistentChannelData): Boolean = - data.commitments.localCommit.index == 0 && - data.commitments.localCommit.spec.toLocal == 0.msat && - data.commitments.remoteCommit.index == 0 && - data.commitments.remoteCommit.spec.toRemote == 0.msat && - data.commitments.remoteNextCommitInfo.isRight + def nothingAtStake(data: PersistentChannelData): Boolean = data match { + case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => (d.commitments +: d.previousFundingTxs.map(_.commitments)).forall(commitments => nothingAtStake(commitments)) + case _ => nothingAtStake(data.commitments) + } + + def nothingAtStake(commitments: Commitments): Boolean = + commitments.localCommit.index == 0 && + commitments.localCommit.spec.toLocal == 0.msat && + commitments.remoteCommit.index == 0 && + commitments.remoteCommit.spec.toRemote == 0.msat && + commitments.remoteNextCommitInfo.isRight /** * As soon as a tx spending the funding tx has reached min_depth, we know what the closing type will be, before diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index 8f4333a41a..5c16008acd 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -136,6 +136,7 @@ object Channel { private[channel] sealed trait BitcoinEvent extends PossiblyHarmful private[channel] case object BITCOIN_FUNDING_PUBLISH_FAILED extends BitcoinEvent private[channel] case object BITCOIN_FUNDING_TIMEOUT extends BitcoinEvent + private[channel] case class BITCOIN_FUNDING_DOUBLE_SPENT(fundingTxIds: Set[ByteVector32]) extends BitcoinEvent // @formatter:on case object TickChannelOpenTimeout @@ -268,7 +269,11 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val // if commitment number is zero, we also need to make sure that the funding tx has been published if (closing.commitments.localCommit.index == 0 && closing.commitments.remoteCommit.index == 0) { - blockchain ! GetTxWithMeta(self, closing.commitments.commitInput.outPoint.txid) + if (closing.commitments.channelFeatures.hasFeature(Features.DualFunding)) { + closing.fundingTx.foreach(tx => wallet.publishTransaction(tx)) + } else { + blockchain ! GetTxWithMeta(self, closing.commitments.commitInput.outPoint.txid) + } } } // no need to go OFFLINE, we can directly switch to CLOSING @@ -309,6 +314,12 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val goto(OFFLINE) using funding } + case funding: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => + // we make sure that the funding tx with the highest feerate has been published + // NB: with dual-funding, we only watch the funding tx once it has been confirmed + publishFundingTx(funding.fundingParams, funding.fundingTx) + goto(OFFLINE) using funding + case _ => watchFundingTx(data.commitments) goto(OFFLINE) using data @@ -1264,9 +1275,6 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val goto(SYNCING) using d1 sending channelReestablish - // note: this can only happen if state is NORMAL or SHUTDOWN - // -> in NEGOTIATING there are no more htlcs - // -> in CLOSING we either have mutual closed (so no more htlcs), or already have unilaterally closed (so no action required), and we can't be in OFFLINE state anyway case Event(ProcessCurrentBlockHeight(c), d: PersistentChannelData) => handleNewBlock(c, d) case Event(c: CurrentFeerates, d: PersistentChannelData) => handleCurrentFeerateDisconnected(c, d) @@ -1281,6 +1289,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val case Event(BITCOIN_FUNDING_TIMEOUT, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) => handleFundingTimeout(d) + case Event(e: BITCOIN_FUNDING_DOUBLE_SPENT, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleDualFundingDoubleSpent(e, d) + // just ignore this, we will put a new watch when we reconnect, and we'll be notified again case Event(WatchFundingConfirmedTriggered(_, _, _), _) => stay() @@ -1318,6 +1328,18 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val blockchain ! WatchFundingConfirmed(self, d.commitments.commitInput.outPoint.txid, minDepth) goto(WAIT_FOR_FUNDING_CONFIRMED) + case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => + val minDepth_opt = Helpers.Funding.minDepthDualFunding(nodeParams.channelConf, d.commitments.channelFeatures, d.fundingParams) + val minDepth = minDepth_opt.getOrElse { + val defaultMinDepth = nodeParams.channelConf.minDepthBlocks + // If we are in state WAIT_FOR_FUNDING_CONFIRMED, then the computed minDepth should be > 0, otherwise we would + // have skipped this state. Maybe the computation method was changed and eclair was restarted? + log.warning("min_depth should be defined since we're waiting for the funding tx to confirm, using default minDepth={}", defaultMinDepth) + defaultMinDepth.toLong + } + (d.commitments +: d.previousFundingTxs.map(_.commitments)).foreach(commitments => blockchain ! WatchFundingConfirmed(self, commitments.commitInput.outPoint.txid, minDepth)) + goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) sending d.fundingTx.localSigs + case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_CHANNEL_READY) => log.debug("re-sending channelReady") val channelKeyPath = keyManager.keyPath(d.commitments.localParams, d.commitments.channelConfig) @@ -1325,6 +1347,13 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val val channelReady = ChannelReady(d.commitments.channelId, nextPerCommitmentPoint) goto(WAIT_FOR_CHANNEL_READY) sending channelReady + case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) => + log.debug("re-sending channelReady") + val channelKeyPath = keyManager.keyPath(d.commitments.localParams, d.commitments.channelConfig) + val nextPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1) + val channelReady = ChannelReady(d.commitments.channelId, nextPerCommitmentPoint) + goto(WAIT_FOR_DUAL_FUNDING_READY) sending channelReady + case Event(channelReestablish: ChannelReestablish, d: DATA_NORMAL) => Syncing.checkSync(keyManager, d, channelReestablish) match { case syncFailure: SyncResult.Failure => @@ -1607,8 +1636,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val // We only send the channel_update directly to the peer if we are connected AND the channel hasn't been announced val emitEvent_opt: Option[EmitLocalChannelEvent] = (state, nextState, stateData, nextStateData) match { case (WAIT_FOR_INIT_INTERNAL, OFFLINE, _, d: DATA_NORMAL) => Some(EmitLocalChannelUpdate("restore", d, sendToPeer = false)) - case (WAIT_FOR_FUNDING_CONFIRMED, NORMAL, _, d: DATA_NORMAL) => Some(EmitLocalChannelUpdate("initial", d, sendToPeer = true)) - case (WAIT_FOR_CHANNEL_READY, NORMAL, _, d: DATA_NORMAL) => Some(EmitLocalChannelUpdate("initial", d, sendToPeer = true)) + case (WAIT_FOR_FUNDING_CONFIRMED | WAIT_FOR_DUAL_FUNDING_CONFIRMED, NORMAL, _, d: DATA_NORMAL) => Some(EmitLocalChannelUpdate("initial", d, sendToPeer = true)) + case (WAIT_FOR_CHANNEL_READY | WAIT_FOR_DUAL_FUNDING_READY, NORMAL, _, d: DATA_NORMAL) => Some(EmitLocalChannelUpdate("initial", d, sendToPeer = true)) case (NORMAL, NORMAL, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.shortIds.real.toOption != d2.shortIds.real.toOption || d1.channelUpdate != d2.channelUpdate || d1.channelAnnouncement != d2.channelAnnouncement => Some(EmitLocalChannelUpdate("normal->normal", d2, sendToPeer = d2.channelAnnouncement.isEmpty && d1.channelUpdate != d2.channelUpdate)) case (SYNCING, NORMAL, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.channelUpdate != d2.channelUpdate || d1.channelAnnouncement != d2.channelAnnouncement => Some(EmitLocalChannelUpdate("syncing->normal", d2, sendToPeer = d2.channelAnnouncement.isEmpty)) case (NORMAL, OFFLINE, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.channelUpdate != d2.channelUpdate || d1.channelAnnouncement != d2.channelAnnouncement => Some(EmitLocalChannelUpdate("normal->offline", d2, sendToPeer = false)) @@ -1636,7 +1665,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val case (d1: DATA_NORMAL, d2: DATA_NORMAL) => maybeEmitChannelUpdateChangedEvent(newUpdate = d2.channelUpdate, oldUpdate_opt = Some(d1.channelUpdate), d2) // WAIT_FOR_FUNDING_CONFIRMED->NORMAL, WAIT_FOR_CHANNEL_READY->NORMAL case (_: DATA_WAIT_FOR_FUNDING_CONFIRMED, d2: DATA_NORMAL) => maybeEmitChannelUpdateChangedEvent(newUpdate = d2.channelUpdate, oldUpdate_opt = None, d2) + case (_: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED, d2: DATA_NORMAL) => maybeEmitChannelUpdateChangedEvent(newUpdate = d2.channelUpdate, oldUpdate_opt = None, d2) case (_: DATA_WAIT_FOR_CHANNEL_READY, d2: DATA_NORMAL) => maybeEmitChannelUpdateChangedEvent(newUpdate = d2.channelUpdate, oldUpdate_opt = None, d2) + case (_: DATA_WAIT_FOR_DUAL_FUNDING_READY, d2: DATA_NORMAL) => maybeEmitChannelUpdateChangedEvent(newUpdate = d2.channelUpdate, oldUpdate_opt = None, d2) case _ => () } } @@ -1837,31 +1868,38 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val } private def handleNewBlock(c: CurrentBlockHeight, d: PersistentChannelData) = { - val timedOutOutgoing = d.commitments.timedOutOutgoingHtlcs(c.blockHeight) - val almostTimedOutIncoming = d.commitments.almostTimedOutIncomingHtlcs(c.blockHeight, nodeParams.channelConf.fulfillSafetyBeforeTimeout) - if (timedOutOutgoing.nonEmpty) { - // Downstream timed out. - handleLocalError(HtlcsTimedoutDownstream(d.channelId, timedOutOutgoing), d, Some(c)) - } else if (almostTimedOutIncoming.nonEmpty) { - // Upstream is close to timing out, we need to test if we have funds at risk: htlcs for which we know the preimage - // that are still in our commitment (upstream will try to timeout on-chain). - val relayedFulfills = d.commitments.localChanges.all.collect { case u: UpdateFulfillHtlc => u.id }.toSet - val offendingRelayedHtlcs = almostTimedOutIncoming.filter(htlc => relayedFulfills.contains(htlc.id)) - if (offendingRelayedHtlcs.nonEmpty) { - handleLocalError(HtlcsWillTimeoutUpstream(d.channelId, offendingRelayedHtlcs), d, Some(c)) - } else { - // There might be pending fulfill commands that we haven't relayed yet. - // Since this involves a DB call, we only want to check it if all the previous checks failed (this is the slow path). - val pendingRelayFulfills = nodeParams.db.pendingCommands.listSettlementCommands(d.channelId).collect { case c: CMD_FULFILL_HTLC => c.id } - val offendingPendingRelayFulfills = almostTimedOutIncoming.filter(htlc => pendingRelayFulfills.contains(htlc.id)) - if (offendingPendingRelayFulfills.nonEmpty) { - handleLocalError(HtlcsWillTimeoutUpstream(d.channelId, offendingPendingRelayFulfills), d, Some(c)) + d match { + case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => handleNewBlockDualFundingUnconfirmed(c, d) + case _ => + // note: this can only happen if state is NORMAL or SHUTDOWN + // -> in NEGOTIATING there are no more htlcs + // -> in CLOSING we either have mutual closed (so no more htlcs), or already have unilaterally closed (so no action required), and we can't be in OFFLINE state anyway + val timedOutOutgoing = d.commitments.timedOutOutgoingHtlcs(c.blockHeight) + val almostTimedOutIncoming = d.commitments.almostTimedOutIncomingHtlcs(c.blockHeight, nodeParams.channelConf.fulfillSafetyBeforeTimeout) + if (timedOutOutgoing.nonEmpty) { + // Downstream timed out. + handleLocalError(HtlcsTimedoutDownstream(d.channelId, timedOutOutgoing), d, Some(c)) + } else if (almostTimedOutIncoming.nonEmpty) { + // Upstream is close to timing out, we need to test if we have funds at risk: htlcs for which we know the preimage + // that are still in our commitment (upstream will try to timeout on-chain). + val relayedFulfills = d.commitments.localChanges.all.collect { case u: UpdateFulfillHtlc => u.id }.toSet + val offendingRelayedHtlcs = almostTimedOutIncoming.filter(htlc => relayedFulfills.contains(htlc.id)) + if (offendingRelayedHtlcs.nonEmpty) { + handleLocalError(HtlcsWillTimeoutUpstream(d.channelId, offendingRelayedHtlcs), d, Some(c)) + } else { + // There might be pending fulfill commands that we haven't relayed yet. + // Since this involves a DB call, we only want to check it if all the previous checks failed (this is the slow path). + val pendingRelayFulfills = nodeParams.db.pendingCommands.listSettlementCommands(d.channelId).collect { case c: CMD_FULFILL_HTLC => c.id } + val offendingPendingRelayFulfills = almostTimedOutIncoming.filter(htlc => pendingRelayFulfills.contains(htlc.id)) + if (offendingPendingRelayFulfills.nonEmpty) { + handleLocalError(HtlcsWillTimeoutUpstream(d.channelId, offendingPendingRelayFulfills), d, Some(c)) + } else { + stay() + } + } } else { stay() } - } - } else { - stay() } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala index 8ff8e24cbe..9513085cdf 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala @@ -16,17 +16,23 @@ package fr.acinq.eclair.channel.fsm +import akka.actor.ActorRef import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapter} -import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Script} -import fr.acinq.eclair.Features +import fr.acinq.bitcoin.ScriptFlags +import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Script, Transaction} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ -import fr.acinq.eclair.channel.Helpers.Funding +import fr.acinq.eclair.channel.Helpers.{Funding, getRelayFees} import fr.acinq.eclair.channel.InteractiveTxBuilder.{FullySignedSharedTransaction, InteractiveTxParams, PartiallySignedSharedTransaction} import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel._ import fr.acinq.eclair.channel.publish.TxPublisher.SetChannelId +import fr.acinq.eclair.router.Announcements import fr.acinq.eclair.transactions.Scripts import fr.acinq.eclair.wire.protocol._ +import fr.acinq.eclair.{Features, RealShortChannelId, ToMilliSatoshiConversion} + +import scala.concurrent.duration.DurationInt +import scala.util.{Failure, Success, Try} /** * Created by t-bast on 19/04/2022. @@ -306,21 +312,19 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { Funding.minDepthDualFunding(nodeParams.channelConf, commitments.channelFeatures, fundingParams) match { case Some(fundingMinDepth) => blockchain ! WatchFundingConfirmed(self, commitments.commitInput.outPoint.txid, fundingMinDepth) - val nextData = DATA_WAIT_FOR_DUAL_FUNDING_PLACEHOLDER(commitments, fundingTx, fundingParams, Nil, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, None, None) + val nextData = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, fundingTx, fundingParams, Nil, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, None, None) fundingTx match { - case fundingTx: PartiallySignedSharedTransaction => - goto(WAIT_FOR_DUAL_FUNDING_PLACEHOLDER) using nextData sending fundingTx.localSigs - case fundingTx: FullySignedSharedTransaction => - goto(WAIT_FOR_DUAL_FUNDING_PLACEHOLDER) using nextData sending fundingTx.localSigs calling publishFundingTx(nextData) + case fundingTx: PartiallySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using nextData storing() sending fundingTx.localSigs + case fundingTx: FullySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using nextData storing() sending fundingTx.localSigs calling publishFundingTx(fundingParams, fundingTx) } case None => - val (_, channelReady) = acceptFundingTx(commitments, RealScidStatus.Unknown) - val nextData = DATA_WAIT_FOR_DUAL_FUNDING_PLACEHOLDER(commitments, fundingTx, fundingParams, Nil, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, None, None) + val commitTxs = Set(commitments.localCommit.commitTxAndRemoteSig.commitTx.tx.txid, commitments.remoteCommit.txid) + blockchain ! WatchFundingSpent(self, commitments.commitInput.outPoint.txid, commitments.commitInput.outPoint.index.toInt, commitTxs) + val (shortIds, channelReady) = acceptFundingTx(commitments, RealScidStatus.Unknown) + val nextData = DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments, shortIds, Nil, channelReady) fundingTx match { - case fundingTx: PartiallySignedSharedTransaction => - goto(WAIT_FOR_DUAL_FUNDING_PLACEHOLDER) using nextData sending Seq(fundingTx.localSigs, channelReady) - case fundingTx: FullySignedSharedTransaction => - goto(WAIT_FOR_DUAL_FUNDING_PLACEHOLDER) using nextData sending Seq(fundingTx.localSigs, channelReady) calling publishFundingTx(nextData) + case fundingTx: PartiallySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_READY) using nextData storing() sending Seq(fundingTx.localSigs, channelReady) + case fundingTx: FullySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_READY) using nextData storing() sending Seq(fundingTx.localSigs, channelReady) calling publishFundingTx(fundingParams, fundingTx) } } case f: InteractiveTxBuilder.Failed => @@ -349,8 +353,144 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { goto(CLOSED) }) - when(WAIT_FOR_DUAL_FUNDING_PLACEHOLDER)(handleExceptions { - case Event(_, _) => ??? + when(WAIT_FOR_DUAL_FUNDING_CONFIRMED)(handleExceptions { + case Event(txSigs: TxSignatures, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => + d.fundingTx match { + case fundingTx: PartiallySignedSharedTransaction => InteractiveTxBuilder.addRemoteSigs(d.fundingParams, fundingTx, txSigs) match { + case Left(cause) => + val unsignedFundingTx = fundingTx.tx.buildUnsignedTx() + log.warning("received invalid tx_signatures for txid={} (current funding txid={}): {}", txSigs.txId, unsignedFundingTx.txid, cause.getMessage) + // The funding transaction may still confirm (since our peer should be able to generate valid signatures), + // so we cannot close the channel yet. + stay() sending Error(d.channelId, InvalidFundingSignature(d.channelId, Some(unsignedFundingTx)).getMessage) + case Right(fundingTx) => + log.info("publishing funding tx for channelId={} fundingTxId={}", d.channelId, fundingTx.signedTx.txid) + val nextData = d.copy(fundingTx = fundingTx) + stay() using nextData storing() calling publishFundingTx(d.fundingParams, fundingTx) + } + case _: FullySignedSharedTransaction => + log.warning("received duplicate tx_signatures") + stay() + } + + case Event(_: TxInitRbf, _: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => + log.info("rbf not supported yet") + stay() + + case Event(_: TxAckRbf, _: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => + log.info("rbf not supported yet") + stay() + + case Event(msg: InteractiveTxConstructionMessage, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => + stay() sending Warning(d.channelId, UnexpectedInteractiveTxMessage(d.channelId, msg).getMessage) + + case Event(msg: TxAbort, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => + stay() sending Warning(d.channelId, UnexpectedInteractiveTxMessage(d.channelId, msg).getMessage) + + case Event(WatchFundingConfirmedTriggered(blockHeight, txIndex, confirmedTx), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => + // We find which funding transaction got confirmed. + val allFundingTxs = DualFundingTx(d.fundingTx, d.commitments) +: d.previousFundingTxs + allFundingTxs.find(_.commitments.commitInput.outPoint.txid == confirmedTx.txid) match { + case Some(DualFundingTx(_, commitments)) => + Try(Transaction.correctlySpends(commitments.fullySignedLocalCommitTx(keyManager).tx, Seq(confirmedTx), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) match { + case Success(_) => + log.info(s"channelId=${commitments.channelId} was confirmed at blockHeight=$blockHeight txIndex=$txIndex with funding txid=${commitments.commitInput.outPoint.txid}") + val commitTxs = Set(commitments.localCommit.commitTxAndRemoteSig.commitTx.tx.txid, commitments.remoteCommit.txid) + blockchain ! WatchFundingSpent(self, commitments.commitInput.outPoint.txid, commitments.commitInput.outPoint.index.toInt, commitTxs) + context.system.eventStream.publish(TransactionConfirmed(commitments.channelId, remoteNodeId, confirmedTx)) + val realScidStatus = RealScidStatus.Temporary(RealShortChannelId(blockHeight, txIndex, commitments.commitInput.outPoint.index.toInt)) + val (shortIds, channelReady) = acceptFundingTx(commitments, realScidStatus = realScidStatus) + d.deferred.foreach(self ! _) + val otherFundingTxs = allFundingTxs.filter(_.commitments.commitInput.outPoint.txid != confirmedTx.txid) + goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments, shortIds, otherFundingTxs, channelReady) storing() sending channelReady + case Failure(t) => + log.error(t, s"rejecting channel with invalid funding tx: ${confirmedTx.bin}") + allFundingTxs.foreach(f => wallet.rollback(f.fundingTx.tx.buildUnsignedTx())) + goto(CLOSED) + } + case None => + log.error(s"rejecting channel with invalid funding tx that doesn't match any of our funding txs: ${confirmedTx.bin}") + allFundingTxs.foreach(f => wallet.rollback(f.fundingTx.tx.buildUnsignedTx())) + goto(CLOSED) + } + + case Event(ProcessCurrentBlockHeight(c), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleNewBlockDualFundingUnconfirmed(c, d) + + case Event(e: BITCOIN_FUNDING_DOUBLE_SPENT, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleDualFundingDoubleSpent(e, d) + + case Event(remoteChannelReady: ChannelReady, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => + // We can skip waiting for confirmations if: + // - there is a single version of the funding tx (otherwise we don't know which one to use) + // - they didn't contribute to the funding output or we trust them to not double-spend + val canUseZeroConf = remoteChannelReady.alias_opt.isDefined && + d.previousFundingTxs.isEmpty && + (d.fundingParams.remoteAmount == 0.sat || d.commitments.localParams.initFeatures.hasFeature(Features.ZeroConf)) + if (canUseZeroConf) { + log.info("this chanel isn't zero-conf, but they sent an early channel_ready with an alias: no need to wait for confirmations") + val commitTxs = Set(d.commitments.localCommit.commitTxAndRemoteSig.commitTx.tx.txid, d.commitments.remoteCommit.txid) + blockchain ! WatchFundingSpent(self, d.commitments.commitInput.outPoint.txid, d.commitments.commitInput.outPoint.index.toInt, commitTxs) + val (shortIds, localChannelReady) = acceptFundingTx(d.commitments, RealScidStatus.Unknown) + self ! remoteChannelReady + // NB: we will receive a WatchFundingConfirmedTriggered later that will simply be ignored + goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(d.commitments, shortIds, Nil, localChannelReady) storing() sending localChannelReady + } else { + log.info("received their channel_ready, deferring message") + stay() using d.copy(deferred = Some(remoteChannelReady)) // no need to store, they will re-send if we get disconnected + } + + case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) if d.commitments.announceChannel => + log.debug("received remote announcement signatures, delaying") + // we may receive their announcement sigs before our watcher notifies us that the channel has reached min_conf (especially during testing when blocks are generated in bulk) + // note: no need to persist their message, in case of disconnection they will resend it + context.system.scheduler.scheduleOnce(2 seconds, self, remoteAnnSigs) + stay() + + case Event(c: CMD_FORCECLOSE, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => + // We can't easily force-close until we know which funding transaction confirms. + // A better option would be to double-spend the funding transaction(s). + log.warning("cannot force-close while dual-funded transactions are unconfirmed") + val replyTo = if (c.replyTo == ActorRef.noSender) sender() else c.replyTo + replyTo ! RES_FAILURE(c, CommandUnavailableInThisState(d.channelId, "force-close", stateName)) + stay() + + case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleRemoteError(e, d) + }) + + when(WAIT_FOR_DUAL_FUNDING_READY)(handleExceptions { + case Event(channelReady: ChannelReady, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) => + val shortIds1 = d.shortIds.copy(remoteAlias_opt = channelReady.alias_opt) + shortIds1.remoteAlias_opt.foreach { remoteAlias => + log.info("received remoteAlias={}", remoteAlias) + context.system.eventStream.publish(ShortChannelIdAssigned(self, d.commitments.channelId, shortIds = shortIds1, remoteNodeId = remoteNodeId)) + } + log.info("shortIds: real={} localAlias={} remoteAlias={}", shortIds1.real.toOption.getOrElse("none"), shortIds1.localAlias, shortIds1.remoteAlias_opt.getOrElse("none")) + // we create a channel_update early so that we can use it to send payments through this channel, but it won't be propagated to other nodes since the channel is not yet announced + val scidForChannelUpdate = Helpers.scidForChannelUpdate(channelAnnouncement_opt = None, shortIds1.localAlias) + log.info("using shortChannelId={} for initial channel_update", scidForChannelUpdate) + val fees = getRelayFees(nodeParams, remoteNodeId, d.commitments) + val initialChannelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, scidForChannelUpdate, nodeParams.channelConf.expiryDelta, d.commitments.remoteParams.htlcMinimum, fees.feeBase, fees.feeProportionalMillionths, d.commitments.capacity.toMilliSatoshi, enable = Helpers.aboveReserve(d.commitments)) + // we need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network + context.system.scheduler.scheduleWithFixedDelay(initialDelay = REFRESH_CHANNEL_UPDATE_INTERVAL, delay = REFRESH_CHANNEL_UPDATE_INTERVAL, receiver = self, message = BroadcastChannelUpdate(PeriodicRefresh)) + // used to get the final shortChannelId, used in announcements (if minDepth >= ANNOUNCEMENTS_MINCONF this event will fire instantly) + blockchain ! WatchFundingDeeplyBuried(self, d.commitments.commitInput.outPoint.txid, ANNOUNCEMENTS_MINCONF) + goto(NORMAL) using DATA_NORMAL(d.commitments.copy(remoteNextCommitInfo = Right(channelReady.nextPerCommitmentPoint)), shortIds1, None, initialChannelUpdate, None, None, None) storing() + + case Event(_: TxInitRbf, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) => + // Our peer may not have received the funding transaction confirmation. + stay() sending TxAbort(d.channelId, InvalidRbfTxConfirmed(d.channelId).getMessage) + + case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) if d.commitments.announceChannel => + log.debug("received remote announcement signatures, delaying") + // we may receive their announcement sigs before our watcher notifies us that the channel has reached min_conf (especially during testing when blocks are generated in bulk) + // note: no need to persist their message, in case of disconnection they will resend it + context.system.scheduler.scheduleOnce(2 seconds, self, remoteAnnSigs) + stay() + + case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_DUAL_FUNDING_READY) if tx.txid == d.commitments.remoteCommit.txid => handleRemoteSpentCurrent(tx, d) + + case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_DUAL_FUNDING_READY) => handleInformationLeak(tx, d) + + case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) => handleRemoteError(e, d) }) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala index 2318dd46b0..8302d81bed 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala @@ -379,7 +379,6 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers { case Event(remoteChannelReady: ChannelReady, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) => if (remoteChannelReady.alias_opt.isDefined && d.commitments.localParams.isInitiator) { log.info("this channel isn't zero-conf, but we are funder and they sent an early channel_ready with an alias: no need to wait for confirmations") - // No need to emit ShortChannelIdAssigned: we will emit it when handling their channel_ready in WAIT_FOR_CHANNEL_READY val (shortIds, localChannelReady) = acceptFundingTx(d.commitments, RealScidStatus.Unknown) self ! remoteChannelReady // NB: we will receive a WatchFundingConfirmedTriggered later that will simply be ignored diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala index 6b09ee2688..17da029991 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala @@ -16,9 +16,14 @@ package fr.acinq.eclair.channel.fsm -import fr.acinq.eclair.channel.InteractiveTxBuilder.{FullySignedSharedTransaction, PartiallySignedSharedTransaction} +import fr.acinq.eclair.blockchain.CurrentBlockHeight +import fr.acinq.eclair.channel.Helpers.Closing +import fr.acinq.eclair.channel.InteractiveTxBuilder.{FullySignedSharedTransaction, InteractiveTxParams, PartiallySignedSharedTransaction, SignedSharedTransaction} import fr.acinq.eclair.channel._ +import fr.acinq.eclair.channel.fsm.Channel.BITCOIN_FUNDING_DOUBLE_SPENT +import fr.acinq.eclair.wire.protocol.Error +import scala.concurrent.Future import scala.util.{Failure, Success} /** @@ -32,8 +37,8 @@ trait DualFundingHandlers extends CommonHandlers { this: Channel => - def publishFundingTx(d: DATA_WAIT_FOR_DUAL_FUNDING_PLACEHOLDER): Unit = { - d.fundingTx match { + def publishFundingTx(fundingParams: InteractiveTxParams, fundingTx: SignedSharedTransaction): Unit = { + fundingTx match { case _: PartiallySignedSharedTransaction => log.info("we haven't received remote funding signatures yet: we cannot publish the funding transaction but our peer should publish it") case fundingTx: FullySignedSharedTransaction => @@ -41,8 +46,8 @@ trait DualFundingHandlers extends CommonHandlers { // to publish and we may be able to RBF. wallet.publishTransaction(fundingTx.signedTx).onComplete { case Success(_) => - context.system.eventStream.publish(TransactionPublished(d.commitments.channelId, remoteNodeId, fundingTx.signedTx, fundingTx.tx.localFees(d.fundingParams), "funding")) - channelOpenReplyToUser(Right(ChannelOpenResponse.ChannelOpened(d.commitments.channelId))) + context.system.eventStream.publish(TransactionPublished(fundingParams.channelId, remoteNodeId, fundingTx.signedTx, fundingTx.tx.localFees(fundingParams), "funding")) + channelOpenReplyToUser(Right(ChannelOpenResponse.ChannelOpened(fundingParams.channelId))) case Failure(t) => channelOpenReplyToUser(Left(LocalError(t))) log.warning("error while publishing funding tx: {}", t.getMessage) // tx may be published by our peer, we can't fail-fast @@ -50,4 +55,38 @@ trait DualFundingHandlers extends CommonHandlers { } } + def handleNewBlockDualFundingUnconfirmed(c: CurrentBlockHeight, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) = { + if (Channel.FUNDING_TIMEOUT_FUNDEE < c.blockHeight - d.waitingSince && Closing.nothingAtStake(d)) { + log.warning("funding transaction did not confirm in {} blocks and we have nothing at stake, forgetting channel", Channel.FUNDING_TIMEOUT_FUNDEE) + handleFundingTimeout(d) + } else if (d.lastChecked + 6 < c.blockHeight) { + log.debug("checking if funding transactions have been double-spent") + val fundingTxs = (d.fundingTx +: d.previousFundingTxs.map(_.fundingTx)).map(_.tx.buildUnsignedTx()) + // We check whether *all* funding attempts have been double-spent. + // Since we only consider a transaction double-spent when the spending transaction is confirmed, this will not + // return false positives when one of our transactions is confirmed, because its individual result will be false. + Future.sequence(fundingTxs.map(tx => wallet.doubleSpent(tx))).map(_.forall(_ == true)).map { + case true => self ! BITCOIN_FUNDING_DOUBLE_SPENT(fundingTxs.map(_.txid).toSet) + case false => publishFundingTx(d.fundingParams, d.fundingTx) // we republish the highest feerate funding transaction + } + stay() using d.copy(lastChecked = c.blockHeight) storing() + } else { + stay() + } + } + + def handleDualFundingDoubleSpent(e: BITCOIN_FUNDING_DOUBLE_SPENT, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) = { + val fundingTxIds = (d.commitments +: d.previousFundingTxs.map(_.commitments)).map(_.commitInput.outPoint.txid).toSet + if (fundingTxIds.subsetOf(e.fundingTxIds)) { + log.warning("{} funding attempts have been double-spent, forgetting channel", fundingTxIds.size) + (d.fundingTx +: d.previousFundingTxs.map(_.fundingTx)).foreach(tx => wallet.rollback(tx.tx.buildUnsignedTx())) + channelOpenReplyToUser(Left(LocalError(FundingTxDoubleSpent(d.channelId)))) + goto(CLOSED) sending Error(d.channelId, FundingTxDoubleSpent(d.channelId).getMessage) + } else { + // Not all funding attempts have been double-spent, the channel may still confirm. + // For example, we may have published an RBF attempt while we were checking if funding attempts were double-spent. + stay() + } + } + } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala index eb70e335e8..c8764fe5f8 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala @@ -132,13 +132,20 @@ trait ErrorHandlers extends CommonHandlers { case negotiating@DATA_NEGOTIATING(_, _, _, _, Some(bestUnpublishedClosingTx)) => // if we were in the process of closing and already received a closing sig from the counterparty, it's always better to use that handleMutualClose(bestUnpublishedClosingTx, Left(negotiating)) + // The channel was never used and the funding tx could be double-spent: we don't need to publish our commitment + // since we don't have funds in the channel, but it's a nice thing to do because it lets our peer get their + // funds back without delays if they can't double-spend the funding tx. case d: DATA_WAIT_FOR_FUNDING_CONFIRMED if Closing.nothingAtStake(d) => - // The channel was never used and the funding tx could be double-spent: we don't need to publish our commitment - // since we don't have funds in the channel, but it's a nice thing to do because it lets our peer get their - // funds back without delays if they can't double-spend the funding tx. val commitTx = d.commitments.fullySignedLocalCommitTx(keyManager) txPublisher ! PublishFinalTx(commitTx, 0 sat, None) goto(CLOSED) + case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED if Closing.nothingAtStake(d) => + val commitTx = d.commitments.fullySignedLocalCommitTx(keyManager) + txPublisher ! PublishFinalTx(commitTx, 0 sat, None) + goto(CLOSED) + case _: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => + log.info("cannot close channel while dual-funding txs are unconfirmed: waiting for a transaction to confirm or be double-spent") + stay() case hasCommitments: PersistentChannelData => spendLocalCurrent(hasCommitments) // NB: we publish the commitment even if we have nothing at stake (in a dataloss situation our peer will send us an error just for that) case _: TransientChannelData => goto(CLOSED) // when there is no commitment yet, we just go to CLOSED state in case an error occurs } @@ -187,15 +194,21 @@ trait ErrorHandlers extends CommonHandlers { log.warning("we have an outdated commitment: will not publish our local tx") stay() } else { - val commitTx = d.commitments.fullySignedLocalCommitTx(keyManager).tx - val localCommitPublished = Closing.LocalClose.claimCommitTxOutputs(keyManager, d.commitments, commitTx, nodeParams.currentBlockHeight, nodeParams.onChainFeeConf) - val nextData = d match { - case closing: DATA_CLOSING => closing.copy(localCommitPublished = Some(localCommitPublished)) - case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, negotiating.closingTxProposed.flatten.map(_.unsignedTx), localCommitPublished = Some(localCommitPublished)) - case waitForFundingConfirmed: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.fundingTx, waitingSince = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) - case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) + d match { + case _: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => + log.info("cannot spend our commitment while dual-funding txs are unconfirmed: waiting for a transaction to confirm or be double-spent") + stay() + case _ => + val commitTx = d.commitments.fullySignedLocalCommitTx(keyManager).tx + val localCommitPublished = Closing.LocalClose.claimCommitTxOutputs(keyManager, d.commitments, commitTx, nodeParams.currentBlockHeight, nodeParams.onChainFeeConf) + val nextData = d match { + case closing: DATA_CLOSING => closing.copy(localCommitPublished = Some(localCommitPublished)) + case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, negotiating.closingTxProposed.flatten.map(_.unsignedTx), localCommitPublished = Some(localCommitPublished)) + case waitForFundingConfirmed: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.fundingTx, waitingSince = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) + case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) + } + goto(CLOSING) using nextData storing() calling doPublish(localCommitPublished, d.commitments) } - goto(CLOSING) using nextData storing() calling doPublish(localCommitPublished, d.commitments) } } @@ -238,6 +251,12 @@ trait ErrorHandlers extends CommonHandlers { case closing: DATA_CLOSING => closing.copy(remoteCommitPublished = Some(remoteCommitPublished)) case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, negotiating.closingTxProposed.flatten.map(_.unsignedTx), remoteCommitPublished = Some(remoteCommitPublished)) case waitForFundingConfirmed: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.fundingTx, waitingSince = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished)) + case waitForFundingConfirmed: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => + val dualFundedTx_opt = waitForFundingConfirmed.fundingTx match { + case _: InteractiveTxBuilder.PartiallySignedSharedTransaction => None + case tx: InteractiveTxBuilder.FullySignedSharedTransaction => Some(tx.signedTx) + } + DATA_CLOSING(d.commitments, fundingTx = dualFundedTx_opt, waitingSince = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished)) case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished)) } goto(CLOSING) using nextData storing() calling doPublish(remoteCommitPublished, d.commitments) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/DbEventHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/DbEventHandler.scala index a1f63f57ca..9a8972c13b 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/DbEventHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/DbEventHandler.scala @@ -108,7 +108,7 @@ class DbEventHandler(nodeParams: NodeParams) extends Actor with DiagnosticActorL case e: ChannelStateChanged => // NB: order matters! e match { - case ChannelStateChanged(_, channelId, _, remoteNodeId, WAIT_FOR_CHANNEL_READY, NORMAL, Some(commitments: Commitments)) => + case ChannelStateChanged(_, channelId, _, remoteNodeId, WAIT_FOR_CHANNEL_READY | WAIT_FOR_DUAL_FUNDING_READY, NORMAL, Some(commitments: Commitments)) => ChannelMetrics.ChannelLifecycleEvents.withTag(ChannelTags.Event, ChannelTags.Events.Created).increment() val event = ChannelEvent.EventType.Created auditDb.add(ChannelEvent(channelId, remoteNodeId, commitments.capacity, commitments.localParams.isInitiator, !commitments.announceChannel, event)) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala index 26cf2d496a..77e5b8f573 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/json/JsonSerializers.scala @@ -31,12 +31,12 @@ import fr.acinq.eclair.io.Peer import fr.acinq.eclair.message.OnionMessages import fr.acinq.eclair.payment.PaymentFailure.PaymentFailedSummary import fr.acinq.eclair.payment._ -import fr.acinq.eclair.router.Router.{ChannelHop, ChannelRelayParams, Route} +import fr.acinq.eclair.router.Router.{ChannelRelayParams, Route} import fr.acinq.eclair.transactions.DirectedHtlc import fr.acinq.eclair.transactions.Transactions._ import fr.acinq.eclair.wire.protocol.MessageOnionCodecs.blindedRouteCodec import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Feature, FeatureSupport, MilliSatoshi, ShortChannelId, TimestampMilli, TimestampSecond, UInt64, UnknownFeature, channel} +import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Feature, FeatureSupport, MilliSatoshi, ShortChannelId, TimestampMilli, TimestampSecond, UInt64, UnknownFeature} import org.json4s import org.json4s.JsonAST._ import org.json4s.jackson.Serialization @@ -152,7 +152,7 @@ object CltvExpiryDeltaSerializer extends MinimalSerializer({ }) object BlockHeightSerializer extends MinimalSerializer({ - case x: BlockHeight => JInt(x.toInt) + case h: BlockHeight => JLong(h.toLong) }) object FeeratePerKwSerializer extends MinimalSerializer({ @@ -503,8 +503,13 @@ object CustomTypeHints { classOf[DATA_WAIT_FOR_FUNDING_INTERNAL], classOf[DATA_WAIT_FOR_FUNDING_CREATED], classOf[DATA_WAIT_FOR_FUNDING_SIGNED], - classOf[DATA_WAIT_FOR_CHANNEL_READY], classOf[DATA_WAIT_FOR_FUNDING_CONFIRMED], + classOf[DATA_WAIT_FOR_CHANNEL_READY], + classOf[DATA_WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL], + classOf[DATA_WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL], + classOf[DATA_WAIT_FOR_DUAL_FUNDING_CREATED], + classOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED], + classOf[DATA_WAIT_FOR_DUAL_FUNDING_READY], classOf[DATA_NORMAL], classOf[DATA_SHUTDOWN], classOf[DATA_NEGOTIATING], diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala index 822e224116..fa20a95d9a 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala @@ -16,8 +16,10 @@ package fr.acinq.eclair.wire.internal.channel.version3 +import akka.actor.typed import fr.acinq.bitcoin.scalacompat.DeterministicWallet.{ExtendedPrivateKey, KeyPath} import fr.acinq.bitcoin.scalacompat.{OutPoint, Transaction, TxOut} +import fr.acinq.eclair.channel.InteractiveTxBuilder._ import fr.acinq.eclair.channel._ import fr.acinq.eclair.crypto.ShaChain import fr.acinq.eclair.transactions.Transactions._ @@ -25,7 +27,7 @@ import fr.acinq.eclair.transactions.{CommitmentSpec, DirectedHtlc, IncomingHtlc, import fr.acinq.eclair.wire.protocol.CommonCodecs._ import fr.acinq.eclair.wire.protocol.LightningMessageCodecs._ import fr.acinq.eclair.wire.protocol.UpdateMessage -import fr.acinq.eclair.{BlockHeight, FeatureSupport, Features, Alias, PermanentChannelFeature} +import fr.acinq.eclair.{Alias, BlockHeight, FeatureSupport, Features, PermanentChannelFeature} import scodec.bits.{BitVector, ByteVector} import scodec.codecs._ import scodec.{Attempt, Codec} @@ -336,6 +338,67 @@ private[channel] object ChannelCodecs3 { ("shortIds" | shortids) :: ("lastSent" | lengthDelimited(channelReadyCodec))).as[DATA_WAIT_FOR_CHANNEL_READY] + private val remoteTxAddInputCodec: Codec[RemoteTxAddInput] = ( + ("serialId" | uint64) :: + ("outPoint" | outPointCodec) :: + ("txOut" | txOutCodec) :: + ("sequence" | uint32)).as[RemoteTxAddInput] + + private val remoteTxAddOutputCodec: Codec[RemoteTxAddOutput] = ( + ("serialId" | uint64) :: + ("amount" | satoshi) :: + ("scriptPubKey" | lengthDelimited(bytes))).as[RemoteTxAddOutput] + + private val sharedTransactionCodec: Codec[SharedTransaction] = ( + ("localInputs" | seqOfN(uint16, lengthDelimited(txAddInputCodec))) :: + ("remoteInputs" | seqOfN(uint16, remoteTxAddInputCodec)) :: + ("localOutputs" | seqOfN(uint16, lengthDelimited(txAddOutputCodec))) :: + ("remoteOutputs" | seqOfN(uint16, remoteTxAddOutputCodec)) :: + ("lockTime" | uint32)).as[SharedTransaction] + + private val partiallySignedSharedTransactionCodec: Codec[PartiallySignedSharedTransaction] = ( + ("sharedTx" | sharedTransactionCodec) :: + ("localSigs" | lengthDelimited(txSignaturesCodec))).as[PartiallySignedSharedTransaction] + + private val fullySignedSharedTransactionCodec: Codec[FullySignedSharedTransaction] = ( + ("sharedTx" | sharedTransactionCodec) :: + ("localSigs" | lengthDelimited(txSignaturesCodec)) :: + ("remoteSigs" | lengthDelimited(txSignaturesCodec))).as[FullySignedSharedTransaction] + + private val signedSharedTransactionCodec: Codec[SignedSharedTransaction] = discriminated[SignedSharedTransaction].by(uint16) + .typecase(0x01, partiallySignedSharedTransactionCodec) + .typecase(0x02, fullySignedSharedTransactionCodec) + + private val dualFundingTxCodec: Codec[DualFundingTx] = ( + ("fundingTx" | signedSharedTransactionCodec) :: + ("commitments" | commitmentsCodec)).as[DualFundingTx] + + private val fundingParamsCodec: Codec[InteractiveTxParams] = ( + ("channelId" | bytes32) :: + ("isInitiator" | bool8) :: + ("localAmount" | satoshi) :: + ("remoteAmount" | satoshi) :: + ("fundingPubkeyScript" | lengthDelimited(bytes)) :: + ("lockTime" | uint32) :: + ("dustLimit" | satoshi) :: + ("targetFeerate" | feeratePerKw)).as[InteractiveTxParams] + + val DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED_0b_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] = ( + ("commitments" | commitmentsCodec) :: + ("fundingTx" | signedSharedTransactionCodec) :: + ("fundingParams" | fundingParamsCodec) :: + ("previousFundingTxs" | seqOfN(uint16, dualFundingTxCodec)) :: + ("waitingSince" | blockHeight) :: + ("lastChecked" | blockHeight) :: + ("rbfAttempt" | provide(Option.empty[typed.ActorRef[InteractiveTxBuilder.Command]])) :: + ("deferred" | optional(bool8, lengthDelimited(channelReadyCodec)))).as[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] + + val DATA_WAIT_FOR_DUAL_FUNDING_READY_0c_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_READY] = ( + ("commitments" | commitmentsCodec) :: + ("shortIds" | shortids) :: + ("otherFundingTxs" | seqOfN(uint16, dualFundingTxCodec)) :: + ("lastSent" | lengthDelimited(channelReadyCodec))).as[DATA_WAIT_FOR_DUAL_FUNDING_READY] + val DATA_NORMAL_02_Codec: Codec[DATA_NORMAL] = ( ("commitments" | commitmentsCodec) :: ("shortChannelId" | realshortchannelid) :: @@ -410,6 +473,8 @@ private[channel] object ChannelCodecs3 { // Order matters! val channelDataCodec: Codec[PersistentChannelData] = discriminated[PersistentChannelData].by(uint16) + .typecase(0x0c, Codecs.DATA_WAIT_FOR_DUAL_FUNDING_READY_0c_Codec) + .typecase(0x0b, Codecs.DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED_0b_Codec) .typecase(0x0a, Codecs.DATA_WAIT_FOR_CHANNEL_READY_0a_Codec) .typecase(0x09, Codecs.DATA_NORMAL_09_Codec) .typecase(0x08, Codecs.DATA_SHUTDOWN_08_Codec) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/CommonCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/CommonCodecs.scala index 314fbaf4bf..e8d3bdbe5d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/CommonCodecs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/CommonCodecs.scala @@ -110,6 +110,8 @@ object CommonCodecs { val listofsignatures: Codec[List[ByteVector64]] = listOfN(uint16, bytes64) + def seqOfN[A](countCodec: Codec[Int], valueCodec: Codec[A]): Codec[Seq[A]] = listOfN(countCodec, valueCodec).xmap(_.toSeq, _.toList) + val channelflags: Codec[ChannelFlags] = (ignore(7) dropLeft bool).as[ChannelFlags] val ipv4address: Codec[Inet4Address] = bytes(4).xmap(b => InetAddress.getByAddress(b.toArray).asInstanceOf[Inet4Address], a => ByteVector(a.getAddress)) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/TestDatabases.scala b/eclair-core/src/test/scala/fr/acinq/eclair/TestDatabases.scala index 91206d6721..c3e9c5184d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/TestDatabases.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/TestDatabases.scala @@ -48,7 +48,6 @@ object TestDatabases { dbs.copy(channels = new SqliteChannelsDbWithValidation(dbs.channels)) } - /** * ChannelsDb instance that wraps around an actual db instance and does additional checks * This can be thought of as fuzzing and fills a gap between codec unit tests and database tests, by checking that channel state can be written and read consistently @@ -75,6 +74,8 @@ object TestDatabases { def freeze3(input: PersistentChannelData): PersistentChannelData = input match { case d: DATA_WAIT_FOR_FUNDING_CONFIRMED => d.copy(commitments = freeze2(d.commitments)) case d: DATA_WAIT_FOR_CHANNEL_READY => d.copy(commitments = freeze2(d.commitments)) + case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => d.copy(commitments = freeze2(d.commitments)) + case d: DATA_WAIT_FOR_DUAL_FUNDING_READY => d.copy(commitments = freeze2(d.commitments)) case d: DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT => d.copy(commitments = freeze2(d.commitments)) case d: DATA_NORMAL => d.copy(commitments = freeze2(d.commitments)) case d: DATA_CLOSING => d.copy(commitments = freeze2(d.commitments)) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala index 2991b89efa..0891c5ae23 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala @@ -124,8 +124,9 @@ class SingleKeyOnChainWallet extends OnChainWallet { val inputAmount = amountOut + 100_000.sat val inputTx = Transaction(2, Seq(TxIn(OutPoint(randomBytes32(), 1), Nil, 0)), Seq(TxOut(inputAmount, Script.pay2wpkh(pubkey))), 0) inputs = inputs :+ inputTx + val dummyWitness = Script.witnessPay2wpkh(pubkey, ByteVector.fill(73)(0)) val dummySignedTx = tx.copy( - txIn = tx.txIn :+ TxIn(OutPoint(inputTx, 0), ByteVector.empty, 0, Script.witnessPay2wpkh(pubkey, ByteVector.fill(73)(0))), + txIn = tx.txIn.map(_.copy(witness = dummyWitness)) :+ TxIn(OutPoint(inputTx, 0), ByteVector.empty, 0, dummyWitness), txOut = tx.txOut :+ TxOut(inputAmount, Script.pay2wpkh(pubkey)), ) val fee = Transactions.weight2fee(feeRate, dummySignedTx.weight()) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala index 92dfe5f5a1..f344e6c807 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala @@ -237,34 +237,86 @@ trait ChannelStateTestsBase extends Assertions with Eventually { assert(alice2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId == ByteVector32.Zeroes) bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, nonInitiatorFundingAmount, dualFunded, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) assert(bob2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId == ByteVector32.Zeroes) - alice2bob.expectMsgType[OpenChannel] - alice2bob.forward(bob) - bob2alice.expectMsgType[AcceptChannel] - bob2alice.forward(alice) - alice2bob.expectMsgType[FundingCreated] - alice2bob.forward(bob) - bob2alice.expectMsgType[FundingSigned] - bob2alice.forward(alice) - assert(alice2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId != ByteVector32.Zeroes) - alice2blockchain.expectMsgType[WatchFundingSpent] - assert(bob2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId != ByteVector32.Zeroes) - bob2blockchain.expectMsgType[WatchFundingSpent] - val fundingTx = eventListener.expectMsgType[TransactionPublished].tx - if (!channelType.features.contains(Features.ZeroConf)) { - alice2blockchain.expectMsgType[WatchFundingConfirmed] - bob2blockchain.expectMsgType[WatchFundingConfirmed] - eventually(assert(alice.stateName == WAIT_FOR_FUNDING_CONFIRMED)) - alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx) - bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx) + + val fundingTx = if (!dualFunded) { + alice2bob.expectMsgType[OpenChannel] + alice2bob.forward(bob) + bob2alice.expectMsgType[AcceptChannel] + bob2alice.forward(alice) + alice2bob.expectMsgType[FundingCreated] + alice2bob.forward(bob) + bob2alice.expectMsgType[FundingSigned] + bob2alice.forward(alice) + assert(alice2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId != ByteVector32.Zeroes) + alice2blockchain.expectMsgType[WatchFundingSpent] + assert(bob2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId != ByteVector32.Zeroes) + bob2blockchain.expectMsgType[WatchFundingSpent] + val fundingTx = eventListener.expectMsgType[TransactionPublished].tx + if (!channelType.features.contains(Features.ZeroConf)) { + alice2blockchain.expectMsgType[WatchFundingConfirmed] + bob2blockchain.expectMsgType[WatchFundingConfirmed] + eventually(assert(alice.stateName == WAIT_FOR_FUNDING_CONFIRMED)) + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx) + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx) + } + eventually(assert(alice.stateName == WAIT_FOR_CHANNEL_READY)) + eventually(assert(bob.stateName == WAIT_FOR_CHANNEL_READY)) + alice2blockchain.expectMsgType[WatchFundingLost] + bob2blockchain.expectMsgType[WatchFundingLost] + alice2bob.expectMsgType[ChannelReady] + alice2bob.forward(bob) + bob2alice.expectMsgType[ChannelReady] + bob2alice.forward(alice) + fundingTx + } else { + alice2bob.expectMsgType[OpenDualFundedChannel] + alice2bob.forward(bob) + bob2alice.expectMsgType[AcceptDualFundedChannel] + bob2alice.forward(alice) + assert(alice2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId != ByteVector32.Zeroes) + assert(bob2blockchain.expectMsgType[TxPublisher.SetChannelId].channelId != ByteVector32.Zeroes) + alice2bob.expectMsgType[TxAddInput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxAddInput] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddOutput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxAddOutput] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddOutput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxComplete] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxComplete] + alice2bob.forward(bob) + bob2alice.expectMsgType[CommitSig] + bob2alice.forward(alice) + alice2bob.expectMsgType[CommitSig] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxSignatures] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxSignatures] + alice2bob.forward(bob) + val fundingTx = eventListener.expectMsgType[TransactionPublished].tx + if (!channelType.features.contains(Features.ZeroConf)) { + eventually(assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)) + eventually(assert(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)) + alice2blockchain.expectMsgType[WatchFundingConfirmed] + bob2blockchain.expectMsgType[WatchFundingConfirmed] + alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx) + bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx) + } + alice2blockchain.expectMsgType[WatchFundingSpent] + bob2blockchain.expectMsgType[WatchFundingSpent] + alice2blockchain.expectMsgType[WatchFundingLost] + bob2blockchain.expectMsgType[WatchFundingLost] + alice2bob.expectMsgType[ChannelReady] + alice2bob.forward(bob) + bob2alice.expectMsgType[ChannelReady] + bob2alice.forward(alice) + fundingTx } - eventually(assert(alice.stateName == WAIT_FOR_CHANNEL_READY)) - eventually(assert(bob.stateName == WAIT_FOR_CHANNEL_READY)) - alice2blockchain.expectMsgType[WatchFundingLost] - bob2blockchain.expectMsgType[WatchFundingLost] - alice2bob.expectMsgType[ChannelReady] - alice2bob.forward(bob) - bob2alice.expectMsgType[ChannelReady] - bob2alice.forward(alice) + if (interceptChannelUpdates) { // we don't forward the channel updates, in reality they would be processed by the router alice2bob.expectMsgType[ChannelUpdate] @@ -274,7 +326,10 @@ trait ChannelStateTestsBase extends Assertions with Eventually { bob2blockchain.expectMsgType[WatchFundingDeeplyBuried] eventually(assert(alice.stateName == NORMAL)) eventually(assert(bob.stateName == NORMAL)) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.availableBalanceForSend == (pushMsat - aliceParams.requestedChannelReserve_opt.getOrElse(0 sat)).max(0 msat)) + + val aliceCommitments = alice.stateData.asInstanceOf[DATA_NORMAL].commitments + val bobCommitments = bob.stateData.asInstanceOf[DATA_NORMAL].commitments + assert(bobCommitments.availableBalanceForSend == (nonInitiatorFundingAmount.getOrElse(0 sat) + pushMsat - aliceCommitments.remoteChannelReserve).max(0 msat)) // x2 because alice and bob share the same relayer channelUpdateListener.expectMsgType[LocalChannelUpdate] channelUpdateListener.expectMsgType[LocalChannelUpdate] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala index 17d43c4623..a65e8cd05c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala @@ -20,7 +20,7 @@ import akka.actor.Status import akka.testkit.{TestFSMRef, TestProbe} import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, SatoshiLong, Script} import fr.acinq.eclair.blockchain.SingleKeyOnChainWallet -import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchFundingLost} +import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchFundingLost, WatchFundingSpent} import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.channel.InteractiveTxBuilder.{FullySignedSharedTransaction, PartiallySignedSharedTransaction} import fr.acinq.eclair.channel._ @@ -97,23 +97,23 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn // Bob sends its signatures first as he contributed less than Alice. bob2alice.expectMsgType[TxSignatures] - awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_PLACEHOLDER) - val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_PLACEHOLDER] + awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) + val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] assert(bobData.commitments.channelFeatures.hasFeature(Features.DualFunding)) assert(bobData.fundingTx.isInstanceOf[PartiallySignedSharedTransaction]) val fundingTxId = bobData.fundingTx.asInstanceOf[PartiallySignedSharedTransaction].tx.buildUnsignedTx().txid - assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId === fundingTxId) + assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTxId) // Alice receives Bob's signatures and sends her own signatures. bob2alice.forward(alice) - assert(listener.expectMsgType[TransactionPublished].tx.txid === fundingTxId) - assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId === fundingTxId) + assert(listener.expectMsgType[TransactionPublished].tx.txid == fundingTxId) + assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTxId) alice2bob.expectMsgType[TxSignatures] - awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_PLACEHOLDER) - val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_PLACEHOLDER] + awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) + val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] assert(aliceData.commitments.channelFeatures.hasFeature(Features.DualFunding)) assert(aliceData.fundingTx.isInstanceOf[FullySignedSharedTransaction]) - assert(aliceData.fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx.txid === fundingTxId) + assert(aliceData.fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx.txid == fundingTxId) } test("complete interactive-tx protocol (zero-conf)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.ZeroConf), Tag(ChannelStateTestsTags.ScidAlias), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => @@ -148,29 +148,27 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn val bobSigs = bob2alice.expectMsgType[TxSignatures] bob2alice.expectMsgType[ChannelReady] assert(bobListener.expectMsgType[ShortChannelIdAssigned].shortIds.real == RealScidStatus.Unknown) - awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_PLACEHOLDER) - val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_PLACEHOLDER] + awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_READY) + val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY] assert(bobData.commitments.channelFeatures.hasFeature(Features.DualFunding)) assert(bobData.commitments.channelFeatures.hasFeature(Features.ZeroConf)) - assert(bobData.fundingTx.isInstanceOf[PartiallySignedSharedTransaction]) - val fundingTxId = bobData.fundingTx.asInstanceOf[PartiallySignedSharedTransaction].tx.buildUnsignedTx().txid - assert(bob2blockchain.expectMsgType[WatchFundingLost].txId === fundingTxId) + bob2blockchain.expectMsgType[WatchFundingSpent] + bob2blockchain.expectMsgType[WatchFundingLost] bob2blockchain.expectNoMessage(100 millis) // Alice receives Bob's signatures and sends her own signatures. bob2alice.forward(alice, bobSigs) assert(aliceListener.expectMsgType[ShortChannelIdAssigned].shortIds.real == RealScidStatus.Unknown) - assert(aliceListener.expectMsgType[TransactionPublished].tx.txid === fundingTxId) - assert(alice2blockchain.expectMsgType[WatchFundingLost].txId === fundingTxId) + val fundingTx = aliceListener.expectMsgType[TransactionPublished].tx + assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid) + assert(alice2blockchain.expectMsgType[WatchFundingLost].txId == fundingTx.txid) alice2blockchain.expectNoMessage(100 millis) alice2bob.expectMsgType[TxSignatures] alice2bob.expectMsgType[ChannelReady] - awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_PLACEHOLDER) - val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_PLACEHOLDER] + awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY) + val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY] assert(aliceData.commitments.channelFeatures.hasFeature(Features.DualFunding)) assert(aliceData.commitments.channelFeatures.hasFeature(Features.ZeroConf)) - assert(aliceData.fundingTx.isInstanceOf[FullySignedSharedTransaction]) - assert(aliceData.fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx.txid === fundingTxId) } test("recv invalid interactive-tx message", Tag(ChannelStateTestsTags.DualFunding)) { f => @@ -253,6 +251,12 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn awaitCond(wallet.rolledback.size == 1) awaitCond(alice.stateName == CLOSED) aliceOrigin.expectMsgType[Status.Failure] + + // Bob has sent his signatures already, so he cannot close the channel yet. + alice2bob.forward(bob, TxSignatures(channelId(alice), randomBytes32(), Nil)) + bob2alice.expectMsgType[Error] + bob2blockchain.expectNoMessage(100 millis) + assert(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) } test("recv TxAbort", Tag(ChannelStateTestsTags.DualFunding)) { f => diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala new file mode 100644 index 0000000000..613685911c --- /dev/null +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala @@ -0,0 +1,315 @@ +/* + * Copyright 2022 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.states.c + +import akka.testkit.{TestFSMRef, TestProbe} +import fr.acinq.bitcoin.scalacompat.ByteVector32 +import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ +import fr.acinq.eclair.blockchain.{CurrentBlockHeight, SingleKeyOnChainWallet} +import fr.acinq.eclair.channel.InteractiveTxBuilder.FullySignedSharedTransaction +import fr.acinq.eclair.channel._ +import fr.acinq.eclair.channel.fsm.Channel +import fr.acinq.eclair.channel.fsm.Channel.ProcessCurrentBlockHeight +import fr.acinq.eclair.channel.publish.TxPublisher.{PublishFinalTx, SetChannelId} +import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags} +import fr.acinq.eclair.wire.protocol._ +import fr.acinq.eclair.{BlockHeight, TestConstants, TestKitBaseClass} +import org.scalatest.funsuite.FixtureAnyFunSuiteLike +import org.scalatest.{Outcome, Tag} + +import scala.concurrent.duration.DurationInt + +class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ChannelStateTestsBase { + + case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, bob2blockchain: TestProbe, aliceListener: TestProbe, bobListener: TestProbe, wallet: SingleKeyOnChainWallet) + + override def withFixture(test: OneArgTest): Outcome = { + val wallet = new SingleKeyOnChainWallet() + val setup = init(wallet_opt = Some(wallet), tags = test.tags) + import setup._ + + val aliceListener = TestProbe() + alice.underlying.system.eventStream.subscribe(aliceListener.ref, classOf[TransactionPublished]) + alice.underlying.system.eventStream.subscribe(aliceListener.ref, classOf[TransactionConfirmed]) + val bobListener = TestProbe() + bob.underlying.system.eventStream.subscribe(bobListener.ref, classOf[TransactionPublished]) + bob.underlying.system.eventStream.subscribe(bobListener.ref, classOf[TransactionConfirmed]) + + val channelConfig = ChannelConfig.standard + val channelFlags = ChannelFlags.Private + val (aliceParams, bobParams, channelType) = computeFeatures(setup, test.tags, channelFlags) + val aliceInit = Init(aliceParams.initFeatures) + val bobInit = Init(bobParams.initFeatures) + val bobContribution = if (test.tags.contains("no-funding-contribution")) None else Some(TestConstants.nonInitiatorFundingSatoshis) + within(30 seconds) { + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType) + bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, bobContribution, dualFunded = true, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) + alice2blockchain.expectMsgType[SetChannelId] // temporary channel id + bob2blockchain.expectMsgType[SetChannelId] // temporary channel id + alice2bob.expectMsgType[OpenDualFundedChannel] + alice2bob.forward(bob) + bob2alice.expectMsgType[AcceptDualFundedChannel] + bob2alice.forward(alice) + alice2blockchain.expectMsgType[SetChannelId] // final channel id + bob2blockchain.expectMsgType[SetChannelId] // final channel id + + alice2bob.expectMsgType[TxAddInput] + alice2bob.forward(bob) + bobContribution match { + case Some(_) => bob2alice.expectMsgType[TxAddInput] + case None => bob2alice.expectMsgType[TxComplete] + } + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddOutput] + alice2bob.forward(bob) + bobContribution match { + case Some(_) => bob2alice.expectMsgType[TxAddOutput] + case None => bob2alice.expectMsgType[TxComplete] + } + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddOutput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxComplete] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxComplete] + alice2bob.forward(bob) + bob2alice.expectMsgType[CommitSig] + bob2alice.forward(alice) + alice2bob.expectMsgType[CommitSig] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxSignatures] + bob2alice.forward(alice) + // Alice publishes the funding tx. + val fundingTx = aliceListener.expectMsgType[TransactionPublished].tx + assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid) + alice2bob.expectMsgType[TxSignatures] + alice2bob.forward(bob) + awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) + // Bob publishes the funding tx. + assert(bobListener.expectMsgType[TransactionPublished].tx.txid == fundingTx.txid) + assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid) + withFixture(test.toNoArgTest(FixtureParam(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain, aliceListener, bobListener, wallet))) + } + } + + test("recv TxSignatures (duplicate)", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + + val aliceSigs = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.localSigs + val bobSigs = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.localSigs + alice2bob.forward(bob, aliceSigs) + bob2alice.forward(alice, bobSigs) + alice2bob.expectNoMessage(100 millis) + bob2alice.expectNoMessage(100 millis) + assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) + assert(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) + } + + test("re-transmit TxSignatures on reconnection", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + + val aliceInit = Init(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localParams.initFeatures) + val bobInit = Init(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localParams.initFeatures) + val aliceSigs = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.localSigs + val bobSigs = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.localSigs + + alice ! INPUT_DISCONNECTED + awaitCond(alice.stateName == OFFLINE) + bob ! INPUT_DISCONNECTED + awaitCond(bob.stateName == OFFLINE) + alice ! INPUT_RECONNECTED(alice2bob.ref, aliceInit, bobInit) + bob ! INPUT_RECONNECTED(bob2alice.ref, bobInit, aliceInit) + alice2bob.expectMsgType[ChannelReestablish] + alice2bob.forward(bob) + bob2alice.expectMsgType[ChannelReestablish] + bob2alice.forward(alice) + alice2bob.expectMsg(aliceSigs) + bob2alice.expectMsg(bobSigs) + + awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) + awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) + } + + test("recv WatchFundingConfirmedTriggered (initiator)", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx + alice ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx) + assert(aliceListener.expectMsgType[TransactionConfirmed].tx == fundingTx) + assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid) + alice2bob.expectMsgType[ChannelReady] + awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY) + } + + test("recv WatchFundingConfirmedTriggered (non-initiator)", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx + bob ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx) + assert(bobListener.expectMsgType[TransactionConfirmed].tx == fundingTx) + assert(bob2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid) + bob2alice.expectMsgType[ChannelReady] + awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_READY) + } + + test("recv CurrentBlockCount (funding in progress)", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx + val currentBlock = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].waitingSince + 10 + alice ! ProcessCurrentBlockHeight(CurrentBlockHeight(currentBlock)) + // Alice republishes the highest feerate funding tx. + assert(aliceListener.expectMsgType[TransactionPublished].tx.txid == fundingTx.txid) + alice2bob.expectNoMessage(100 millis) + alice2blockchain.expectNoMessage(100 millis) + assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) + } + + test("recv CurrentBlockCount (funding in progress while offline)", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx + val currentBlock = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].waitingSince + 10 + alice ! INPUT_DISCONNECTED + awaitCond(alice.stateName == OFFLINE) + alice ! ProcessCurrentBlockHeight(CurrentBlockHeight(currentBlock)) + // Alice republishes the highest feerate funding tx. + assert(aliceListener.expectMsgType[TransactionPublished].tx.txid == fundingTx.txid) + alice2bob.expectNoMessage(100 millis) + alice2blockchain.expectNoMessage(100 millis) + assert(alice.stateName == OFFLINE) + } + + test("recv CurrentBlockCount (funding double-spent)", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx + val currentBlock = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].waitingSince + 10 + wallet.doubleSpent = Set(fundingTx.txid) + alice ! ProcessCurrentBlockHeight(CurrentBlockHeight(currentBlock)) + alice2bob.expectMsgType[Error] + alice2blockchain.expectNoMessage(100 millis) + awaitCond(wallet.rolledback.map(_.txid) == Seq(fundingTx.txid)) + awaitCond(alice.stateName == CLOSED) + } + + test("recv CurrentBlockCount (funding double-spent while offline)", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx + val currentBlock = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].waitingSince + 10 + alice ! INPUT_DISCONNECTED + awaitCond(alice.stateName == OFFLINE) + wallet.doubleSpent = Set(fundingTx.txid) + alice ! ProcessCurrentBlockHeight(CurrentBlockHeight(currentBlock)) + alice2bob.expectMsgType[Error] + alice2blockchain.expectNoMessage(100 millis) + awaitCond(wallet.rolledback.map(_.txid) == Seq(fundingTx.txid)) + awaitCond(alice.stateName == CLOSED) + } + + test("recv CurrentBlockCount (funding timeout reached)", Tag(ChannelStateTestsTags.DualFunding), Tag("no-funding-contribution")) { f => + import f._ + val timeoutBlock = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].waitingSince + Channel.FUNDING_TIMEOUT_FUNDEE + 1 + bob ! ProcessCurrentBlockHeight(CurrentBlockHeight(timeoutBlock)) + bob2alice.expectMsgType[Error] + bob2blockchain.expectNoMessage(100 millis) + awaitCond(bob.stateName == CLOSED) + } + + test("recv CurrentBlockCount (funding timeout reached while offline)", Tag(ChannelStateTestsTags.DualFunding), Tag("no-funding-contribution")) { f => + import f._ + val timeoutBlock = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].waitingSince + Channel.FUNDING_TIMEOUT_FUNDEE + 1 + bob ! INPUT_DISCONNECTED + awaitCond(bob.stateName == OFFLINE) + bob ! ProcessCurrentBlockHeight(CurrentBlockHeight(timeoutBlock)) + bob2alice.expectMsgType[Error] + bob2blockchain.expectNoMessage(100 millis) + awaitCond(bob.stateName == CLOSED) + } + + test("recv ChannelReady (initiator)", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx + bob ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx) + val channelReady = bob2alice.expectMsgType[ChannelReady] + assert(channelReady.alias_opt.isDefined) + bob2alice.forward(alice) + alice2bob.expectNoMessage(100 millis) + awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].deferred.contains(channelReady)) + awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) + awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_READY) + } + + test("recv ChannelReady (initiator, no remote contribution)", Tag(ChannelStateTestsTags.DualFunding), Tag("no-funding-contribution")) { f => + import f._ + val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx + bob ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx) + val bobChannelReady = bob2alice.expectMsgType[ChannelReady] + assert(bobChannelReady.alias_opt.isDefined) + bob2alice.forward(alice) + val aliceChannelReady = alice2bob.expectMsgType[ChannelReady] + assert(aliceChannelReady.alias_opt.isDefined) + assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid) + assert(alice2blockchain.expectMsgType[WatchFundingLost].txId == fundingTx.txid) + assert(alice2blockchain.expectMsgType[WatchFundingDeeplyBuried].txId == fundingTx.txid) + awaitCond(alice.stateName == NORMAL) + } + + test("recv ChannelReady (non-initiator)", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx + alice ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx) + val channelReady = alice2bob.expectMsgType[ChannelReady] + assert(channelReady.alias_opt.isDefined) + alice2bob.forward(bob) + bob2alice.expectNoMessage(100 millis) + awaitCond(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].deferred.contains(channelReady)) + awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) + awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY) + } + + test("recv Error", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + alice ! Error(ByteVector32.Zeroes, "dual funding d34d") + // We don't force-close yet because we don't know which funding tx will be confirmed. + alice2blockchain.expectNoMessage(100 millis) + alice2bob.expectNoMessage(100 millis) + assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) + } + + test("recv Error (nothing at stake)", Tag(ChannelStateTestsTags.DualFunding), Tag("no-funding-contribution")) { f => + import f._ + val tx = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx + bob ! Error(ByteVector32.Zeroes, "please help me recover my funds") + // We have nothing at stake, but we publish our commitment to help our peer recover their funds more quickly. + assert(bob2blockchain.expectMsgType[PublishFinalTx].tx.txid == tx.txid) + bob2blockchain.expectNoMessage(100 millis) + awaitCond(bob.stateName == CLOSED) + } + + test("recv CMD_CLOSE", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + val sender = TestProbe() + val c = CMD_CLOSE(sender.ref, None, None) + alice ! c + sender.expectMsg(RES_FAILURE(c, CommandUnavailableInThisState(channelId(alice), "close", WAIT_FOR_DUAL_FUNDING_CONFIRMED))) + } + + test("recv CMD_FORCECLOSE", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + val sender = TestProbe() + val c = CMD_FORCECLOSE(sender.ref) + alice ! c + sender.expectMsg(RES_FAILURE(c, CommandUnavailableInThisState(channelId(alice), "force-close", WAIT_FOR_DUAL_FUNDING_CONFIRMED))) + } + +} diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala new file mode 100644 index 0000000000..b391612ebd --- /dev/null +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala @@ -0,0 +1,223 @@ +/* + * Copyright 2022 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.states.c + +import akka.testkit.{TestFSMRef, TestProbe} +import fr.acinq.bitcoin.scalacompat.{ByteVector32, Transaction} +import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ +import fr.acinq.eclair.channel.InteractiveTxBuilder.FullySignedSharedTransaction +import fr.acinq.eclair.channel._ +import fr.acinq.eclair.channel.fsm.Channel +import fr.acinq.eclair.channel.publish.TxPublisher +import fr.acinq.eclair.channel.publish.TxPublisher.SetChannelId +import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags} +import fr.acinq.eclair.payment.relay.Relayer.RelayFees +import fr.acinq.eclair.wire.protocol._ +import fr.acinq.eclair.{BlockHeight, MilliSatoshiLong, TestConstants, TestKitBaseClass} +import org.scalatest.OptionValues.convertOptionToValuable +import org.scalatest.funsuite.FixtureAnyFunSuiteLike +import org.scalatest.{Outcome, Tag} + +import scala.concurrent.duration.DurationInt + +class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ChannelStateTestsBase { + + case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, bob2blockchain: TestProbe) + + override def withFixture(test: OneArgTest): Outcome = { + val setup = init(tags = test.tags) + import setup._ + + val channelConfig = ChannelConfig.standard + val channelFlags = ChannelFlags(announceChannel = test.tags.contains(ChannelStateTestsTags.ChannelsPublic)) + val (aliceParams, bobParams, channelType) = computeFeatures(setup, test.tags, channelFlags) + val aliceInit = Init(aliceParams.initFeatures) + val bobInit = Init(bobParams.initFeatures) + within(30 seconds) { + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.anchorOutputsFeeratePerKw, TestConstants.feeratePerKw, None, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType) + bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, Some(TestConstants.nonInitiatorFundingSatoshis), dualFunded = true, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType) + alice2blockchain.expectMsgType[SetChannelId] // temporary channel id + bob2blockchain.expectMsgType[SetChannelId] // temporary channel id + alice2bob.expectMsgType[OpenDualFundedChannel] + alice2bob.forward(bob) + bob2alice.expectMsgType[AcceptDualFundedChannel] + bob2alice.forward(alice) + alice2blockchain.expectMsgType[SetChannelId] // final channel id + bob2blockchain.expectMsgType[SetChannelId] // final channel id + + alice2bob.expectMsgType[TxAddInput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxAddInput] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddOutput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxAddOutput] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxAddOutput] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxComplete] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxComplete] + alice2bob.forward(bob) + bob2alice.expectMsgType[CommitSig] + bob2alice.forward(alice) + alice2bob.expectMsgType[CommitSig] + alice2bob.forward(bob) + bob2alice.expectMsgType[TxSignatures] + bob2alice.forward(alice) + alice2bob.expectMsgType[TxSignatures] + alice2bob.forward(bob) + if (!test.tags.contains(ChannelStateTestsTags.ZeroConf)) { + val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx + assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid) + assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid) + alice ! WatchFundingConfirmedTriggered(BlockHeight(TestConstants.defaultBlockHeight), 42, fundingTx) + bob ! WatchFundingConfirmedTriggered(BlockHeight(TestConstants.defaultBlockHeight), 42, fundingTx) + } + alice2blockchain.expectMsgType[WatchFundingSpent] + alice2blockchain.expectMsgType[WatchFundingLost] + bob2blockchain.expectMsgType[WatchFundingSpent] + bob2blockchain.expectMsgType[WatchFundingLost] + awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY) + awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_READY) + withFixture(test.toNoArgTest(FixtureParam(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain))) + } + } + + test("recv ChannelReady", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => + import f._ + + alice.underlyingActor.nodeParams.db.peers.addOrUpdateRelayFees(bob.underlyingActor.nodeParams.nodeId, RelayFees(20 msat, 125)) + bob.underlyingActor.nodeParams.db.peers.addOrUpdateRelayFees(alice.underlyingActor.nodeParams.nodeId, RelayFees(25 msat, 90)) + + val aliceChannelReady = alice2bob.expectMsgType[ChannelReady] + alice2bob.forward(bob, aliceChannelReady) + awaitCond(bob.stateName == NORMAL) + val bobChannelReady = bob2alice.expectMsgType[ChannelReady] + bob2alice.forward(alice, bobChannelReady) + awaitCond(alice.stateName == NORMAL) + + assert(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Temporary]) + val aliceCommitments = alice.stateData.asInstanceOf[DATA_NORMAL].commitments + val aliceUpdate = alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate + assert(aliceUpdate.shortChannelId == aliceChannelReady.alias_opt.value) + assert(aliceUpdate.feeBaseMsat == 20.msat) + assert(aliceUpdate.feeProportionalMillionths == 125) + assert(aliceCommitments.localChannelReserve == aliceCommitments.commitInput.txOut.amount / 100) + assert(aliceCommitments.localChannelReserve == aliceCommitments.remoteChannelReserve) + assert(bob.stateData.asInstanceOf[DATA_NORMAL].shortIds.real.isInstanceOf[RealScidStatus.Temporary]) + val bobCommitments = bob.stateData.asInstanceOf[DATA_NORMAL].commitments + val bobUpdate = bob.stateData.asInstanceOf[DATA_NORMAL].channelUpdate + assert(bobUpdate.shortChannelId == bobChannelReady.alias_opt.value) + assert(bobUpdate.feeBaseMsat == 25.msat) + assert(bobUpdate.feeProportionalMillionths == 90) + assert(bobCommitments.localChannelReserve == aliceCommitments.remoteChannelReserve) + assert(bobCommitments.localChannelReserve == bobCommitments.remoteChannelReserve) + + assert(alice2bob.expectMsgType[ChannelUpdate].shortChannelId == bobChannelReady.alias_opt.value) + assert(bob2alice.expectMsgType[ChannelUpdate].shortChannelId == aliceChannelReady.alias_opt.value) + alice2bob.expectNoMessage(100 millis) + bob2alice.expectNoMessage(100 millis) + } + + test("recv ChannelReady (zero-conf)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.ZeroConf)) { f => + import f._ + + val aliceChannelReady = alice2bob.expectMsgType[ChannelReady] + alice2bob.forward(bob, aliceChannelReady) + awaitCond(bob.stateName == NORMAL) + val bobChannelReady = bob2alice.expectMsgType[ChannelReady] + bob2alice.forward(alice, bobChannelReady) + awaitCond(alice.stateName == NORMAL) + + assert(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real == RealScidStatus.Unknown) + val aliceCommitments = alice.stateData.asInstanceOf[DATA_NORMAL].commitments + assert(alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.shortChannelId == aliceChannelReady.alias_opt.value) + assert(aliceCommitments.localChannelReserve == aliceCommitments.commitInput.txOut.amount / 100) + assert(aliceCommitments.localChannelReserve == aliceCommitments.remoteChannelReserve) + assert(bob.stateData.asInstanceOf[DATA_NORMAL].shortIds.real == RealScidStatus.Unknown) + val bobCommitments = bob.stateData.asInstanceOf[DATA_NORMAL].commitments + assert(bob.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.shortChannelId == bobChannelReady.alias_opt.value) + assert(bobCommitments.localChannelReserve == aliceCommitments.remoteChannelReserve) + assert(bobCommitments.localChannelReserve == bobCommitments.remoteChannelReserve) + + assert(alice2bob.expectMsgType[ChannelUpdate].shortChannelId == bobChannelReady.alias_opt.value) + assert(bob2alice.expectMsgType[ChannelUpdate].shortChannelId == aliceChannelReady.alias_opt.value) + alice2bob.expectNoMessage(100 millis) + bob2alice.expectNoMessage(100 millis) + } + + test("recv TxInitRbf", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => + import f._ + alice2bob.expectMsgType[ChannelReady] + alice ! TxInitRbf(channelId(alice), 0, TestConstants.feeratePerKw * 1.1) + alice2bob.expectMsgType[TxAbort] + assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY) + } + + test("recv WatchFundingSpentTriggered (remote commit)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => + import f._ + // bob publishes his commitment tx + val bobCommitTx = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx + alice ! WatchFundingSpentTriggered(bobCommitTx) + alice2blockchain.expectMsgType[TxPublisher.PublishTx] + assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == bobCommitTx.txid) + awaitCond(alice.stateName == CLOSING) + } + + test("recv WatchFundingSpentTriggered (other commit)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => + import f._ + alice2bob.expectMsgType[ChannelReady] + val commitTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx + alice ! WatchFundingSpentTriggered(Transaction(0, Nil, Nil, 0)) + alice2bob.expectMsgType[Error] + assert(alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx].tx.txid == commitTx.txid) + awaitCond(alice.stateName == ERR_INFORMATION_LEAK) + } + + test("recv Error", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => + import f._ + val commitTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx + alice ! Error(ByteVector32.Zeroes, "dual funding failure") + awaitCond(alice.stateName == CLOSING) + assert(alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx].tx.txid == commitTx.txid) + alice2blockchain.expectMsgType[TxPublisher.PublishTx] // commit tx + alice2blockchain.expectMsgType[TxPublisher.PublishTx] // local anchor + assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == commitTx.txid) + } + + test("recv CMD_CLOSE", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => + import f._ + val sender = TestProbe() + val c = CMD_CLOSE(sender.ref, None, None) + alice ! c + sender.expectMsg(RES_FAILURE(c, CommandUnavailableInThisState(channelId(alice), "close", WAIT_FOR_DUAL_FUNDING_READY))) + } + + test("recv CMD_FORCECLOSE", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f => + import f._ + val sender = TestProbe() + val commitTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx + alice ! CMD_FORCECLOSE(sender.ref) + awaitCond(alice.stateName == CLOSING) + assert(alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx].tx.txid == commitTx.txid) + alice2blockchain.expectMsgType[TxPublisher.PublishTx] // commit tx + alice2blockchain.expectMsgType[TxPublisher.PublishTx] // local anchor + assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == commitTx.txid) + } + +} diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala index e76b673459..a4ef930b3d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala @@ -70,7 +70,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with } } - test("recv CMD_ADD_HTLC (empty origin)") { f => + private def testRecvCmdAddHtlcEmptyOrigin(f: FixtureParam): Unit = { import f._ val initialState = alice.stateData.asInstanceOf[DATA_NORMAL] val sender = TestProbe() @@ -92,6 +92,14 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ))) } + test("recv CMD_ADD_HTLC (empty origin)") { f => + testRecvCmdAddHtlcEmptyOrigin(f) + } + + test("recv CMD_ADD_HTLC (empty origin, dual funding)", Tag(ChannelStateTestsTags.DualFunding)) { f => + testRecvCmdAddHtlcEmptyOrigin(f) + } + test("recv CMD_ADD_HTLC (incrementing ids)") { f => import f._ val sender = TestProbe() @@ -3439,6 +3447,17 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with assert(alice.stateData.asInstanceOf[DATA_NORMAL].channelUpdate.shortChannelId == aliceIds.localAlias) } + test("recv WatchFundingDeeplyBuriedTriggered (public channel, dual funding)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.ChannelsPublic)) { f => + import f._ + alice ! WatchFundingDeeplyBuriedTriggered(BlockHeight(400000), 42, null) + val annSigs = alice2bob.expectMsgType[AnnouncementSignatures] + // public channel: we don't send the channel_update directly to the peer + alice2bob.expectNoMessage(1 second) + awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].shortIds.real == RealScidStatus.Final(annSigs.shortChannelId)) + // we don't re-publish the same channel_update if there was no change + channelUpdateListener.expectNoMessage(1 second) + } + test("recv WatchFundingDeeplyBuriedTriggered (public channel, zero-conf)", Tag(ChannelStateTestsTags.ChannelsPublic), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.ZeroConf)) { f => import f._ // in zero-conf channel we don't have a real short channel id when going to NORMAL state diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala index b05f48ec21..2005c37a9a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala @@ -119,6 +119,10 @@ abstract class IntegrationSpec extends TestKitBaseClass with BitcoindService wit s"eclair.features.${AnchorOutputsZeroFeeHtlcTx.rfcName}" -> "optional" ).asJava).withFallback(withStaticRemoteKey) + val withDualFunding = ConfigFactory.parseMap(Map( + s"eclair.features.${DualFunding.rfcName}" -> "optional" + ).asJava).withFallback(withAnchorOutputsZeroFeeHtlcTxs) + implicit val formats: Formats = DefaultFormats override def beforeAll(): Unit = { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala index 635a17013c..ac4414b23c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala @@ -59,9 +59,9 @@ class PaymentIntegrationSpec extends IntegrationSpec { test("start eclair nodes") { instantiateEclairNode("A", ConfigFactory.parseMap(Map("eclair.node-alias" -> "A", "eclair.channel.expiry-delta-blocks" -> 130, "eclair.server.port" -> 29730, "eclair.api.port" -> 28080, "eclair.channel.channel-flags.announce-channel" -> false).asJava).withFallback(withDefaultCommitment).withFallback(commonConfig)) // A's channels are private instantiateEclairNode("B", ConfigFactory.parseMap(Map("eclair.node-alias" -> "B", "eclair.channel.expiry-delta-blocks" -> 131, "eclair.server.port" -> 29731, "eclair.api.port" -> 28081, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(withDefaultCommitment).withFallback(commonConfig)) - instantiateEclairNode("C", ConfigFactory.parseMap(Map("eclair.node-alias" -> "C", "eclair.channel.expiry-delta-blocks" -> 132, "eclair.server.port" -> 29732, "eclair.api.port" -> 28082, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(withAnchorOutputsZeroFeeHtlcTxs).withFallback(commonConfig)) + instantiateEclairNode("C", ConfigFactory.parseMap(Map("eclair.node-alias" -> "C", "eclair.channel.expiry-delta-blocks" -> 132, "eclair.server.port" -> 29732, "eclair.api.port" -> 28082, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(withDualFunding).withFallback(commonConfig)) instantiateEclairNode("D", ConfigFactory.parseMap(Map("eclair.node-alias" -> "D", "eclair.channel.expiry-delta-blocks" -> 133, "eclair.server.port" -> 29733, "eclair.api.port" -> 28083, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(withDefaultCommitment).withFallback(commonConfig)) - instantiateEclairNode("E", ConfigFactory.parseMap(Map("eclair.node-alias" -> "E", "eclair.channel.expiry-delta-blocks" -> 134, "eclair.server.port" -> 29734, "eclair.api.port" -> 28084).asJava).withFallback(withAnchorOutputsZeroFeeHtlcTxs).withFallback(commonConfig)) + instantiateEclairNode("E", ConfigFactory.parseMap(Map("eclair.node-alias" -> "E", "eclair.channel.expiry-delta-blocks" -> 134, "eclair.server.port" -> 29734, "eclair.api.port" -> 28084).asJava).withFallback(withDualFunding).withFallback(commonConfig)) instantiateEclairNode("F", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F", "eclair.channel.expiry-delta-blocks" -> 135, "eclair.server.port" -> 29735, "eclair.api.port" -> 28085, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(commonConfig)) instantiateEclairNode("G", ConfigFactory.parseMap(Map("eclair.node-alias" -> "G", "eclair.channel.expiry-delta-blocks" -> 136, "eclair.server.port" -> 29736, "eclair.api.port" -> 28086, "eclair.relay.fees.public-channels.fee-base-msat" -> 1010, "eclair.relay.fees.public-channels.fee-proportional-millionths" -> 102, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(commonConfig)) } From 075a606665582c54862b63202d2c0fd68b3c4b1a Mon Sep 17 00:00:00 2001 From: t-bast Date: Wed, 17 Aug 2022 18:05:50 +0200 Subject: [PATCH 02/11] Allow force-closing unconfirmed dual funded channels As suggested by @pm47, it's actually not that hard to handle this. Note that we don't allow mutual closing an unconfirmed channel, but that is also the case for single-funded channels. We can improve that in the future if necessary. --- .../fr/acinq/eclair/channel/ChannelData.scala | 8 ++- .../fr/acinq/eclair/channel/fsm/Channel.scala | 19 ++++++- .../channel/fsm/ChannelOpenDualFunded.scala | 8 --- .../eclair/channel/fsm/ErrorHandlers.scala | 51 +++++++------------ .../channel/version0/ChannelCodecs0.scala | 12 +++-- .../channel/version1/ChannelCodecs1.scala | 7 ++- .../channel/version2/ChannelCodecs2.scala | 7 ++- .../channel/version3/ChannelCodecs3.scala | 18 ++++++- .../fr/acinq/eclair/channel/HelpersSpec.scala | 10 ++++ ...WaitForDualFundingConfirmedStateSpec.scala | 44 ++++++++++++---- .../payment/PostRestartHtlcCleanerSpec.scala | 5 +- 11 files changed, 127 insertions(+), 62 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala index 0823908eb5..981a63ed5f 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala @@ -486,7 +486,12 @@ final case class DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments: Commitments, waitingSince: BlockHeight, // how long have we been waiting for a funding tx to confirm lastChecked: BlockHeight, // last time we checked if the channel was double-spent rbfAttempt: Option[typed.ActorRef[InteractiveTxBuilder.Command]], - deferred: Option[ChannelReady]) extends PersistentChannelData + deferred: Option[ChannelReady]) extends PersistentChannelData { + val signedFundingTx_opt: Option[Transaction] = fundingTx match { + case _: PartiallySignedSharedTransaction => None + case tx: FullySignedSharedTransaction => Some(tx.signedTx) + } +} final case class DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments: Commitments, shortIds: ShortIds, otherFundingTxs: Seq[DualFundingTx], @@ -510,6 +515,7 @@ final case class DATA_NEGOTIATING(commitments: Commitments, final case class DATA_CLOSING(commitments: Commitments, fundingTx: Option[Transaction], // this will be non-empty if we are the initiator and we got in closing while waiting for our own tx to be published waitingSince: BlockHeight, // how long since we initiated the closing + alternativeCommitments: List[Commitments], // commitments we signed that spend a different funding output mutualCloseProposed: List[ClosingTx], // all exchanged closing sigs are flattened, we use this only to keep track of what publishable tx they have mutualClosePublished: List[ClosingTx] = Nil, localCommitPublished: Option[LocalCommitPublished] = None, diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index 5c16008acd..fc2e090792 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -1084,6 +1084,23 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val case Event(BITCOIN_FUNDING_TIMEOUT, d: DATA_CLOSING) => handleFundingTimeout(d) + case Event(w: WatchFundingConfirmedTriggered, d: DATA_CLOSING) => + d.alternativeCommitments.find(_.commitInput.outPoint.txid == w.tx.txid) match { + case Some(alternativeCommitments) => + log.info("an alternative funding tx with txid={} got confirmed", w.tx.txid) + val commitTx = alternativeCommitments.fullySignedLocalCommitTx(keyManager).tx + val commitTxs = Set(commitTx.txid, alternativeCommitments.remoteCommit.txid) + blockchain ! WatchFundingSpent(self, alternativeCommitments.commitInput.outPoint.txid, alternativeCommitments.commitInput.outPoint.index.toInt, commitTxs) + context.system.eventStream.publish(TransactionConfirmed(d.channelId, remoteNodeId, w.tx)) + val localCommitPublished = Closing.LocalClose.claimCommitTxOutputs(keyManager, alternativeCommitments, commitTx, nodeParams.currentBlockHeight, nodeParams.onChainFeeConf) + stay() using d.copy(commitments = alternativeCommitments, localCommitPublished = Some(localCommitPublished)) storing() calling doPublish(localCommitPublished, alternativeCommitments) + case None => + if (d.commitments.commitInput.outPoint.txid != w.tx.txid) { + log.warning("an unknown funding tx with txid={} got confirmed, this should not happen", w.tx.txid) + } + stay() + } + case Event(WatchFundingSpentTriggered(tx), d: DATA_CLOSING) => if (d.mutualClosePublished.exists(_.tx.txid == tx.txid)) { // we already know about this tx, probably because we have published it ourselves after successful negotiation @@ -1332,7 +1349,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val val minDepth_opt = Helpers.Funding.minDepthDualFunding(nodeParams.channelConf, d.commitments.channelFeatures, d.fundingParams) val minDepth = minDepth_opt.getOrElse { val defaultMinDepth = nodeParams.channelConf.minDepthBlocks - // If we are in state WAIT_FOR_FUNDING_CONFIRMED, then the computed minDepth should be > 0, otherwise we would + // If we are in state WAIT_FOR_DUAL_FUNDING_CONFIRMED, then the computed minDepth should be > 0, otherwise we would // have skipped this state. Maybe the computation method was changed and eclair was restarted? log.warning("min_depth should be defined since we're waiting for the funding tx to confirm, using default minDepth={}", defaultMinDepth) defaultMinDepth.toLong diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala index 9513085cdf..f697c03f3b 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala @@ -445,14 +445,6 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { context.system.scheduler.scheduleOnce(2 seconds, self, remoteAnnSigs) stay() - case Event(c: CMD_FORCECLOSE, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - // We can't easily force-close until we know which funding transaction confirms. - // A better option would be to double-spend the funding transaction(s). - log.warning("cannot force-close while dual-funded transactions are unconfirmed") - val replyTo = if (c.replyTo == ActorRef.noSender) sender() else c.replyTo - replyTo ! RES_FAILURE(c, CommandUnavailableInThisState(d.channelId, "force-close", stateName)) - stay() - case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleRemoteError(e, d) }) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala index c8764fe5f8..e946b3e306 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala @@ -52,7 +52,7 @@ trait ErrorHandlers extends CommonHandlers { def handleMutualClose(closingTx: ClosingTx, d: Either[DATA_NEGOTIATING, DATA_CLOSING]) = { log.info(s"closing tx published: closingTxId=${closingTx.tx.txid}") val nextData = d match { - case Left(negotiating) => DATA_CLOSING(negotiating.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, negotiating.closingTxProposed.flatten.map(_.unsignedTx), mutualClosePublished = closingTx :: Nil) + case Left(negotiating) => DATA_CLOSING(negotiating.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, mutualCloseProposed = negotiating.closingTxProposed.flatten.map(_.unsignedTx), mutualClosePublished = closingTx :: Nil) case Right(closing) => closing.copy(mutualClosePublished = closing.mutualClosePublished :+ closingTx) } goto(CLOSING) using nextData storing() calling doPublish(closingTx, nextData.commitments.localParams.isInitiator) @@ -143,9 +143,6 @@ trait ErrorHandlers extends CommonHandlers { val commitTx = d.commitments.fullySignedLocalCommitTx(keyManager) txPublisher ! PublishFinalTx(commitTx, 0 sat, None) goto(CLOSED) - case _: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => - log.info("cannot close channel while dual-funding txs are unconfirmed: waiting for a transaction to confirm or be double-spent") - stay() case hasCommitments: PersistentChannelData => spendLocalCurrent(hasCommitments) // NB: we publish the commitment even if we have nothing at stake (in a dataloss situation our peer will send us an error just for that) case _: TransientChannelData => goto(CLOSED) // when there is no commitment yet, we just go to CLOSED state in case an error occurs } @@ -194,21 +191,16 @@ trait ErrorHandlers extends CommonHandlers { log.warning("we have an outdated commitment: will not publish our local tx") stay() } else { - d match { - case _: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => - log.info("cannot spend our commitment while dual-funding txs are unconfirmed: waiting for a transaction to confirm or be double-spent") - stay() - case _ => - val commitTx = d.commitments.fullySignedLocalCommitTx(keyManager).tx - val localCommitPublished = Closing.LocalClose.claimCommitTxOutputs(keyManager, d.commitments, commitTx, nodeParams.currentBlockHeight, nodeParams.onChainFeeConf) - val nextData = d match { - case closing: DATA_CLOSING => closing.copy(localCommitPublished = Some(localCommitPublished)) - case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, negotiating.closingTxProposed.flatten.map(_.unsignedTx), localCommitPublished = Some(localCommitPublished)) - case waitForFundingConfirmed: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.fundingTx, waitingSince = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) - case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) - } - goto(CLOSING) using nextData storing() calling doPublish(localCommitPublished, d.commitments) + val commitTx = d.commitments.fullySignedLocalCommitTx(keyManager).tx + val localCommitPublished = Closing.LocalClose.claimCommitTxOutputs(keyManager, d.commitments, commitTx, nodeParams.currentBlockHeight, nodeParams.onChainFeeConf) + val nextData = d match { + case closing: DATA_CLOSING => closing.copy(localCommitPublished = Some(localCommitPublished)) + case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, negotiating.closingTxProposed.flatten.map(_.unsignedTx), localCommitPublished = Some(localCommitPublished)) + case waitForFundingConfirmed: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.fundingTx, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) + case waitForFundingConfirmed: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.signedFundingTx_opt, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = waitForFundingConfirmed.previousFundingTxs.map(_.commitments).toList, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) + case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) } + goto(CLOSING) using nextData storing() calling doPublish(localCommitPublished, d.commitments) } } @@ -249,15 +241,10 @@ trait ErrorHandlers extends CommonHandlers { val remoteCommitPublished = Closing.RemoteClose.claimCommitTxOutputs(keyManager, d.commitments, d.commitments.remoteCommit, commitTx, nodeParams.currentBlockHeight, nodeParams.onChainFeeConf) val nextData = d match { case closing: DATA_CLOSING => closing.copy(remoteCommitPublished = Some(remoteCommitPublished)) - case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, negotiating.closingTxProposed.flatten.map(_.unsignedTx), remoteCommitPublished = Some(remoteCommitPublished)) - case waitForFundingConfirmed: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.fundingTx, waitingSince = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished)) - case waitForFundingConfirmed: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => - val dualFundedTx_opt = waitForFundingConfirmed.fundingTx match { - case _: InteractiveTxBuilder.PartiallySignedSharedTransaction => None - case tx: InteractiveTxBuilder.FullySignedSharedTransaction => Some(tx.signedTx) - } - DATA_CLOSING(d.commitments, fundingTx = dualFundedTx_opt, waitingSince = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished)) - case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished)) + case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, mutualCloseProposed = negotiating.closingTxProposed.flatten.map(_.unsignedTx), remoteCommitPublished = Some(remoteCommitPublished)) + case waitForFundingConfirmed: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.fundingTx, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished)) + case waitForFundingConfirmed: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.signedFundingTx_opt, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = waitForFundingConfirmed.previousFundingTxs.map(_.commitments).toList, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished)) + case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished)) } goto(CLOSING) using nextData storing() calling doPublish(remoteCommitPublished, d.commitments) } @@ -272,7 +259,7 @@ trait ErrorHandlers extends CommonHandlers { claimHtlcTxs = Map.empty, claimAnchorTxs = List.empty, irrevocablySpent = Map.empty) - val nextData = DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, Nil, futureRemoteCommitPublished = Some(remoteCommitPublished)) + val nextData = DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, Nil, Nil, futureRemoteCommitPublished = Some(remoteCommitPublished)) goto(CLOSING) using nextData storing() calling doPublish(remoteCommitPublished, d.commitments) } @@ -287,9 +274,9 @@ trait ErrorHandlers extends CommonHandlers { val remoteCommitPublished = Closing.RemoteClose.claimCommitTxOutputs(keyManager, d.commitments, remoteCommit, commitTx, nodeParams.currentBlockHeight, nodeParams.onChainFeeConf) val nextData = d match { case closing: DATA_CLOSING => closing.copy(nextRemoteCommitPublished = Some(remoteCommitPublished)) - case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, negotiating.closingTxProposed.flatten.map(_.unsignedTx), nextRemoteCommitPublished = Some(remoteCommitPublished)) + case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, mutualCloseProposed = negotiating.closingTxProposed.flatten.map(_.unsignedTx), nextRemoteCommitPublished = Some(remoteCommitPublished)) // NB: if there is a next commitment, we can't be in DATA_WAIT_FOR_FUNDING_CONFIRMED so we don't have the case where fundingTx is defined - case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, nextRemoteCommitPublished = Some(remoteCommitPublished)) + case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, mutualCloseProposed = Nil, nextRemoteCommitPublished = Some(remoteCommitPublished)) } goto(CLOSING) using nextData storing() calling doPublish(remoteCommitPublished, d.commitments) } @@ -323,9 +310,9 @@ trait ErrorHandlers extends CommonHandlers { val nextData = d match { case closing: DATA_CLOSING => closing.copy(revokedCommitPublished = closing.revokedCommitPublished :+ revokedCommitPublished) - case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, negotiating.closingTxProposed.flatten.map(_.unsignedTx), revokedCommitPublished = revokedCommitPublished :: Nil) + case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, mutualCloseProposed = negotiating.closingTxProposed.flatten.map(_.unsignedTx), revokedCommitPublished = revokedCommitPublished :: Nil) // NB: if there is a revoked commitment, we can't be in DATA_WAIT_FOR_FUNDING_CONFIRMED so we don't have the case where fundingTx is defined - case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, mutualCloseProposed = Nil, revokedCommitPublished = revokedCommitPublished :: Nil) + case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, mutualCloseProposed = Nil, revokedCommitPublished = revokedCommitPublished :: Nil) } goto(CLOSING) using nextData storing() calling doPublish(revokedCommitPublished) sending error case None => diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version0/ChannelCodecs0.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version0/ChannelCodecs0.scala index c67fe3573d..9acb5c8bd6 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version0/ChannelCodecs0.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version0/ChannelCodecs0.scala @@ -26,7 +26,7 @@ import fr.acinq.eclair.wire.internal.channel.version0.ChannelTypes0.{HtlcTxAndSi import fr.acinq.eclair.wire.protocol.CommonCodecs._ import fr.acinq.eclair.wire.protocol.LightningMessageCodecs.{channelAnnouncementCodec, channelUpdateCodec, combinedFeaturesCodec} import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{BlockHeight, Alias, TimestampSecond} +import fr.acinq.eclair.{Alias, BlockHeight, TimestampSecond} import scodec.Codec import scodec.bits.{BitVector, ByteVector} import scodec.codecs._ @@ -433,7 +433,10 @@ private[channel] object ChannelCodecs0 { ("remoteCommitPublished" | optional(bool, remoteCommitPublishedCodec)) :: ("nextRemoteCommitPublished" | optional(bool, remoteCommitPublishedCodec)) :: ("futureRemoteCommitPublished" | optional(bool, remoteCommitPublishedCodec)) :: - ("revokedCommitPublished" | listOfN(uint16, revokedCommitPublishedCodec))).as[DATA_CLOSING].decodeOnly + ("revokedCommitPublished" | listOfN(uint16, revokedCommitPublishedCodec))).map { + case commitments :: fundingTx :: waitingSince :: mutualCloseProposed :: mutualClosePublished :: localCommitPublished :: remoteCommitPublished :: nextRemoteCommitPublished :: futureRemoteCommitPublished :: revokedCommitPublished :: HNil => + DATA_CLOSING(commitments, fundingTx, waitingSince, Nil, mutualCloseProposed, mutualClosePublished, localCommitPublished, remoteCommitPublished, nextRemoteCommitPublished, futureRemoteCommitPublished, revokedCommitPublished) + }.decodeOnly val DATA_CLOSING_09_Codec: Codec[DATA_CLOSING] = ( ("commitments" | commitmentsCodec) :: @@ -445,7 +448,10 @@ private[channel] object ChannelCodecs0 { ("remoteCommitPublished" | optional(bool, remoteCommitPublishedCodec)) :: ("nextRemoteCommitPublished" | optional(bool, remoteCommitPublishedCodec)) :: ("futureRemoteCommitPublished" | optional(bool, remoteCommitPublishedCodec)) :: - ("revokedCommitPublished" | listOfN(uint16, revokedCommitPublishedCodec))).as[DATA_CLOSING].decodeOnly + ("revokedCommitPublished" | listOfN(uint16, revokedCommitPublishedCodec))).map { + case commitments :: fundingTx :: waitingSince :: mutualCloseProposed :: mutualClosePublished :: localCommitPublished :: remoteCommitPublished :: nextRemoteCommitPublished :: futureRemoteCommitPublished :: revokedCommitPublished :: HNil => + DATA_CLOSING(commitments, fundingTx, waitingSince, Nil, mutualCloseProposed, mutualClosePublished, localCommitPublished, remoteCommitPublished, nextRemoteCommitPublished, futureRemoteCommitPublished, revokedCommitPublished) + }.decodeOnly val channelReestablishCodec: Codec[ChannelReestablish] = ( ("channelId" | bytes32) :: diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version1/ChannelCodecs1.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version1/ChannelCodecs1.scala index 97be45a049..e6cc8fe002 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version1/ChannelCodecs1.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version1/ChannelCodecs1.scala @@ -18,7 +18,6 @@ package fr.acinq.eclair.wire.internal.channel.version1 import fr.acinq.bitcoin.scalacompat.DeterministicWallet.{ExtendedPrivateKey, KeyPath} import fr.acinq.bitcoin.scalacompat.{ByteVector32, OutPoint, Transaction, TxOut} -import fr.acinq.eclair.{Alias, BlockHeight} import fr.acinq.eclair.channel._ import fr.acinq.eclair.crypto.ShaChain import fr.acinq.eclair.transactions.Transactions._ @@ -28,6 +27,7 @@ import fr.acinq.eclair.wire.internal.channel.version0.ChannelTypes0.{HtlcTxAndSi import fr.acinq.eclair.wire.protocol.CommonCodecs._ import fr.acinq.eclair.wire.protocol.LightningMessageCodecs._ import fr.acinq.eclair.wire.protocol._ +import fr.acinq.eclair.{Alias, BlockHeight} import scodec.bits.ByteVector import scodec.codecs._ import scodec.{Attempt, Codec} @@ -283,7 +283,10 @@ private[channel] object ChannelCodecs1 { ("remoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) :: ("nextRemoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) :: ("futureRemoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) :: - ("revokedCommitPublished" | listOfN(uint16, revokedCommitPublishedCodec))).as[DATA_CLOSING] + ("revokedCommitPublished" | listOfN(uint16, revokedCommitPublishedCodec))).map { + case commitments :: fundingTx :: waitingSince :: mutualCloseProposed :: mutualClosePublished :: localCommitPublished :: remoteCommitPublished :: nextRemoteCommitPublished :: futureRemoteCommitPublished :: revokedCommitPublished :: HNil => + DATA_CLOSING(commitments, fundingTx, waitingSince, Nil, mutualCloseProposed, mutualClosePublished, localCommitPublished, remoteCommitPublished, nextRemoteCommitPublished, futureRemoteCommitPublished, revokedCommitPublished) + }.decodeOnly val DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT_26_Codec: Codec[DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT] = ( ("commitments" | commitmentsCodec) :: diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version2/ChannelCodecs2.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version2/ChannelCodecs2.scala index fb132d33ba..bfeb9ee0b3 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version2/ChannelCodecs2.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version2/ChannelCodecs2.scala @@ -18,7 +18,6 @@ package fr.acinq.eclair.wire.internal.channel.version2 import fr.acinq.bitcoin.scalacompat.DeterministicWallet.{ExtendedPrivateKey, KeyPath} import fr.acinq.bitcoin.scalacompat.{OutPoint, Transaction, TxOut} -import fr.acinq.eclair.{Alias, BlockHeight} import fr.acinq.eclair.channel._ import fr.acinq.eclair.crypto.ShaChain import fr.acinq.eclair.transactions.Transactions._ @@ -28,6 +27,7 @@ import fr.acinq.eclair.wire.internal.channel.version0.ChannelTypes0.{HtlcTxAndSi import fr.acinq.eclair.wire.protocol.CommonCodecs._ import fr.acinq.eclair.wire.protocol.LightningMessageCodecs._ import fr.acinq.eclair.wire.protocol._ +import fr.acinq.eclair.{Alias, BlockHeight} import scodec.bits.ByteVector import scodec.codecs._ import scodec.{Attempt, Codec} @@ -318,7 +318,10 @@ private[channel] object ChannelCodecs2 { ("remoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) :: ("nextRemoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) :: ("futureRemoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) :: - ("revokedCommitPublished" | listOfN(uint16, revokedCommitPublishedCodec))).as[DATA_CLOSING] + ("revokedCommitPublished" | listOfN(uint16, revokedCommitPublishedCodec))).map { + case commitments :: fundingTx :: waitingSince :: mutualCloseProposed :: mutualClosePublished :: localCommitPublished :: remoteCommitPublished :: nextRemoteCommitPublished :: futureRemoteCommitPublished :: revokedCommitPublished :: HNil => + DATA_CLOSING(commitments, fundingTx, waitingSince, Nil, mutualCloseProposed, mutualClosePublished, localCommitPublished, remoteCommitPublished, nextRemoteCommitPublished, futureRemoteCommitPublished, revokedCommitPublished) + }.decodeOnly val DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT_06_Codec: Codec[DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT] = ( ("commitments" | commitmentsCodec) :: diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala index fa20a95d9a..9c288304d6 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala @@ -456,7 +456,6 @@ private[channel] object ChannelCodecs3 { val DATA_CLOSING_05_Codec: Codec[DATA_CLOSING] = ( ("commitments" | commitmentsCodec) :: ("fundingTx" | optional(bool8, txCodec)) :: - // TODO: next time we define a new channel codec version, we should use the blockHeight codec here (32 bytes) ("waitingSince" | int64.as[BlockHeight]) :: ("mutualCloseProposed" | listOfN(uint16, closingTxCodec)) :: ("mutualClosePublished" | listOfN(uint16, closingTxCodec)) :: @@ -464,6 +463,22 @@ private[channel] object ChannelCodecs3 { ("remoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) :: ("nextRemoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) :: ("futureRemoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) :: + ("revokedCommitPublished" | listOfN(uint16, revokedCommitPublishedCodec))).map { + case commitments :: fundingTx :: waitingSince :: mutualCloseProposed :: mutualClosePublished :: localCommitPublished :: remoteCommitPublished :: nextRemoteCommitPublished :: futureRemoteCommitPublished :: revokedCommitPublished :: HNil => + DATA_CLOSING(commitments, fundingTx, waitingSince, Nil, mutualCloseProposed, mutualClosePublished, localCommitPublished, remoteCommitPublished, nextRemoteCommitPublished, futureRemoteCommitPublished, revokedCommitPublished) + }.decodeOnly + + val DATA_CLOSING_0d_Codec: Codec[DATA_CLOSING] = ( + ("commitments" | commitmentsCodec) :: + ("fundingTx" | optional(bool8, txCodec)) :: + ("waitingSince" | blockHeight) :: + ("alternativeCommitments" | listOfN(uint16, commitmentsCodec)) :: + ("mutualCloseProposed" | listOfN(uint16, closingTxCodec)) :: + ("mutualClosePublished" | listOfN(uint16, closingTxCodec)) :: + ("localCommitPublished" | optional(bool8, localCommitPublishedCodec)) :: + ("remoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) :: + ("nextRemoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) :: + ("futureRemoteCommitPublished" | optional(bool8, remoteCommitPublishedCodec)) :: ("revokedCommitPublished" | listOfN(uint16, revokedCommitPublishedCodec))).as[DATA_CLOSING] val DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT_06_Codec: Codec[DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT] = ( @@ -473,6 +488,7 @@ private[channel] object ChannelCodecs3 { // Order matters! val channelDataCodec: Codec[PersistentChannelData] = discriminated[PersistentChannelData].by(uint16) + .typecase(0x0d, Codecs.DATA_CLOSING_0d_Codec) .typecase(0x0c, Codecs.DATA_WAIT_FOR_DUAL_FUNDING_READY_0c_Codec) .typecase(0x0b, Codecs.DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED_0b_Codec) .typecase(0x0a, Codecs.DATA_WAIT_FOR_CHANNEL_READY_0a_Codec) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/HelpersSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/HelpersSpec.scala index c5c51257c3..c2736562ae 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/HelpersSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/HelpersSpec.scala @@ -258,6 +258,7 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat commitments = commitments, fundingTx = None, waitingSince = BlockHeight(0), + alternativeCommitments = Nil, mutualCloseProposed = tx1 :: tx2 :: tx3 :: Nil, mutualClosePublished = tx2 :: tx3 :: Nil, localCommitPublished = None, @@ -273,6 +274,7 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat commitments = commitments, fundingTx = None, waitingSince = BlockHeight(0), + alternativeCommitments = Nil, mutualCloseProposed = tx1 :: Nil, mutualClosePublished = tx1 :: Nil, localCommitPublished = Some(LocalCommitPublished( @@ -295,6 +297,7 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat commitments = commitments, fundingTx = None, waitingSince = BlockHeight(0), + alternativeCommitments = Nil, mutualCloseProposed = tx1 :: Nil, mutualClosePublished = tx1 :: Nil, localCommitPublished = Some(LocalCommitPublished( @@ -317,6 +320,7 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat commitments = commitments, fundingTx = None, waitingSince = BlockHeight(0), + alternativeCommitments = Nil, mutualCloseProposed = Nil, mutualClosePublished = Nil, localCommitPublished = Some(LocalCommitPublished( @@ -345,6 +349,7 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat commitments = commitments, fundingTx = None, waitingSince = BlockHeight(0), + alternativeCommitments = Nil, mutualCloseProposed = tx1 :: Nil, mutualClosePublished = tx1 :: Nil, localCommitPublished = Some(LocalCommitPublished( @@ -373,6 +378,7 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat commitments = commitments.copy(remoteNextCommitInfo = Left(WaitingForRevocation(commitments.remoteCommit, null, 7L))), fundingTx = None, waitingSince = BlockHeight(0), + alternativeCommitments = Nil, mutualCloseProposed = tx1 :: Nil, mutualClosePublished = tx1 :: Nil, localCommitPublished = Some(LocalCommitPublished( @@ -407,6 +413,7 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat commitments = commitments, fundingTx = None, waitingSince = BlockHeight(0), + alternativeCommitments = Nil, mutualCloseProposed = Nil, mutualClosePublished = Nil, localCommitPublished = None, @@ -428,6 +435,7 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat commitments = commitments, fundingTx = None, waitingSince = BlockHeight(0), + alternativeCommitments = Nil, mutualCloseProposed = Nil, mutualClosePublished = Nil, localCommitPublished = None, @@ -449,6 +457,7 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat commitments = commitments, fundingTx = None, waitingSince = BlockHeight(0), + alternativeCommitments = Nil, mutualCloseProposed = Nil, mutualClosePublished = Nil, localCommitPublished = Some(LocalCommitPublished( @@ -496,6 +505,7 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat commitments = commitments, fundingTx = None, waitingSince = BlockHeight(0), + alternativeCommitments = Nil, mutualCloseProposed = Nil, mutualClosePublished = Nil, localCommitPublished = Some(LocalCommitPublished( diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala index 613685911c..20edebc79f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala @@ -24,6 +24,7 @@ import fr.acinq.eclair.channel.InteractiveTxBuilder.FullySignedSharedTransaction import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel import fr.acinq.eclair.channel.fsm.Channel.ProcessCurrentBlockHeight +import fr.acinq.eclair.channel.publish.TxPublisher import fr.acinq.eclair.channel.publish.TxPublisher.{PublishFinalTx, SetChannelId} import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags} import fr.acinq.eclair.wire.protocol._ @@ -279,19 +280,39 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture test("recv Error", Tag(ChannelStateTestsTags.DualFunding)) { f => import f._ + val tx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx alice ! Error(ByteVector32.Zeroes, "dual funding d34d") - // We don't force-close yet because we don't know which funding tx will be confirmed. - alice2blockchain.expectNoMessage(100 millis) - alice2bob.expectNoMessage(100 millis) - assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) + awaitCond(alice.stateName == CLOSING) + assert(alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx].tx.txid == tx.txid) + alice2blockchain.expectMsgType[TxPublisher.PublishTx] // claim-main-delayed + assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == tx.txid) + } + + test("recv Error (remote commit published)", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + val aliceCommitTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx + alice ! Error(ByteVector32.Zeroes, "force-closing channel, bye-bye") + awaitCond(alice.stateName == CLOSING) + assert(alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx].tx.txid == aliceCommitTx.txid) + val claimMainLocal = alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx] + assert(claimMainLocal.input.txid == aliceCommitTx.txid) + assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == aliceCommitTx.txid) + assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMainLocal.tx.txid) + // Bob broadcasts his commit tx as well. + val bobCommitTx = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx + alice ! WatchFundingSpentTriggered(bobCommitTx) + val claimMainRemote = alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx] + assert(claimMainRemote.input.txid == bobCommitTx.txid) + assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == bobCommitTx.txid) + assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMainRemote.tx.txid) } test("recv Error (nothing at stake)", Tag(ChannelStateTestsTags.DualFunding), Tag("no-funding-contribution")) { f => import f._ - val tx = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx + val commitTx = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx bob ! Error(ByteVector32.Zeroes, "please help me recover my funds") // We have nothing at stake, but we publish our commitment to help our peer recover their funds more quickly. - assert(bob2blockchain.expectMsgType[PublishFinalTx].tx.txid == tx.txid) + assert(bob2blockchain.expectMsgType[PublishFinalTx].tx.txid == commitTx.txid) bob2blockchain.expectNoMessage(100 millis) awaitCond(bob.stateName == CLOSED) } @@ -307,9 +328,14 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture test("recv CMD_FORCECLOSE", Tag(ChannelStateTestsTags.DualFunding)) { f => import f._ val sender = TestProbe() - val c = CMD_FORCECLOSE(sender.ref) - alice ! c - sender.expectMsg(RES_FAILURE(c, CommandUnavailableInThisState(channelId(alice), "force-close", WAIT_FOR_DUAL_FUNDING_CONFIRMED))) + val commitTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx + alice ! CMD_FORCECLOSE(sender.ref) + awaitCond(alice.stateName == CLOSING) + assert(alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx].tx.txid == commitTx.txid) + val claimMain = alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx] + assert(claimMain.input.txid == commitTx.txid) + assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == commitTx.txid) + assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMain.tx.txid) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PostRestartHtlcCleanerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PostRestartHtlcCleanerSpec.scala index 65219b3433..5704e160ff 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PostRestartHtlcCleanerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PostRestartHtlcCleanerSpec.scala @@ -31,7 +31,6 @@ import fr.acinq.eclair.payment.OutgoingPaymentPacket.{Upstream, buildCommand} import fr.acinq.eclair.payment.PaymentPacketSpec._ import fr.acinq.eclair.payment.relay.{PostRestartHtlcCleaner, Relayer} import fr.acinq.eclair.router.BaseRouterSpec.channelHopFromUpdate -import fr.acinq.eclair.router.Router.ChannelHop import fr.acinq.eclair.transactions.Transactions.{ClaimRemoteDelayedOutputTx, InputInfo} import fr.acinq.eclair.transactions.{DirectedHtlc, IncomingHtlc, OutgoingHtlc} import fr.acinq.eclair.wire.internal.channel.ChannelCodecsSpec @@ -474,12 +473,12 @@ class PostRestartHtlcCleanerSpec extends TestKitBaseClass with FixtureAnyFunSuit val dummyClaimMainTx = Transaction(2, Seq(TxIn(OutPoint(revokedCommitTx, 0), Nil, 0)), Seq(revokedCommitTx.txOut.head.copy(amount = 4000 sat)), 0) val dummyClaimMain = ClaimRemoteDelayedOutputTx(InputInfo(OutPoint(revokedCommitTx, 0), revokedCommitTx.txOut.head, Nil), dummyClaimMainTx) val rcp = RevokedCommitPublished(revokedCommitTx, Some(dummyClaimMain), None, Nil, Nil, Map(revokedCommitTx.txIn.head.outPoint -> revokedCommitTx)) - DATA_CLOSING(normal.commitments, None, BlockHeight(0), Nil, revokedCommitPublished = List(rcp)) + DATA_CLOSING(normal.commitments, None, BlockHeight(0), Nil, Nil, revokedCommitPublished = List(rcp)) } nodeParams.db.channels.addOrUpdateChannel(upstreamChannel) nodeParams.db.channels.addOrUpdateChannel(downstreamChannel) - assert(Closing.isClosed(downstreamChannel, None) == None) + assert(Closing.isClosed(downstreamChannel, None).isEmpty) val (_, postRestart) = f.createRelayer(nodeParams) sender.send(postRestart, PostRestartHtlcCleaner.GetBrokenHtlcs) From bd479921eae2270b731cd1d03457bdae20acb58c Mon Sep 17 00:00:00 2001 From: t-bast Date: Thu, 18 Aug 2022 11:18:38 +0200 Subject: [PATCH 03/11] Watch funding txs as soon as published This aligns with what is done in the single-funded case and lets us detect channel force-close while offline. --- .../fr/acinq/eclair/channel/ChannelData.scala | 1 - .../fr/acinq/eclair/channel/fsm/Channel.scala | 17 +++++--- .../channel/fsm/ChannelOpenDualFunded.scala | 29 ++++++++----- .../eclair/channel/fsm/ErrorHandlers.scala | 20 ++++++++- .../channel/version3/ChannelCodecs3.scala | 1 - .../ChannelStateTestsHelperMethods.scala | 6 ++- ...WaitForDualFundingConfirmedStateSpec.scala | 41 ++++++++++++++++++- .../c/WaitForDualFundingReadyStateSpec.scala | 6 ++- 8 files changed, 95 insertions(+), 26 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala index 981a63ed5f..4bc6385580 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala @@ -494,7 +494,6 @@ final case class DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments: Commitments, } final case class DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments: Commitments, shortIds: ShortIds, - otherFundingTxs: Seq[DualFundingTx], lastSent: ChannelReady) extends PersistentChannelData final case class DATA_NORMAL(commitments: Commitments, diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index fc2e090792..f7f291f221 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -315,8 +315,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val } case funding: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => + (funding.commitments +: funding.previousFundingTxs.map(_.commitments)).foreach(c => watchFundingTx(c)) // we make sure that the funding tx with the highest feerate has been published - // NB: with dual-funding, we only watch the funding tx once it has been confirmed publishFundingTx(funding.fundingParams, funding.fundingTx) goto(OFFLINE) using funding @@ -1088,10 +1088,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val d.alternativeCommitments.find(_.commitInput.outPoint.txid == w.tx.txid) match { case Some(alternativeCommitments) => log.info("an alternative funding tx with txid={} got confirmed", w.tx.txid) - val commitTx = alternativeCommitments.fullySignedLocalCommitTx(keyManager).tx - val commitTxs = Set(commitTx.txid, alternativeCommitments.remoteCommit.txid) - blockchain ! WatchFundingSpent(self, alternativeCommitments.commitInput.outPoint.txid, alternativeCommitments.commitInput.outPoint.index.toInt, commitTxs) + watchFundingTx(alternativeCommitments) context.system.eventStream.publish(TransactionConfirmed(d.channelId, remoteNodeId, w.tx)) + val commitTx = alternativeCommitments.fullySignedLocalCommitTx(keyManager).tx val localCommitPublished = Closing.LocalClose.claimCommitTxOutputs(keyManager, alternativeCommitments, commitTx, nodeParams.currentBlockHeight, nodeParams.onChainFeeConf) stay() using d.copy(commitments = alternativeCommitments, localCommitPublished = Some(localCommitPublished)) storing() calling doPublish(localCommitPublished, alternativeCommitments) case None => @@ -1127,6 +1126,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val } else if (d.commitments.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)) { // counterparty may attempt to spend its last commit tx at any time handleRemoteSpentNext(tx, d) + } else if (d.alternativeCommitments.exists(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid))) { + // counterparty may attempt to spend an alternative unconfirmed funding tx at any time + handleRemoteSpentAlternative(tx, d.alternativeCommitments, d) } else { // counterparty may attempt to spend a revoked commit tx at any time handleRemoteSpentOther(tx, d) @@ -1320,6 +1322,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val case Event(WatchFundingSpentTriggered(tx), d: PersistentChannelData) if d.commitments.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid) => handleRemoteSpentNext(tx, d) + case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) if d.previousFundingTxs.map(_.commitments).exists(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)) => handleRemoteSpentAlternative(tx, d.previousFundingTxs.map(_.commitments).toList, d) + case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT) => handleRemoteSpentFuture(tx, d) case Event(WatchFundingSpentTriggered(tx), d: PersistentChannelData) => handleRemoteSpentOther(tx, d) @@ -1515,13 +1519,14 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val case Event(WatchFundingDeeplyBuriedTriggered(_, _, _), _) => stay() - case Event(WatchFundingSpentTriggered(tx), d: DATA_NEGOTIATING) if d.closingTxProposed.flatten.exists(_.unsignedTx.tx.txid == tx.txid) => - handleMutualClose(getMutualClosePublished(tx, d.closingTxProposed), Left(d)) + case Event(WatchFundingSpentTriggered(tx), d: DATA_NEGOTIATING) if d.closingTxProposed.flatten.exists(_.unsignedTx.tx.txid == tx.txid) => handleMutualClose(getMutualClosePublished(tx, d.closingTxProposed), Left(d)) case Event(WatchFundingSpentTriggered(tx), d: PersistentChannelData) if tx.txid == d.commitments.remoteCommit.txid => handleRemoteSpentCurrent(tx, d) case Event(WatchFundingSpentTriggered(tx), d: PersistentChannelData) if d.commitments.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid) => handleRemoteSpentNext(tx, d) + case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) if d.previousFundingTxs.map(_.commitments).exists(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)) => handleRemoteSpentAlternative(tx, d.previousFundingTxs.map(_.commitments).toList, d) + case Event(WatchFundingSpentTriggered(tx), d: PersistentChannelData) => handleRemoteSpentOther(tx, d) case Event(e: Error, d: PersistentChannelData) => handleRemoteError(e, d) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala index f697c03f3b..5404be23a6 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala @@ -16,7 +16,6 @@ package fr.acinq.eclair.channel.fsm -import akka.actor.ActorRef import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapter} import fr.acinq.bitcoin.ScriptFlags import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Script, Transaction} @@ -309,6 +308,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { case InteractiveTxBuilder.SendMessage(msg) => stay() sending msg case InteractiveTxBuilder.Succeeded(fundingParams, fundingTx, commitments) => d.deferred.foreach(self ! _) + watchFundingTx(commitments) Funding.minDepthDualFunding(nodeParams.channelConf, commitments.channelFeatures, fundingParams) match { case Some(fundingMinDepth) => blockchain ! WatchFundingConfirmed(self, commitments.commitInput.outPoint.txid, fundingMinDepth) @@ -318,10 +318,8 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { case fundingTx: FullySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using nextData storing() sending fundingTx.localSigs calling publishFundingTx(fundingParams, fundingTx) } case None => - val commitTxs = Set(commitments.localCommit.commitTxAndRemoteSig.commitTx.tx.txid, commitments.remoteCommit.txid) - blockchain ! WatchFundingSpent(self, commitments.commitInput.outPoint.txid, commitments.commitInput.outPoint.index.toInt, commitTxs) val (shortIds, channelReady) = acceptFundingTx(commitments, RealScidStatus.Unknown) - val nextData = DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments, shortIds, Nil, channelReady) + val nextData = DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments, shortIds, channelReady) fundingTx match { case fundingTx: PartiallySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_READY) using nextData storing() sending Seq(fundingTx.localSigs, channelReady) case fundingTx: FullySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_READY) using nextData storing() sending Seq(fundingTx.localSigs, channelReady) calling publishFundingTx(fundingParams, fundingTx) @@ -395,14 +393,12 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { Try(Transaction.correctlySpends(commitments.fullySignedLocalCommitTx(keyManager).tx, Seq(confirmedTx), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) match { case Success(_) => log.info(s"channelId=${commitments.channelId} was confirmed at blockHeight=$blockHeight txIndex=$txIndex with funding txid=${commitments.commitInput.outPoint.txid}") - val commitTxs = Set(commitments.localCommit.commitTxAndRemoteSig.commitTx.tx.txid, commitments.remoteCommit.txid) - blockchain ! WatchFundingSpent(self, commitments.commitInput.outPoint.txid, commitments.commitInput.outPoint.index.toInt, commitTxs) + watchFundingTx(commitments) context.system.eventStream.publish(TransactionConfirmed(commitments.channelId, remoteNodeId, confirmedTx)) val realScidStatus = RealScidStatus.Temporary(RealShortChannelId(blockHeight, txIndex, commitments.commitInput.outPoint.index.toInt)) val (shortIds, channelReady) = acceptFundingTx(commitments, realScidStatus = realScidStatus) d.deferred.foreach(self ! _) - val otherFundingTxs = allFundingTxs.filter(_.commitments.commitInput.outPoint.txid != confirmedTx.txid) - goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments, shortIds, otherFundingTxs, channelReady) storing() sending channelReady + goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments, shortIds, channelReady) storing() sending channelReady case Failure(t) => log.error(t, s"rejecting channel with invalid funding tx: ${confirmedTx.bin}") allFundingTxs.foreach(f => wallet.rollback(f.fundingTx.tx.buildUnsignedTx())) @@ -427,12 +423,10 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { (d.fundingParams.remoteAmount == 0.sat || d.commitments.localParams.initFeatures.hasFeature(Features.ZeroConf)) if (canUseZeroConf) { log.info("this chanel isn't zero-conf, but they sent an early channel_ready with an alias: no need to wait for confirmations") - val commitTxs = Set(d.commitments.localCommit.commitTxAndRemoteSig.commitTx.tx.txid, d.commitments.remoteCommit.txid) - blockchain ! WatchFundingSpent(self, d.commitments.commitInput.outPoint.txid, d.commitments.commitInput.outPoint.index.toInt, commitTxs) val (shortIds, localChannelReady) = acceptFundingTx(d.commitments, RealScidStatus.Unknown) self ! remoteChannelReady // NB: we will receive a WatchFundingConfirmedTriggered later that will simply be ignored - goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(d.commitments, shortIds, Nil, localChannelReady) storing() sending localChannelReady + goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(d.commitments, shortIds, localChannelReady) storing() sending localChannelReady } else { log.info("received their channel_ready, deferring message") stay() using d.copy(deferred = Some(remoteChannelReady)) // no need to store, they will re-send if we get disconnected @@ -445,6 +439,19 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { context.system.scheduler.scheduleOnce(2 seconds, self, remoteAnnSigs) stay() + case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => + // We wait for one of the funding transactions to confirm before going to the closing state, as the spent funding + // tx and the associated commit tx could be replaced by a new version of the funding tx. + if (tx.txid == d.commitments.remoteCommit.txid) { + log.warning("funding tx spent by txid={} while still unconfirmed", tx.txid) + stay() + } else if (d.previousFundingTxs.exists(_.commitments.remoteCommit.txid == tx.txid)) { + log.warning("previous funding tx spent by txid while still unconfirmed", tx.txid) + stay() + } else { + handleInformationLeak(tx, d) + } + case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleRemoteError(e, d) }) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala index e946b3e306..641026c3b2 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala @@ -21,7 +21,7 @@ import akka.actor.{ActorRef, FSM} import fr.acinq.bitcoin.scalacompat.{ByteVector32, OutPoint, SatoshiLong, Transaction} import fr.acinq.eclair.NotificationsLogger import fr.acinq.eclair.NotificationsLogger.NotifyNodeOperator -import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchOutputSpent, WatchTxConfirmed} +import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchOutputSpent, WatchTxConfirmed} import fr.acinq.eclair.channel.Helpers.Closing import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel.UnhandledExceptionStrategy @@ -281,6 +281,24 @@ trait ErrorHandlers extends CommonHandlers { goto(CLOSING) using nextData storing() calling doPublish(remoteCommitPublished, d.commitments) } + def handleRemoteSpentAlternative(commitTx: Transaction, alternativeCommitments: List[Commitments], d: PersistentChannelData) = { + val commitments_opt = alternativeCommitments.find(c => c.remoteCommit.txid == commitTx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == commitTx.txid)) + require(commitments_opt.nonEmpty, "there should be a commit spending an alternative funding tx matching this transaction") + val commitments = commitments_opt.get + log.warning("they published their commit with txid={} spending an alternative funding tx with fundingTxid={}", commitTx.txid, commitments.commitInput.outPoint.txid) + + context.system.eventStream.publish(TransactionPublished(d.channelId, remoteNodeId, commitTx, Closing.commitTxFee(commitments.commitInput, commitTx, commitments.localParams.isInitiator), "remote-commit")) + // We wait for this alternative funding tx to be confirmed before claiming the commit tx outputs, as it could still + // be replaced by another funding tx. + blockchain ! WatchFundingConfirmed(self, commitments.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks) + val nextData = d match { + case waitForFundingConfirmed: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, waitForFundingConfirmed.signedFundingTx_opt, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments, mutualCloseProposed = Nil) + case closing: DATA_CLOSING => closing + case _ => DATA_CLOSING(d.commitments, None, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments, mutualCloseProposed = Nil) + } + goto(CLOSING) using nextData storing() + } + def doPublish(remoteCommitPublished: RemoteCommitPublished, commitments: Commitments): Unit = { import remoteCommitPublished._ diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala index 9c288304d6..4425be68e2 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala @@ -396,7 +396,6 @@ private[channel] object ChannelCodecs3 { val DATA_WAIT_FOR_DUAL_FUNDING_READY_0c_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_READY] = ( ("commitments" | commitmentsCodec) :: ("shortIds" | shortids) :: - ("otherFundingTxs" | seqOfN(uint16, dualFundingTxCodec)) :: ("lastSent" | lengthDelimited(channelReadyCodec))).as[DATA_WAIT_FOR_DUAL_FUNDING_READY] val DATA_NORMAL_02_Codec: Codec[DATA_NORMAL] = ( diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala index f344e6c807..547526800d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala @@ -298,6 +298,8 @@ trait ChannelStateTestsBase extends Assertions with Eventually { alice2bob.expectMsgType[TxSignatures] alice2bob.forward(bob) val fundingTx = eventListener.expectMsgType[TransactionPublished].tx + alice2blockchain.expectMsgType[WatchFundingSpent] + bob2blockchain.expectMsgType[WatchFundingSpent] if (!channelType.features.contains(Features.ZeroConf)) { eventually(assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)) eventually(assert(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)) @@ -305,9 +307,9 @@ trait ChannelStateTestsBase extends Assertions with Eventually { bob2blockchain.expectMsgType[WatchFundingConfirmed] alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx) bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx) + alice2blockchain.expectMsgType[WatchFundingSpent] + bob2blockchain.expectMsgType[WatchFundingSpent] } - alice2blockchain.expectMsgType[WatchFundingSpent] - bob2blockchain.expectMsgType[WatchFundingSpent] alice2blockchain.expectMsgType[WatchFundingLost] bob2blockchain.expectMsgType[WatchFundingLost] alice2bob.expectMsgType[ChannelReady] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala index 20edebc79f..c007e97a48 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala @@ -17,7 +17,7 @@ package fr.acinq.eclair.channel.states.c import akka.testkit.{TestFSMRef, TestProbe} -import fr.acinq.bitcoin.scalacompat.ByteVector32 +import fr.acinq.bitcoin.scalacompat.{ByteVector32, Transaction} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ import fr.acinq.eclair.blockchain.{CurrentBlockHeight, SingleKeyOnChainWallet} import fr.acinq.eclair.channel.InteractiveTxBuilder.FullySignedSharedTransaction @@ -96,12 +96,14 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture bob2alice.forward(alice) // Alice publishes the funding tx. val fundingTx = aliceListener.expectMsgType[TransactionPublished].tx + assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid) assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid) alice2bob.expectMsgType[TxSignatures] alice2bob.forward(bob) awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) // Bob publishes the funding tx. assert(bobListener.expectMsgType[TransactionPublished].tx.txid == fundingTx.txid) + assert(bob2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid) assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid) withFixture(test.toNoArgTest(FixtureParam(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain, aliceListener, bobListener, wallet))) } @@ -259,7 +261,6 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture bob2alice.forward(alice) val aliceChannelReady = alice2bob.expectMsgType[ChannelReady] assert(aliceChannelReady.alias_opt.isDefined) - assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid) assert(alice2blockchain.expectMsgType[WatchFundingLost].txId == fundingTx.txid) assert(alice2blockchain.expectMsgType[WatchFundingDeeplyBuried].txId == fundingTx.txid) awaitCond(alice.stateName == NORMAL) @@ -278,6 +279,42 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY) } + test("recv WatchFundingSpentTriggered (remote commit)", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + // bob publishes his commitment tx + val bobCommitTx = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx + alice ! WatchFundingSpentTriggered(bobCommitTx) + // alice doesn't react since the spent funding tx is still unconfirmed + alice2blockchain.expectNoMessage(100 millis) + assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) + } + + test("recv WatchFundingSpentTriggered (other commit)", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + val commitTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx + alice ! WatchFundingSpentTriggered(Transaction(0, Nil, Nil, 0)) + alice2bob.expectMsgType[Error] + assert(alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx].tx.txid == commitTx.txid) + awaitCond(alice.stateName == ERR_INFORMATION_LEAK) + } + + test("recv WatchFundingSpentTriggered while offline (remote commit)", Tag(ChannelStateTestsTags.DualFunding)) { f => + import f._ + val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx + val aliceCommitTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx + alice ! INPUT_DISCONNECTED + awaitCond(alice.stateName == OFFLINE) + // bob publishes his commitment tx + val bobCommitTx = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx + alice ! WatchFundingSpentTriggered(bobCommitTx.tx) + val claimMain = alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx] + assert(claimMain.input.txid == bobCommitTx.tx.txid) + assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == bobCommitTx.tx.txid) + assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMain.tx.txid) + alice2blockchain.expectNoMessage(100 millis) + awaitCond(alice.stateName == CLOSING) + } + test("recv Error", Tag(ChannelStateTestsTags.DualFunding)) { f => import f._ val tx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala index b391612ebd..1cf113db0f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala @@ -81,16 +81,18 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF bob2alice.forward(alice) alice2bob.expectMsgType[TxSignatures] alice2bob.forward(bob) + alice2blockchain.expectMsgType[WatchFundingSpent] + bob2blockchain.expectMsgType[WatchFundingSpent] if (!test.tags.contains(ChannelStateTestsTags.ZeroConf)) { val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid) assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid) alice ! WatchFundingConfirmedTriggered(BlockHeight(TestConstants.defaultBlockHeight), 42, fundingTx) bob ! WatchFundingConfirmedTriggered(BlockHeight(TestConstants.defaultBlockHeight), 42, fundingTx) + alice2blockchain.expectMsgType[WatchFundingSpent] + bob2blockchain.expectMsgType[WatchFundingSpent] } - alice2blockchain.expectMsgType[WatchFundingSpent] alice2blockchain.expectMsgType[WatchFundingLost] - bob2blockchain.expectMsgType[WatchFundingSpent] bob2blockchain.expectMsgType[WatchFundingLost] awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY) awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_READY) From 01119f23be00dbe97d65217063fca180d1007d2e Mon Sep 17 00:00:00 2001 From: t-bast Date: Thu, 18 Aug 2022 11:41:32 +0200 Subject: [PATCH 04/11] fixup! Watch funding txs as soon as published --- .../channel/states/b/WaitForDualFundingCreatedStateSpec.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala index a65e8cd05c..0d87cd1f0d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala @@ -102,11 +102,13 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn assert(bobData.commitments.channelFeatures.hasFeature(Features.DualFunding)) assert(bobData.fundingTx.isInstanceOf[PartiallySignedSharedTransaction]) val fundingTxId = bobData.fundingTx.asInstanceOf[PartiallySignedSharedTransaction].tx.buildUnsignedTx().txid + assert(bob2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTxId) assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTxId) // Alice receives Bob's signatures and sends her own signatures. bob2alice.forward(alice) assert(listener.expectMsgType[TransactionPublished].tx.txid == fundingTxId) + assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTxId) assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTxId) alice2bob.expectMsgType[TxSignatures] awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) @@ -245,6 +247,7 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn alice2bob.forward(bob) val bobSigs = bob2alice.expectMsgType[TxSignatures] + bob2blockchain.expectMsgType[WatchFundingSpent] bob2blockchain.expectMsgType[WatchFundingConfirmed] bob2alice.forward(alice, bobSigs.copy(txId = randomBytes32(), witnesses = Nil)) alice2bob.expectMsgType[TxAbort] From 1d846d22cd1884d6ec1e929d06709f820a42309e Mon Sep 17 00:00:00 2001 From: t-bast Date: Thu, 18 Aug 2022 12:36:40 +0200 Subject: [PATCH 05/11] Various nits --- .../scala/fr/acinq/eclair/channel/ChannelData.scala | 2 +- .../fr/acinq/eclair/channel/InteractiveTxBuilder.scala | 6 ++++-- .../eclair/channel/fsm/ChannelOpenDualFunded.scala | 4 ++-- .../main/scala/fr/acinq/eclair/router/Validation.scala | 2 +- .../internal/channel/version3/ChannelCodecs3.scala | 10 +++++----- .../fr/acinq/eclair/wire/protocol/CommonCodecs.scala | 2 -- .../eclair/wire/protocol/LightningMessageCodecs.scala | 4 ++-- .../eclair/channel/InteractiveTxBuilderSpec.scala | 2 +- 8 files changed, 16 insertions(+), 16 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala index 4bc6385580..af2d8fd2b9 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala @@ -482,7 +482,7 @@ final case class DATA_WAIT_FOR_DUAL_FUNDING_CREATED(channelId: ByteVector32, final case class DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments: Commitments, fundingTx: SignedSharedTransaction, fundingParams: InteractiveTxParams, - previousFundingTxs: Seq[DualFundingTx], + previousFundingTxs: List[DualFundingTx], waitingSince: BlockHeight, // how long have we been waiting for a funding tx to confirm lastChecked: BlockHeight, // last time we checked if the channel was double-spent rbfAttempt: Option[typed.ActorRef[InteractiveTxBuilder.Command]], diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/InteractiveTxBuilder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/InteractiveTxBuilder.scala index 78a50f9552..219f52b64f 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/InteractiveTxBuilder.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/InteractiveTxBuilder.scala @@ -141,7 +141,7 @@ object InteractiveTxBuilder { case class UnusableInput(outpoint: OutPoint) /** Unsigned transaction created collaboratively. */ - case class SharedTransaction(localInputs: Seq[TxAddInput], remoteInputs: Seq[RemoteTxAddInput], localOutputs: Seq[TxAddOutput], remoteOutputs: Seq[RemoteTxAddOutput], lockTime: Long) { + case class SharedTransaction(localInputs: List[TxAddInput], remoteInputs: List[RemoteTxAddInput], localOutputs: List[TxAddOutput], remoteOutputs: List[RemoteTxAddOutput], lockTime: Long) { val localAmountIn: Satoshi = localInputs.map(i => i.previousTx.txOut(i.previousTxOutput.toInt).amount).sum val remoteAmountIn: Satoshi = remoteInputs.map(_.txOut.amount).sum val totalAmountIn: Satoshi = localAmountIn + remoteAmountIn @@ -566,7 +566,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon } def validateTx(session: InteractiveTxSession): Either[ChannelException, (SharedTransaction, Int)] = { - val sharedTx = SharedTransaction(session.localInputs, session.remoteInputs.map(i => RemoteTxAddInput(i)), session.localOutputs, session.remoteOutputs.map(o => RemoteTxAddOutput(o)), fundingParams.lockTime) + val sharedTx = SharedTransaction(session.localInputs.toList, session.remoteInputs.map(i => RemoteTxAddInput(i)).toList, session.localOutputs.toList, session.remoteOutputs.map(o => RemoteTxAddOutput(o)).toList, fundingParams.lockTime) val tx = sharedTx.buildUnsignedTx() if (tx.txIn.length > 252 || tx.txOut.length > 252) { @@ -689,6 +689,8 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon Behaviors.stopped } case SignTransactionResult(signedTx, None) => + // We return as soon as we sign the tx, because we need to be able to handle the case where remote publishes the + // tx right away without properly sending us their signature. log.info("interactive-tx partially signed with {} local inputs, {} remote inputs, {} local outputs and {} remote outputs", signedTx.tx.localInputs.length, signedTx.tx.remoteInputs.length, signedTx.tx.localOutputs.length, signedTx.tx.remoteOutputs.length) replyTo ! Succeeded(fundingParams, signedTx, commitments) Behaviors.stopped diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala index 5404be23a6..1b06277cf2 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala @@ -405,7 +405,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { goto(CLOSED) } case None => - log.error(s"rejecting channel with invalid funding tx that doesn't match any of our funding txs: ${confirmedTx.bin}") + log.error(s"internal error: the funding tx that confirmed doesn't match any of our funding txs: ${confirmedTx.bin}") allFundingTxs.foreach(f => wallet.rollback(f.fundingTx.tx.buildUnsignedTx())) goto(CLOSED) } @@ -422,7 +422,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { d.previousFundingTxs.isEmpty && (d.fundingParams.remoteAmount == 0.sat || d.commitments.localParams.initFeatures.hasFeature(Features.ZeroConf)) if (canUseZeroConf) { - log.info("this chanel isn't zero-conf, but they sent an early channel_ready with an alias: no need to wait for confirmations") + log.info("this channel isn't zero-conf, but they sent an early channel_ready with an alias: no need to wait for confirmations") val (shortIds, localChannelReady) = acceptFundingTx(d.commitments, RealScidStatus.Unknown) self ! remoteChannelReady // NB: we will receive a WatchFundingConfirmedTriggered later that will simply be ignored diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/Validation.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/Validation.scala index b55356a422..cc12e26ed1 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/Validation.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/Validation.scala @@ -516,7 +516,7 @@ object Validation { } case None => // should never happen, we log a warning and handle the update, it will be rejected since there is no related channel - log.warning("unrecognized local chanel update for channelId={} localAlias={}", lcu.channelId, lcu.shortIds.localAlias) + log.warning("unrecognized local channel update for channelId={} localAlias={}", lcu.channelId, lcu.shortIds.localAlias) handleChannelUpdate(d, db, nodeParams.routerConf, Left(lcu)) } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala index 4425be68e2..8647df564b 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala @@ -350,10 +350,10 @@ private[channel] object ChannelCodecs3 { ("scriptPubKey" | lengthDelimited(bytes))).as[RemoteTxAddOutput] private val sharedTransactionCodec: Codec[SharedTransaction] = ( - ("localInputs" | seqOfN(uint16, lengthDelimited(txAddInputCodec))) :: - ("remoteInputs" | seqOfN(uint16, remoteTxAddInputCodec)) :: - ("localOutputs" | seqOfN(uint16, lengthDelimited(txAddOutputCodec))) :: - ("remoteOutputs" | seqOfN(uint16, remoteTxAddOutputCodec)) :: + ("localInputs" | listOfN(uint16, lengthDelimited(txAddInputCodec))) :: + ("remoteInputs" | listOfN(uint16, remoteTxAddInputCodec)) :: + ("localOutputs" | listOfN(uint16, lengthDelimited(txAddOutputCodec))) :: + ("remoteOutputs" | listOfN(uint16, remoteTxAddOutputCodec)) :: ("lockTime" | uint32)).as[SharedTransaction] private val partiallySignedSharedTransactionCodec: Codec[PartiallySignedSharedTransaction] = ( @@ -387,7 +387,7 @@ private[channel] object ChannelCodecs3 { ("commitments" | commitmentsCodec) :: ("fundingTx" | signedSharedTransactionCodec) :: ("fundingParams" | fundingParamsCodec) :: - ("previousFundingTxs" | seqOfN(uint16, dualFundingTxCodec)) :: + ("previousFundingTxs" | listOfN(uint16, dualFundingTxCodec)) :: ("waitingSince" | blockHeight) :: ("lastChecked" | blockHeight) :: ("rbfAttempt" | provide(Option.empty[typed.ActorRef[InteractiveTxBuilder.Command]])) :: diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/CommonCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/CommonCodecs.scala index e8d3bdbe5d..314fbaf4bf 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/CommonCodecs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/CommonCodecs.scala @@ -110,8 +110,6 @@ object CommonCodecs { val listofsignatures: Codec[List[ByteVector64]] = listOfN(uint16, bytes64) - def seqOfN[A](countCodec: Codec[Int], valueCodec: Codec[A]): Codec[Seq[A]] = listOfN(countCodec, valueCodec).xmap(_.toSeq, _.toList) - val channelflags: Codec[ChannelFlags] = (ignore(7) dropLeft bool).as[ChannelFlags] val ipv4address: Codec[Inet4Address] = bytes(4).xmap(b => InetAddress.getByAddress(b.toArray).asInstanceOf[Inet4Address], a => ByteVector(a.getAddress)) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageCodecs.scala index 57259011f8..13995723ad 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageCodecs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/protocol/LightningMessageCodecs.scala @@ -368,7 +368,7 @@ object LightningMessageCodecs { ("shortChannelIds" | variableSizeBytes(uint16, encodedShortChannelIdsCodec)) :: ("tlvStream" | QueryShortChannelIdsTlv.codec)).as[QueryShortChannelIds] - val replyShortChanelIdsEndCodec: Codec[ReplyShortChannelIdsEnd] = ( + val replyShortChannelIdsEndCodec: Codec[ReplyShortChannelIdsEnd] = ( ("chainHash" | bytes32) :: ("complete" | byte) :: ("tlvStream" | ReplyShortChannelIdsEndTlv.replyShortChannelIdsEndTlvCodec)).as[ReplyShortChannelIdsEnd] @@ -458,7 +458,7 @@ object LightningMessageCodecs { .typecase(258, channelUpdateCodec) .typecase(259, announcementSignaturesCodec) .typecase(261, queryShortChannelIdsCodec) - .typecase(262, replyShortChanelIdsEndCodec) + .typecase(262, replyShortChannelIdsEndCodec) .typecase(263, queryChannelRangeCodec) .typecase(264, replyChannelRangeCodec) .typecase(265, gossipTimestampFilterCodec) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala index aee6f12fb8..1d473933b2 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/InteractiveTxBuilderSpec.scala @@ -979,7 +979,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit val wallet = new SingleKeyOnChainWallet() val params = createChannelParams(100_000 sat, 0 sat, FeeratePerKw(5000 sat), 330 sat, 0) val validScript = Script.write(Script.pay2wpkh(randomKey().publicKey)) - val firstAttempt = PartiallySignedSharedTransaction(SharedTransaction(Seq(createInput(params.channelId, UInt64(2), 125_000 sat)), Nil, Nil, Nil, 0), null) + val firstAttempt = PartiallySignedSharedTransaction(SharedTransaction(List(createInput(params.channelId, UInt64(2), 125_000 sat)), Nil, Nil, Nil, 0), null) val secondAttempt = PartiallySignedSharedTransaction(SharedTransaction(firstAttempt.tx.localInputs :+ createInput(params.channelId, UInt64(4), 150_000 sat), Nil, Nil, Nil, 0), null) val bob = params.spawnTxBuilderBob(params.fundingParamsB, TestConstants.anchorOutputsFeeratePerKw, wallet) bob ! Start(probe.ref, Seq(firstAttempt, secondAttempt)) From 33b60156078b406135955af9dbd130d0556941fe Mon Sep 17 00:00:00 2001 From: t-bast Date: Thu, 18 Aug 2022 12:52:51 +0200 Subject: [PATCH 06/11] Factorize waiting for channel ready handlers --- .../channel/fsm/ChannelOpenDualFunded.scala | 28 ++---- .../channel/fsm/ChannelOpenSingleFunded.scala | 28 ++---- .../channel/fsm/CommonFundingHandlers.scala | 85 +++++++++++++++++++ .../channel/fsm/DualFundingHandlers.scala | 2 +- .../channel/fsm/SingleFundingHandlers.scala | 30 ++----- 5 files changed, 101 insertions(+), 72 deletions(-) create mode 100644 eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/CommonFundingHandlers.scala diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala index 1b06277cf2..aff595e9d4 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala @@ -20,15 +20,14 @@ import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapte import fr.acinq.bitcoin.ScriptFlags import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Script, Transaction} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ -import fr.acinq.eclair.channel.Helpers.{Funding, getRelayFees} +import fr.acinq.eclair.channel.Helpers.Funding import fr.acinq.eclair.channel.InteractiveTxBuilder.{FullySignedSharedTransaction, InteractiveTxParams, PartiallySignedSharedTransaction} import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel._ import fr.acinq.eclair.channel.publish.TxPublisher.SetChannelId -import fr.acinq.eclair.router.Announcements import fr.acinq.eclair.transactions.Scripts import fr.acinq.eclair.wire.protocol._ -import fr.acinq.eclair.{Features, RealShortChannelId, ToMilliSatoshiConversion} +import fr.acinq.eclair.{Features, RealShortChannelId} import scala.concurrent.duration.DurationInt import scala.util.{Failure, Success, Try} @@ -457,32 +456,15 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { when(WAIT_FOR_DUAL_FUNDING_READY)(handleExceptions { case Event(channelReady: ChannelReady, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) => - val shortIds1 = d.shortIds.copy(remoteAlias_opt = channelReady.alias_opt) - shortIds1.remoteAlias_opt.foreach { remoteAlias => - log.info("received remoteAlias={}", remoteAlias) - context.system.eventStream.publish(ShortChannelIdAssigned(self, d.commitments.channelId, shortIds = shortIds1, remoteNodeId = remoteNodeId)) - } - log.info("shortIds: real={} localAlias={} remoteAlias={}", shortIds1.real.toOption.getOrElse("none"), shortIds1.localAlias, shortIds1.remoteAlias_opt.getOrElse("none")) - // we create a channel_update early so that we can use it to send payments through this channel, but it won't be propagated to other nodes since the channel is not yet announced - val scidForChannelUpdate = Helpers.scidForChannelUpdate(channelAnnouncement_opt = None, shortIds1.localAlias) - log.info("using shortChannelId={} for initial channel_update", scidForChannelUpdate) - val fees = getRelayFees(nodeParams, remoteNodeId, d.commitments) - val initialChannelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, scidForChannelUpdate, nodeParams.channelConf.expiryDelta, d.commitments.remoteParams.htlcMinimum, fees.feeBase, fees.feeProportionalMillionths, d.commitments.capacity.toMilliSatoshi, enable = Helpers.aboveReserve(d.commitments)) - // we need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network - context.system.scheduler.scheduleWithFixedDelay(initialDelay = REFRESH_CHANNEL_UPDATE_INTERVAL, delay = REFRESH_CHANNEL_UPDATE_INTERVAL, receiver = self, message = BroadcastChannelUpdate(PeriodicRefresh)) - // used to get the final shortChannelId, used in announcements (if minDepth >= ANNOUNCEMENTS_MINCONF this event will fire instantly) - blockchain ! WatchFundingDeeplyBuried(self, d.commitments.commitInput.outPoint.txid, ANNOUNCEMENTS_MINCONF) - goto(NORMAL) using DATA_NORMAL(d.commitments.copy(remoteNextCommitInfo = Right(channelReady.nextPerCommitmentPoint)), shortIds1, None, initialChannelUpdate, None, None, None) storing() + val nextData = receiveChannelReady(d.shortIds, channelReady, d.commitments) + goto(NORMAL) using nextData storing() case Event(_: TxInitRbf, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) => // Our peer may not have received the funding transaction confirmation. stay() sending TxAbort(d.channelId, InvalidRbfTxConfirmed(d.channelId).getMessage) case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) if d.commitments.announceChannel => - log.debug("received remote announcement signatures, delaying") - // we may receive their announcement sigs before our watcher notifies us that the channel has reached min_conf (especially during testing when blocks are generated in bulk) - // note: no need to persist their message, in case of disconnection they will resend it - context.system.scheduler.scheduleOnce(2 seconds, self, remoteAnnSigs) + delayEarlyAnnouncementSigs(remoteAnnSigs) stay() case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_DUAL_FUNDING_READY) if tx.txid == d.commitments.remoteCommit.txid => handleRemoteSpentCurrent(tx, d) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala index 8302d81bed..da92af8d3d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala @@ -23,16 +23,15 @@ import fr.acinq.bitcoin.ScriptFlags import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Script, Transaction} import fr.acinq.eclair.blockchain.OnChainWallet.MakeFundingTxResponse import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ -import fr.acinq.eclair.channel.Helpers.{Funding, getRelayFees} +import fr.acinq.eclair.channel.Helpers.Funding import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel._ import fr.acinq.eclair.channel.publish.TxPublisher.SetChannelId import fr.acinq.eclair.crypto.ShaChain -import fr.acinq.eclair.router.Announcements import fr.acinq.eclair.transactions.Transactions.TxOwner import fr.acinq.eclair.transactions.{Scripts, Transactions} import fr.acinq.eclair.wire.protocol.{AcceptChannel, AnnouncementSignatures, ChannelReady, ChannelTlv, Error, FundingCreated, FundingSigned, OpenChannel, TlvStream} -import fr.acinq.eclair.{Features, MilliSatoshiLong, RealShortChannelId, ToMilliSatoshiConversion, randomKey, toLongId} +import fr.acinq.eclair.{Features, MilliSatoshiLong, RealShortChannelId, randomKey, toLongId} import scodec.bits.ByteVector import scala.concurrent.duration.DurationInt @@ -435,28 +434,11 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers { when(WAIT_FOR_CHANNEL_READY)(handleExceptions { case Event(channelReady: ChannelReady, d: DATA_WAIT_FOR_CHANNEL_READY) => - val shortIds1 = d.shortIds.copy(remoteAlias_opt = channelReady.alias_opt) - shortIds1.remoteAlias_opt.foreach { remoteAlias => - log.info("received remoteAlias={}", remoteAlias) - context.system.eventStream.publish(ShortChannelIdAssigned(self, d.commitments.channelId, shortIds = shortIds1, remoteNodeId = remoteNodeId)) - } - log.info("shortIds: real={} localAlias={} remoteAlias={}", shortIds1.real.toOption.getOrElse("none"), shortIds1.localAlias, shortIds1.remoteAlias_opt.getOrElse("none")) - // we create a channel_update early so that we can use it to send payments through this channel, but it won't be propagated to other nodes since the channel is not yet announced - val scidForChannelUpdate = Helpers.scidForChannelUpdate(channelAnnouncement_opt = None, shortIds1.localAlias) - log.info("using shortChannelId={} for initial channel_update", scidForChannelUpdate) - val relayFees = getRelayFees(nodeParams, remoteNodeId, d.commitments) - val initialChannelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, scidForChannelUpdate, nodeParams.channelConf.expiryDelta, d.commitments.remoteParams.htlcMinimum, relayFees.feeBase, relayFees.feeProportionalMillionths, d.commitments.capacity.toMilliSatoshi, enable = Helpers.aboveReserve(d.commitments)) - // we need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network - context.system.scheduler.scheduleWithFixedDelay(initialDelay = REFRESH_CHANNEL_UPDATE_INTERVAL, delay = REFRESH_CHANNEL_UPDATE_INTERVAL, receiver = self, message = BroadcastChannelUpdate(PeriodicRefresh)) - // used to get the final shortChannelId, used in announcements (if minDepth >= ANNOUNCEMENTS_MINCONF this event will fire instantly) - blockchain ! WatchFundingDeeplyBuried(self, d.commitments.commitInput.outPoint.txid, ANNOUNCEMENTS_MINCONF) - goto(NORMAL) using DATA_NORMAL(d.commitments.copy(remoteNextCommitInfo = Right(channelReady.nextPerCommitmentPoint)), shortIds1, None, initialChannelUpdate, None, None, None) storing() + val nextData = receiveChannelReady(d.shortIds, channelReady, d.commitments) + goto(NORMAL) using nextData storing() case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_WAIT_FOR_CHANNEL_READY) if d.commitments.announceChannel => - log.debug("received remote announcement signatures, delaying") - // we may receive their announcement sigs before our watcher notifies us that the channel has reached min_conf (especially during testing when blocks are generated in bulk) - // note: no need to persist their message, in case of disconnection they will resend it - context.system.scheduler.scheduleOnce(2 seconds, self, remoteAnnSigs) + delayEarlyAnnouncementSigs(remoteAnnSigs) stay() case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_CHANNEL_READY) if tx.txid == d.commitments.remoteCommit.txid => handleRemoteSpentCurrent(tx, d) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/CommonFundingHandlers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/CommonFundingHandlers.scala new file mode 100644 index 0000000000..579e68e025 --- /dev/null +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/CommonFundingHandlers.scala @@ -0,0 +1,85 @@ +/* + * Copyright 2022 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.fsm + +import akka.actor.typed.scaladsl.adapter.actorRefAdapter +import fr.acinq.bitcoin.scalacompat.ByteVector32 +import fr.acinq.eclair.{ShortChannelId, ToMilliSatoshiConversion} +import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingDeeplyBuried, WatchFundingLost, WatchFundingSpent} +import fr.acinq.eclair.channel.Helpers.getRelayFees +import fr.acinq.eclair.channel._ +import fr.acinq.eclair.channel.fsm.Channel.{ANNOUNCEMENTS_MINCONF, BroadcastChannelUpdate, PeriodicRefresh, REFRESH_CHANNEL_UPDATE_INTERVAL} +import fr.acinq.eclair.router.Announcements +import fr.acinq.eclair.wire.protocol.{AnnouncementSignatures, ChannelReady, ChannelReadyTlv, TlvStream} + +import scala.concurrent.duration.DurationInt + +/** + * Created by t-bast on 18/08/2022. + */ + +trait CommonFundingHandlers extends CommonHandlers { + + this: Channel => + + def watchFundingTx(commitments: Commitments, additionalKnownSpendingTxs: Set[ByteVector32] = Set.empty): Unit = { + // TODO: should we wait for an acknowledgment from the watcher? + // TODO: implement WatchFundingLost? + val knownSpendingTxs = Set(commitments.localCommit.commitTxAndRemoteSig.commitTx.tx.txid, commitments.remoteCommit.txid) ++ commitments.remoteNextCommitInfo.left.toSeq.map(_.nextRemoteCommit.txid).toSet ++ additionalKnownSpendingTxs + blockchain ! WatchFundingSpent(self, commitments.commitInput.outPoint.txid, commitments.commitInput.outPoint.index.toInt, knownSpendingTxs) + } + + def acceptFundingTx(commitments: Commitments, realScidStatus: RealScidStatus): (ShortIds, ChannelReady) = { + blockchain ! WatchFundingLost(self, commitments.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks) + val channelKeyPath = keyManager.keyPath(commitments.localParams, commitments.channelConfig) + val nextPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1) + // the alias will use in our peer's channel_update message, the goal is to be able to use our channel as soon + // as it reaches NORMAL state, and before it is announced on the network + val shortIds = ShortIds(realScidStatus, ShortChannelId.generateLocalAlias(), remoteAlias_opt = None) + context.system.eventStream.publish(ShortChannelIdAssigned(self, commitments.channelId, shortIds, remoteNodeId)) + // we always send our local alias, even if it isn't explicitly supported, that's an optional TLV anyway + val channelReady = ChannelReady(commitments.channelId, nextPerCommitmentPoint, TlvStream(ChannelReadyTlv.ShortChannelIdTlv(shortIds.localAlias))) + (shortIds, channelReady) + } + + def receiveChannelReady(shortIds: ShortIds, channelReady: ChannelReady, commitments: Commitments): DATA_NORMAL = { + val shortIds1 = shortIds.copy(remoteAlias_opt = channelReady.alias_opt) + shortIds1.remoteAlias_opt.foreach { remoteAlias => + log.info("received remoteAlias={}", remoteAlias) + context.system.eventStream.publish(ShortChannelIdAssigned(self, commitments.channelId, shortIds = shortIds1, remoteNodeId = remoteNodeId)) + } + log.info("shortIds: real={} localAlias={} remoteAlias={}", shortIds1.real.toOption.getOrElse("none"), shortIds1.localAlias, shortIds1.remoteAlias_opt.getOrElse("none")) + // we create a channel_update early so that we can use it to send payments through this channel, but it won't be propagated to other nodes since the channel is not yet announced + val scidForChannelUpdate = Helpers.scidForChannelUpdate(channelAnnouncement_opt = None, shortIds1.localAlias) + log.info("using shortChannelId={} for initial channel_update", scidForChannelUpdate) + val relayFees = getRelayFees(nodeParams, remoteNodeId, commitments) + val initialChannelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, scidForChannelUpdate, nodeParams.channelConf.expiryDelta, commitments.remoteParams.htlcMinimum, relayFees.feeBase, relayFees.feeProportionalMillionths, commitments.capacity.toMilliSatoshi, enable = Helpers.aboveReserve(commitments)) + // we need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network + context.system.scheduler.scheduleWithFixedDelay(initialDelay = REFRESH_CHANNEL_UPDATE_INTERVAL, delay = REFRESH_CHANNEL_UPDATE_INTERVAL, receiver = self, message = BroadcastChannelUpdate(PeriodicRefresh)) + // used to get the final shortChannelId, used in announcements (if minDepth >= ANNOUNCEMENTS_MINCONF this event will fire instantly) + blockchain ! WatchFundingDeeplyBuried(self, commitments.commitInput.outPoint.txid, ANNOUNCEMENTS_MINCONF) + DATA_NORMAL(commitments.copy(remoteNextCommitInfo = Right(channelReady.nextPerCommitmentPoint)), shortIds1, None, initialChannelUpdate, None, None, None) + } + + def delayEarlyAnnouncementSigs(remoteAnnSigs: AnnouncementSignatures): Unit = { + log.debug("received remote announcement signatures, delaying") + // we may receive their announcement sigs before our watcher notifies us that the channel has reached min_conf (especially during testing when blocks are generated in bulk) + // note: no need to persist their message, in case of disconnection they will resend it + context.system.scheduler.scheduleOnce(2 seconds, self, remoteAnnSigs) + } + +} diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala index 17da029991..5eeec2dbd2 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala @@ -33,7 +33,7 @@ import scala.util.{Failure, Success} /** * This trait contains handlers related to dual-funding channel transactions. */ -trait DualFundingHandlers extends CommonHandlers { +trait DualFundingHandlers extends CommonFundingHandlers { this: Channel => diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/SingleFundingHandlers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/SingleFundingHandlers.scala index f9506d5f52..7bbb047ba0 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/SingleFundingHandlers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/SingleFundingHandlers.scala @@ -17,13 +17,13 @@ package fr.acinq.eclair.channel.fsm import akka.actor.typed.scaladsl.adapter.{TypedActorRefOps, actorRefAdapter} -import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, SatoshiLong, Transaction} -import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{GetTxWithMeta, GetTxWithMetaResponse, WatchFundingLost, WatchFundingSpent} +import fr.acinq.bitcoin.scalacompat.{Satoshi, SatoshiLong, Transaction} +import fr.acinq.eclair.BlockHeight +import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{GetTxWithMeta, GetTxWithMetaResponse} import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel.{BITCOIN_FUNDING_PUBLISH_FAILED, BITCOIN_FUNDING_TIMEOUT, FUNDING_TIMEOUT_FUNDEE} import fr.acinq.eclair.channel.publish.TxPublisher.PublishFinalTx -import fr.acinq.eclair.wire.protocol.{ChannelReady, ChannelReadyTlv, Error, TlvStream} -import fr.acinq.eclair.{BlockHeight, ShortChannelId} +import fr.acinq.eclair.wire.protocol.Error import scala.concurrent.duration.DurationInt import scala.util.{Failure, Success} @@ -35,7 +35,7 @@ import scala.util.{Failure, Success} /** * This trait contains handlers related to single-funder channel transactions. */ -trait SingleFundingHandlers extends CommonHandlers { +trait SingleFundingHandlers extends CommonFundingHandlers { this: Channel => @@ -53,26 +53,6 @@ trait SingleFundingHandlers extends CommonHandlers { } } - def watchFundingTx(commitments: Commitments, additionalKnownSpendingTxs: Set[ByteVector32] = Set.empty): Unit = { - // TODO: should we wait for an acknowledgment from the watcher? - // TODO: implement WatchFundingLost? - val knownSpendingTxs = Set(commitments.localCommit.commitTxAndRemoteSig.commitTx.tx.txid, commitments.remoteCommit.txid) ++ commitments.remoteNextCommitInfo.left.toSeq.map(_.nextRemoteCommit.txid).toSet ++ additionalKnownSpendingTxs - blockchain ! WatchFundingSpent(self, commitments.commitInput.outPoint.txid, commitments.commitInput.outPoint.index.toInt, knownSpendingTxs) - } - - def acceptFundingTx(commitments: Commitments, realScidStatus: RealScidStatus): (ShortIds, ChannelReady) = { - blockchain ! WatchFundingLost(self, commitments.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks) - val channelKeyPath = keyManager.keyPath(commitments.localParams, commitments.channelConfig) - val nextPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1) - // the alias will use in our peer's channel_update message, the goal is to be able to use our channel as soon - // as it reaches NORMAL state, and before it is announced on the network - val shortIds = ShortIds(realScidStatus, ShortChannelId.generateLocalAlias(), remoteAlias_opt = None) - context.system.eventStream.publish(ShortChannelIdAssigned(self, commitments.channelId, shortIds, remoteNodeId)) - // we always send our local alias, even if it isn't explicitly supported, that's an optional TLV anyway - val channelReady = ChannelReady(commitments.channelId, nextPerCommitmentPoint, TlvStream(ChannelReadyTlv.ShortChannelIdTlv(shortIds.localAlias))) - (shortIds, channelReady) - } - /** * When we are funder, we use this function to detect when our funding tx has been double-spent (by another transaction * that we made for some reason). If the funding tx has been double spent we can forget about the channel. From 7b7bfac5bb44092cac0a7439723abff63ae4f356 Mon Sep 17 00:00:00 2001 From: t-bast Date: Thu, 18 Aug 2022 14:30:52 +0200 Subject: [PATCH 07/11] Spend latest funding when remote spends previous It's likely that we'll win that race since the latest funding tx pays more fees. --- .../fr/acinq/eclair/channel/fsm/Channel.scala | 40 +++++++++++++++---- .../channel/fsm/ChannelOpenDualFunded.scala | 20 +++++----- .../channel/fsm/ChannelOpenSingleFunded.scala | 4 +- .../eclair/channel/fsm/ErrorHandlers.scala | 24 ++--------- 4 files changed, 47 insertions(+), 41 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index f7f291f221..7141924ac5 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -1086,13 +1086,26 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val case Event(w: WatchFundingConfirmedTriggered, d: DATA_CLOSING) => d.alternativeCommitments.find(_.commitInput.outPoint.txid == w.tx.txid) match { - case Some(alternativeCommitments) => + case Some(commitments1) => + // This is a corner case where: + // - we are using dual funding + // - *and* the funding tx was RBF-ed + // - *and* we went to CLOSING before any funding tx got confirmed (probably due to a local or remote error) + // - *and* an older version of the funding tx confirmed and reached min depth (it won't be re-orged out) + // + // This means that: + // - the whole current commitment tree has been double-spent and can safely be forgotten + // - from now on, we only need to keep track of the commitment associated to the funding tx that got confirmed + // + // Force-closing is our only option here, if we are in this state the channel was closing and it is too late + // to negotiate a mutual close. log.info("an alternative funding tx with txid={} got confirmed", w.tx.txid) - watchFundingTx(alternativeCommitments) + watchFundingTx(commitments1) context.system.eventStream.publish(TransactionConfirmed(d.channelId, remoteNodeId, w.tx)) - val commitTx = alternativeCommitments.fullySignedLocalCommitTx(keyManager).tx - val localCommitPublished = Closing.LocalClose.claimCommitTxOutputs(keyManager, alternativeCommitments, commitTx, nodeParams.currentBlockHeight, nodeParams.onChainFeeConf) - stay() using d.copy(commitments = alternativeCommitments, localCommitPublished = Some(localCommitPublished)) storing() calling doPublish(localCommitPublished, alternativeCommitments) + val commitTx = commitments1.fullySignedLocalCommitTx(keyManager).tx + val localCommitPublished = Closing.LocalClose.claimCommitTxOutputs(keyManager, commitments1, commitTx, nodeParams.currentBlockHeight, nodeParams.onChainFeeConf) + val d1 = DATA_CLOSING(commitments1, None, d.waitingSince, alternativeCommitments = Nil, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) + stay() using d1 storing() calling doPublish(localCommitPublished, commitments1) case None => if (d.commitments.commitInput.outPoint.txid != w.tx.txid) { log.warning("an unknown funding tx with txid={} got confirmed, this should not happen", w.tx.txid) @@ -1128,7 +1141,10 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val handleRemoteSpentNext(tx, d) } else if (d.alternativeCommitments.exists(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid))) { // counterparty may attempt to spend an alternative unconfirmed funding tx at any time - handleRemoteSpentAlternative(tx, d.alternativeCommitments, d) + val commitments1 = d.alternativeCommitments.find(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)).get + log.warning("they published their commit with txid={} spending an alternative unconfirmed funding tx with fundingTxid={}", tx.txid, commitments1.commitInput.outPoint.txid) + blockchain ! WatchFundingConfirmed(self, commitments1.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks) + stay() } else { // counterparty may attempt to spend a revoked commit tx at any time handleRemoteSpentOther(tx, d) @@ -1322,7 +1338,11 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val case Event(WatchFundingSpentTriggered(tx), d: PersistentChannelData) if d.commitments.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid) => handleRemoteSpentNext(tx, d) - case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) if d.previousFundingTxs.map(_.commitments).exists(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)) => handleRemoteSpentAlternative(tx, d.previousFundingTxs.map(_.commitments).toList, d) + case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) if d.previousFundingTxs.map(_.commitments).exists(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)) => + val commitments1 = d.previousFundingTxs.map(_.commitments).find(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)).get + log.warning("they published their commit with txid={} spending an alternative unconfirmed funding tx with fundingTxid={}", tx.txid, commitments1.commitInput.outPoint.txid) + blockchain ! WatchFundingConfirmed(self, commitments1.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks) + spendLocalCurrent(d) case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT) => handleRemoteSpentFuture(tx, d) @@ -1525,7 +1545,11 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val case Event(WatchFundingSpentTriggered(tx), d: PersistentChannelData) if d.commitments.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid) => handleRemoteSpentNext(tx, d) - case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) if d.previousFundingTxs.map(_.commitments).exists(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)) => handleRemoteSpentAlternative(tx, d.previousFundingTxs.map(_.commitments).toList, d) + case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) if d.previousFundingTxs.map(_.commitments).exists(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)) => + val commitments1 = d.previousFundingTxs.map(_.commitments).find(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)).get + log.warning("they published their commit with txid={} spending an alternative unconfirmed funding tx with fundingTxid={}", tx.txid, commitments1.commitInput.outPoint.txid) + blockchain ! WatchFundingConfirmed(self, commitments1.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks) + spendLocalCurrent(d) case Event(WatchFundingSpentTriggered(tx), d: PersistentChannelData) => handleRemoteSpentOther(tx, d) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala index aff595e9d4..be717e67f3 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala @@ -311,17 +311,17 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { Funding.minDepthDualFunding(nodeParams.channelConf, commitments.channelFeatures, fundingParams) match { case Some(fundingMinDepth) => blockchain ! WatchFundingConfirmed(self, commitments.commitInput.outPoint.txid, fundingMinDepth) - val nextData = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, fundingTx, fundingParams, Nil, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, None, None) + val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, fundingTx, fundingParams, Nil, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, None, None) fundingTx match { - case fundingTx: PartiallySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using nextData storing() sending fundingTx.localSigs - case fundingTx: FullySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using nextData storing() sending fundingTx.localSigs calling publishFundingTx(fundingParams, fundingTx) + case fundingTx: PartiallySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d1 storing() sending fundingTx.localSigs + case fundingTx: FullySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d1 storing() sending fundingTx.localSigs calling publishFundingTx(fundingParams, fundingTx) } case None => val (shortIds, channelReady) = acceptFundingTx(commitments, RealScidStatus.Unknown) - val nextData = DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments, shortIds, channelReady) + val d1 = DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments, shortIds, channelReady) fundingTx match { - case fundingTx: PartiallySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_READY) using nextData storing() sending Seq(fundingTx.localSigs, channelReady) - case fundingTx: FullySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_READY) using nextData storing() sending Seq(fundingTx.localSigs, channelReady) calling publishFundingTx(fundingParams, fundingTx) + case fundingTx: PartiallySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_READY) using d1 storing() sending Seq(fundingTx.localSigs, channelReady) + case fundingTx: FullySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_READY) using d1 storing() sending Seq(fundingTx.localSigs, channelReady) calling publishFundingTx(fundingParams, fundingTx) } } case f: InteractiveTxBuilder.Failed => @@ -362,8 +362,8 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { stay() sending Error(d.channelId, InvalidFundingSignature(d.channelId, Some(unsignedFundingTx)).getMessage) case Right(fundingTx) => log.info("publishing funding tx for channelId={} fundingTxId={}", d.channelId, fundingTx.signedTx.txid) - val nextData = d.copy(fundingTx = fundingTx) - stay() using nextData storing() calling publishFundingTx(d.fundingParams, fundingTx) + val d1 = d.copy(fundingTx = fundingTx) + stay() using d1 storing() calling publishFundingTx(d.fundingParams, fundingTx) } case _: FullySignedSharedTransaction => log.warning("received duplicate tx_signatures") @@ -456,8 +456,8 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { when(WAIT_FOR_DUAL_FUNDING_READY)(handleExceptions { case Event(channelReady: ChannelReady, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) => - val nextData = receiveChannelReady(d.shortIds, channelReady, d.commitments) - goto(NORMAL) using nextData storing() + val d1 = receiveChannelReady(d.shortIds, channelReady, d.commitments) + goto(NORMAL) using d1 storing() case Event(_: TxInitRbf, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) => // Our peer may not have received the funding transaction confirmation. diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala index da92af8d3d..cb843569fa 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala @@ -434,8 +434,8 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers { when(WAIT_FOR_CHANNEL_READY)(handleExceptions { case Event(channelReady: ChannelReady, d: DATA_WAIT_FOR_CHANNEL_READY) => - val nextData = receiveChannelReady(d.shortIds, channelReady, d.commitments) - goto(NORMAL) using nextData storing() + val d1 = receiveChannelReady(d.shortIds, channelReady, d.commitments) + goto(NORMAL) using d1 storing() case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_WAIT_FOR_CHANNEL_READY) if d.commitments.announceChannel => delayEarlyAnnouncementSigs(remoteAnnSigs) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala index 641026c3b2..f89b340520 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ErrorHandlers.scala @@ -21,7 +21,7 @@ import akka.actor.{ActorRef, FSM} import fr.acinq.bitcoin.scalacompat.{ByteVector32, OutPoint, SatoshiLong, Transaction} import fr.acinq.eclair.NotificationsLogger import fr.acinq.eclair.NotificationsLogger.NotifyNodeOperator -import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchOutputSpent, WatchTxConfirmed} +import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchOutputSpent, WatchTxConfirmed} import fr.acinq.eclair.channel.Helpers.Closing import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel.UnhandledExceptionStrategy @@ -197,7 +197,7 @@ trait ErrorHandlers extends CommonHandlers { case closing: DATA_CLOSING => closing.copy(localCommitPublished = Some(localCommitPublished)) case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, negotiating.closingTxProposed.flatten.map(_.unsignedTx), localCommitPublished = Some(localCommitPublished)) case waitForFundingConfirmed: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.fundingTx, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) - case waitForFundingConfirmed: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.signedFundingTx_opt, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = waitForFundingConfirmed.previousFundingTxs.map(_.commitments).toList, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) + case waitForFundingConfirmed: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.signedFundingTx_opt, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = waitForFundingConfirmed.previousFundingTxs.map(_.commitments), mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) } goto(CLOSING) using nextData storing() calling doPublish(localCommitPublished, d.commitments) @@ -243,7 +243,7 @@ trait ErrorHandlers extends CommonHandlers { case closing: DATA_CLOSING => closing.copy(remoteCommitPublished = Some(remoteCommitPublished)) case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, mutualCloseProposed = negotiating.closingTxProposed.flatten.map(_.unsignedTx), remoteCommitPublished = Some(remoteCommitPublished)) case waitForFundingConfirmed: DATA_WAIT_FOR_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.fundingTx, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished)) - case waitForFundingConfirmed: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.signedFundingTx_opt, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = waitForFundingConfirmed.previousFundingTxs.map(_.commitments).toList, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished)) + case waitForFundingConfirmed: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, fundingTx = waitForFundingConfirmed.signedFundingTx_opt, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = waitForFundingConfirmed.previousFundingTxs.map(_.commitments), mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished)) case _ => DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments = Nil, mutualCloseProposed = Nil, remoteCommitPublished = Some(remoteCommitPublished)) } goto(CLOSING) using nextData storing() calling doPublish(remoteCommitPublished, d.commitments) @@ -281,24 +281,6 @@ trait ErrorHandlers extends CommonHandlers { goto(CLOSING) using nextData storing() calling doPublish(remoteCommitPublished, d.commitments) } - def handleRemoteSpentAlternative(commitTx: Transaction, alternativeCommitments: List[Commitments], d: PersistentChannelData) = { - val commitments_opt = alternativeCommitments.find(c => c.remoteCommit.txid == commitTx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == commitTx.txid)) - require(commitments_opt.nonEmpty, "there should be a commit spending an alternative funding tx matching this transaction") - val commitments = commitments_opt.get - log.warning("they published their commit with txid={} spending an alternative funding tx with fundingTxid={}", commitTx.txid, commitments.commitInput.outPoint.txid) - - context.system.eventStream.publish(TransactionPublished(d.channelId, remoteNodeId, commitTx, Closing.commitTxFee(commitments.commitInput, commitTx, commitments.localParams.isInitiator), "remote-commit")) - // We wait for this alternative funding tx to be confirmed before claiming the commit tx outputs, as it could still - // be replaced by another funding tx. - blockchain ! WatchFundingConfirmed(self, commitments.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks) - val nextData = d match { - case waitForFundingConfirmed: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => DATA_CLOSING(d.commitments, waitForFundingConfirmed.signedFundingTx_opt, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments, mutualCloseProposed = Nil) - case closing: DATA_CLOSING => closing - case _ => DATA_CLOSING(d.commitments, None, waitingSince = nodeParams.currentBlockHeight, alternativeCommitments, mutualCloseProposed = Nil) - } - goto(CLOSING) using nextData storing() - } - def doPublish(remoteCommitPublished: RemoteCommitPublished, commitments: Commitments): Unit = { import remoteCommitPublished._ From 578b6af9a8f1e74777faab036af004463a53e425 Mon Sep 17 00:00:00 2001 From: t-bast Date: Thu, 18 Aug 2022 15:10:49 +0200 Subject: [PATCH 08/11] Refactor channel_ready creation Note that when re-transmitting, we had a small bug where we didn't send our local alias. --- .../fr/acinq/eclair/channel/fsm/Channel.scala | 10 ++++------ .../eclair/channel/fsm/CommonFundingHandlers.scala | 14 +++++++++----- 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index 7141924ac5..98c21e3fd0 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -1383,16 +1383,12 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_CHANNEL_READY) => log.debug("re-sending channelReady") - val channelKeyPath = keyManager.keyPath(d.commitments.localParams, d.commitments.channelConfig) - val nextPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1) - val channelReady = ChannelReady(d.commitments.channelId, nextPerCommitmentPoint) + val channelReady = createChannelReady(d.shortIds, d.commitments) goto(WAIT_FOR_CHANNEL_READY) sending channelReady case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) => log.debug("re-sending channelReady") - val channelKeyPath = keyManager.keyPath(d.commitments.localParams, d.commitments.channelConfig) - val nextPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1) - val channelReady = ChannelReady(d.commitments.channelId, nextPerCommitmentPoint) + val channelReady = createChannelReady(d.shortIds, d.commitments) goto(WAIT_FOR_DUAL_FUNDING_READY) sending channelReady case Event(channelReestablish: ChannelReestablish, d: DATA_NORMAL) => @@ -1534,6 +1530,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val case Event(BITCOIN_FUNDING_TIMEOUT, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) => handleFundingTimeout(d) + case Event(e: BITCOIN_FUNDING_DOUBLE_SPENT, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleDualFundingDoubleSpent(e, d) + // just ignore this, we will put a new watch when we reconnect, and we'll be notified again case Event(WatchFundingConfirmedTriggered(_, _, _), _) => stay() diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/CommonFundingHandlers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/CommonFundingHandlers.scala index 579e68e025..ebf72ddb58 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/CommonFundingHandlers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/CommonFundingHandlers.scala @@ -18,13 +18,13 @@ package fr.acinq.eclair.channel.fsm import akka.actor.typed.scaladsl.adapter.actorRefAdapter import fr.acinq.bitcoin.scalacompat.ByteVector32 -import fr.acinq.eclair.{ShortChannelId, ToMilliSatoshiConversion} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingDeeplyBuried, WatchFundingLost, WatchFundingSpent} import fr.acinq.eclair.channel.Helpers.getRelayFees import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel.{ANNOUNCEMENTS_MINCONF, BroadcastChannelUpdate, PeriodicRefresh, REFRESH_CHANNEL_UPDATE_INTERVAL} import fr.acinq.eclair.router.Announcements import fr.acinq.eclair.wire.protocol.{AnnouncementSignatures, ChannelReady, ChannelReadyTlv, TlvStream} +import fr.acinq.eclair.{ShortChannelId, ToMilliSatoshiConversion} import scala.concurrent.duration.DurationInt @@ -45,17 +45,21 @@ trait CommonFundingHandlers extends CommonHandlers { def acceptFundingTx(commitments: Commitments, realScidStatus: RealScidStatus): (ShortIds, ChannelReady) = { blockchain ! WatchFundingLost(self, commitments.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks) - val channelKeyPath = keyManager.keyPath(commitments.localParams, commitments.channelConfig) - val nextPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1) // the alias will use in our peer's channel_update message, the goal is to be able to use our channel as soon // as it reaches NORMAL state, and before it is announced on the network val shortIds = ShortIds(realScidStatus, ShortChannelId.generateLocalAlias(), remoteAlias_opt = None) context.system.eventStream.publish(ShortChannelIdAssigned(self, commitments.channelId, shortIds, remoteNodeId)) - // we always send our local alias, even if it isn't explicitly supported, that's an optional TLV anyway - val channelReady = ChannelReady(commitments.channelId, nextPerCommitmentPoint, TlvStream(ChannelReadyTlv.ShortChannelIdTlv(shortIds.localAlias))) + val channelReady = createChannelReady(shortIds, commitments) (shortIds, channelReady) } + def createChannelReady(shortIds: ShortIds, commitments: Commitments): ChannelReady = { + val channelKeyPath = keyManager.keyPath(commitments.localParams, commitments.channelConfig) + val nextPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1) + // we always send our local alias, even if it isn't explicitly supported, that's an optional TLV anyway + ChannelReady(commitments.channelId, nextPerCommitmentPoint, TlvStream(ChannelReadyTlv.ShortChannelIdTlv(shortIds.localAlias))) + } + def receiveChannelReady(shortIds: ShortIds, channelReady: ChannelReady, commitments: Commitments): DATA_NORMAL = { val shortIds1 = shortIds.copy(remoteAlias_opt = channelReady.alias_opt) shortIds1.remoteAlias_opt.foreach { remoteAlias => From d5d0e36ccdf018dff3d845aa9cec8beb84089b97 Mon Sep 17 00:00:00 2001 From: t-bast Date: Thu, 18 Aug 2022 17:56:42 +0200 Subject: [PATCH 09/11] Check funding confirmation in offline Instead of watching funding txs being spent. --- .../fr/acinq/eclair/channel/fsm/Channel.scala | 47 +++++++++---------- .../channel/fsm/ChannelOpenDualFunded.scala | 16 +------ .../channel/fsm/DualFundingHandlers.scala | 20 ++++++++ .../ChannelStateTestsHelperMethods.scala | 6 +-- .../WaitForDualFundingCreatedStateSpec.scala | 3 -- ...WaitForDualFundingConfirmedStateSpec.scala | 31 +++--------- .../c/WaitForDualFundingReadyStateSpec.scala | 6 +-- 7 files changed, 54 insertions(+), 75 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index 98c21e3fd0..3e1bb247b1 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -260,6 +260,12 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val case None => // in all other cases we need to be ready for any type of closing watchFundingTx(data.commitments, closing.spendingTxs.map(_.txid).toSet) + if (closing.alternativeCommitments.nonEmpty) { + // There are unconfirmed, alternative funding transactions, so we wait for one to confirm before + // watching transactions spending it. + blockchain ! WatchFundingConfirmed(self, data.commitments.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks) + closing.alternativeCommitments.foreach(c => blockchain ! WatchFundingConfirmed(self, c.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks)) + } closing.mutualClosePublished.foreach(mcp => doPublish(mcp, isInitiator)) closing.localCommitPublished.foreach(lcp => doPublish(lcp, closing.commitments)) closing.remoteCommitPublished.foreach(rcp => doPublish(rcp, closing.commitments)) @@ -315,9 +321,11 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val } case funding: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => - (funding.commitments +: funding.previousFundingTxs.map(_.commitments)).foreach(c => watchFundingTx(c)) // we make sure that the funding tx with the highest feerate has been published publishFundingTx(funding.fundingParams, funding.fundingTx) + // we watch confirmation of all funding candidates, and once one of them confirms we will watch spending txs + blockchain ! WatchFundingConfirmed(self, funding.commitments.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks) + funding.previousFundingTxs.map(_.commitments).foreach(c => blockchain ! WatchFundingConfirmed(self, c.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks)) goto(OFFLINE) using funding case _ => @@ -1106,10 +1114,11 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val val localCommitPublished = Closing.LocalClose.claimCommitTxOutputs(keyManager, commitments1, commitTx, nodeParams.currentBlockHeight, nodeParams.onChainFeeConf) val d1 = DATA_CLOSING(commitments1, None, d.waitingSince, alternativeCommitments = Nil, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) stay() using d1 storing() calling doPublish(localCommitPublished, commitments1) + case None if d.commitments.commitInput.outPoint.txid == w.tx.txid => + // The best funding tx candidate has been confirmed, we can forget alternative commitments. + stay() using d.copy(alternativeCommitments = Nil) case None => - if (d.commitments.commitInput.outPoint.txid != w.tx.txid) { - log.warning("an unknown funding tx with txid={} got confirmed, this should not happen", w.tx.txid) - } + log.warning("an unknown funding tx with txid={} got confirmed, this should not happen", w.tx.txid) stay() } @@ -1139,12 +1148,6 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val } else if (d.commitments.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)) { // counterparty may attempt to spend its last commit tx at any time handleRemoteSpentNext(tx, d) - } else if (d.alternativeCommitments.exists(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid))) { - // counterparty may attempt to spend an alternative unconfirmed funding tx at any time - val commitments1 = d.alternativeCommitments.find(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)).get - log.warning("they published their commit with txid={} spending an alternative unconfirmed funding tx with fundingTxid={}", tx.txid, commitments1.commitInput.outPoint.txid) - blockchain ! WatchFundingConfirmed(self, commitments1.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks) - stay() } else { // counterparty may attempt to spend a revoked commit tx at any time handleRemoteSpentOther(tx, d) @@ -1326,10 +1329,12 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val case Event(e: BITCOIN_FUNDING_DOUBLE_SPENT, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleDualFundingDoubleSpent(e, d) + case Event(w: WatchFundingConfirmedTriggered, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleDualFundingConfirmedOffline(w, d) + // just ignore this, we will put a new watch when we reconnect, and we'll be notified again - case Event(WatchFundingConfirmedTriggered(_, _, _), _) => stay() + case Event(_: WatchFundingConfirmedTriggered, _) => stay() - case Event(WatchFundingDeeplyBuriedTriggered(_, _, _), _) => stay() + case Event(_: WatchFundingDeeplyBuriedTriggered, _) => stay() case Event(WatchFundingSpentTriggered(tx), d: DATA_NEGOTIATING) if d.closingTxProposed.flatten.exists(_.unsignedTx.tx.txid == tx.txid) => handleMutualClose(getMutualClosePublished(tx, d.closingTxProposed), Left(d)) @@ -1338,12 +1343,6 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val case Event(WatchFundingSpentTriggered(tx), d: PersistentChannelData) if d.commitments.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid) => handleRemoteSpentNext(tx, d) - case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) if d.previousFundingTxs.map(_.commitments).exists(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)) => - val commitments1 = d.previousFundingTxs.map(_.commitments).find(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)).get - log.warning("they published their commit with txid={} spending an alternative unconfirmed funding tx with fundingTxid={}", tx.txid, commitments1.commitInput.outPoint.txid) - blockchain ! WatchFundingConfirmed(self, commitments1.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks) - spendLocalCurrent(d) - case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT) => handleRemoteSpentFuture(tx, d) case Event(WatchFundingSpentTriggered(tx), d: PersistentChannelData) => handleRemoteSpentOther(tx, d) @@ -1532,10 +1531,12 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val case Event(e: BITCOIN_FUNDING_DOUBLE_SPENT, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleDualFundingDoubleSpent(e, d) + case Event(w: WatchFundingConfirmedTriggered, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleDualFundingConfirmedOffline(w, d) + // just ignore this, we will put a new watch when we reconnect, and we'll be notified again - case Event(WatchFundingConfirmedTriggered(_, _, _), _) => stay() + case Event(_: WatchFundingConfirmedTriggered, _) => stay() - case Event(WatchFundingDeeplyBuriedTriggered(_, _, _), _) => stay() + case Event(_: WatchFundingDeeplyBuriedTriggered, _) => stay() case Event(WatchFundingSpentTriggered(tx), d: DATA_NEGOTIATING) if d.closingTxProposed.flatten.exists(_.unsignedTx.tx.txid == tx.txid) => handleMutualClose(getMutualClosePublished(tx, d.closingTxProposed), Left(d)) @@ -1543,12 +1544,6 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val case Event(WatchFundingSpentTriggered(tx), d: PersistentChannelData) if d.commitments.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid) => handleRemoteSpentNext(tx, d) - case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) if d.previousFundingTxs.map(_.commitments).exists(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)) => - val commitments1 = d.previousFundingTxs.map(_.commitments).find(c => c.remoteCommit.txid == tx.txid || c.remoteNextCommitInfo.left.toOption.exists(_.nextRemoteCommit.txid == tx.txid)).get - log.warning("they published their commit with txid={} spending an alternative unconfirmed funding tx with fundingTxid={}", tx.txid, commitments1.commitInput.outPoint.txid) - blockchain ! WatchFundingConfirmed(self, commitments1.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks) - spendLocalCurrent(d) - case Event(WatchFundingSpentTriggered(tx), d: PersistentChannelData) => handleRemoteSpentOther(tx, d) case Event(e: Error, d: PersistentChannelData) => handleRemoteError(e, d) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala index be717e67f3..b9762ef966 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala @@ -307,7 +307,6 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { case InteractiveTxBuilder.SendMessage(msg) => stay() sending msg case InteractiveTxBuilder.Succeeded(fundingParams, fundingTx, commitments) => d.deferred.foreach(self ! _) - watchFundingTx(commitments) Funding.minDepthDualFunding(nodeParams.channelConf, commitments.channelFeatures, fundingParams) match { case Some(fundingMinDepth) => blockchain ! WatchFundingConfirmed(self, commitments.commitInput.outPoint.txid, fundingMinDepth) @@ -317,6 +316,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { case fundingTx: FullySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d1 storing() sending fundingTx.localSigs calling publishFundingTx(fundingParams, fundingTx) } case None => + watchFundingTx(commitments) val (shortIds, channelReady) = acceptFundingTx(commitments, RealScidStatus.Unknown) val d1 = DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments, shortIds, channelReady) fundingTx match { @@ -422,6 +422,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { (d.fundingParams.remoteAmount == 0.sat || d.commitments.localParams.initFeatures.hasFeature(Features.ZeroConf)) if (canUseZeroConf) { log.info("this channel isn't zero-conf, but they sent an early channel_ready with an alias: no need to wait for confirmations") + watchFundingTx(d.commitments) val (shortIds, localChannelReady) = acceptFundingTx(d.commitments, RealScidStatus.Unknown) self ! remoteChannelReady // NB: we will receive a WatchFundingConfirmedTriggered later that will simply be ignored @@ -438,19 +439,6 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { context.system.scheduler.scheduleOnce(2 seconds, self, remoteAnnSigs) stay() - case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => - // We wait for one of the funding transactions to confirm before going to the closing state, as the spent funding - // tx and the associated commit tx could be replaced by a new version of the funding tx. - if (tx.txid == d.commitments.remoteCommit.txid) { - log.warning("funding tx spent by txid={} while still unconfirmed", tx.txid) - stay() - } else if (d.previousFundingTxs.exists(_.commitments.remoteCommit.txid == tx.txid)) { - log.warning("previous funding tx spent by txid while still unconfirmed", tx.txid) - stay() - } else { - handleInformationLeak(tx, d) - } - case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleRemoteError(e, d) }) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala index 5eeec2dbd2..dda1d3e1ff 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala @@ -17,6 +17,7 @@ package fr.acinq.eclair.channel.fsm import fr.acinq.eclair.blockchain.CurrentBlockHeight +import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.WatchFundingConfirmedTriggered import fr.acinq.eclair.channel.Helpers.Closing import fr.acinq.eclair.channel.InteractiveTxBuilder.{FullySignedSharedTransaction, InteractiveTxParams, PartiallySignedSharedTransaction, SignedSharedTransaction} import fr.acinq.eclair.channel._ @@ -55,6 +56,25 @@ trait DualFundingHandlers extends CommonFundingHandlers { } } + def handleDualFundingConfirmedOffline(w: WatchFundingConfirmedTriggered, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) = { + if (w.tx.txid == d.commitments.commitInput.outPoint.txid) { + watchFundingTx(d.commitments) + context.system.eventStream.publish(TransactionConfirmed(d.channelId, remoteNodeId, w.tx)) + // We can forget previous funding attempts now that the funding tx is confirmed. + stay() using d.copy(previousFundingTxs = Nil) storing() + } else if (d.previousFundingTxs.exists(_.commitments.commitInput.outPoint.txid == w.tx.txid)) { + log.info("a previous funding tx with txid={} has been confirmed", w.tx.txid) + val confirmed = d.previousFundingTxs.find(_.commitments.commitInput.outPoint.txid == w.tx.txid).get + watchFundingTx(confirmed.commitments) + context.system.eventStream.publish(TransactionConfirmed(d.channelId, remoteNodeId, w.tx)) + // We can forget other funding attempts now that one of the funding txs is confirmed. + stay() using d.copy(commitments = confirmed.commitments, fundingTx = confirmed.fundingTx, previousFundingTxs = Nil) storing() + } else { + log.error(s"internal error: a funding tx confirmed that doesn't match any of our funding txs: ${w.tx.txid}") + stay() + } + } + def handleNewBlockDualFundingUnconfirmed(c: CurrentBlockHeight, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) = { if (Channel.FUNDING_TIMEOUT_FUNDEE < c.blockHeight - d.waitingSince && Closing.nothingAtStake(d)) { log.warning("funding transaction did not confirm in {} blocks and we have nothing at stake, forgetting channel", Channel.FUNDING_TIMEOUT_FUNDEE) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala index 547526800d..620caad416 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/ChannelStateTestsHelperMethods.scala @@ -298,8 +298,6 @@ trait ChannelStateTestsBase extends Assertions with Eventually { alice2bob.expectMsgType[TxSignatures] alice2bob.forward(bob) val fundingTx = eventListener.expectMsgType[TransactionPublished].tx - alice2blockchain.expectMsgType[WatchFundingSpent] - bob2blockchain.expectMsgType[WatchFundingSpent] if (!channelType.features.contains(Features.ZeroConf)) { eventually(assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)) eventually(assert(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)) @@ -307,10 +305,10 @@ trait ChannelStateTestsBase extends Assertions with Eventually { bob2blockchain.expectMsgType[WatchFundingConfirmed] alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx) bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx) - alice2blockchain.expectMsgType[WatchFundingSpent] - bob2blockchain.expectMsgType[WatchFundingSpent] } + alice2blockchain.expectMsgType[WatchFundingSpent] alice2blockchain.expectMsgType[WatchFundingLost] + bob2blockchain.expectMsgType[WatchFundingSpent] bob2blockchain.expectMsgType[WatchFundingLost] alice2bob.expectMsgType[ChannelReady] alice2bob.forward(bob) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala index 0d87cd1f0d..a65e8cd05c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingCreatedStateSpec.scala @@ -102,13 +102,11 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn assert(bobData.commitments.channelFeatures.hasFeature(Features.DualFunding)) assert(bobData.fundingTx.isInstanceOf[PartiallySignedSharedTransaction]) val fundingTxId = bobData.fundingTx.asInstanceOf[PartiallySignedSharedTransaction].tx.buildUnsignedTx().txid - assert(bob2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTxId) assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTxId) // Alice receives Bob's signatures and sends her own signatures. bob2alice.forward(alice) assert(listener.expectMsgType[TransactionPublished].tx.txid == fundingTxId) - assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTxId) assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTxId) alice2bob.expectMsgType[TxSignatures] awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) @@ -247,7 +245,6 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn alice2bob.forward(bob) val bobSigs = bob2alice.expectMsgType[TxSignatures] - bob2blockchain.expectMsgType[WatchFundingSpent] bob2blockchain.expectMsgType[WatchFundingConfirmed] bob2alice.forward(alice, bobSigs.copy(txId = randomBytes32(), witnesses = Nil)) alice2bob.expectMsgType[TxAbort] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala index c007e97a48..f002243f55 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingConfirmedStateSpec.scala @@ -17,7 +17,7 @@ package fr.acinq.eclair.channel.states.c import akka.testkit.{TestFSMRef, TestProbe} -import fr.acinq.bitcoin.scalacompat.{ByteVector32, Transaction} +import fr.acinq.bitcoin.scalacompat.ByteVector32 import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ import fr.acinq.eclair.blockchain.{CurrentBlockHeight, SingleKeyOnChainWallet} import fr.acinq.eclair.channel.InteractiveTxBuilder.FullySignedSharedTransaction @@ -96,14 +96,12 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture bob2alice.forward(alice) // Alice publishes the funding tx. val fundingTx = aliceListener.expectMsgType[TransactionPublished].tx - assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid) assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid) alice2bob.expectMsgType[TxSignatures] alice2bob.forward(bob) awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) // Bob publishes the funding tx. assert(bobListener.expectMsgType[TransactionPublished].tx.txid == fundingTx.txid) - assert(bob2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid) assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid) withFixture(test.toNoArgTest(FixtureParam(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain, aliceListener, bobListener, wallet))) } @@ -261,6 +259,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture bob2alice.forward(alice) val aliceChannelReady = alice2bob.expectMsgType[ChannelReady] assert(aliceChannelReady.alias_opt.isDefined) + assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid) assert(alice2blockchain.expectMsgType[WatchFundingLost].txId == fundingTx.txid) assert(alice2blockchain.expectMsgType[WatchFundingDeeplyBuried].txId == fundingTx.txid) awaitCond(alice.stateName == NORMAL) @@ -279,32 +278,16 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY) } - test("recv WatchFundingSpentTriggered (remote commit)", Tag(ChannelStateTestsTags.DualFunding)) { f => - import f._ - // bob publishes his commitment tx - val bobCommitTx = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx - alice ! WatchFundingSpentTriggered(bobCommitTx) - // alice doesn't react since the spent funding tx is still unconfirmed - alice2blockchain.expectNoMessage(100 millis) - assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED) - } - - test("recv WatchFundingSpentTriggered (other commit)", Tag(ChannelStateTestsTags.DualFunding)) { f => - import f._ - val commitTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx - alice ! WatchFundingSpentTriggered(Transaction(0, Nil, Nil, 0)) - alice2bob.expectMsgType[Error] - assert(alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx].tx.txid == commitTx.txid) - awaitCond(alice.stateName == ERR_INFORMATION_LEAK) - } - test("recv WatchFundingSpentTriggered while offline (remote commit)", Tag(ChannelStateTestsTags.DualFunding)) { f => import f._ val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx - val aliceCommitTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx alice ! INPUT_DISCONNECTED awaitCond(alice.stateName == OFFLINE) - // bob publishes his commitment tx + // The funding tx confirms while we're offline. + alice ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx) + assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid) + alice2blockchain.expectNoMessage(100 millis) + // Bob broadcasts his commit tx. val bobCommitTx = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx alice ! WatchFundingSpentTriggered(bobCommitTx.tx) val claimMain = alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala index 1cf113db0f..b391612ebd 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForDualFundingReadyStateSpec.scala @@ -81,18 +81,16 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF bob2alice.forward(alice) alice2bob.expectMsgType[TxSignatures] alice2bob.forward(bob) - alice2blockchain.expectMsgType[WatchFundingSpent] - bob2blockchain.expectMsgType[WatchFundingSpent] if (!test.tags.contains(ChannelStateTestsTags.ZeroConf)) { val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid) assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid) alice ! WatchFundingConfirmedTriggered(BlockHeight(TestConstants.defaultBlockHeight), 42, fundingTx) bob ! WatchFundingConfirmedTriggered(BlockHeight(TestConstants.defaultBlockHeight), 42, fundingTx) - alice2blockchain.expectMsgType[WatchFundingSpent] - bob2blockchain.expectMsgType[WatchFundingSpent] } + alice2blockchain.expectMsgType[WatchFundingSpent] alice2blockchain.expectMsgType[WatchFundingLost] + bob2blockchain.expectMsgType[WatchFundingSpent] bob2blockchain.expectMsgType[WatchFundingLost] awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY) awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_READY) From f49116ab7af3733bd2b1055c66e8eaa27236276b Mon Sep 17 00:00:00 2001 From: t-bast Date: Thu, 18 Aug 2022 18:45:33 +0200 Subject: [PATCH 10/11] fixup! Check funding confirmation in offline --- .../fr/acinq/eclair/channel/fsm/Channel.scala | 12 +++++---- .../channel/fsm/ChannelOpenDualFunded.scala | 25 ++++++------------- 2 files changed, 15 insertions(+), 22 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index 3e1bb247b1..351b57a7f5 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -1114,12 +1114,14 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val val localCommitPublished = Closing.LocalClose.claimCommitTxOutputs(keyManager, commitments1, commitTx, nodeParams.currentBlockHeight, nodeParams.onChainFeeConf) val d1 = DATA_CLOSING(commitments1, None, d.waitingSince, alternativeCommitments = Nil, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished)) stay() using d1 storing() calling doPublish(localCommitPublished, commitments1) - case None if d.commitments.commitInput.outPoint.txid == w.tx.txid => - // The best funding tx candidate has been confirmed, we can forget alternative commitments. - stay() using d.copy(alternativeCommitments = Nil) case None => - log.warning("an unknown funding tx with txid={} got confirmed, this should not happen", w.tx.txid) - stay() + if (d.commitments.commitInput.outPoint.txid == w.tx.txid) { + // The best funding tx candidate has been confirmed, we can forget alternative commitments. + stay() using d.copy(alternativeCommitments = Nil) storing() + } else { + log.warning("an unknown funding tx with txid={} got confirmed, this should not happen", w.tx.txid) + stay() + } } case Event(WatchFundingSpentTriggered(tx), d: DATA_CLOSING) => diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala index b9762ef966..d9afe82447 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala @@ -17,8 +17,7 @@ package fr.acinq.eclair.channel.fsm import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapter} -import fr.acinq.bitcoin.ScriptFlags -import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Script, Transaction} +import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Script} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ import fr.acinq.eclair.channel.Helpers.Funding import fr.acinq.eclair.channel.InteractiveTxBuilder.{FullySignedSharedTransaction, InteractiveTxParams, PartiallySignedSharedTransaction} @@ -30,7 +29,6 @@ import fr.acinq.eclair.wire.protocol._ import fr.acinq.eclair.{Features, RealShortChannelId} import scala.concurrent.duration.DurationInt -import scala.util.{Failure, Success, Try} /** * Created by t-bast on 19/04/2022. @@ -389,20 +387,13 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { val allFundingTxs = DualFundingTx(d.fundingTx, d.commitments) +: d.previousFundingTxs allFundingTxs.find(_.commitments.commitInput.outPoint.txid == confirmedTx.txid) match { case Some(DualFundingTx(_, commitments)) => - Try(Transaction.correctlySpends(commitments.fullySignedLocalCommitTx(keyManager).tx, Seq(confirmedTx), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) match { - case Success(_) => - log.info(s"channelId=${commitments.channelId} was confirmed at blockHeight=$blockHeight txIndex=$txIndex with funding txid=${commitments.commitInput.outPoint.txid}") - watchFundingTx(commitments) - context.system.eventStream.publish(TransactionConfirmed(commitments.channelId, remoteNodeId, confirmedTx)) - val realScidStatus = RealScidStatus.Temporary(RealShortChannelId(blockHeight, txIndex, commitments.commitInput.outPoint.index.toInt)) - val (shortIds, channelReady) = acceptFundingTx(commitments, realScidStatus = realScidStatus) - d.deferred.foreach(self ! _) - goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments, shortIds, channelReady) storing() sending channelReady - case Failure(t) => - log.error(t, s"rejecting channel with invalid funding tx: ${confirmedTx.bin}") - allFundingTxs.foreach(f => wallet.rollback(f.fundingTx.tx.buildUnsignedTx())) - goto(CLOSED) - } + log.info(s"channelId=${commitments.channelId} was confirmed at blockHeight=$blockHeight txIndex=$txIndex with funding txid=${commitments.commitInput.outPoint.txid}") + watchFundingTx(commitments) + context.system.eventStream.publish(TransactionConfirmed(commitments.channelId, remoteNodeId, confirmedTx)) + val realScidStatus = RealScidStatus.Temporary(RealShortChannelId(blockHeight, txIndex, commitments.commitInput.outPoint.index.toInt)) + val (shortIds, channelReady) = acceptFundingTx(commitments, realScidStatus = realScidStatus) + d.deferred.foreach(self ! _) + goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments, shortIds, channelReady) storing() sending channelReady case None => log.error(s"internal error: the funding tx that confirmed doesn't match any of our funding txs: ${confirmedTx.bin}") allFundingTxs.foreach(f => wallet.rollback(f.fundingTx.tx.buildUnsignedTx())) From ccf43f9202609d78d053d36f1e27949365461944 Mon Sep 17 00:00:00 2001 From: t-bast Date: Fri, 19 Aug 2022 11:10:11 +0200 Subject: [PATCH 11/11] Harmonize logs And a few nits. --- .../main/scala/fr/acinq/eclair/channel/fsm/Channel.scala | 2 +- .../acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala | 8 ++------ .../eclair/channel/fsm/ChannelOpenSingleFunded.scala | 5 +---- .../fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala | 2 +- 4 files changed, 5 insertions(+), 12 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index 351b57a7f5..31dadfbb0c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -1107,7 +1107,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val // // Force-closing is our only option here, if we are in this state the channel was closing and it is too late // to negotiate a mutual close. - log.info("an alternative funding tx with txid={} got confirmed", w.tx.txid) + log.info("channelId={} was confirmed at blockHeight={} txIndex={} with a previous funding txid={}", d.channelId, w.blockHeight, w.txIndex, w.tx.txid) watchFundingTx(commitments1) context.system.eventStream.publish(TransactionConfirmed(d.channelId, remoteNodeId, w.tx)) val commitTx = commitments1.fullySignedLocalCommitTx(keyManager).tx diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala index d9afe82447..d65877f496 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenDualFunded.scala @@ -387,7 +387,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { val allFundingTxs = DualFundingTx(d.fundingTx, d.commitments) +: d.previousFundingTxs allFundingTxs.find(_.commitments.commitInput.outPoint.txid == confirmedTx.txid) match { case Some(DualFundingTx(_, commitments)) => - log.info(s"channelId=${commitments.channelId} was confirmed at blockHeight=$blockHeight txIndex=$txIndex with funding txid=${commitments.commitInput.outPoint.txid}") + log.info("channelId={} was confirmed at blockHeight={} txIndex={} with funding txid={}", d.channelId, blockHeight, txIndex, confirmedTx.txid) watchFundingTx(commitments) context.system.eventStream.publish(TransactionConfirmed(commitments.channelId, remoteNodeId, confirmedTx)) val realScidStatus = RealScidStatus.Temporary(RealShortChannelId(blockHeight, txIndex, commitments.commitInput.outPoint.index.toInt)) @@ -396,7 +396,6 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments, shortIds, channelReady) storing() sending channelReady case None => log.error(s"internal error: the funding tx that confirmed doesn't match any of our funding txs: ${confirmedTx.bin}") - allFundingTxs.foreach(f => wallet.rollback(f.fundingTx.tx.buildUnsignedTx())) goto(CLOSED) } @@ -424,10 +423,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { } case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) if d.commitments.announceChannel => - log.debug("received remote announcement signatures, delaying") - // we may receive their announcement sigs before our watcher notifies us that the channel has reached min_conf (especially during testing when blocks are generated in bulk) - // note: no need to persist their message, in case of disconnection they will resend it - context.system.scheduler.scheduleOnce(2 seconds, self, remoteAnnSigs) + delayEarlyAnnouncementSigs(remoteAnnSigs) stay() case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleRemoteError(e, d) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala index cb843569fa..8b641db369 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunded.scala @@ -404,10 +404,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers { } case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if d.commitments.announceChannel => - log.debug("received remote announcement signatures, delaying") - // we may receive their announcement sigs before our watcher notifies us that the channel has reached min_conf (especially during testing when blocks are generated in bulk) - // note: no need to persist their message, in case of disconnection they will resend it - context.system.scheduler.scheduleOnce(2 seconds, self, remoteAnnSigs) + delayEarlyAnnouncementSigs(remoteAnnSigs) stay() case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.commitInput.outPoint.txid => handleGetFundingTx(getTxResponse, d.waitingSince, d.fundingTx) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala index dda1d3e1ff..8037f6cc64 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/DualFundingHandlers.scala @@ -63,7 +63,7 @@ trait DualFundingHandlers extends CommonFundingHandlers { // We can forget previous funding attempts now that the funding tx is confirmed. stay() using d.copy(previousFundingTxs = Nil) storing() } else if (d.previousFundingTxs.exists(_.commitments.commitInput.outPoint.txid == w.tx.txid)) { - log.info("a previous funding tx with txid={} has been confirmed", w.tx.txid) + log.info("channelId={} was confirmed at blockHeight={} txIndex={} with a previous funding txid={}", d.channelId, w.blockHeight, w.txIndex, w.tx.txid) val confirmed = d.previousFundingTxs.find(_.commitments.commitInput.outPoint.txid == w.tx.txid).get watchFundingTx(confirmed.commitments) context.system.eventStream.publish(TransactionConfirmed(d.channelId, remoteNodeId, w.tx))