Skip to content

Commit

Permalink
Update to select funding inputs before sending open_channel2 and spli…
Browse files Browse the repository at this point in the history
…ce_init

 - If a changeless set of inputs is found, excess funding (if any) will be added to the proposed `fundingAmount`/`fundingContribution` before sending `open_channel2`/`splice_init` respectively.
 - InteractiveTxFunder sets `excess_opt` with the input value in excess of what is needed to achieve the requested target feerate.
 - InteractiveTxFunder assumes all inputs are p2wpkh to compute their weight for the purpose of computing the feerate of the funding transaction.
 - 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.
  • Loading branch information
remyers committed Oct 22, 2024
1 parent 96d0c9a commit fc86a9f
Show file tree
Hide file tree
Showing 16 changed files with 466 additions and 116 deletions.
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 @@ -524,7 +525,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. */
Expand Down Expand Up @@ -601,10 +602,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,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -950,7 +950,26 @@ 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
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 = Transactions.PlaceHolderPubKey,
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(Transactions.PlaceHolderPubKey, Transactions.PlaceHolderPubKey)))
val txFunder = context.spawnAnonymous(InteractiveTxFunder(remoteNodeId, fundingParams, dummyFundingPubkeyScript, purpose = InteractiveTxBuilder.SpliceTx(parentCommitment, d.commitments.changes), wallet))
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) =>
Expand All @@ -977,6 +996,28 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending Warning(d.channelId, InvalidSpliceNotQuiescent(d.channelId).getMessage)
}

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.excess_opt.getOrElse(0 sat))
stay() using d.copy(spliceStatus = SpliceStatus.SpliceRequested(cmd, spliceInit1, Some(fundingContributions))) sending spliceInit1
}
case _ =>
msg match {
case InteractiveTxFunder.FundingFailed =>
log.warning("received unexpected response from txFunder: {}, current splice status is {}.", msg, d.spliceStatus)
case fundingContributions: InteractiveTxFunder.FundingContributions =>
log.warning("received unexpected response from txFunder: {}, current splice status is {}. Rolling back funding contributions.", msg, d.spliceStatus)
rollbackOpenAttempt(fundingContributions)
}
stay()
}

case Event(_: QuiescenceTimeout, d: DATA_NORMAL) => handleQuiescenceTimeout(d)

case Event(msg: SpliceInit, d: DATA_NORMAL) =>
Expand Down Expand Up @@ -1034,6 +1075,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
purpose = InteractiveTxBuilder.SpliceTx(parentCommitment, d.commitments.changes),
localPushAmount = spliceAck.pushAmount, remotePushAmount = msg.pushAmount,
liquidityPurchase_opt = willFund_opt.map(_.purchase),
None, // TODO: select funding inputs and add excess to liquidity purchase before sending spliceAck
wallet
))
txBuilder ! InteractiveTxBuilder.Start(self)
Expand All @@ -1053,7 +1095,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(
Expand Down Expand Up @@ -1084,6 +1126,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
purpose = InteractiveTxBuilder.SpliceTx(parentCommitment, d.commitments.changes),
localPushAmount = cmd.pushAmount, remotePushAmount = msg.pushAmount,
liquidityPurchase_opt = liquidityPurchase_opt,
fundingContributions_opt = fundingContributions_opt,
wallet
))
txBuilder ! InteractiveTxBuilder.Start(self)
Expand Down Expand Up @@ -1152,6 +1195,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
purpose = rbf,
localPushAmount = 0 msat, remotePushAmount = 0 msat,
willFund_opt.map(_.purchase),
None,
wallet
))
txBuilder ! InteractiveTxBuilder.Start(self)
Expand Down Expand Up @@ -1205,6 +1249,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
purpose = rbf,
localPushAmount = 0 msat, remotePushAmount = 0 msat,
liquidityPurchase_opt = liquidityPurchase_opt,
None,
wallet
))
txBuilder ! InteractiveTxBuilder.Start(self)
Expand All @@ -1230,7 +1275,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)
Expand Down Expand Up @@ -3032,13 +3077,19 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
}

private def handleQuiescenceTimeout(d: DATA_NORMAL): State = {
if (d.spliceStatus == SpliceStatus.NoSplice) {
log.warning("quiescence timed out with no ongoing splice, did we forget to cancel the timer?")
stay()
} else {
log.warning("quiescence timed out in state {}, closing connection", d.spliceStatus.getClass.getSimpleName)
context.system.scheduler.scheduleOnce(2 second, peer, Peer.Disconnect(remoteNodeId))
stay() sending Warning(d.channelId, SpliceAttemptTimedOut(d.channelId).getMessage)
d.spliceStatus match {
case SpliceStatus.NoSplice =>
log.warning("quiescence timed out with no ongoing splice, did we forget to cancel the timer?")
stay()
case spliceRequested: SpliceStatus.SpliceRequested if spliceRequested.fundingContributions_opt.isDefined =>
log.warning("quiescence timed out after sending splice request, rolling back funding contributions and closing connection")
rollbackOpenAttempt(spliceRequested.fundingContributions_opt.get)
context.system.scheduler.scheduleOnce(2 second, peer, Peer.Disconnect(remoteNodeId))
stay() sending Warning(d.channelId, SpliceAttemptTimedOut(d.channelId).getMessage)
case _ =>
log.warning("quiescence timed out in state {}, closing connection", d.spliceStatus.getClass.getSimpleName)
context.system.scheduler.scheduleOnce(2 second, peer, Peer.Disconnect(remoteNodeId))
stay() sending Warning(d.channelId, SpliceAttemptTimedOut(d.channelId).getMessage)
}
}

Expand All @@ -3054,7 +3105,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
Expand Down
Loading

0 comments on commit fc86a9f

Please sign in to comment.