Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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._
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -521,9 +522,11 @@ 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
/** Our peer initiated a spice */
case class SpliceInitiated(init: SpliceInit, willFund_opt: Option[LiquidityAds.WillFundPurchase]) extends SpliceStatus
/** We both agreed to splice/rbf and are building the corresponding transaction. */
case class SpliceInProgress(cmd_opt: Option[ChannelFundingCommand], sessionId: ByteVector32, splice: typed.ActorRef[InteractiveTxBuilder.Command], remoteCommitSig: Option[CommitSig]) extends SpliceStatus
/** The splice transaction has been negotiated, we're exchanging signatures. */
Expand Down Expand Up @@ -598,10 +601,14 @@ final case class DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments: Commitments,
}
final case class DATA_WAIT_FOR_CHANNEL_READY(commitments: Commitments, aliases: ShortIdAliases) 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,
Expand Down
161 changes: 126 additions & 35 deletions eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -16,17 +16,19 @@

package fr.acinq.eclair.channel.fsm

import akka.actor.Status
import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapter}
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.Helpers.Funding
import fr.acinq.eclair.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.{LiquidityPurchaseSigned, OpenChannelResponse}
import fr.acinq.eclair.transactions.{Scripts, Transactions}
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{ToMilliSatoshiConversion, UInt64, randomBytes32}

Expand Down Expand Up @@ -104,38 +106,88 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {

when(WAIT_FOR_INIT_DUAL_FUNDED_CHANNEL)(handleExceptions {
case Event(input: INPUT_INIT_CHANNEL_INITIATOR, _) =>
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)),
if (input.requireConfirmedInputs) Some(ChannelTlv.RequireConfirmedInputsTlv()) else None,
input.requestFunding_opt.map(ChannelTlv.RequestFundingTlv),
input.pushAmount_opt.map(amount => ChannelTlv.PushAmountTlv(amount)),
).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 = Transactions.PlaceHolderPubKey,
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(Transactions.PlaceHolderPubKey, Transactions.PlaceHolderPubKey)))
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)),
if (d.input.requireConfirmedInputs) Some(ChannelTlv.RequireConfirmedInputsTlv()) else None,
d.input.requestFunding_opt.map(ChannelTlv.RequestFundingTlv),
d.input.pushAmount_opt.map(amount => ChannelTlv.PushAmountTlv(amount)),
).flatten
val fundingAmount1 = d.input.fundingAmount + fundingContributions.excess_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 {
Expand Down Expand Up @@ -219,6 +271,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
channelParams, purpose,
localPushAmount = accept.pushAmount, remotePushAmount = open.pushAmount,
willFund_opt.map(_.purchase),
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
Expand All @@ -236,6 +289,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
import d.init.{localParams, remoteInit}
Helpers.validateParamsDualFundedInitiator(nodeParams, remoteNodeId, 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, liquidityPurchase_opt)) =>
Expand Down Expand Up @@ -283,6 +337,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
channelParams, purpose,
localPushAmount = d.lastSent.pushAmount, remotePushAmount = accept.pushAmount,
liquidityPurchase_opt = liquidityPurchase_opt,
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))
Expand Down Expand Up @@ -576,6 +631,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
purpose = InteractiveTxBuilder.FundingTxRbf(d.commitments.active.head, previousTransactions = d.allFundingTxs.map(_.sharedTx), feeBudget_opt = None),
localPushAmount = d.localPushAmount, remotePushAmount = d.remotePushAmount,
liquidityPurchase_opt = willFund_opt.map(_.purchase),
fundingContributions_opt = None,
wallet))
txBuilder ! InteractiveTxBuilder.Start(self)
val toSend = Seq(
Expand Down Expand Up @@ -623,6 +679,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
purpose = InteractiveTxBuilder.FundingTxRbf(d.commitments.active.head, previousTransactions = d.allFundingTxs.map(_.sharedTx), feeBudget_opt = Some(cmd.fundingFeeBudget)),
localPushAmount = d.localPushAmount, remotePushAmount = d.remotePushAmount,
liquidityPurchase_opt = liquidityPurchase_opt,
fundingContributions_opt = None,
wallet))
txBuilder ! InteractiveTxBuilder.Start(self)
stay() using d.copy(status = DualFundingStatus.RbfInProgress(cmd_opt = Some(cmd), txBuilder, remoteCommitSig = None))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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) {
Expand Down
Loading