From 38ae09a1c8a4c03e4d11e0d00f44509022a826eb Mon Sep 17 00:00:00 2001 From: Richard Myers Date: Tue, 27 Aug 2024 14:51:05 +0200 Subject: [PATCH] Update to select funding inputs before sending open_channel2 and splice_init - If `addExcessToRecipientPosition_opt` is set, excess from funding (if any) will be added to the proposed `fundingAmount`/`fundingContribution` before sending `open_channel2`/`splice_init` respectively. - We assume our peer requires confirmed inputs. In the future we could add a heuristic for this, but it's safer to assume they want confirmed inputs. --- .../fr/acinq/eclair/channel/ChannelData.scala | 11 +- .../fr/acinq/eclair/channel/fsm/Channel.scala | 51 ++++++- .../channel/fsm/ChannelOpenDualFunded.scala | 125 +++++++++++++----- .../channel/fsm/DualFundingHandlers.scala | 9 +- .../channel/fund/InteractiveTxBuilder.scala | 81 +++++++----- .../channel/fund/InteractiveTxFunder.scala | 37 +++--- .../acinq/eclair/io/PeerReadyNotifier.scala | 1 + .../blockchain/DummyOnChainWallet.scala | 10 ++ .../channel/InteractiveTxBuilderSpec.scala | 8 ++ .../ChannelStateTestsHelperMethods.scala | 4 +- ...ngInternalDualFundedChannelStateSpec.scala | 107 +++++++++++++++ .../WaitForDualFundingCreatedStateSpec.scala | 6 +- .../b/WaitForDualFundingSignedStateSpec.scala | 6 +- ...WaitForDualFundingConfirmedStateSpec.scala | 6 +- .../states/e/NormalSplicesStateSpec.scala | 13 ++ .../basic/fixtures/MinimalNodeFixture.scala | 6 +- 16 files changed, 377 insertions(+), 104 deletions(-) create mode 100644 eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForFundingInternalDualFundedChannelStateSpec.scala 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 ff3809ff0d..097837c707 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 @@ -22,7 +22,7 @@ import fr.acinq.bitcoin.scalacompat.{ByteVector32, DeterministicWallet, OutPoint import fr.acinq.eclair.blockchain.fee.{ConfirmationTarget, FeeratePerKw} import fr.acinq.eclair.channel.LocalFundingStatus.DualFundedUnconfirmedFundingTx import fr.acinq.eclair.channel.fund.InteractiveTxBuilder._ -import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession} +import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxFunder, InteractiveTxSigningSession} import fr.acinq.eclair.io.Peer import fr.acinq.eclair.transactions.CommitmentSpec import fr.acinq.eclair.transactions.Transactions._ @@ -62,6 +62,7 @@ case object WAIT_FOR_FUNDING_CONFIRMED extends ChannelState case object WAIT_FOR_CHANNEL_READY extends ChannelState // Dual-funded channel opening: case object WAIT_FOR_INIT_DUAL_FUNDED_CHANNEL extends ChannelState +case object WAIT_FOR_DUAL_FUNDING_INTERNAL 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 @@ -506,7 +507,7 @@ object SpliceStatus { /** The channel is quiescent, we wait for our peer to send splice_init or tx_init_rbf. */ case object NonInitiatorQuiescent extends SpliceStatus /** We told our peer we want to splice funds in the channel. */ - case class SpliceRequested(cmd: CMD_SPLICE, init: SpliceInit) extends SpliceStatus + case class SpliceRequested(cmd: CMD_SPLICE, init: SpliceInit, fundingContributions_opt: Option[InteractiveTxFunder.FundingContributions]) extends SpliceStatus /** We told our peer we want to RBF the latest splice transaction. */ case class RbfRequested(cmd: CMD_BUMP_FUNDING_FEE, rbf: TxInitRbf) extends SpliceStatus /** We both agreed to splice/rbf and are building the corresponding transaction. */ @@ -583,10 +584,14 @@ final case class DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments: Commitments, } final case class DATA_WAIT_FOR_CHANNEL_READY(commitments: Commitments, shortIds: ShortIds) extends ChannelDataWithCommitments +final case class DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL(input: INPUT_INIT_CHANNEL_INITIATOR) extends TransientChannelData { + val channelId: ByteVector32 = input.temporaryChannelId +} + final case class DATA_WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL(init: INPUT_INIT_CHANNEL_NON_INITIATOR) extends TransientChannelData { val channelId: ByteVector32 = init.temporaryChannelId } -final case class DATA_WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL(init: INPUT_INIT_CHANNEL_INITIATOR, lastSent: OpenDualFundedChannel) extends TransientChannelData { +final case class DATA_WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL(init: INPUT_INIT_CHANNEL_INITIATOR, lastSent: OpenDualFundedChannel, fundingContributions: InteractiveTxFunder.FundingContributions) extends TransientChannelData { val channelId: ByteVector32 = lastSent.temporaryChannelId } final case class DATA_WAIT_FOR_DUAL_FUNDING_CREATED(channelId: ByteVector32, 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 356415856b..6378b4cb8e 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 @@ -21,7 +21,7 @@ import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapte import akka.actor.{Actor, ActorContext, ActorRef, FSM, OneForOneStrategy, PossiblyHarmful, Props, SupervisorStrategy, typed} import akka.event.Logging.MDC import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} -import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, SatoshiLong, Transaction, TxId} +import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, SatoshiLong, Script, Transaction, TxId} import fr.acinq.eclair.Logs.LogCategory import fr.acinq.eclair._ import fr.acinq.eclair.blockchain.OnChainWallet.MakeFundingTxResponse @@ -944,7 +944,28 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with context.system.scheduler.scheduleOnce(2 second, peer, Peer.Disconnect(remoteNodeId)) stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending Warning(d.channelId, f.getMessage) case Right(spliceInit) => - stay() using d.copy(spliceStatus = SpliceStatus.SpliceRequested(cmd, spliceInit)) sending spliceInit + // use our fundingPubKey as a placeholder for the remote funder's pubkey + val fundingPubKey = spliceInit.fundingPubKey + val parentCommitment = d.commitments.latest.commitment + // assume our peer requires confirmed inputs when we initiate a splice + val requireConfirmedInputs = RequireConfirmedInputs(forLocal = true, forRemote = nodeParams.channelConf.requireConfirmedInputsForDualFunding) + val fundingParams = InteractiveTxParams( + channelId = spliceInit.channelId, + isInitiator = true, + localContribution = spliceInit.fundingContribution, + remoteContribution = 0 sat, + sharedInput_opt = Some(Multisig2of2Input(parentCommitment)), + remoteFundingPubKey = fundingPubKey, + localOutputs = cmd.spliceOutputs, + lockTime = nodeParams.currentBlockHeight.toLong, + dustLimit = d.commitments.params.localParams.dustLimit, + targetFeerate = spliceInit.feerate, + requireConfirmedInputs = requireConfirmedInputs + ) + val dummyFundingPubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(fundingPubKey, fundingPubKey))) + val txFunder = context.spawnAnonymous(InteractiveTxFunder(remoteNodeId, fundingParams, dummyFundingPubkeyScript, purpose = InteractiveTxBuilder.SpliceTx(parentCommitment), wallet, nodeParams.channelConf.maxExcess_opt)) + txFunder ! InteractiveTxFunder.FundTransaction(self) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceRequested(cmd, spliceInit, None)) } case cmd: CMD_BUMP_FUNDING_FEE => initiateSpliceRbf(cmd, d) match { case Left(f) => @@ -1018,6 +1039,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with channelParams = d.commitments.params, purpose = InteractiveTxBuilder.SpliceTx(parentCommitment), localPushAmount = spliceAck.pushAmount, remotePushAmount = msg.pushAmount, + None, wallet )) txBuilder ! InteractiveTxBuilder.Start(self) @@ -1036,7 +1058,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with case Event(msg: SpliceAck, d: DATA_NORMAL) => d.spliceStatus match { - case SpliceStatus.SpliceRequested(cmd, spliceInit) => + case SpliceStatus.SpliceRequested(cmd, spliceInit, fundingContributions_opt) => log.info("our peer accepted our splice request and will contribute {} to the funding transaction", msg.fundingContribution) val parentCommitment = d.commitments.latest.commitment val fundingParams = InteractiveTxParams( @@ -1059,6 +1081,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with channelParams = d.commitments.params, purpose = InteractiveTxBuilder.SpliceTx(parentCommitment), localPushAmount = cmd.pushAmount, remotePushAmount = msg.pushAmount, + fundingContributions_opt = fundingContributions_opt, wallet )) txBuilder ! InteractiveTxBuilder.Start(self) @@ -1068,6 +1091,22 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with stay() } + case Event(msg: InteractiveTxFunder.Response, d: DATA_NORMAL) => + d.spliceStatus match { + case SpliceStatus.SpliceRequested(cmd, spliceInit, _) => + msg match { + case InteractiveTxFunder.FundingFailed => + cmd.replyTo ! RES_FAILURE(cmd, ChannelFundingError(d.channelId)) + stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) calling endQuiescence(d) + case fundingContributions: InteractiveTxFunder.FundingContributions => + val spliceInit1 = spliceInit.copy(fundingContribution = spliceInit.fundingContribution + fundingContributions.addExcess_opt.getOrElse(0 sat)) + stay() using d.copy(spliceStatus = SpliceStatus.SpliceRequested(cmd, spliceInit1, Some(fundingContributions))) sending spliceInit1 + } + case _ => + log.warning("received unexpected response from txFunder: {}, current splice status is {}", msg, d.spliceStatus) + stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) calling endQuiescence(d) + } + case Event(msg: InteractiveTxConstructionMessage, d: DATA_NORMAL) => d.spliceStatus match { case SpliceStatus.SpliceInProgress(_, _, txBuilder, _) => @@ -1116,6 +1155,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with channelParams = d.commitments.params, purpose = rbf, localPushAmount = 0 msat, remotePushAmount = 0 msat, + None, wallet )) txBuilder ! InteractiveTxBuilder.Start(self) @@ -1160,6 +1200,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with channelParams = d.commitments.params, purpose = rbf, localPushAmount = 0 msat, remotePushAmount = 0 msat, + None, wallet )) txBuilder ! InteractiveTxBuilder.Start(self) @@ -1183,7 +1224,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with log.info("our peer aborted the splice attempt: ascii='{}' bin={}", msg.toAscii, msg.data) rollbackFundingAttempt(signingSession.fundingTx.tx, previousTxs = Seq.empty) // no splice rbf yet stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage) calling endQuiescence(d) - case SpliceStatus.SpliceRequested(cmd, _) => + case SpliceStatus.SpliceRequested(cmd, _, _) => log.info("our peer rejected our splice attempt: ascii='{}' bin={}", msg.toAscii, msg.data) cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException(s"splice attempt rejected by our peer: ${msg.toAscii}")) stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage) calling endQuiescence(d) @@ -2996,7 +3037,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with private def reportSpliceFailure(spliceStatus: SpliceStatus, f: Throwable): Unit = { val cmd_opt = spliceStatus match { case SpliceStatus.NegotiatingQuiescence(cmd_opt, _) => cmd_opt - case SpliceStatus.SpliceRequested(cmd, _) => Some(cmd) + case SpliceStatus.SpliceRequested(cmd, _, _) => Some(cmd) case SpliceStatus.RbfRequested(cmd, _) => Some(cmd) case SpliceStatus.SpliceInProgress(cmd_opt, _, txBuilder, _) => txBuilder ! InteractiveTxBuilder.Abort 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 35f7cf6297..5d0b4cbbe7 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,19 @@ package fr.acinq.eclair.channel.fsm +import akka.actor.Status import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapter} import com.softwaremill.quicklens.{ModifyPimp, QuicklensAt} -import fr.acinq.bitcoin.scalacompat.SatoshiLong +import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Script} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ -import fr.acinq.eclair.channel._ +import fr.acinq.eclair.channel.{WAIT_FOR_DUAL_FUNDING_INTERNAL, WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL, _} import fr.acinq.eclair.channel.fsm.Channel._ -import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTransaction, InteractiveTxParams, PartiallySignedSharedTransaction, RequireConfirmedInputs} -import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession} +import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTransaction, InteractiveTxParams, LocalFailure, Output, PartiallySignedSharedTransaction, RequireConfirmedInputs} +import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxFunder, InteractiveTxSigningSession} import fr.acinq.eclair.channel.publish.TxPublisher.SetChannelId import fr.acinq.eclair.crypto.ShaChain import fr.acinq.eclair.io.Peer.OpenChannelResponse +import fr.acinq.eclair.transactions.Scripts import fr.acinq.eclair.wire.protocol._ import fr.acinq.eclair.{MilliSatoshiLong, RealShortChannelId, ToMilliSatoshiConversion, UInt64, randomBytes32} @@ -104,37 +106,89 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { when(WAIT_FOR_INIT_DUAL_FUNDED_CHANNEL)(handleExceptions { case Event(input: INPUT_INIT_CHANNEL_INITIATOR, _) => + // use our fundingPubKey as a placeholder for the remote funder's pubkey val fundingPubKey = keyManager.fundingPublicKey(input.localParams.fundingKeyPath, fundingTxIndex = 0).publicKey - val channelKeyPath = keyManager.keyPath(input.localParams, input.channelConfig) - val upfrontShutdownScript_opt = input.localParams.upfrontShutdownScript_opt.map(scriptPubKey => ChannelTlv.UpfrontShutdownScriptTlv(scriptPubKey)) - val tlvs: Set[OpenDualFundedChannelTlv] = Set( - upfrontShutdownScript_opt, - Some(ChannelTlv.ChannelTypeTlv(input.channelType)), - input.pushAmount_opt.map(amount => ChannelTlv.PushAmountTlv(amount)), - if (input.requireConfirmedInputs) Some(ChannelTlv.RequireConfirmedInputsTlv()) else None, - ).flatten - val open = OpenDualFundedChannel( - chainHash = nodeParams.chainHash, - temporaryChannelId = input.temporaryChannelId, - fundingFeerate = input.fundingTxFeerate, - commitmentFeerate = input.commitTxFeerate, - fundingAmount = input.fundingAmount, - dustLimit = input.localParams.dustLimit, - maxHtlcValueInFlightMsat = UInt64(input.localParams.maxHtlcValueInFlightMsat.toLong), - htlcMinimum = input.localParams.htlcMinimum, - toSelfDelay = input.localParams.toSelfDelay, - maxAcceptedHtlcs = input.localParams.maxAcceptedHtlcs, + // assume our peer requires confirmed inputs when we initiate a dual funded channel open + val requireConfirmedInputs = RequireConfirmedInputs(forLocal = true, forRemote = nodeParams.channelConf.requireConfirmedInputsForDualFunding) + val fundingParams = InteractiveTxParams( + channelId = input.temporaryChannelId, + isInitiator = true, + localContribution = input.fundingAmount, + remoteContribution = 0 sat, + sharedInput_opt = None, + remoteFundingPubKey = fundingPubKey, + localOutputs = Nil, lockTime = nodeParams.currentBlockHeight.toLong, - fundingPubkey = fundingPubKey, - revocationBasepoint = keyManager.revocationPoint(channelKeyPath).publicKey, - paymentBasepoint = input.localParams.walletStaticPaymentBasepoint.getOrElse(keyManager.paymentPoint(channelKeyPath).publicKey), - delayedPaymentBasepoint = keyManager.delayedPaymentPoint(channelKeyPath).publicKey, - htlcBasepoint = keyManager.htlcPoint(channelKeyPath).publicKey, - firstPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 0), - secondPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1), - channelFlags = input.channelFlags, - tlvStream = TlvStream(tlvs)) - goto(WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL) using DATA_WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL(input, open) sending open + dustLimit = input.localParams.dustLimit, + targetFeerate = input.fundingTxFeerate, + requireConfirmedInputs = requireConfirmedInputs + ) + val dummyPurpose = InteractiveTxBuilder.DummyFundingTx(feeBudget_opt = input.fundingTxFeeBudget_opt) + val dummyFundingPubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(fundingPubKey, fundingPubKey))) + val txFunder = context.spawnAnonymous(InteractiveTxFunder(remoteNodeId, fundingParams, dummyFundingPubkeyScript, dummyPurpose, wallet)) + txFunder ! InteractiveTxFunder.FundTransaction(self) + goto(WAIT_FOR_DUAL_FUNDING_INTERNAL) using DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL(input) + }) + + when(WAIT_FOR_DUAL_FUNDING_INTERNAL)(handleExceptions { + case Event(msg: InteractiveTxFunder.Response, d: DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL) => msg match { + case InteractiveTxFunder.FundingFailed => + d.input.replyTo ! OpenChannelResponse.Rejected(LocalFailure(ChannelFundingError(d.channelId)).cause.getMessage) + goto(CLOSED) + case fundingContributions: InteractiveTxFunder.FundingContributions => + val fundingPubKey = keyManager.fundingPublicKey(d.input.localParams.fundingKeyPath, fundingTxIndex = 0).publicKey + val channelKeyPath = keyManager.keyPath(d.input.localParams, d.input.channelConfig) + val upfrontShutdownScript_opt = d.input.localParams.upfrontShutdownScript_opt.map(scriptPubKey => ChannelTlv.UpfrontShutdownScriptTlv(scriptPubKey)) + val tlvs: Set[OpenDualFundedChannelTlv] = Set( + upfrontShutdownScript_opt, + Some(ChannelTlv.ChannelTypeTlv(d.input.channelType)), + d.input.pushAmount_opt.map(amount => ChannelTlv.PushAmountTlv(amount)), + if (d.input.requireConfirmedInputs) Some(ChannelTlv.RequireConfirmedInputsTlv()) else None, + ).flatten + val fundingAmount1 = d.input.fundingAmount + fundingContributions.addExcess_opt.getOrElse(0 sat) + val open = OpenDualFundedChannel( + chainHash = nodeParams.chainHash, + temporaryChannelId = d.input.temporaryChannelId, + fundingFeerate = d.input.fundingTxFeerate, + commitmentFeerate = d.input.commitTxFeerate, + fundingAmount = fundingAmount1, + dustLimit = d.input.localParams.dustLimit, + maxHtlcValueInFlightMsat = UInt64(d.input.localParams.maxHtlcValueInFlightMsat.toLong), + htlcMinimum = d.input.localParams.htlcMinimum, + toSelfDelay = d.input.localParams.toSelfDelay, + maxAcceptedHtlcs = d.input.localParams.maxAcceptedHtlcs, + lockTime = nodeParams.currentBlockHeight.toLong, + fundingPubkey = fundingPubKey, + revocationBasepoint = keyManager.revocationPoint(channelKeyPath).publicKey, + paymentBasepoint = d.input.localParams.walletStaticPaymentBasepoint.getOrElse(keyManager.paymentPoint(channelKeyPath).publicKey), + delayedPaymentBasepoint = keyManager.delayedPaymentPoint(channelKeyPath).publicKey, + htlcBasepoint = keyManager.htlcPoint(channelKeyPath).publicKey, + firstPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 0), + secondPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1), + channelFlags = d.input.channelFlags, + tlvStream = TlvStream(tlvs)) + goto(WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL) using DATA_WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL(d.input.copy(fundingAmount = fundingAmount1), open, fundingContributions) sending open + } + case Event(Status.Failure(t), d: DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL) => + log.error(t, s"wallet returned error: ") + d.input.replyTo ! OpenChannelResponse.Rejected(s"wallet error: ${t.getMessage}") + goto(CLOSED) + + case Event(c: CloseCommand, d: DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL) => + d.input.replyTo ! OpenChannelResponse.Cancelled + handleFastClose(c, d.channelId) + + case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL) => + d.input.replyTo ! OpenChannelResponse.RemoteError(e.toAscii) + handleRemoteError(e, d) + + case Event(INPUT_DISCONNECTED, d: DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL) => + d.input.replyTo ! OpenChannelResponse.Disconnected + goto(CLOSED) + + case Event(TickChannelOpenTimeout, d: DATA_WAIT_FOR_DUAL_FUNDING_INTERNAL) => + d.input.replyTo ! OpenChannelResponse.TimedOut + goto(CLOSED) }) when(WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL)(handleExceptions { @@ -218,6 +272,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { nodeParams, fundingParams, channelParams, purpose, localPushAmount = accept.pushAmount, remotePushAmount = open.pushAmount, + fundingContributions_opt = None, wallet)) txBuilder ! InteractiveTxBuilder.Start(self) goto(WAIT_FOR_DUAL_FUNDING_CREATED) using DATA_WAIT_FOR_DUAL_FUNDING_CREATED(channelId, channelParams, open.secondPerCommitmentPoint, accept.pushAmount, open.pushAmount, txBuilder, deferred = None, replyTo_opt = None) sending accept @@ -235,6 +290,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { import d.init.{localParams, remoteInit} Helpers.validateParamsDualFundedInitiator(nodeParams, d.init.channelType, localParams.initFeatures, remoteInit.features, d.lastSent, accept) match { case Left(t) => + rollbackOpenAttempt(d.fundingContributions) d.init.replyTo ! OpenChannelResponse.Rejected(t.getMessage) handleLocalError(t, d, Some(accept)) case Right((channelFeatures, remoteShutdownScript)) => @@ -281,6 +337,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { nodeParams, fundingParams, channelParams, purpose, localPushAmount = d.lastSent.pushAmount, remotePushAmount = accept.pushAmount, + fundingContributions_opt = Some(d.fundingContributions), wallet)) txBuilder ! InteractiveTxBuilder.Start(self) goto(WAIT_FOR_DUAL_FUNDING_CREATED) using DATA_WAIT_FOR_DUAL_FUNDING_CREATED(channelId, channelParams, accept.secondPerCommitmentPoint, d.lastSent.pushAmount, accept.pushAmount, txBuilder, deferred = None, replyTo_opt = Some(d.init.replyTo)) @@ -551,6 +608,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { channelParams = d.commitments.params, purpose = InteractiveTxBuilder.FundingTxRbf(d.commitments.active.head, previousTransactions = d.allFundingTxs.map(_.sharedTx), feeBudget_opt = None), localPushAmount = d.localPushAmount, remotePushAmount = d.remotePushAmount, + fundingContributions_opt = None, wallet)) txBuilder ! InteractiveTxBuilder.Start(self) val toSend = Seq( @@ -589,6 +647,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers { channelParams = d.commitments.params, purpose = InteractiveTxBuilder.FundingTxRbf(d.commitments.active.head, previousTransactions = d.allFundingTxs.map(_.sharedTx), feeBudget_opt = Some(cmd.fundingFeeBudget)), localPushAmount = d.localPushAmount, remotePushAmount = d.remotePushAmount, + fundingContributions_opt = None, wallet)) txBuilder ! InteractiveTxBuilder.Start(self) stay() using d.copy(status = DualFundingStatus.RbfInProgress(cmd_opt = Some(cmd), txBuilder, remoteCommitSig = None)) 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 eb26fa4df9..c56ddcba01 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 @@ -25,7 +25,7 @@ import fr.acinq.eclair.channel.LocalFundingStatus.DualFundedUnconfirmedFundingTx import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel.BITCOIN_FUNDING_DOUBLE_SPENT import fr.acinq.eclair.channel.fund.InteractiveTxBuilder._ -import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession} +import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxFunder, InteractiveTxSigningSession} import fr.acinq.eclair.wire.protocol.{ChannelReady, Error} import scala.concurrent.Future @@ -116,6 +116,13 @@ trait DualFundingHandlers extends CommonFundingHandlers { * bitcoind when transactions are published. But if we couldn't publish those transactions (e.g. because our peer * never sent us their signatures, or the transaction wasn't accepted in our mempool), their inputs may still be locked. */ + def rollbackOpenAttempt(fundingContributions: InteractiveTxFunder.FundingContributions): Unit = { + val inputs = fundingContributions.inputs.map(i => TxIn(i.outPoint, Nil, 0)) + if (inputs.nonEmpty) { + wallet.rollback(Transaction(2, inputs, Nil, 0)) + } + } + def rollbackDualFundingTxs(txs: Seq[SignedSharedTransaction]): Unit = { val inputs = txs.flatMap(sharedTx => sharedTx.tx.localInputs ++ sharedTx.tx.sharedInput_opt.toSeq).distinctBy(_.serialId).map(i => TxIn(i.outPoint, Nil, 0)) if (inputs.nonEmpty) { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala index 7b1de61349..c9cce516a2 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxBuilder.scala @@ -157,17 +157,26 @@ object InteractiveTxBuilder { } // @formatter:off - sealed trait Purpose { + sealed trait FundingInfo { def previousLocalBalance: MilliSatoshi def previousRemoteBalance: MilliSatoshi def previousFundingAmount: Satoshi + def localHtlcs: Set[DirectedHtlc] + def htlcBalance: MilliSatoshi = localHtlcs.toSeq.map(_.add.amountMsat).sum + } + sealed trait CommitmentInfo { def localCommitIndex: Long def remoteCommitIndex: Long def remotePerCommitmentPoint: PublicKey def commitTxFeerate: FeeratePerKw def fundingTxIndex: Long - def localHtlcs: Set[DirectedHtlc] - def htlcBalance: MilliSatoshi = localHtlcs.toSeq.map(_.add.amountMsat).sum + } + sealed trait Purpose extends FundingInfo with CommitmentInfo + case class DummyFundingTx(feeBudget_opt: Option[Satoshi]) extends FundingInfo { + override val previousLocalBalance: MilliSatoshi = 0 msat + override val previousRemoteBalance: MilliSatoshi = 0 msat + override val previousFundingAmount: Satoshi = 0 sat + override val localHtlcs: Set[DirectedHtlc] = Set.empty } case class FundingTx(commitTxFeerate: FeeratePerKw, remotePerCommitmentPoint: PublicKey, feeBudget_opt: Option[Satoshi]) extends Purpose { override val previousLocalBalance: MilliSatoshi = 0 msat @@ -364,6 +373,7 @@ object InteractiveTxBuilder { purpose: Purpose, localPushAmount: MilliSatoshi, remotePushAmount: MilliSatoshi, + fundingContributions_opt: Option[InteractiveTxFunder.FundingContributions], wallet: OnChainChannelFunder)(implicit ec: ExecutionContext): Behavior[Command] = { Behaviors.setup { context => // The stash is used to buffer messages that arrive while we're funding the transaction. @@ -384,7 +394,7 @@ object InteractiveTxBuilder { Behaviors.stopped } else { val actor = new InteractiveTxBuilder(replyTo, sessionId, nodeParams, channelParams, fundingParams, purpose, localPushAmount, remotePushAmount, wallet, stash, context) - actor.start() + actor.start(fundingContributions_opt) } case Abort => Behaviors.stopped } @@ -423,34 +433,42 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon case _ => Nil } - def start(): Behavior[Command] = { - val txFunder = context.spawnAnonymous(InteractiveTxFunder(remoteNodeId, fundingParams, fundingPubkeyScript, purpose, wallet)) - txFunder ! InteractiveTxFunder.FundTransaction(context.messageAdapter[InteractiveTxFunder.Response](r => FundTransactionResult(r))) - Behaviors.receiveMessagePartial { - case FundTransactionResult(result) => result match { - case InteractiveTxFunder.FundingFailed => - if (previousTransactions.nonEmpty && !fundingParams.isInitiator) { - // We don't have enough funds to reach the desired feerate, but this is an RBF attempt that we did not initiate. - // It still makes sense for us to contribute whatever we're able to (by using our previous set of inputs and - // outputs): the final feerate will be less than what the initiator intended, but it's still better than being - // stuck with a low feerate transaction that won't confirm. - log.warn("could not fund interactive tx at {}, re-using previous inputs and outputs", fundingParams.targetFeerate) - val previousTx = previousTransactions.head.tx - stash.unstashAll(buildTx(InteractiveTxFunder.FundingContributions(previousTx.localInputs, previousTx.localOutputs))) - } else { - // We use a generic exception and don't send the internal error to the peer. - replyTo ! LocalFailure(ChannelFundingError(fundingParams.channelId)) - Behaviors.stopped - } - case fundingContributions: InteractiveTxFunder.FundingContributions => - stash.unstashAll(buildTx(fundingContributions)) + def start(fundingContributions_opt: Option[InteractiveTxFunder.FundingContributions]): Behavior[Command] = { + if (fundingContributions_opt.isDefined) { + val fundingContributions1 = fundingContributions_opt.get.copy(outputs = fundingContributions_opt.get.outputs.map { + case o: InteractiveTxBuilder.Output.Shared => Output.Shared(o.serialId, fundingPubkeyScript, purpose.previousLocalBalance + fundingParams.localContribution, purpose.previousRemoteBalance + fundingParams.remoteContribution, purpose.htlcBalance) + case o => o + }) + stash.unstashAll(buildTx(fundingContributions1)) + } else { + val txFunder = context.spawnAnonymous(InteractiveTxFunder(remoteNodeId, fundingParams, fundingPubkeyScript, purpose, wallet)) + txFunder ! InteractiveTxFunder.FundTransaction(context.messageAdapter[InteractiveTxFunder.Response](r => FundTransactionResult(r))) + Behaviors.receiveMessagePartial { + case FundTransactionResult(result) => result match { + case InteractiveTxFunder.FundingFailed => + if (previousTransactions.nonEmpty && !fundingParams.isInitiator) { + // We don't have enough funds to reach the desired feerate, but this is an RBF attempt that we did not initiate. + // It still makes sense for us to contribute whatever we're able to (by using our previous set of inputs and + // outputs): the final feerate will be less than what the initiator intended, but it's still better than being + // stuck with a low feerate transaction that won't confirm. + log.warn("could not fund interactive tx at {}, re-using previous inputs and outputs", fundingParams.targetFeerate) + val previousTx = previousTransactions.head.tx + stash.unstashAll(buildTx(InteractiveTxFunder.FundingContributions(previousTx.localInputs, previousTx.localOutputs, addExcess_opt = None))) + } else { + // We use a generic exception and don't send the internal error to the peer. + replyTo ! LocalFailure(ChannelFundingError(fundingParams.channelId)) + Behaviors.stopped + } + case fundingContributions: InteractiveTxFunder.FundingContributions => + stash.unstashAll(buildTx(fundingContributions)) + } + case msg: ReceiveMessage => + stash.stash(msg) + Behaviors.same + case Abort => + stash.stash(Abort) + Behaviors.same } - case msg: ReceiveMessage => - stash.stash(msg) - Behaviors.same - case Abort => - stash.stash(Abort) - Behaviors.same } } @@ -769,6 +787,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon private def signCommitTx(completeTx: SharedTransaction): Behavior[Command] = { val fundingTx = completeTx.buildUnsignedTx() val fundingOutputIndex = fundingTx.txOut.indexWhere(_.publicKeyScript == fundingPubkeyScript) + require(fundingOutputIndex >= 0, "shared output not found in funding tx!") Funding.makeCommitTxs(keyManager, channelParams, fundingAmount = fundingParams.fundingAmount, toLocal = completeTx.sharedOutput.localAmount - localPushAmount + remotePushAmount, diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala index 26341b618b..059dfee933 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fund/InteractiveTxFunder.scala @@ -45,22 +45,22 @@ object InteractiveTxFunder { // @formatter:off sealed trait Command case class FundTransaction(replyTo: ActorRef[Response]) extends Command - private case class FundTransactionResult(tx: Transaction, changePosition: Option[Int]) extends Command + private case class FundTransactionResult(tx: Transaction, changePosition: Option[Int], addExcess_opt: Option[Satoshi] ) extends Command private case class InputDetails(usableInputs: Seq[OutgoingInput], unusableInputs: Set[UnusableInput]) extends Command private case class WalletFailure(t: Throwable) extends Command private case object UtxosUnlocked extends Command sealed trait Response - case class FundingContributions(inputs: Seq[OutgoingInput], outputs: Seq[OutgoingOutput]) extends Response + case class FundingContributions(inputs: Seq[OutgoingInput], outputs: Seq[OutgoingOutput], addExcess_opt: Option[Satoshi]) extends Response case object FundingFailed extends Response // @formatter:on - def apply(remoteNodeId: PublicKey, fundingParams: InteractiveTxParams, fundingPubkeyScript: ByteVector, purpose: InteractiveTxBuilder.Purpose, wallet: OnChainChannelFunder)(implicit ec: ExecutionContext): Behavior[Command] = { + def apply(remoteNodeId: PublicKey, fundingParams: InteractiveTxParams, fundingPubkeyScript: ByteVector, purpose: InteractiveTxBuilder.FundingInfo, wallet: OnChainChannelFunder, maxExcess_opt: Option[Satoshi] = None)(implicit ec: ExecutionContext): Behavior[Command] = { Behaviors.setup { context => Behaviors.withMdc(Logs.mdc(remoteNodeId_opt = Some(remoteNodeId), channelId_opt = Some(fundingParams.channelId))) { Behaviors.receiveMessagePartial { case FundTransaction(replyTo) => - val actor = new InteractiveTxFunder(replyTo, fundingParams, fundingPubkeyScript, purpose, wallet, context) + val actor = new InteractiveTxFunder(replyTo, fundingParams, fundingPubkeyScript, purpose, wallet, maxExcess_opt, context) actor.start() } } @@ -121,7 +121,7 @@ object InteractiveTxFunder { previousTxSizeOk && isNativeSegwit && confirmationsOk } - private def sortFundingContributions(fundingParams: InteractiveTxParams, inputs: Seq[OutgoingInput], outputs: Seq[OutgoingOutput]): FundingContributions = { + private def sortFundingContributions(fundingParams: InteractiveTxParams, inputs: Seq[OutgoingInput], outputs: Seq[OutgoingOutput], addExcess_opt: Option[Satoshi]): FundingContributions = { // We always randomize the order of inputs and outputs. val sortedInputs = Random.shuffle(inputs).zipWithIndex.map { case (input, i) => val serialId = UInt64(2 * i + fundingParams.serialIdParity) @@ -138,7 +138,7 @@ object InteractiveTxFunder { case output: Output.Shared => output.copy(serialId = serialId) } } - FundingContributions(sortedInputs, sortedOutputs) + FundingContributions(sortedInputs, sortedOutputs, addExcess_opt) } } @@ -146,8 +146,9 @@ object InteractiveTxFunder { private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response], fundingParams: InteractiveTxParams, fundingPubkeyScript: ByteVector, - purpose: InteractiveTxBuilder.Purpose, + purpose: InteractiveTxBuilder.FundingInfo, wallet: OnChainChannelFunder, + maxExcess_opt: Option[Satoshi], context: ActorContext[InteractiveTxFunder.Command])(implicit ec: ExecutionContext) { import InteractiveTxFunder._ @@ -158,6 +159,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response case rbf: InteractiveTxBuilder.SpliceTxRbf => rbf.previousTransactions case _ => Nil } + private val addExcessToRecipientPosition_opt = maxExcess_opt.map(_ => 0) def start(): Behavior[Command] = { // We always double-spend all our previous inputs. It's technically overkill because we only really need to double @@ -177,12 +179,12 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response val sharedInput = fundingParams.sharedInput_opt.toSeq.map(sharedInput => Input.Shared(UInt64(0), sharedInput.info.outPoint, sharedInput.info.txOut.publicKeyScript, 0xfffffffdL, purpose.previousLocalBalance, purpose.previousRemoteBalance, purpose.htlcBalance)) val sharedOutput = Output.Shared(UInt64(0), fundingPubkeyScript, purpose.previousLocalBalance + fundingParams.localContribution, purpose.previousRemoteBalance + fundingParams.remoteContribution, purpose.htlcBalance) val nonChangeOutputs = fundingParams.localOutputs.map(txOut => Output.Local.NonChange(UInt64(0), txOut.amount, txOut.publicKeyScript)) - val fundingContributions = sortFundingContributions(fundingParams, sharedInput ++ previousWalletInputs, sharedOutput +: nonChangeOutputs) + val fundingContributions = sortFundingContributions(fundingParams, sharedInput ++ previousWalletInputs, sharedOutput +: nonChangeOutputs, addExcess_opt = Some(0 sat)) replyTo ! fundingContributions Behaviors.stopped } else { val nonChangeOutputs = fundingParams.localOutputs.map(txOut => Output.Local.NonChange(UInt64(0), txOut.amount, txOut.publicKeyScript)) - val fundingContributions = sortFundingContributions(fundingParams, previousWalletInputs, nonChangeOutputs) + val fundingContributions = sortFundingContributions(fundingParams, previousWalletInputs, nonChangeOutputs, addExcess_opt = Some(0 sat)) replyTo ! fundingContributions Behaviors.stopped } @@ -207,17 +209,18 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response private def fund(txNotFunded: Transaction, currentInputs: Seq[OutgoingInput], unusableInputs: Set[UnusableInput]): Behavior[Command] = { val sharedInputWeight = fundingParams.sharedInput_opt.toSeq.map(i => i.info.outPoint -> i.weight.toLong).toMap val feeBudget_opt = purpose match { + case p: DummyFundingTx => p.feeBudget_opt case p: FundingTx => p.feeBudget_opt case p: FundingTxRbf => p.feeBudget_opt case p: SpliceTxRbf => p.feeBudget_opt case _ => None } - context.pipeToSelf(wallet.fundTransaction(txNotFunded, fundingParams.targetFeerate, replaceable = true, externalInputsWeight = sharedInputWeight, feeBudget_opt = feeBudget_opt, addExcessToRecipientPosition_opt = None, maxExcess_opt = None)) { + context.pipeToSelf(wallet.fundTransaction(txNotFunded, fundingParams.targetFeerate, replaceable = true, externalInputsWeight = sharedInputWeight, feeBudget_opt = feeBudget_opt, addExcessToRecipientPosition_opt = addExcessToRecipientPosition_opt, maxExcess_opt = maxExcess_opt)) { case Failure(t) => WalletFailure(t) - case Success(result) => FundTransactionResult(result.tx, result.changePosition) + case Success(result) => FundTransactionResult(result.tx, result.changePosition, addExcessToRecipientPosition_opt.map(pos => result.tx.txOut(pos).amount - txNotFunded.txOut(pos).amount)) } Behaviors.receiveMessagePartial { - case FundTransactionResult(fundedTx, changePosition) => + case FundTransactionResult(fundedTx, changePosition, addExcess_opt) => // Those inputs were already selected by bitcoind and considered unsuitable for interactive tx. val lockedUnusableInputs = fundedTx.txIn.map(_.outPoint).filter(o => unusableInputs.map(_.outpoint).contains(o)) if (lockedUnusableInputs.nonEmpty) { @@ -226,7 +229,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response log.error("could not fund interactive tx: bitcoind included already known unusable inputs that should have been locked: {}", lockedUnusableInputs.mkString(",")) sendResultAndStop(FundingFailed, currentInputs.map(_.outPoint).toSet ++ fundedTx.txIn.map(_.outPoint) ++ unusableInputs.map(_.outpoint)) } else { - filterInputs(fundedTx, changePosition, currentInputs, unusableInputs) + filterInputs(fundedTx, changePosition, currentInputs, unusableInputs, addExcess_opt) } case WalletFailure(t) => log.error("could not fund interactive tx: ", t) @@ -235,7 +238,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response } /** Not all inputs are suitable for interactive tx construction. */ - private def filterInputs(fundedTx: Transaction, changePosition: Option[Int], currentInputs: Seq[OutgoingInput], unusableInputs: Set[UnusableInput]): Behavior[Command] = { + private def filterInputs(fundedTx: Transaction, changePosition: Option[Int], currentInputs: Seq[OutgoingInput], unusableInputs: Set[UnusableInput], addExcess_opt: Option[Satoshi]): Behavior[Command] = { context.pipeToSelf(Future.sequence(fundedTx.txIn.map(txIn => getInputDetails(txIn, currentInputs)))) { case Failure(t) => WalletFailure(t) case Success(results) => InputDetails(results.collect { case Right(i) => i }, results.collect { case Left(i) => i }.toSet) @@ -256,9 +259,9 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response val fundingContributions = if (fundingParams.isInitiator) { // The initiator is responsible for adding the shared output and the shared input. val inputs = inputDetails.usableInputs - val fundingOutput = Output.Shared(UInt64(0), fundingPubkeyScript, purpose.previousLocalBalance + fundingParams.localContribution, purpose.previousRemoteBalance + fundingParams.remoteContribution, purpose.htlcBalance) + val fundingOutput = Output.Shared(UInt64(0), fundingPubkeyScript, purpose.previousLocalBalance + fundingParams.localContribution + addExcess_opt.getOrElse(0 sat), purpose.previousRemoteBalance + fundingParams.remoteContribution, purpose.htlcBalance) val outputs = Seq(fundingOutput) ++ nonChangeOutputs ++ changeOutput_opt.toSeq - sortFundingContributions(fundingParams, inputs, outputs) + sortFundingContributions(fundingParams, inputs, outputs, addExcess_opt) } else { // The non-initiator must not include the shared input or the shared output. val inputs = inputDetails.usableInputs.filterNot(_.isInstanceOf[Input.Shared]) @@ -279,7 +282,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response nonChangeOutputs :+ changeOutput.copy(amount = changeOutput.amount + overpaidFees) case None => nonChangeOutputs } - sortFundingContributions(fundingParams, inputs, outputs) + sortFundingContributions(fundingParams, inputs, outputs, addExcess_opt) } log.debug("added {} inputs and {} outputs to interactive tx", fundingContributions.inputs.length, fundingContributions.outputs.length) // We unlock the unusable inputs (if any) as they can be used outside of interactive-tx sessions. diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/io/PeerReadyNotifier.scala b/eclair-core/src/main/scala/fr/acinq/eclair/io/PeerReadyNotifier.scala index 81d6c71b5c..57d168a8ab 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/io/PeerReadyNotifier.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/io/PeerReadyNotifier.scala @@ -174,6 +174,7 @@ object PeerReadyNotifier { case channel.WAIT_FOR_INIT_INTERNAL => false case channel.WAIT_FOR_INIT_SINGLE_FUNDED_CHANNEL => false case channel.WAIT_FOR_INIT_DUAL_FUNDED_CHANNEL => false + case channel.WAIT_FOR_DUAL_FUNDING_INTERNAL => false case channel.OFFLINE => false case channel.SYNCING => false case channel.WAIT_FOR_OPEN_CHANNEL => true 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 672747aea4..8e33d2257f 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 @@ -22,6 +22,7 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.bitcoin.scalacompat.{Crypto, OutPoint, Satoshi, SatoshiLong, Script, Transaction, TxId, TxIn, TxOut} import fr.acinq.bitcoin.{Bech32, SigHash, SigVersion} import fr.acinq.eclair.TestUtils.randomTxId +import fr.acinq.eclair.blockchain.DummyOnChainWallet.SingleKeyOnChainWallet.invalidFundingAmount import fr.acinq.eclair.blockchain.OnChainWallet.{FundTransactionResponse, MakeFundingTxResponse, OnChainBalance, ProcessPsbtResponse} import fr.acinq.eclair.blockchain.bitcoind.BitcoindService.SignTransactionResponse import fr.acinq.eclair.blockchain.fee.FeeratePerKw @@ -155,6 +156,7 @@ class SingleKeyOnChainWallet extends OnChainWallet with OnchainPubkeyCache { override def fundTransaction(tx: Transaction, feeRate: FeeratePerKw, replaceable: Boolean, externalInputsWeight: Map[OutPoint, Long], feeBudget_opt: Option[Satoshi], addExcessToRecipientPosition_opt: Option[Int], maxExcess_opt: Option[Satoshi])(implicit ec: ExecutionContext): Future[FundTransactionResponse] = synchronized { val currentAmountIn = tx.txIn.flatMap(txIn => inputs.find(_.txid == txIn.outPoint.txid).flatMap(_.txOut.lift(txIn.outPoint.index.toInt))).map(_.amount).sum val amountOut = tx.txOut.map(_.amount).sum + if (amountOut >= invalidFundingAmount) return Future.failed(new RuntimeException(s"invalid funding amount")) // We add a single input to reach the desired feerate. val inputAmount = amountOut + 100_000.sat val inputTx = Transaction(2, Seq(TxIn(OutPoint(randomTxId(), 1), Nil, 0)), Seq(TxOut(inputAmount, Script.pay2wpkh(pubkey))), 0) @@ -249,6 +251,10 @@ class SingleKeyOnChainWallet extends OnChainWallet with OnchainPubkeyCache { override def getP2wpkhPubkey(renew: Boolean): PublicKey = pubkey } +class SingleKeyOnChainWalletWithConfirmedInputs extends SingleKeyOnChainWallet { + override def getTxConfirmations(txid: TxId)(implicit ec: ExecutionContext): Future[Option[Int]] = Future.successful(Some(6)) +} + object DummyOnChainWallet { val dummyReceiveAddress: String = "bcrt1qwcv8naajwn8fjhu8z59q9e6ucrqr068rlcenux" @@ -264,4 +270,8 @@ object DummyOnChainWallet { MakeFundingTxResponse(fundingTx, 0, 420 sat) } + object SingleKeyOnChainWallet { + val invalidFundingAmount: Satoshi = 2_100_000_000 sat + } + } \ No newline at end of file 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 54a8f1c29e..2de129ad0f 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 @@ -127,6 +127,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsA, fundingParams, channelParamsA, FundingTx(commitFeerate, firstPerCommitmentPointB, feeBudget_opt = None), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def spawnTxBuilderRbfAlice(fundingParams: InteractiveTxParams, commitment: Commitment, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( @@ -134,6 +135,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsA, fundingParams, channelParamsA, FundingTxRbf(commitment, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def spawnTxBuilderSpliceAlice(fundingParams: InteractiveTxParams, commitment: Commitment, wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( @@ -141,6 +143,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsA, fundingParams, channelParamsA, SpliceTx(commitment), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def spawnTxBuilderSpliceRbfAlice(fundingParams: InteractiveTxParams, parentCommitment: Commitment, latestFundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( @@ -148,6 +151,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsA, fundingParams, channelParamsA, SpliceTxRbf(parentCommitment, latestFundingTx, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def spawnTxBuilderBob(wallet: OnChainWallet, fundingParams: InteractiveTxParams = fundingParamsB): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( @@ -155,6 +159,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsB, fundingParams, channelParamsB, FundingTx(commitFeerate, firstPerCommitmentPointA, feeBudget_opt = None), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def spawnTxBuilderRbfBob(fundingParams: InteractiveTxParams, commitment: Commitment, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( @@ -162,6 +167,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsB, fundingParams, channelParamsB, FundingTxRbf(commitment, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def spawnTxBuilderSpliceBob(fundingParams: InteractiveTxParams, commitment: Commitment, wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( @@ -169,6 +175,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsB, fundingParams, channelParamsB, SpliceTx(commitment), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def spawnTxBuilderSpliceRbfBob(fundingParams: InteractiveTxParams, parentCommitment: Commitment, latestFundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx, previousTransactions: Seq[InteractiveTxBuilder.SignedSharedTransaction], wallet: OnChainWallet): ActorRef[InteractiveTxBuilder.Command] = system.spawnAnonymous(InteractiveTxBuilder( @@ -176,6 +183,7 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit nodeParamsB, fundingParams, channelParamsB, SpliceTxRbf(parentCommitment, latestFundingTx, previousTransactions, feeBudget_opt = None), 0 msat, 0 msat, + fundingContributions_opt = None, wallet)) def exchangeSigsAliceFirst(fundingParams: InteractiveTxParams, successA: InteractiveTxBuilder.Succeeded, successB: InteractiveTxBuilder.Succeeded): (FullySignedSharedTransaction, Commitment, FullySignedSharedTransaction, Commitment) = { 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 d6142fda40..76f969d39e 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 @@ -27,7 +27,7 @@ import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair._ import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ import fr.acinq.eclair.blockchain.fee.{FeeratePerKw, FeeratesPerKw} -import fr.acinq.eclair.blockchain.{DummyOnChainWallet, OnChainWallet, OnchainPubkeyCache, SingleKeyOnChainWallet} +import fr.acinq.eclair.blockchain.{DummyOnChainWallet, OnChainWallet, OnchainPubkeyCache, SingleKeyOnChainWalletWithConfirmedInputs} import fr.acinq.eclair.channel.fsm.Channel import fr.acinq.eclair.channel.publish.TxPublisher import fr.acinq.eclair.channel.publish.TxPublisher.PublishReplaceableTx @@ -159,7 +159,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually { .modify(_.channelConf.balanceThresholds).setToIf(tags.contains(ChannelStateTestsTags.AdaptMaxHtlcAmount))(Seq(Channel.BalanceThreshold(1_000 sat, 0 sat), Channel.BalanceThreshold(5_000 sat, 1_000 sat), Channel.BalanceThreshold(10_000 sat, 5_000 sat))) val wallet = wallet_opt match { case Some(wallet) => wallet - case None => if (tags.contains(ChannelStateTestsTags.DualFunding)) new SingleKeyOnChainWallet() else new DummyOnChainWallet() + case None => if (tags.contains(ChannelStateTestsTags.DualFunding)) new SingleKeyOnChainWalletWithConfirmedInputs() else new DummyOnChainWallet() } val alice: TestFSMRef[ChannelState, ChannelData, Channel] = { implicit val system: ActorSystem = systemA diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForFundingInternalDualFundedChannelStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForFundingInternalDualFundedChannelStateSpec.scala new file mode 100644 index 0000000000..30c36d470a --- /dev/null +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForFundingInternalDualFundedChannelStateSpec.scala @@ -0,0 +1,107 @@ +/* + * Copyright 2024 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.a + +import akka.actor.Status +import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps +import akka.testkit.{TestFSMRef, TestProbe} +import fr.acinq.bitcoin.scalacompat.ByteVector32 +import fr.acinq.eclair.blockchain.NoOpOnChainWallet +import fr.acinq.eclair.channel._ +import fr.acinq.eclair.channel.fsm.Channel +import fr.acinq.eclair.channel.fsm.Channel.TickChannelOpenTimeout +import fr.acinq.eclair.channel.fund.InteractiveTxFunder +import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags} +import fr.acinq.eclair.io.Peer.OpenChannelResponse +import fr.acinq.eclair.wire.protocol._ +import fr.acinq.eclair.{TestConstants, TestKitBaseClass} +import org.scalatest.Outcome +import org.scalatest.funsuite.FixtureAnyFunSuiteLike + +import scala.concurrent.duration._ + +class WaitForFundingInternalDualFundedChannelStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ChannelStateTestsBase { + + case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], aliceOpenReplyTo: TestProbe, alice2bob: TestProbe, listener: TestProbe) + + override def withFixture(test: OneArgTest): Outcome = { + val setup = init(wallet_opt = Some(new NoOpOnChainWallet()), tags = test.tags + ChannelStateTestsTags.DualFunding) + import setup._ + val channelConfig = ChannelConfig.standard + val channelFlags = ChannelFlags(announceChannel = false) + val (aliceParams, bobParams, channelType) = computeFeatures(setup, test.tags, channelFlags) + val bobInit = Init(bobParams.initFeatures) + val listener = TestProbe() + within(30 seconds) { + alice.underlying.system.eventStream.subscribe(listener.ref, classOf[ChannelAborted]) + alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, maxExcess_opt = None, dualFunded = true, TestConstants.feeratePerKw, TestConstants.feeratePerKw, fundingTxFeeBudget_opt = None, Some(TestConstants.initiatorPushAmount), requireConfirmedInputs = true, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped) + withFixture(test.toNoArgTest(FixtureParam(alice, aliceOpenReplyTo, alice2bob, listener))) + } + } + + test("recv Status.Failure (wallet error)") { f => + import f._ + awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_INTERNAL) + alice ! Status.Failure(new RuntimeException("insufficient funds")) + listener.expectMsgType[ChannelAborted] + awaitCond(alice.stateName == CLOSED) + aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected] + } + + test("recv Error") { f => + import f._ + alice ! Error(ByteVector32.Zeroes, "oops") + listener.expectMsgType[ChannelAborted] + awaitCond(alice.stateName == CLOSED) + aliceOpenReplyTo.expectMsgType[OpenChannelResponse.RemoteError] + } + + test("recv CMD_CLOSE") { f => + import f._ + val sender = TestProbe() + val c = CMD_CLOSE(sender.ref, None, None) + alice ! c + sender.expectMsg(RES_SUCCESS(c, ByteVector32.Zeroes)) + listener.expectMsgType[ChannelAborted] + awaitCond(alice.stateName == CLOSED) + aliceOpenReplyTo.expectMsg(OpenChannelResponse.Cancelled) + } + + test("recv INPUT_DISCONNECTED") { f => + import f._ + alice ! INPUT_DISCONNECTED + listener.expectMsgType[ChannelAborted] + awaitCond(alice.stateName == CLOSED) + aliceOpenReplyTo.expectMsg(OpenChannelResponse.Disconnected) + } + + test("recv TickChannelOpenTimeout") { f => + import f._ + alice ! TickChannelOpenTimeout + listener.expectMsgType[ChannelAborted] + awaitCond(alice.stateName == CLOSED) + aliceOpenReplyTo.expectMsg(OpenChannelResponse.TimedOut) + } + + test("recv funding success") { f => + import f._ + alice ! InteractiveTxFunder.FundingContributions(Seq(), Seq(), None) + alice2bob.expectMsgType[OpenDualFundedChannel] + awaitCond(alice.stateName == WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL) + } + +} 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 239dea6d4c..a69946e944 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 @@ -19,7 +19,7 @@ package fr.acinq.eclair.channel.states.b import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps import akka.testkit.{TestFSMRef, TestProbe} import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong, Script} -import fr.acinq.eclair.blockchain.SingleKeyOnChainWallet +import fr.acinq.eclair.blockchain.SingleKeyOnChainWalletWithConfirmedInputs import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel @@ -37,10 +37,10 @@ import scala.concurrent.duration.DurationInt class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ChannelStateTestsBase { - case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], aliceOpenReplyTo: TestProbe, alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, bob2blockchain: TestProbe, wallet: SingleKeyOnChainWallet, aliceListener: TestProbe, bobListener: TestProbe) + case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], aliceOpenReplyTo: TestProbe, alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, bob2blockchain: TestProbe, wallet: SingleKeyOnChainWalletWithConfirmedInputs, aliceListener: TestProbe, bobListener: TestProbe) override def withFixture(test: OneArgTest): Outcome = { - val wallet = new SingleKeyOnChainWallet() + val wallet = new SingleKeyOnChainWalletWithConfirmedInputs() val setup = init(wallet_opt = Some(wallet), tags = test.tags) import setup._ val channelConfig = ChannelConfig.standard diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingSignedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingSignedStateSpec.scala index 72b18f3a07..74a63c2d39 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingSignedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForDualFundingSignedStateSpec.scala @@ -20,7 +20,7 @@ import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps import akka.testkit.{TestFSMRef, TestProbe} import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, SatoshiLong, TxId} import fr.acinq.eclair.TestUtils.randomTxId -import fr.acinq.eclair.blockchain.SingleKeyOnChainWallet +import fr.acinq.eclair.blockchain.SingleKeyOnChainWalletWithConfirmedInputs import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchPublished} import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.channel._ @@ -39,10 +39,10 @@ import scala.concurrent.duration.DurationInt class WaitForDualFundingSignedStateSpec 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, wallet: SingleKeyOnChainWallet, aliceListener: TestProbe, bobListener: TestProbe) + case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, bob2blockchain: TestProbe, wallet: SingleKeyOnChainWalletWithConfirmedInputs, aliceListener: TestProbe, bobListener: TestProbe) override def withFixture(test: OneArgTest): Outcome = { - val wallet = new SingleKeyOnChainWallet() + val wallet = new SingleKeyOnChainWalletWithConfirmedInputs() val setup = init(wallet_opt = Some(wallet), tags = test.tags) import setup._ val channelConfig = ChannelConfig.standard 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 ca772d62aa..460736b2d3 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 @@ -21,7 +21,7 @@ import akka.testkit.{TestFSMRef, TestProbe} import com.softwaremill.quicklens.{ModifyPimp, QuicklensAt} import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong, Transaction} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ -import fr.acinq.eclair.blockchain.{CurrentBlockHeight, SingleKeyOnChainWallet} +import fr.acinq.eclair.blockchain.{CurrentBlockHeight, SingleKeyOnChainWalletWithConfirmedInputs} import fr.acinq.eclair.channel._ import fr.acinq.eclair.channel.fsm.Channel import fr.acinq.eclair.channel.fsm.Channel.ProcessCurrentBlockHeight @@ -40,10 +40,10 @@ 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) + 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: SingleKeyOnChainWalletWithConfirmedInputs) override def withFixture(test: OneArgTest): Outcome = { - val wallet = new SingleKeyOnChainWallet() + val wallet = new SingleKeyOnChainWalletWithConfirmedInputs() val setup = init(wallet_opt = Some(wallet), tags = test.tags) import setup._ diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala index c8abe3e2fe..a2c2b0ef53 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalSplicesStateSpec.scala @@ -23,6 +23,7 @@ import fr.acinq.bitcoin.ScriptFlags import fr.acinq.bitcoin.scalacompat.NumericSatoshi.abs import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, SatoshiLong, Transaction, TxIn} import fr.acinq.eclair._ +import fr.acinq.eclair.blockchain.DummyOnChainWallet.SingleKeyOnChainWallet.invalidFundingAmount import fr.acinq.eclair.blockchain.SingleKeyOnChainWallet import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ import fr.acinq.eclair.blockchain.fee.FeeratePerKw @@ -1060,6 +1061,18 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.forall(_.localCommit.spec.htlcs.size == 1)) } + test("Funding failed before a splice is requested from our peer") { f => + import f._ + val sender = TestProbe() + val cmd = CMD_SPLICE(sender.ref, spliceIn_opt = Some(SpliceIn(invalidFundingAmount, pushAmount = 0 msat)), spliceOut_opt = None) + alice ! cmd + exchangeStfu(f) + awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus.isInstanceOf[SpliceStatus.SpliceRequested]) + sender.expectMsg(RES_FAILURE(cmd, ChannelFundingError(channelId(alice)))) + awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus == SpliceStatus.NoSplice) + alice2bob.expectNoMessage(100 millis) + } + test("recv CMD_ADD_HTLC while a splice is requested") { f => import f._ val sender = TestProbe() diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala index 27a4cd58de..ce2636ce23 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/basic/fixtures/MinimalNodeFixture.scala @@ -10,7 +10,7 @@ import com.typesafe.config.ConfigFactory import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, Satoshi, SatoshiLong, Transaction, TxId} import fr.acinq.eclair.ShortChannelId.txIndex -import fr.acinq.eclair.blockchain.SingleKeyOnChainWallet +import fr.acinq.eclair.blockchain.SingleKeyOnChainWalletWithConfirmedInputs import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchFundingConfirmedTriggered, WatchFundingDeeplyBuried, WatchFundingDeeplyBuriedTriggered} import fr.acinq.eclair.blockchain.fee.{FeeratePerKw, FeeratesPerKw} @@ -56,7 +56,7 @@ case class MinimalNodeFixture private(nodeParams: NodeParams, offerManager: typed.ActorRef[OfferManager.Command], postman: typed.ActorRef[Postman.Command], watcher: TestProbe, - wallet: SingleKeyOnChainWallet, + wallet: SingleKeyOnChainWalletWithConfirmedInputs, bitcoinClient: TestBitcoinCoreClient) { val nodeId = nodeParams.nodeId val routeParams = nodeParams.routerConf.pathFindingExperimentConf.experiments.values.head.getDefaultRouteParams @@ -88,7 +88,7 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat val readyListener = TestProbe("ready-listener") system.eventStream.subscribe(readyListener.ref, classOf[SubscriptionsComplete]) val bitcoinClient = new TestBitcoinCoreClient() - val wallet = new SingleKeyOnChainWallet() + val wallet = new SingleKeyOnChainWalletWithConfirmedInputs() val watcher = TestProbe("watcher") val triggerer = TestProbe("payment-triggerer") val watcherTyped = watcher.ref.toTyped[ZmqWatcher.Command]