mirror of
https://github.com/ACINQ/eclair.git
synced 2025-03-13 11:35:47 +01:00
Dual funding RBF support (#2275)
Add support for bumping the fees of a dual funding transaction. We spawn a transient dedicated actor: if the RBF attempt fails, or if we are disconnected before completing the protocol, we should forget it. Add more tests for scenarios where an unconfirmed channel is force-closed, where the funding transaction that confirms may not be the last one.
This commit is contained in:
parent
ad19a665a1
commit
a735ba86b6
20 changed files with 654 additions and 63 deletions
|
@ -73,6 +73,27 @@ $ ./eclair-cli open --nodeId=03864e... --fundingSatoshis=100000 --channelType=an
|
|||
$ ./eclair-cli open --nodeId=03864e... --fundingSatoshis=100000 --channelType=anchor_outputs_zero_fee_htlc_tx+scid_alias+zeroconf --announceChannel=false
|
||||
```
|
||||
|
||||
### Experimental support for dual-funding
|
||||
|
||||
This release adds experimental support for dual-funded channels, as specified [here](https://github.com/lightning/bolts/pull/851).
|
||||
Dual-funded channels have many benefits:
|
||||
|
||||
- both peers can contribute to channel funding
|
||||
- the funding transaction can be RBF-ed
|
||||
|
||||
This feature is turned off by default, because there may still be breaking changes in the specification.
|
||||
To turn it on, simply enable the feature in your `eclair.conf`:
|
||||
|
||||
```conf
|
||||
eclair.features.option_dual_fund = optional
|
||||
```
|
||||
|
||||
If your peer also supports the feature, eclair will automatically use dual-funding when opening a channel.
|
||||
If the channel doesn't confirm, you can use the `rbfopen` RPC to initiate an RBF attempt and speed up confirmation.
|
||||
|
||||
In this first version, the non-initiator cannot yet contribute funds to the channel.
|
||||
This will be added in future updates.
|
||||
|
||||
### Changes to features override
|
||||
|
||||
Eclair supports overriding features on a per-peer basis, using the `eclair.override-init-features` field in `eclair.conf`.
|
||||
|
@ -91,9 +112,10 @@ upgrading to this release.
|
|||
|
||||
### API changes
|
||||
|
||||
- `channelbalances`: retrieves information about the balances of all local channels (#2196)
|
||||
- `stop`: stops eclair. Please note that the recommended way of stopping eclair is simply to kill its process (#2233)
|
||||
- `channelbalances` retrieves information about the balances of all local channels (#2196)
|
||||
- `channelbalances` and `usablebalances` return a `shortIds` object instead of a single `shortChannelId` (#2323)
|
||||
- `stop` stops eclair: please note that the recommended way of stopping eclair is simply to kill its process (#2233)
|
||||
- `rbfopen` lets the initiator of a dual-funded channel RBF the funding transaction (#2275)
|
||||
|
||||
### Miscellaneous improvements and bug fixes
|
||||
|
||||
|
|
|
@ -38,6 +38,7 @@ and COMMAND is one of the available commands:
|
|||
|
||||
=== Channel ===
|
||||
- open
|
||||
- rbfopen
|
||||
- close
|
||||
- forceclose
|
||||
- channel
|
||||
|
|
|
@ -88,6 +88,8 @@ trait Eclair {
|
|||
|
||||
def open(nodeId: PublicKey, fundingAmount: Satoshi, pushAmount_opt: Option[MilliSatoshi], channelType_opt: Option[SupportedChannelType], fundingFeeratePerByte_opt: Option[FeeratePerByte], announceChannel_opt: Option[Boolean], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[ChannelOpenResponse]
|
||||
|
||||
def rbfOpen(channelId: ByteVector32, targetFeerate: FeeratePerKw, lockTime_opt: Option[Long])(implicit timeout: Timeout): Future[CommandResponse[CMD_BUMP_FUNDING_FEE]]
|
||||
|
||||
def close(channels: List[ApiTypes.ChannelIdentifier], scriptPubKey_opt: Option[ByteVector], closingFeerates_opt: Option[ClosingFeerates])(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_CLOSE]]]]
|
||||
|
||||
def forceClose(channels: List[ApiTypes.ChannelIdentifier])(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_FORCECLOSE]]]]
|
||||
|
@ -196,6 +198,11 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
|
|||
} yield res
|
||||
}
|
||||
|
||||
override def rbfOpen(channelId: ByteVector32, targetFeerate: FeeratePerKw, lockTime_opt: Option[Long])(implicit timeout: Timeout): Future[CommandResponse[CMD_BUMP_FUNDING_FEE]] = {
|
||||
val cmd = CMD_BUMP_FUNDING_FEE(ActorRef.noSender, targetFeerate, lockTime_opt.getOrElse(appKit.nodeParams.currentBlockHeight.toLong))
|
||||
sendToChannel(Left(channelId), cmd)
|
||||
}
|
||||
|
||||
override def close(channels: List[ApiTypes.ChannelIdentifier], scriptPubKey_opt: Option[ByteVector], closingFeerates_opt: Option[ClosingFeerates])(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_CLOSE]]]] = {
|
||||
sendToChannels(channels, CMD_CLOSE(ActorRef.noSender, scriptPubKey_opt, closingFeerates_opt))
|
||||
}
|
||||
|
|
|
@ -194,6 +194,8 @@ final case class ClosingFeerates(preferred: FeeratePerKw, min: FeeratePerKw, max
|
|||
sealed trait CloseCommand extends HasReplyToCommand
|
||||
final case class CMD_CLOSE(replyTo: ActorRef, scriptPubKey: Option[ByteVector], feerates: Option[ClosingFeerates]) extends CloseCommand
|
||||
final case class CMD_FORCECLOSE(replyTo: ActorRef) extends CloseCommand
|
||||
|
||||
final case class CMD_BUMP_FUNDING_FEE(replyTo: ActorRef, targetFeerate: FeeratePerKw, lockTime: Long) extends HasReplyToCommand
|
||||
final case class CMD_UPDATE_RELAY_FEE(replyTo: ActorRef, feeBase: MilliSatoshi, feeProportionalMillionths: Long, cltvExpiryDelta_opt: Option[CltvExpiryDelta]) extends HasReplyToCommand
|
||||
final case class CMD_GET_CHANNEL_STATE(replyTo: ActorRef) extends HasReplyToCommand
|
||||
final case class CMD_GET_CHANNEL_DATA(replyTo: ActorRef) extends HasReplyToCommand
|
||||
|
@ -413,6 +415,13 @@ case class DualFundedUnconfirmedFundingTx(sharedTx: SignedSharedTransaction) ext
|
|||
/** Once a dual funding tx has been signed, we must remember the associated commitments. */
|
||||
case class DualFundingTx(fundingTx: SignedSharedTransaction, commitments: Commitments)
|
||||
|
||||
sealed trait RbfStatus
|
||||
object RbfStatus {
|
||||
case object NoRbf extends RbfStatus
|
||||
case class RbfRequested(cmd: CMD_BUMP_FUNDING_FEE) extends RbfStatus
|
||||
case class RbfInProgress(rbf: typed.ActorRef[InteractiveTxBuilder.Command]) extends RbfStatus
|
||||
}
|
||||
|
||||
sealed trait ChannelData extends PossiblyHarmful {
|
||||
def channelId: ByteVector32
|
||||
}
|
||||
|
@ -495,7 +504,7 @@ final case class DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments: Commitments,
|
|||
previousFundingTxs: List[DualFundingTx],
|
||||
waitingSince: BlockHeight, // how long have we been waiting for a funding tx to confirm
|
||||
lastChecked: BlockHeight, // last time we checked if the channel was double-spent
|
||||
rbfAttempt: Option[typed.ActorRef[InteractiveTxBuilder.Command]],
|
||||
rbfStatus: RbfStatus,
|
||||
deferred: Option[ChannelReady]) extends PersistentChannelData
|
||||
final case class DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments: Commitments,
|
||||
shortIds: ShortIds,
|
||||
|
|
|
@ -69,6 +69,7 @@ case class InvalidFundingSignature (override val channelId: Byte
|
|||
case class InvalidRbfFeerate (override val channelId: ByteVector32, proposed: FeeratePerKw, expected: FeeratePerKw) extends ChannelException(channelId, s"invalid rbf attempt: the feerate must be at least $expected, you proposed $proposed")
|
||||
case class InvalidRbfAlreadyInProgress (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid rbf attempt: the current rbf attempt must be completed or aborted first")
|
||||
case class InvalidRbfTxConfirmed (override val channelId: ByteVector32) extends ChannelException(channelId, "no need to rbf, transaction is already confirmed")
|
||||
case class InvalidRbfNonInitiator (override val channelId: ByteVector32) extends ChannelException(channelId, "cannot initiate rbf: we're not the initiator of this interactive-tx attempt")
|
||||
case class InvalidRbfAttempt (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid rbf attempt")
|
||||
case class NoMoreHtlcsClosingInProgress (override val channelId: ByteVector32) extends ChannelException(channelId, "cannot send new htlcs, closing in progress")
|
||||
case class NoMoreFeeUpdateClosingInProgress (override val channelId: ByteVector32) extends ChannelException(channelId, "cannot send new update_fee, closing in progress")
|
||||
|
|
|
@ -199,6 +199,8 @@ case class Commitments(channelId: ByteVector32,
|
|||
commitTx
|
||||
}
|
||||
|
||||
val fundingTxId: ByteVector32 = commitInput.outPoint.txid
|
||||
|
||||
val commitmentFormat: CommitmentFormat = channelFeatures.commitmentFormat
|
||||
|
||||
val channelType: SupportedChannelType = channelFeatures.channelType
|
||||
|
|
|
@ -106,6 +106,8 @@ object InteractiveTxBuilder {
|
|||
dustLimit: Satoshi,
|
||||
targetFeerate: FeeratePerKw) {
|
||||
val fundingAmount: Satoshi = localAmount + remoteAmount
|
||||
// BOLT 2: MUST set `feerate` greater than or equal to 25/24 times the `feerate` of the previously constructed transaction, rounded down.
|
||||
val minNextFeerate: FeeratePerKw = targetFeerate * 25 / 24
|
||||
}
|
||||
|
||||
case class InteractiveTxSession(toSend: Seq[Either[TxAddInput, TxAddOutput]],
|
||||
|
|
|
@ -263,8 +263,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
|
|||
if (closing.alternativeCommitments.nonEmpty) {
|
||||
// There are unconfirmed, alternative funding transactions, so we wait for one to confirm before
|
||||
// watching transactions spending it.
|
||||
blockchain ! WatchFundingConfirmed(self, data.commitments.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks)
|
||||
closing.alternativeCommitments.foreach(c => blockchain ! WatchFundingConfirmed(self, c.commitments.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks))
|
||||
blockchain ! WatchFundingConfirmed(self, data.commitments.fundingTxId, nodeParams.channelConf.minDepthBlocks)
|
||||
closing.alternativeCommitments.foreach(c => blockchain ! WatchFundingConfirmed(self, c.commitments.fundingTxId, nodeParams.channelConf.minDepthBlocks))
|
||||
}
|
||||
closing.mutualClosePublished.foreach(mcp => doPublish(mcp, isInitiator))
|
||||
closing.localCommitPublished.foreach(lcp => doPublish(lcp, closing.commitments))
|
||||
|
@ -278,7 +278,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
|
|||
if (closing.commitments.channelFeatures.hasFeature(Features.DualFunding)) {
|
||||
closing.fundingTx.flatMap(_.signedTx_opt).foreach(tx => wallet.publishTransaction(tx))
|
||||
} else {
|
||||
blockchain ! GetTxWithMeta(self, closing.commitments.commitInput.outPoint.txid)
|
||||
blockchain ! GetTxWithMeta(self, closing.commitments.fundingTxId)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -312,7 +312,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
|
|||
case funding: DATA_WAIT_FOR_FUNDING_CONFIRMED =>
|
||||
watchFundingTx(funding.commitments)
|
||||
// we make sure that the funding tx has been published
|
||||
blockchain ! GetTxWithMeta(self, funding.commitments.commitInput.outPoint.txid)
|
||||
blockchain ! GetTxWithMeta(self, funding.commitments.fundingTxId)
|
||||
if (funding.waitingSince.toLong > 1_500_000_000) {
|
||||
// we were using timestamps instead of block heights when the channel was created: we reset it *and* we use block heights
|
||||
goto(OFFLINE) using funding.copy(waitingSince = nodeParams.currentBlockHeight) storing()
|
||||
|
@ -324,8 +324,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
|
|||
// we make sure that the funding tx with the highest feerate has been published
|
||||
publishFundingTx(funding.fundingParams, funding.fundingTx)
|
||||
// we watch confirmation of all funding candidates, and once one of them confirms we will watch spending txs
|
||||
blockchain ! WatchFundingConfirmed(self, funding.commitments.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks)
|
||||
funding.previousFundingTxs.map(_.commitments).foreach(c => blockchain ! WatchFundingConfirmed(self, c.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks))
|
||||
blockchain ! WatchFundingConfirmed(self, funding.commitments.fundingTxId, nodeParams.channelConf.minDepthBlocks)
|
||||
funding.previousFundingTxs.map(_.commitments).foreach(c => blockchain ! WatchFundingConfirmed(self, c.fundingTxId, nodeParams.channelConf.minDepthBlocks))
|
||||
goto(OFFLINE) using funding
|
||||
|
||||
case _ =>
|
||||
|
@ -1082,7 +1082,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
|
|||
case Left(cause) => handleCommandError(cause, c)
|
||||
}
|
||||
|
||||
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_CLOSING) if getTxResponse.txid == d.commitments.commitInput.outPoint.txid =>
|
||||
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_CLOSING) if getTxResponse.txid == d.commitments.fundingTxId =>
|
||||
// NB: waitingSinceBlock contains the block at which closing was initiated, not the block at which funding was initiated.
|
||||
// That means we're lenient with our peer and give its funding tx more time to confirm, to avoid having to store two distinct
|
||||
// waitingSinceBlock (e.g. closingWaitingSinceBlock and fundingWaitingSinceBlock).
|
||||
|
@ -1093,7 +1093,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
|
|||
case Event(BITCOIN_FUNDING_TIMEOUT, d: DATA_CLOSING) => handleFundingTimeout(d)
|
||||
|
||||
case Event(w: WatchFundingConfirmedTriggered, d: DATA_CLOSING) =>
|
||||
d.alternativeCommitments.find(_.commitments.commitInput.outPoint.txid == w.tx.txid) match {
|
||||
d.alternativeCommitments.find(_.commitments.fundingTxId == w.tx.txid) match {
|
||||
case Some(DualFundingTx(_, commitments1)) =>
|
||||
// This is a corner case where:
|
||||
// - we are using dual funding
|
||||
|
@ -1112,14 +1112,14 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
|
|||
context.system.eventStream.publish(TransactionConfirmed(d.channelId, remoteNodeId, w.tx))
|
||||
val otherFundingTxs =
|
||||
d.fundingTx.toSeq.collect { case DualFundedUnconfirmedFundingTx(fundingTx) => fundingTx } ++
|
||||
d.alternativeCommitments.filterNot(_.commitments.commitInput.outPoint.txid == w.tx.txid).map(_.fundingTx)
|
||||
d.alternativeCommitments.filterNot(_.commitments.fundingTxId == w.tx.txid).map(_.fundingTx)
|
||||
rollbackDualFundingTxs(otherFundingTxs)
|
||||
val commitTx = commitments1.fullySignedLocalCommitTx(keyManager).tx
|
||||
val localCommitPublished = Closing.LocalClose.claimCommitTxOutputs(keyManager, commitments1, commitTx, nodeParams.currentBlockHeight, nodeParams.onChainFeeConf)
|
||||
val d1 = DATA_CLOSING(commitments1, None, d.waitingSince, alternativeCommitments = Nil, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished))
|
||||
stay() using d1 storing() calling doPublish(localCommitPublished, commitments1)
|
||||
case None =>
|
||||
if (d.commitments.commitInput.outPoint.txid == w.tx.txid) {
|
||||
if (d.commitments.fundingTxId == w.tx.txid) {
|
||||
// The best funding tx candidate has been confirmed, we can forget alternative commitments.
|
||||
stay() using d.copy(alternativeCommitments = Nil) storing()
|
||||
} else {
|
||||
|
@ -1327,7 +1327,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
|
|||
|
||||
case Event(c: CMD_UPDATE_RELAY_FEE, d: DATA_NORMAL) => handleUpdateRelayFeeDisconnected(c, d)
|
||||
|
||||
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.commitInput.outPoint.txid => handleGetFundingTx(getTxResponse, d.waitingSince, d.fundingTx)
|
||||
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.fundingTxId => handleGetFundingTx(getTxResponse, d.waitingSince, d.fundingTx)
|
||||
|
||||
case Event(BITCOIN_FUNDING_PUBLISH_FAILED, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) => handleFundingPublishFailed(d)
|
||||
|
||||
|
@ -1371,7 +1371,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
|
|||
defaultMinDepth.toLong
|
||||
}
|
||||
// we put back the watch (operation is idempotent) because the event may have been fired while we were in OFFLINE
|
||||
blockchain ! WatchFundingConfirmed(self, d.commitments.commitInput.outPoint.txid, minDepth)
|
||||
blockchain ! WatchFundingConfirmed(self, d.commitments.fundingTxId, minDepth)
|
||||
goto(WAIT_FOR_FUNDING_CONFIRMED)
|
||||
|
||||
case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
|
@ -1383,7 +1383,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
|
|||
log.warning("min_depth should be defined since we're waiting for the funding tx to confirm, using default minDepth={}", defaultMinDepth)
|
||||
defaultMinDepth.toLong
|
||||
}
|
||||
(d.commitments +: d.previousFundingTxs.map(_.commitments)).foreach(commitments => blockchain ! WatchFundingConfirmed(self, commitments.commitInput.outPoint.txid, minDepth))
|
||||
(d.commitments +: d.previousFundingTxs.map(_.commitments)).foreach(commitments => blockchain ! WatchFundingConfirmed(self, commitments.fundingTxId, minDepth))
|
||||
goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) sending d.fundingTx.localSigs
|
||||
|
||||
case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_CHANNEL_READY) =>
|
||||
|
@ -1449,7 +1449,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
|
|||
case _ =>
|
||||
// even if we were just disconnected/reconnected, we need to put back the watch because the event may have been
|
||||
// fired while we were in OFFLINE (if not, the operation is idempotent anyway)
|
||||
blockchain ! WatchFundingDeeplyBuried(self, d.commitments.commitInput.outPoint.txid, ANNOUNCEMENTS_MINCONF)
|
||||
blockchain ! WatchFundingDeeplyBuried(self, d.commitments.fundingTxId, ANNOUNCEMENTS_MINCONF)
|
||||
}
|
||||
|
||||
if (d.commitments.announceChannel) {
|
||||
|
@ -1529,7 +1529,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val
|
|||
|
||||
case Event(c: CurrentFeerates, d: PersistentChannelData) => handleCurrentFeerateDisconnected(c, d)
|
||||
|
||||
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.commitInput.outPoint.txid => handleGetFundingTx(getTxResponse, d.waitingSince, d.fundingTx)
|
||||
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.fundingTxId => handleGetFundingTx(getTxResponse, d.waitingSince, d.fundingTx)
|
||||
|
||||
case Event(BITCOIN_FUNDING_PUBLISH_FAILED, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) => handleFundingPublishFailed(d)
|
||||
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package fr.acinq.eclair.channel.fsm
|
||||
|
||||
import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapter}
|
||||
import akka.actor.{ActorRef, Status}
|
||||
import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Script}
|
||||
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
|
||||
import fr.acinq.eclair.channel.Helpers.Funding
|
||||
|
@ -305,8 +306,8 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
d.deferred.foreach(self ! _)
|
||||
Funding.minDepthDualFunding(nodeParams.channelConf, commitments.channelFeatures, fundingParams) match {
|
||||
case Some(fundingMinDepth) =>
|
||||
blockchain ! WatchFundingConfirmed(self, commitments.commitInput.outPoint.txid, fundingMinDepth)
|
||||
val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, fundingTx, fundingParams, Nil, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, None, None)
|
||||
blockchain ! WatchFundingConfirmed(self, commitments.fundingTxId, fundingMinDepth)
|
||||
val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, fundingTx, fundingParams, Nil, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, RbfStatus.NoRbf, None)
|
||||
fundingTx match {
|
||||
case fundingTx: PartiallySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d1 storing() sending fundingTx.localSigs
|
||||
case fundingTx: FullySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d1 storing() sending fundingTx.localSigs calling publishFundingTx(fundingParams, fundingTx)
|
||||
|
@ -362,28 +363,160 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
stay() using d1 storing() calling publishFundingTx(d.fundingParams, fundingTx)
|
||||
}
|
||||
case _: FullySignedSharedTransaction =>
|
||||
log.warning("received duplicate tx_signatures")
|
||||
stay()
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.RbfInProgress(txBuilder) =>
|
||||
txBuilder ! InteractiveTxBuilder.ReceiveTxSigs(txSigs)
|
||||
stay()
|
||||
case _ =>
|
||||
// Signatures are retransmitted on reconnection, but we may have already received them.
|
||||
log.info("ignoring duplicate tx_signatures for txid={}", txSigs.txId)
|
||||
stay()
|
||||
}
|
||||
}
|
||||
|
||||
case Event(_: TxInitRbf, _: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
log.info("rbf not supported yet")
|
||||
stay()
|
||||
case Event(cmd: CMD_BUMP_FUNDING_FEE, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
val replyTo = if (cmd.replyTo == ActorRef.noSender) sender() else cmd.replyTo
|
||||
if (!d.fundingParams.isInitiator) {
|
||||
replyTo ! Status.Failure(InvalidRbfNonInitiator(d.channelId))
|
||||
stay()
|
||||
} else {
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.NoRbf => val minNextFeerate = d.fundingParams.minNextFeerate
|
||||
if (cmd.targetFeerate < minNextFeerate) {
|
||||
replyTo ! Status.Failure(InvalidRbfFeerate(d.channelId, cmd.targetFeerate, minNextFeerate))
|
||||
stay()
|
||||
} else {
|
||||
stay() using d.copy(rbfStatus = RbfStatus.RbfRequested(cmd.copy(replyTo = replyTo))) sending TxInitRbf(d.channelId, cmd.lockTime, cmd.targetFeerate, d.fundingParams.localAmount)
|
||||
}
|
||||
case _ =>
|
||||
log.warning("cannot initiate rbf, another one is already in progress")
|
||||
replyTo ! Status.Failure(InvalidRbfAlreadyInProgress(d.channelId))
|
||||
stay()
|
||||
}
|
||||
}
|
||||
|
||||
case Event(_: TxAckRbf, _: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
log.info("rbf not supported yet")
|
||||
stay()
|
||||
case Event(msg: TxInitRbf, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
if (d.fundingParams.isInitiator) {
|
||||
// Only the initiator is allowed to initiate RBF.
|
||||
log.info("rejecting tx_init_rbf, we're the initiator, not them!")
|
||||
stay() sending TxAbort(d.channelId, InvalidRbfNonInitiator(d.channelId).getMessage)
|
||||
} else {
|
||||
val minNextFeerate = d.fundingParams.minNextFeerate
|
||||
if (d.rbfStatus != RbfStatus.NoRbf) {
|
||||
log.info("rejecting rbf attempt: the current rbf attempt must be completed or aborted first")
|
||||
stay() sending TxAbort(d.channelId, InvalidRbfAlreadyInProgress(d.channelId).getMessage)
|
||||
} else if (msg.feerate < minNextFeerate) {
|
||||
log.info("rejecting rbf attempt: the new feerate must be at least {} (proposed={})", minNextFeerate, msg.feerate)
|
||||
stay() sending TxAbort(d.channelId, InvalidRbfFeerate(d.channelId, msg.feerate, minNextFeerate).getMessage)
|
||||
} else {
|
||||
log.info("our peer wants to raise the feerate of the funding transaction (previous={} target={})", d.fundingParams.targetFeerate, msg.feerate)
|
||||
val fundingParams = InteractiveTxParams(
|
||||
d.channelId,
|
||||
d.fundingParams.isInitiator,
|
||||
d.fundingParams.localAmount, // we don't change our funding contribution
|
||||
msg.fundingContribution,
|
||||
d.fundingParams.fundingPubkeyScript,
|
||||
msg.lockTime,
|
||||
d.fundingParams.dustLimit,
|
||||
msg.feerate
|
||||
)
|
||||
val txBuilder = context.spawnAnonymous(InteractiveTxBuilder(
|
||||
remoteNodeId, fundingParams, keyManager,
|
||||
d.commitments.localParams, d.commitments.remoteParams,
|
||||
d.commitments.localCommit.spec.commitTxFeerate,
|
||||
d.commitments.remoteCommit.remotePerCommitmentPoint,
|
||||
d.commitments.channelFlags, d.commitments.channelConfig, d.commitments.channelFeatures,
|
||||
wallet))
|
||||
txBuilder ! InteractiveTxBuilder.Start(self, d.fundingTx +: d.previousFundingTxs.map(_.fundingTx))
|
||||
stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(txBuilder)) sending TxAckRbf(d.channelId, fundingParams.localAmount)
|
||||
}
|
||||
}
|
||||
|
||||
case Event(msg: TxAckRbf, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.RbfRequested(cmd) =>
|
||||
log.info("our peer accepted our rbf attempt and will contribute {} to the funding transaction", msg.fundingContribution)
|
||||
cmd.replyTo ! RES_SUCCESS(cmd, d.channelId)
|
||||
val fundingParams = InteractiveTxParams(
|
||||
d.channelId,
|
||||
d.fundingParams.isInitiator,
|
||||
d.fundingParams.localAmount, // we don't change our funding contribution
|
||||
msg.fundingContribution,
|
||||
d.fundingParams.fundingPubkeyScript,
|
||||
cmd.lockTime,
|
||||
d.fundingParams.dustLimit,
|
||||
cmd.targetFeerate
|
||||
)
|
||||
val txBuilder = context.spawnAnonymous(InteractiveTxBuilder(
|
||||
remoteNodeId, fundingParams, keyManager,
|
||||
d.commitments.localParams, d.commitments.remoteParams,
|
||||
d.commitments.localCommit.spec.commitTxFeerate,
|
||||
d.commitments.remoteCommit.remotePerCommitmentPoint,
|
||||
d.commitments.channelFlags, d.commitments.channelConfig, d.commitments.channelFeatures,
|
||||
wallet))
|
||||
txBuilder ! InteractiveTxBuilder.Start(self, d.fundingTx +: d.previousFundingTxs.map(_.fundingTx))
|
||||
stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(txBuilder))
|
||||
case _ =>
|
||||
log.info("ignoring unexpected tx_ack_rbf")
|
||||
stay() sending Warning(d.channelId, UnexpectedInteractiveTxMessage(d.channelId, msg).getMessage)
|
||||
}
|
||||
|
||||
case Event(msg: InteractiveTxConstructionMessage, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
stay() sending Warning(d.channelId, UnexpectedInteractiveTxMessage(d.channelId, msg).getMessage)
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.RbfInProgress(txBuilder) =>
|
||||
txBuilder ! InteractiveTxBuilder.ReceiveTxMessage(msg)
|
||||
stay()
|
||||
case _ =>
|
||||
log.info("ignoring unexpected interactive-tx message: {}", msg.getClass.getSimpleName)
|
||||
stay() sending Warning(d.channelId, UnexpectedInteractiveTxMessage(d.channelId, msg).getMessage)
|
||||
}
|
||||
|
||||
case Event(commitSig: CommitSig, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.RbfInProgress(txBuilder) =>
|
||||
txBuilder ! InteractiveTxBuilder.ReceiveCommitSig(commitSig)
|
||||
stay()
|
||||
case _ =>
|
||||
log.info("ignoring unexpected commit_sig")
|
||||
stay() sending Warning(d.channelId, UnexpectedCommitSig(d.channelId).getMessage)
|
||||
}
|
||||
|
||||
case Event(msg: TxAbort, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
stay() sending Warning(d.channelId, UnexpectedInteractiveTxMessage(d.channelId, msg).getMessage)
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.RbfInProgress(txBuilder) =>
|
||||
log.info("our peer aborted the rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data)
|
||||
txBuilder ! InteractiveTxBuilder.Abort
|
||||
stay() using d.copy(rbfStatus = RbfStatus.NoRbf)
|
||||
case RbfStatus.RbfRequested(cmd) =>
|
||||
log.info("our peer rejected our rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data)
|
||||
cmd.replyTo ! Status.Failure(new RuntimeException(s"rbf attempt rejected by our peer: ${msg.toAscii}"))
|
||||
stay() using d.copy(rbfStatus = RbfStatus.NoRbf)
|
||||
case RbfStatus.NoRbf =>
|
||||
log.info("ignoring unexpected tx_abort message")
|
||||
stay() sending Warning(d.channelId, UnexpectedInteractiveTxMessage(d.channelId, msg).getMessage)
|
||||
}
|
||||
|
||||
case Event(msg: InteractiveTxBuilder.Response, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => msg match {
|
||||
case InteractiveTxBuilder.SendMessage(msg) => stay() sending msg
|
||||
case InteractiveTxBuilder.Succeeded(fundingParams, fundingTx, commitments) =>
|
||||
// We now have more than one version of the funding tx, so we cannot use zero-conf.
|
||||
val fundingMinDepth = Funding.minDepthDualFunding(nodeParams.channelConf, commitments.channelFeatures, fundingParams).getOrElse(nodeParams.channelConf.minDepthBlocks.toLong)
|
||||
blockchain ! WatchFundingConfirmed(self, commitments.fundingTxId, fundingMinDepth)
|
||||
val previousFundingTxs = DualFundingTx(d.fundingTx, d.commitments) +: d.previousFundingTxs
|
||||
val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, fundingTx, fundingParams, previousFundingTxs, d.waitingSince, d.lastChecked, RbfStatus.NoRbf, d.deferred)
|
||||
fundingTx match {
|
||||
case fundingTx: PartiallySignedSharedTransaction => stay() using d1 storing() sending fundingTx.localSigs
|
||||
case fundingTx: FullySignedSharedTransaction => stay() using d1 storing() sending fundingTx.localSigs calling publishFundingTx(fundingParams, fundingTx)
|
||||
}
|
||||
case f: InteractiveTxBuilder.Failed =>
|
||||
log.info("rbf attempt failed: {}", f.cause.getMessage)
|
||||
stay() using d.copy(rbfStatus = RbfStatus.NoRbf) sending TxAbort(d.channelId, f.cause.getMessage)
|
||||
}
|
||||
|
||||
case Event(WatchFundingConfirmedTriggered(blockHeight, txIndex, confirmedTx), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
// We find which funding transaction got confirmed.
|
||||
val allFundingTxs = DualFundingTx(d.fundingTx, d.commitments) +: d.previousFundingTxs
|
||||
allFundingTxs.find(_.commitments.commitInput.outPoint.txid == confirmedTx.txid) match {
|
||||
allFundingTxs.find(_.commitments.fundingTxId == confirmedTx.txid) match {
|
||||
case Some(DualFundingTx(_, commitments)) =>
|
||||
log.info("channelId={} was confirmed at blockHeight={} txIndex={} with funding txid={}", d.channelId, blockHeight, txIndex, confirmedTx.txid)
|
||||
watchFundingTx(commitments)
|
||||
|
@ -391,7 +524,19 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
val realScidStatus = RealScidStatus.Temporary(RealShortChannelId(blockHeight, txIndex, commitments.commitInput.outPoint.index.toInt))
|
||||
val (shortIds, channelReady) = acceptFundingTx(commitments, realScidStatus = realScidStatus)
|
||||
d.deferred.foreach(self ! _)
|
||||
goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments, shortIds, channelReady) storing() sending channelReady
|
||||
val otherFundingTxs = allFundingTxs.filter(_.commitments.fundingTxId != confirmedTx.txid).map(_.fundingTx)
|
||||
rollbackDualFundingTxs(otherFundingTxs)
|
||||
val toSend = d.rbfStatus match {
|
||||
case RbfStatus.RbfInProgress(txBuilder) =>
|
||||
txBuilder ! InteractiveTxBuilder.Abort
|
||||
Seq(TxAbort(d.channelId, InvalidRbfTxConfirmed(d.channelId).getMessage), channelReady)
|
||||
case RbfStatus.RbfRequested(cmd) =>
|
||||
cmd.replyTo ! Status.Failure(InvalidRbfTxConfirmed(d.channelId))
|
||||
Seq(TxAbort(d.channelId, InvalidRbfTxConfirmed(d.channelId).getMessage), channelReady)
|
||||
case RbfStatus.NoRbf =>
|
||||
Seq(channelReady)
|
||||
}
|
||||
goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments, shortIds, channelReady) storing() sending toSend
|
||||
case None =>
|
||||
log.error(s"internal error: the funding tx that confirmed doesn't match any of our funding txs: ${confirmedTx.bin}")
|
||||
rollbackDualFundingTxs(allFundingTxs.map(_.fundingTx))
|
||||
|
@ -425,6 +570,14 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
delayEarlyAnnouncementSigs(remoteAnnSigs)
|
||||
stay()
|
||||
|
||||
case Event(INPUT_DISCONNECTED, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.RbfInProgress(txBuilder) => txBuilder ! InteractiveTxBuilder.Abort
|
||||
case RbfStatus.RbfRequested(cmd) => cmd.replyTo ! Status.Failure(new RuntimeException("rbf attempt failed: disconnected"))
|
||||
case RbfStatus.NoRbf => // nothing to do
|
||||
}
|
||||
goto(OFFLINE) using d.copy(rbfStatus = RbfStatus.NoRbf)
|
||||
|
||||
case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleRemoteError(e, d)
|
||||
})
|
||||
|
||||
|
|
|
@ -290,11 +290,11 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
|
|||
context.system.eventStream.publish(ChannelIdAssigned(self, remoteNodeId, temporaryChannelId, channelId))
|
||||
context.system.eventStream.publish(ChannelSignatureReceived(self, commitments))
|
||||
// NB: we don't send a ChannelSignatureSent for the first commit
|
||||
log.info(s"waiting for them to publish the funding tx for channelId=$channelId fundingTxid=${commitInput.outPoint.txid}")
|
||||
log.info(s"waiting for them to publish the funding tx for channelId=$channelId fundingTxid=${commitments.fundingTxId}")
|
||||
watchFundingTx(commitments)
|
||||
Funding.minDepthFundee(nodeParams.channelConf, commitments.channelFeatures, fundingAmount) match {
|
||||
case Some(fundingMinDepth) =>
|
||||
blockchain ! WatchFundingConfirmed(self, commitInput.outPoint.txid, fundingMinDepth)
|
||||
blockchain ! WatchFundingConfirmed(self, commitments.fundingTxId, fundingMinDepth)
|
||||
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, None, nodeParams.currentBlockHeight, None, Right(fundingSigned)) storing() sending fundingSigned
|
||||
case None =>
|
||||
val (shortIds, channelReady) = acceptFundingTx(commitments, RealScidStatus.Unknown)
|
||||
|
@ -335,13 +335,13 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
|
|||
commitInput, ShaChain.init)
|
||||
val blockHeight = nodeParams.currentBlockHeight
|
||||
context.system.eventStream.publish(ChannelSignatureReceived(self, commitments))
|
||||
log.info(s"publishing funding tx for channelId=$channelId fundingTxid=${commitInput.outPoint.txid}")
|
||||
log.info(s"publishing funding tx for channelId=$channelId fundingTxid=${commitments.fundingTxId}")
|
||||
watchFundingTx(commitments)
|
||||
// we will publish the funding tx only after the channel state has been written to disk because we want to
|
||||
// make sure we first persist the commitment that returns back the funds to us in case of problem
|
||||
Funding.minDepthFunder(commitments.channelFeatures) match {
|
||||
case Some(fundingMinDepth) =>
|
||||
blockchain ! WatchFundingConfirmed(self, commitInput.outPoint.txid, fundingMinDepth)
|
||||
blockchain ! WatchFundingConfirmed(self, commitments.fundingTxId, fundingMinDepth)
|
||||
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, Some(fundingTx), blockHeight, None, Left(fundingCreated)) storing() calling publishFundingTx(commitments, fundingTx, fundingTxFee)
|
||||
case None =>
|
||||
val (shortIds, channelReady) = acceptFundingTx(commitments, RealScidStatus.Unknown)
|
||||
|
@ -407,7 +407,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
|
|||
delayEarlyAnnouncementSigs(remoteAnnSigs)
|
||||
stay()
|
||||
|
||||
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.commitInput.outPoint.txid => handleGetFundingTx(getTxResponse, d.waitingSince, d.fundingTx)
|
||||
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.fundingTxId => handleGetFundingTx(getTxResponse, d.waitingSince, d.fundingTx)
|
||||
|
||||
case Event(BITCOIN_FUNDING_PUBLISH_FAILED, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) => handleFundingPublishFailed(d)
|
||||
|
||||
|
|
|
@ -44,7 +44,7 @@ trait CommonFundingHandlers extends CommonHandlers {
|
|||
}
|
||||
|
||||
def acceptFundingTx(commitments: Commitments, realScidStatus: RealScidStatus): (ShortIds, ChannelReady) = {
|
||||
blockchain ! WatchFundingLost(self, commitments.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks)
|
||||
blockchain ! WatchFundingLost(self, commitments.fundingTxId, nodeParams.channelConf.minDepthBlocks)
|
||||
// the alias will use in our peer's channel_update message, the goal is to be able to use our channel as soon
|
||||
// as it reaches NORMAL state, and before it is announced on the network
|
||||
val shortIds = ShortIds(realScidStatus, ShortChannelId.generateLocalAlias(), remoteAlias_opt = None)
|
||||
|
@ -75,7 +75,7 @@ trait CommonFundingHandlers extends CommonHandlers {
|
|||
// we need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network
|
||||
context.system.scheduler.scheduleWithFixedDelay(initialDelay = REFRESH_CHANNEL_UPDATE_INTERVAL, delay = REFRESH_CHANNEL_UPDATE_INTERVAL, receiver = self, message = BroadcastChannelUpdate(PeriodicRefresh))
|
||||
// used to get the final shortChannelId, used in announcements (if minDepth >= ANNOUNCEMENTS_MINCONF this event will fire instantly)
|
||||
blockchain ! WatchFundingDeeplyBuried(self, commitments.commitInput.outPoint.txid, ANNOUNCEMENTS_MINCONF)
|
||||
blockchain ! WatchFundingDeeplyBuried(self, commitments.fundingTxId, ANNOUNCEMENTS_MINCONF)
|
||||
DATA_NORMAL(commitments.copy(remoteNextCommitInfo = Right(channelReady.nextPerCommitmentPoint)), shortIds1, None, initialChannelUpdate, None, None, None)
|
||||
}
|
||||
|
||||
|
|
|
@ -17,6 +17,8 @@
|
|||
package fr.acinq.eclair.channel.fsm
|
||||
|
||||
import fr.acinq.bitcoin.scalacompat.{Transaction, TxIn}
|
||||
import fr.acinq.eclair.NotificationsLogger
|
||||
import fr.acinq.eclair.NotificationsLogger.NotifyNodeOperator
|
||||
import fr.acinq.eclair.blockchain.CurrentBlockHeight
|
||||
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.WatchFundingConfirmedTriggered
|
||||
import fr.acinq.eclair.channel.Helpers.Closing
|
||||
|
@ -58,19 +60,19 @@ trait DualFundingHandlers extends CommonFundingHandlers {
|
|||
}
|
||||
|
||||
def handleDualFundingConfirmedOffline(w: WatchFundingConfirmedTriggered, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) = {
|
||||
if (w.tx.txid == d.commitments.commitInput.outPoint.txid) {
|
||||
if (w.tx.txid == d.commitments.fundingTxId) {
|
||||
watchFundingTx(d.commitments)
|
||||
context.system.eventStream.publish(TransactionConfirmed(d.channelId, remoteNodeId, w.tx))
|
||||
// We can forget previous funding attempts now that the funding tx is confirmed.
|
||||
rollbackDualFundingTxs(d.previousFundingTxs.map(_.fundingTx))
|
||||
stay() using d.copy(previousFundingTxs = Nil) storing()
|
||||
} else if (d.previousFundingTxs.exists(_.commitments.commitInput.outPoint.txid == w.tx.txid)) {
|
||||
} else if (d.previousFundingTxs.exists(_.commitments.fundingTxId == w.tx.txid)) {
|
||||
log.info("channelId={} was confirmed at blockHeight={} txIndex={} with a previous funding txid={}", d.channelId, w.blockHeight, w.txIndex, w.tx.txid)
|
||||
val confirmed = d.previousFundingTxs.find(_.commitments.commitInput.outPoint.txid == w.tx.txid).get
|
||||
val confirmed = d.previousFundingTxs.find(_.commitments.fundingTxId == w.tx.txid).get
|
||||
watchFundingTx(confirmed.commitments)
|
||||
context.system.eventStream.publish(TransactionConfirmed(d.channelId, remoteNodeId, w.tx))
|
||||
// We can forget other funding attempts now that one of the funding txs is confirmed.
|
||||
val otherFundingTxs = d.fundingTx +: d.previousFundingTxs.filter(_.commitments.commitInput.outPoint.txid != w.tx.txid).map(_.fundingTx)
|
||||
val otherFundingTxs = d.fundingTx +: d.previousFundingTxs.filter(_.commitments.fundingTxId != w.tx.txid).map(_.fundingTx)
|
||||
rollbackDualFundingTxs(otherFundingTxs)
|
||||
stay() using d.copy(commitments = confirmed.commitments, fundingTx = confirmed.fundingTx, previousFundingTxs = Nil) storing()
|
||||
} else {
|
||||
|
@ -80,7 +82,12 @@ trait DualFundingHandlers extends CommonFundingHandlers {
|
|||
}
|
||||
|
||||
def handleNewBlockDualFundingUnconfirmed(c: CurrentBlockHeight, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) = {
|
||||
if (Channel.FUNDING_TIMEOUT_FUNDEE < c.blockHeight - d.waitingSince && Closing.nothingAtStake(d)) {
|
||||
// We regularly notify the node operator that they may want to RBF this channel.
|
||||
val blocksSinceOpen = c.blockHeight - d.waitingSince
|
||||
if (d.fundingParams.isInitiator && (blocksSinceOpen % 288 == 0)) { // 288 blocks = 2 days
|
||||
context.system.eventStream.publish(NotifyNodeOperator(NotificationsLogger.Info, s"channelId=${d.channelId} is still unconfirmed after $blocksSinceOpen blocks, you may need to use the rbfopen RPC to make it confirm."))
|
||||
}
|
||||
if (Channel.FUNDING_TIMEOUT_FUNDEE < blocksSinceOpen && Closing.nothingAtStake(d)) {
|
||||
log.warning("funding transaction did not confirm in {} blocks and we have nothing at stake, forgetting channel", Channel.FUNDING_TIMEOUT_FUNDEE)
|
||||
handleFundingTimeout(d)
|
||||
} else if (d.lastChecked + 6 < c.blockHeight) {
|
||||
|
@ -100,7 +107,7 @@ trait DualFundingHandlers extends CommonFundingHandlers {
|
|||
}
|
||||
|
||||
def handleDualFundingDoubleSpent(e: BITCOIN_FUNDING_DOUBLE_SPENT, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) = {
|
||||
val fundingTxIds = (d.commitments +: d.previousFundingTxs.map(_.commitments)).map(_.commitInput.outPoint.txid).toSet
|
||||
val fundingTxIds = (d.commitments +: d.previousFundingTxs.map(_.commitments)).map(_.fundingTxId).toSet
|
||||
if (fundingTxIds.subsetOf(e.fundingTxIds)) {
|
||||
log.warning("{} funding attempts have been double-spent, forgetting channel", fundingTxIds.size)
|
||||
(d.fundingTx +: d.previousFundingTxs.map(_.fundingTx)).foreach(tx => wallet.rollback(tx.tx.buildUnsignedTx()))
|
||||
|
|
|
@ -318,7 +318,7 @@ trait ErrorHandlers extends CommonHandlers {
|
|||
case None =>
|
||||
// the published tx was neither their current commitment nor a revoked one
|
||||
log.error(s"couldn't identify txid=${tx.txid}, something very bad is going on!!!")
|
||||
context.system.eventStream.publish(NotifyNodeOperator(NotificationsLogger.Error, s"funding tx ${d.commitments.commitInput.outPoint.txid} of channel ${d.channelId} was spent by an unknown transaction, indicating that your DB has lost data or your node has been breached: please contact the dev team."))
|
||||
context.system.eventStream.publish(NotifyNodeOperator(NotificationsLogger.Error, s"funding tx ${d.commitments.fundingTxId} of channel ${d.channelId} was spent by an unknown transaction, indicating that your DB has lost data or your node has been breached: please contact the dev team."))
|
||||
goto(ERR_INFORMATION_LEAK)
|
||||
}
|
||||
}
|
||||
|
@ -342,8 +342,8 @@ trait ErrorHandlers extends CommonHandlers {
|
|||
|
||||
def handleInformationLeak(tx: Transaction, d: PersistentChannelData) = {
|
||||
// this is never supposed to happen !!
|
||||
log.error(s"our funding tx ${d.commitments.commitInput.outPoint.txid} was spent by txid=${tx.txid}!!")
|
||||
context.system.eventStream.publish(NotifyNodeOperator(NotificationsLogger.Error, s"funding tx ${d.commitments.commitInput.outPoint.txid} of channel ${d.channelId} was spent by an unknown transaction, indicating that your DB has lost data or your node has been breached: please contact the dev team."))
|
||||
log.error(s"our funding tx ${d.commitments.fundingTxId} was spent by txid=${tx.txid}!!")
|
||||
context.system.eventStream.publish(NotifyNodeOperator(NotificationsLogger.Error, s"funding tx ${d.commitments.fundingTxId} of channel ${d.channelId} was spent by an unknown transaction, indicating that your DB has lost data or your node has been breached: please contact the dev team."))
|
||||
val exc = FundingTxSpent(d.channelId, tx)
|
||||
val error = Error(d.channelId, exc.getMessage)
|
||||
|
||||
|
|
|
@ -504,7 +504,7 @@ object Validation {
|
|||
// since this is a local channel, we can trust the announcement, no need to go through the full
|
||||
// verification process and make calls to bitcoin core
|
||||
val fundingTxId = lcu.commitments match {
|
||||
case commitments: Commitments => commitments.commitInput.outPoint.txid
|
||||
case commitments: Commitments => commitments.fundingTxId
|
||||
case _ => ByteVector32.Zeroes
|
||||
}
|
||||
val d1 = addPublicChannel(d, nodeParams, watcher, ann, fundingTxId, lcu.commitments.capacity, Some(privateChannel))
|
||||
|
|
|
@ -390,7 +390,7 @@ private[channel] object ChannelCodecs3 {
|
|||
("previousFundingTxs" | listOfN(uint16, dualFundingTxCodec)) ::
|
||||
("waitingSince" | blockHeight) ::
|
||||
("lastChecked" | blockHeight) ::
|
||||
("rbfAttempt" | provide(Option.empty[typed.ActorRef[InteractiveTxBuilder.Command]])) ::
|
||||
("rbfStatus" | provide[RbfStatus](RbfStatus.NoRbf)) ::
|
||||
("deferred" | optional(bool8, lengthDelimited(channelReadyCodec)))).as[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
|
||||
val DATA_WAIT_FOR_DUAL_FUNDING_READY_0c_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_READY] = (
|
||||
|
|
|
@ -19,7 +19,7 @@ package fr.acinq.eclair.wire.protocol
|
|||
import com.google.common.base.Charsets
|
||||
import com.google.common.net.InetAddresses
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Satoshi, ScriptWitness, Transaction}
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Satoshi, SatoshiLong, ScriptWitness, Transaction}
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.eclair.channel.{ChannelFlags, ChannelType}
|
||||
import fr.acinq.eclair.payment.relay.Relayer
|
||||
|
@ -117,7 +117,7 @@ case class TxInitRbf(channelId: ByteVector32,
|
|||
lockTime: Long,
|
||||
feerate: FeeratePerKw,
|
||||
tlvStream: TlvStream[TxInitRbfTlv] = TlvStream.empty) extends InteractiveTxMessage with HasChannelId {
|
||||
val fundingContribution_opt: Option[Satoshi] = tlvStream.get[TxRbfTlv.SharedOutputContributionTlv].map(_.amount)
|
||||
val fundingContribution: Satoshi = tlvStream.get[TxRbfTlv.SharedOutputContributionTlv].map(_.amount).getOrElse(0 sat)
|
||||
}
|
||||
|
||||
object TxInitRbf {
|
||||
|
@ -127,7 +127,7 @@ object TxInitRbf {
|
|||
|
||||
case class TxAckRbf(channelId: ByteVector32,
|
||||
tlvStream: TlvStream[TxAckRbfTlv] = TlvStream.empty) extends InteractiveTxMessage with HasChannelId {
|
||||
val fundingContribution_opt: Option[Satoshi] = tlvStream.get[TxRbfTlv.SharedOutputContributionTlv].map(_.amount)
|
||||
val fundingContribution: Satoshi = tlvStream.get[TxRbfTlv.SharedOutputContributionTlv].map(_.amount).getOrElse(0 sat)
|
||||
}
|
||||
|
||||
object TxAckRbf {
|
||||
|
|
|
@ -21,7 +21,7 @@ import akka.actor.typed.scaladsl.adapter.{ClassicActorSystemOps, actorRefAdapter
|
|||
import akka.pattern.pipe
|
||||
import akka.testkit.TestProbe
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Satoshi, SatoshiLong, Script, Transaction, TxOut}
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Satoshi, SatoshiLong, Script, ScriptWitness, Transaction, TxOut}
|
||||
import fr.acinq.eclair.blockchain.OnChainWallet.{FundTransactionResponse, SignTransactionResponse}
|
||||
import fr.acinq.eclair.blockchain.bitcoind.BitcoindService
|
||||
import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient.{MempoolTx, Utxo}
|
||||
|
@ -35,7 +35,7 @@ import fr.acinq.eclair.wire.protocol._
|
|||
import fr.acinq.eclair.{Feature, FeatureSupport, Features, InitFeature, NodeParams, TestConstants, TestKitBaseClass, UInt64, randomBytes32, randomKey}
|
||||
import org.scalatest.BeforeAndAfterAll
|
||||
import org.scalatest.funsuite.AnyFunSuiteLike
|
||||
import scodec.bits.ByteVector
|
||||
import scodec.bits.{ByteVector, HexStringSyntax}
|
||||
|
||||
import java.util.UUID
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
|
@ -1079,4 +1079,35 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit
|
|||
assert(alice2bob.expectMsgType[RemoteFailure].cause.isInstanceOf[InvalidFundingSignature])
|
||||
}
|
||||
|
||||
test("reference test vector") {
|
||||
val channelId = ByteVector32.Zeroes
|
||||
val parentTx = Transaction.read("02000000000101f86fd1d0db3ac5a72df968622f31e6b5e6566a09e29206d7c7a55df90e181de800000000171600141fb9623ffd0d422eacc450fd1e967efc477b83ccffffffff0580b2e60e00000000220020fd89acf65485df89797d9ba7ba7a33624ac4452f00db08107f34257d33e5b94680b2e60e0000000017a9146a235d064786b49e7043e4a042d4cc429f7eb6948780b2e60e00000000160014fbb4db9d85fba5e301f4399e3038928e44e37d3280b2e60e0000000017a9147ecd1b519326bc13b0ec716e469b58ed02b112a087f0006bee0000000017a914f856a70093da3a5b5c4302ade033d4c2171705d387024730440220696f6cee2929f1feb3fd6adf024ca0f9aa2f4920ed6d35fb9ec5b78c8408475302201641afae11242160101c6f9932aeb4fcd1f13a9c6df5d1386def000ea259a35001210381d7d5b1bc0d7600565d827242576d9cb793bfe0754334af82289ee8b65d137600000000")
|
||||
val initiatorInput = TxAddInput(channelId, UInt64(20), parentTx, 0, 4294967293L)
|
||||
val initiatorOutput = TxAddOutput(channelId, UInt64(30), 49999845 sat, hex"00141ca1cca8855bad6bc1ea5436edd8cff10b7e448b")
|
||||
val sharedOutput = TxAddOutput(channelId, UInt64(44), 400000000 sat, hex"0020297b92c238163e820b82486084634b4846b86a3c658d87b9384192e6bea98ec5")
|
||||
val nonInitiatorInput = TxAddInput(channelId, UInt64(11), parentTx, 2, 4294967293L)
|
||||
val nonInitiatorOutput = TxAddOutput(channelId, UInt64(33), 49999900 sat, hex"001444cb0c39f93ecc372b5851725bd29d865d333b10")
|
||||
|
||||
val initiatorParams = InteractiveTxParams(channelId, isInitiator = true, 200_000_000 sat, 200_000_000 sat, hex"0020297b92c238163e820b82486084634b4846b86a3c658d87b9384192e6bea98ec5", 120, 330 sat, FeeratePerKw(253 sat))
|
||||
val initiatorTx = SharedTransaction(List(initiatorInput), List(nonInitiatorInput).map(i => RemoteTxAddInput(i)), List(initiatorOutput, sharedOutput), List(nonInitiatorOutput).map(o => RemoteTxAddOutput(o)), lockTime = 120)
|
||||
assert(initiatorTx.localFees(initiatorParams) == 155.sat)
|
||||
val nonInitiatorParams = initiatorParams.copy(isInitiator = false)
|
||||
val nonInitiatorTx = SharedTransaction(List(nonInitiatorInput), List(initiatorInput).map(i => RemoteTxAddInput(i)), List(nonInitiatorOutput), List(initiatorOutput, sharedOutput).map(o => RemoteTxAddOutput(o)), lockTime = 120)
|
||||
assert(nonInitiatorTx.localFees(nonInitiatorParams) == 100.sat)
|
||||
|
||||
val unsignedTx = Transaction.read("0200000002b932b0669cd0394d0d5bcc27e01ab8c511f1662a6799925b346c0cf18fca03430200000000fdffffffb932b0669cd0394d0d5bcc27e01ab8c511f1662a6799925b346c0cf18fca03430000000000fdffffff03e5effa02000000001600141ca1cca8855bad6bc1ea5436edd8cff10b7e448b1cf0fa020000000016001444cb0c39f93ecc372b5851725bd29d865d333b100084d71700000000220020297b92c238163e820b82486084634b4846b86a3c658d87b9384192e6bea98ec578000000")
|
||||
assert(initiatorTx.buildUnsignedTx().txid == unsignedTx.txid)
|
||||
assert(nonInitiatorTx.buildUnsignedTx().txid == unsignedTx.txid)
|
||||
|
||||
val initiatorSigs = TxSignatures(channelId, unsignedTx.txid, Seq(ScriptWitness(Seq(hex"68656c6c6f2074686572652c2074686973206973206120626974636f6e212121", hex"82012088a820add57dfe5277079d069ca4ad4893c96de91f88ffb981fdc6a2a34d5336c66aff87"))))
|
||||
val nonInitiatorSigs = TxSignatures(channelId, unsignedTx.txid, Seq(ScriptWitness(Seq(hex"304402207de9ba56bb9f641372e805782575ee840a899e61021c8b1572b3ec1d5b5950e9022069e9ba998915dae193d3c25cb89b5e64370e6a3a7755e7f31cf6d7cbc2a49f6d01", hex"034695f5b7864c580bf11f9f8cb1a94eb336f2ce9ef872d2ae1a90ee276c772484"))))
|
||||
val initiatorSignedTx = FullySignedSharedTransaction(initiatorTx, initiatorSigs, nonInitiatorSigs)
|
||||
assert(initiatorSignedTx.feerate == FeeratePerKw(262 sat))
|
||||
val nonInitiatorSignedTx = FullySignedSharedTransaction(nonInitiatorTx, nonInitiatorSigs, initiatorSigs)
|
||||
assert(nonInitiatorSignedTx.feerate == FeeratePerKw(262 sat))
|
||||
val signedTx = Transaction.read("02000000000102b932b0669cd0394d0d5bcc27e01ab8c511f1662a6799925b346c0cf18fca03430200000000fdffffffb932b0669cd0394d0d5bcc27e01ab8c511f1662a6799925b346c0cf18fca03430000000000fdffffff03e5effa02000000001600141ca1cca8855bad6bc1ea5436edd8cff10b7e448b1cf0fa020000000016001444cb0c39f93ecc372b5851725bd29d865d333b100084d71700000000220020297b92c238163e820b82486084634b4846b86a3c658d87b9384192e6bea98ec50247304402207de9ba56bb9f641372e805782575ee840a899e61021c8b1572b3ec1d5b5950e9022069e9ba998915dae193d3c25cb89b5e64370e6a3a7755e7f31cf6d7cbc2a49f6d0121034695f5b7864c580bf11f9f8cb1a94eb336f2ce9ef872d2ae1a90ee276c772484022068656c6c6f2074686572652c2074686973206973206120626974636f6e2121212782012088a820add57dfe5277079d069ca4ad4893c96de91f88ffb981fdc6a2a34d5336c66aff8778000000")
|
||||
assert(initiatorSignedTx.signedTx == signedTx)
|
||||
assert(initiatorSignedTx.signedTx == nonInitiatorSignedTx.signedTx)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.channel.states.c
|
||||
|
||||
import akka.actor.Status
|
||||
import akka.actor.typed.scaladsl.adapter.actorRefAdapter
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.bitcoin.scalacompat.ByteVector32
|
||||
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
|
||||
|
@ -26,6 +28,7 @@ import fr.acinq.eclair.channel.fsm.Channel
|
|||
import fr.acinq.eclair.channel.fsm.Channel.ProcessCurrentBlockHeight
|
||||
import fr.acinq.eclair.channel.publish.TxPublisher
|
||||
import fr.acinq.eclair.channel.publish.TxPublisher.{PublishFinalTx, SetChannelId}
|
||||
import fr.acinq.eclair.channel.states.ChannelStateTestsBase.FakeTxPublisherFactory
|
||||
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
import fr.acinq.eclair.{BlockHeight, TestConstants, TestKitBaseClass}
|
||||
|
@ -165,6 +168,169 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
|
|||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_READY)
|
||||
}
|
||||
|
||||
test("recv WatchFundingConfirmedTriggered (rbf in progress)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val probe = TestProbe()
|
||||
val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx
|
||||
alice ! CMD_BUMP_FUNDING_FEE(probe.ref, TestConstants.feeratePerKw * 1.1, 0)
|
||||
alice2bob.expectMsgType[TxInitRbf]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[TxAckRbf]
|
||||
|
||||
alice ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx)
|
||||
assert(aliceListener.expectMsgType[TransactionConfirmed].tx == fundingTx)
|
||||
assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid)
|
||||
alice2bob.expectMsgType[TxAbort]
|
||||
alice2bob.expectMsgType[ChannelReady]
|
||||
probe.expectMsg(Status.Failure(InvalidRbfTxConfirmed(channelId(alice))))
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY)
|
||||
}
|
||||
|
||||
test("recv WatchFundingConfirmedTriggered after restart", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
val fundingTx = aliceData.fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx
|
||||
val (alice2, bob2) = restartNodes(f, aliceData, bobData)
|
||||
reconnectNodes(f, alice2, aliceData, bob2, bobData)
|
||||
|
||||
alice2 ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx)
|
||||
assert(aliceListener.expectMsgType[TransactionConfirmed].tx == fundingTx)
|
||||
assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid)
|
||||
alice2bob.expectMsgType[ChannelReady]
|
||||
awaitCond(alice2.stateName == WAIT_FOR_DUAL_FUNDING_READY)
|
||||
|
||||
bob2 ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx)
|
||||
assert(bobListener.expectMsgType[TransactionConfirmed].tx == fundingTx)
|
||||
assert(bob2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid)
|
||||
bob2alice.expectMsgType[ChannelReady]
|
||||
awaitCond(bob2.stateName == WAIT_FOR_DUAL_FUNDING_READY)
|
||||
}
|
||||
|
||||
test("recv WatchFundingConfirmedTriggered after restart (previous tx)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val fundingTx1 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx
|
||||
val fundingTx2 = testBumpFundingFees(f).signedTx
|
||||
assert(fundingTx1.txid != fundingTx2.txid)
|
||||
|
||||
val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
val (alice2, bob2) = restartNodes(f, aliceData, bobData)
|
||||
reconnectNodes(f, alice2, aliceData, bob2, bobData)
|
||||
|
||||
alice2 ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx1)
|
||||
assert(aliceListener.expectMsgType[TransactionConfirmed].tx == fundingTx1)
|
||||
assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx1.txid)
|
||||
alice2bob.expectMsgType[ChannelReady]
|
||||
awaitCond(alice2.stateName == WAIT_FOR_DUAL_FUNDING_READY)
|
||||
|
||||
bob2 ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx1)
|
||||
assert(bobListener.expectMsgType[TransactionConfirmed].tx == fundingTx1)
|
||||
assert(bob2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx1.txid)
|
||||
bob2alice.expectMsgType[ChannelReady]
|
||||
awaitCond(bob2.stateName == WAIT_FOR_DUAL_FUNDING_READY)
|
||||
|
||||
assert(alice2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.fundingTxId == fundingTx1.txid)
|
||||
assert(bob2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.fundingTxId == fundingTx1.txid)
|
||||
}
|
||||
|
||||
def testBumpFundingFees(f: FixtureParam): FullySignedSharedTransaction = {
|
||||
import f._
|
||||
|
||||
val probe = TestProbe()
|
||||
val currentFundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction]
|
||||
val previousFundingTxs = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs
|
||||
alice ! CMD_BUMP_FUNDING_FEE(probe.ref, currentFundingTx.feerate * 1.1, 0)
|
||||
assert(alice2bob.expectMsgType[TxInitRbf].fundingContribution == TestConstants.fundingSatoshis)
|
||||
alice2bob.forward(bob)
|
||||
assert(bob2alice.expectMsgType[TxAckRbf].fundingContribution == TestConstants.nonInitiatorFundingSatoshis)
|
||||
bob2alice.forward(alice)
|
||||
probe.expectMsgType[RES_SUCCESS[CMD_BUMP_FUNDING_FEE]]
|
||||
|
||||
// Alice and Bob build a new version of the funding transaction, with one new input every time.
|
||||
val inputCount = previousFundingTxs.length + 2
|
||||
(1 to inputCount).foreach(_ => {
|
||||
alice2bob.expectMsgType[TxAddInput]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[TxAddInput]
|
||||
bob2alice.forward(alice)
|
||||
})
|
||||
alice2bob.expectMsgType[TxAddOutput]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[TxAddOutput]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxAddOutput]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[TxComplete]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxComplete]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[CommitSig]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[TxSignatures]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxSignatures]
|
||||
alice2bob.forward(bob)
|
||||
|
||||
val nextFundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction]
|
||||
assert(aliceListener.expectMsgType[TransactionPublished].tx.txid == nextFundingTx.signedTx.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == nextFundingTx.signedTx.txid)
|
||||
assert(bobListener.expectMsgType[TransactionPublished].tx.txid == nextFundingTx.signedTx.txid)
|
||||
assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == nextFundingTx.signedTx.txid)
|
||||
assert(currentFundingTx.signedTx.txid != nextFundingTx.signedTx.txid)
|
||||
assert(currentFundingTx.feerate < nextFundingTx.feerate)
|
||||
// The new transaction double-spends previous inputs.
|
||||
currentFundingTx.signedTx.txIn.map(_.outPoint).foreach(o => assert(nextFundingTx.signedTx.txIn.exists(_.outPoint == o)))
|
||||
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.length == previousFundingTxs.length + 1)
|
||||
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.head.fundingTx == currentFundingTx)
|
||||
nextFundingTx
|
||||
}
|
||||
|
||||
test("rbf funding attempt", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.isEmpty)
|
||||
testBumpFundingFees(f)
|
||||
testBumpFundingFees(f)
|
||||
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.length == 2)
|
||||
}
|
||||
|
||||
test("rbf funding attempt failure", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val probe = TestProbe()
|
||||
val fundingTxAlice = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction]
|
||||
val fundingTxBob = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction]
|
||||
alice ! CMD_BUMP_FUNDING_FEE(probe.ref, TestConstants.feeratePerKw * 1.1, 0)
|
||||
assert(alice2bob.expectMsgType[TxInitRbf].fundingContribution == TestConstants.fundingSatoshis)
|
||||
alice2bob.forward(bob)
|
||||
assert(bob2alice.expectMsgType[TxAckRbf].fundingContribution == TestConstants.nonInitiatorFundingSatoshis)
|
||||
bob2alice.forward(alice)
|
||||
|
||||
// Alice and Bob build a new version of the funding transaction.
|
||||
alice2bob.expectMsgType[TxAddInput]
|
||||
alice2bob.forward(bob)
|
||||
val bobInput = bob2alice.expectMsgType[TxAddInput]
|
||||
bob2alice.forward(alice, bobInput.copy(previousTxOutput = 42))
|
||||
alice2bob.expectMsgType[TxAbort]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectNoMessage(100 millis)
|
||||
alice2bob.expectNoMessage(100 millis)
|
||||
|
||||
// Alice and Bob clear RBF data from their state.
|
||||
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf)
|
||||
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx == fundingTxAlice)
|
||||
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.isEmpty)
|
||||
assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf)
|
||||
assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx == fundingTxBob)
|
||||
assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].previousFundingTxs.isEmpty)
|
||||
}
|
||||
|
||||
test("recv CurrentBlockCount (funding in progress)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx
|
||||
|
@ -285,8 +451,8 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
|
|||
awaitCond(alice.stateName == OFFLINE)
|
||||
// The funding tx confirms while we're offline.
|
||||
alice ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx)
|
||||
assert(aliceListener.expectMsgType[TransactionConfirmed].tx == fundingTx)
|
||||
assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid)
|
||||
alice2blockchain.expectNoMessage(100 millis)
|
||||
// Bob broadcasts his commit tx.
|
||||
val bobCommitTx = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx
|
||||
alice ! WatchFundingSpentTriggered(bobCommitTx.tx)
|
||||
|
@ -294,10 +460,99 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
|
|||
assert(claimMain.input.txid == bobCommitTx.tx.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == bobCommitTx.tx.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMain.tx.txid)
|
||||
alice2blockchain.expectNoMessage(100 millis)
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
}
|
||||
|
||||
test("recv WatchFundingSpentTriggered while offline (previous tx)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val fundingTx1 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx
|
||||
val bobCommitTx1 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
val fundingTx2 = testBumpFundingFees(f).signedTx
|
||||
assert(fundingTx1.txid != fundingTx2.txid)
|
||||
val bobCommitTx2 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
assert(bobCommitTx1.txid != bobCommitTx2.txid)
|
||||
|
||||
alice ! INPUT_DISCONNECTED
|
||||
awaitCond(alice.stateName == OFFLINE)
|
||||
// A previous funding tx confirms while we're offline.
|
||||
alice ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx1)
|
||||
assert(aliceListener.expectMsgType[TransactionConfirmed].tx == fundingTx1)
|
||||
assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx1.txid)
|
||||
// Bob broadcasts his commit tx.
|
||||
alice ! WatchFundingSpentTriggered(bobCommitTx1)
|
||||
val claimMain = alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx]
|
||||
assert(claimMain.input.txid == bobCommitTx1.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == bobCommitTx1.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMain.tx.txid)
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
}
|
||||
|
||||
test("recv WatchFundingSpentTriggered after restart (remote commit)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
val fundingTx = aliceData.fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx
|
||||
val (alice2, bob2) = restartNodes(f, aliceData, bobData)
|
||||
|
||||
alice2 ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx)
|
||||
assert(aliceListener.expectMsgType[TransactionConfirmed].tx == fundingTx)
|
||||
assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid)
|
||||
alice2 ! WatchFundingSpentTriggered(bobData.commitments.localCommit.commitTxAndRemoteSig.commitTx.tx)
|
||||
val claimMainAlice = alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx]
|
||||
assert(claimMainAlice.input.txid == bobData.commitments.localCommit.commitTxAndRemoteSig.commitTx.tx.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == bobData.commitments.localCommit.commitTxAndRemoteSig.commitTx.tx.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMainAlice.tx.txid)
|
||||
awaitCond(alice2.stateName == CLOSING)
|
||||
|
||||
bob2 ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx)
|
||||
assert(bobListener.expectMsgType[TransactionConfirmed].tx == fundingTx)
|
||||
assert(bob2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx.txid)
|
||||
bob2 ! WatchFundingSpentTriggered(aliceData.commitments.localCommit.commitTxAndRemoteSig.commitTx.tx)
|
||||
val claimMainBob = bob2blockchain.expectMsgType[TxPublisher.PublishFinalTx]
|
||||
assert(claimMainBob.input.txid == aliceData.commitments.localCommit.commitTxAndRemoteSig.commitTx.tx.txid)
|
||||
assert(bob2blockchain.expectMsgType[WatchTxConfirmed].txId == aliceData.commitments.localCommit.commitTxAndRemoteSig.commitTx.tx.txid)
|
||||
assert(bob2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMainBob.tx.txid)
|
||||
awaitCond(bob2.stateName == CLOSING)
|
||||
}
|
||||
|
||||
test("recv WatchFundingSpentTriggered after restart (previous tx)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val fundingTx1 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx
|
||||
val aliceCommitTx1 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
val bobCommitTx1 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
val fundingTx2 = testBumpFundingFees(f).signedTx
|
||||
assert(fundingTx1.txid != fundingTx2.txid)
|
||||
val bobCommitTx2 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
assert(bobCommitTx1.txid != bobCommitTx2.txid)
|
||||
|
||||
val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
val (alice2, bob2) = restartNodes(f, aliceData, bobData)
|
||||
|
||||
alice2 ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx1)
|
||||
assert(aliceListener.expectMsgType[TransactionConfirmed].tx == fundingTx1)
|
||||
assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx1.txid)
|
||||
alice2 ! WatchFundingSpentTriggered(bobCommitTx1)
|
||||
val claimMainAlice = alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx]
|
||||
assert(claimMainAlice.input.txid == bobCommitTx1.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == bobCommitTx1.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMainAlice.tx.txid)
|
||||
awaitCond(alice2.stateName == CLOSING)
|
||||
|
||||
bob2 ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx1)
|
||||
assert(bobListener.expectMsgType[TransactionConfirmed].tx == fundingTx1)
|
||||
assert(bob2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx1.txid)
|
||||
bob2 ! WatchFundingSpentTriggered(aliceCommitTx1)
|
||||
val claimMainBob = bob2blockchain.expectMsgType[TxPublisher.PublishFinalTx]
|
||||
assert(claimMainBob.input.txid == aliceCommitTx1.txid)
|
||||
assert(bob2blockchain.expectMsgType[WatchTxConfirmed].txId == aliceCommitTx1.txid)
|
||||
assert(bob2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMainBob.tx.txid)
|
||||
awaitCond(bob2.stateName == CLOSING)
|
||||
}
|
||||
|
||||
test("recv Error", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
val tx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
|
@ -327,6 +582,47 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
|
|||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMainRemote.tx.txid)
|
||||
}
|
||||
|
||||
test("recv Error (previous tx confirms)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val fundingTx1 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].fundingTx.asInstanceOf[FullySignedSharedTransaction].signedTx
|
||||
val aliceCommitTx1 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx
|
||||
val bobCommitTx1 = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx
|
||||
assert(aliceCommitTx1.input.outPoint.txid == fundingTx1.txid)
|
||||
assert(bobCommitTx1.input.outPoint.txid == fundingTx1.txid)
|
||||
val fundingTx2 = testBumpFundingFees(f).signedTx
|
||||
assert(fundingTx1.txid != fundingTx2.txid)
|
||||
val aliceCommitTx2 = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx
|
||||
assert(aliceCommitTx2.input.outPoint.txid == fundingTx2.txid)
|
||||
|
||||
// Alice receives an error and force-closes using the latest funding transaction.
|
||||
alice ! Error(ByteVector32.Zeroes, "dual funding d34d")
|
||||
awaitCond(alice.stateName == CLOSING)
|
||||
assert(alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx].tx.txid == aliceCommitTx2.tx.txid)
|
||||
val claimMain2 = alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx]
|
||||
assert(claimMain2.input.txid == aliceCommitTx2.tx.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == aliceCommitTx2.tx.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMain2.tx.txid)
|
||||
|
||||
// A previous funding transaction confirms, so Alice publishes the corresponding commit tx.
|
||||
alice ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx1)
|
||||
assert(aliceListener.expectMsgType[TransactionConfirmed].tx == fundingTx1)
|
||||
assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx1.txid)
|
||||
assert(alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx].tx.txid == aliceCommitTx1.tx.txid)
|
||||
val claimMain1 = alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx]
|
||||
assert(claimMain1.input.txid == aliceCommitTx1.tx.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == aliceCommitTx1.tx.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMain1.tx.txid)
|
||||
|
||||
// Bob publishes his commit tx, Alice reacts by spending her remote main output.
|
||||
alice ! WatchFundingSpentTriggered(bobCommitTx1.tx)
|
||||
val claimMainRemote = alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx]
|
||||
assert(claimMainRemote.input.txid == bobCommitTx1.tx.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == bobCommitTx1.tx.txid)
|
||||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMainRemote.tx.txid)
|
||||
assert(alice.stateName == CLOSING)
|
||||
}
|
||||
|
||||
test("recv Error (nothing at stake)", Tag(ChannelStateTestsTags.DualFunding), Tag("no-funding-contribution")) { f =>
|
||||
import f._
|
||||
val commitTx = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
|
@ -358,4 +654,60 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
|
|||
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == claimMain.tx.txid)
|
||||
}
|
||||
|
||||
def restartNodes(f: FixtureParam, aliceData: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED, bobData: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED): (TestFSMRef[ChannelState, ChannelData, Channel], TestFSMRef[ChannelState, ChannelData, Channel]) = {
|
||||
import f._
|
||||
|
||||
val (aliceNodeParams, bobNodeParams) = (alice.underlyingActor.nodeParams, bob.underlyingActor.nodeParams)
|
||||
val (alicePeer, bobPeer) = (alice.getParent, bob.getParent)
|
||||
|
||||
alice.stop()
|
||||
bob.stop()
|
||||
|
||||
val alice2 = TestFSMRef(new Channel(aliceNodeParams, wallet, bobNodeParams.nodeId, alice2blockchain.ref, TestProbe().ref, FakeTxPublisherFactory(alice2blockchain)), alicePeer)
|
||||
alice2 ! INPUT_RESTORED(aliceData)
|
||||
alice2blockchain.expectMsgType[SetChannelId]
|
||||
// When restoring, we watch confirmation of all potential funding transactions to detect offline force-closes.
|
||||
assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == aliceData.commitments.fundingTxId)
|
||||
aliceData.previousFundingTxs.foreach(f => alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == f.commitments.fundingTxId)
|
||||
awaitCond(alice2.stateName == OFFLINE)
|
||||
|
||||
val bob2 = TestFSMRef(new Channel(bobNodeParams, wallet, aliceNodeParams.nodeId, bob2blockchain.ref, TestProbe().ref, FakeTxPublisherFactory(bob2blockchain)), bobPeer)
|
||||
bob2 ! INPUT_RESTORED(bobData)
|
||||
bob2blockchain.expectMsgType[SetChannelId]
|
||||
assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == bobData.commitments.fundingTxId)
|
||||
bobData.previousFundingTxs.foreach(f => bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == f.commitments.fundingTxId)
|
||||
awaitCond(bob2.stateName == OFFLINE)
|
||||
|
||||
alice2.underlying.system.eventStream.subscribe(aliceListener.ref, classOf[TransactionPublished])
|
||||
alice2.underlying.system.eventStream.subscribe(aliceListener.ref, classOf[TransactionConfirmed])
|
||||
bob2.underlying.system.eventStream.subscribe(bobListener.ref, classOf[TransactionPublished])
|
||||
bob2.underlying.system.eventStream.subscribe(bobListener.ref, classOf[TransactionConfirmed])
|
||||
|
||||
(alice2, bob2)
|
||||
}
|
||||
|
||||
def reconnectNodes(f: FixtureParam, alice2: TestFSMRef[ChannelState, ChannelData, Channel], aliceData: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED, bob2: TestFSMRef[ChannelState, ChannelData, Channel], bobData: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED): Unit = {
|
||||
import f._
|
||||
|
||||
val aliceInit = Init(alice2.underlyingActor.nodeParams.features.initFeatures())
|
||||
val bobInit = Init(bob2.underlyingActor.nodeParams.features.initFeatures())
|
||||
alice2 ! INPUT_RECONNECTED(alice2bob.ref, aliceInit, bobInit)
|
||||
val aliceChannelReestablish = alice2bob.expectMsgType[ChannelReestablish]
|
||||
bob2 ! INPUT_RECONNECTED(bob2alice.ref, bobInit, aliceInit)
|
||||
val bobChannelReestablish = bob2alice.expectMsgType[ChannelReestablish]
|
||||
alice2 ! bobChannelReestablish
|
||||
// When reconnecting, we watch confirmation again, otherwise if a transaction was confirmed while we were offline,
|
||||
// we won't be notified again and won't be able to transition to the next state.
|
||||
assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == aliceData.commitments.fundingTxId)
|
||||
aliceData.previousFundingTxs.foreach(f => alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == f.commitments.fundingTxId)
|
||||
alice2bob.expectMsgType[TxSignatures]
|
||||
bob2 ! aliceChannelReestablish
|
||||
assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == bobData.commitments.fundingTxId)
|
||||
bobData.previousFundingTxs.foreach(f => bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == f.commitments.fundingTxId)
|
||||
bob2alice.expectMsgType[TxSignatures]
|
||||
|
||||
awaitCond(alice2.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
awaitCond(bob2.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -165,7 +165,7 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat
|
|||
}
|
||||
|
||||
def fundingTx(node: MinimalNodeFixture, channelId: ByteVector32)(implicit system: ActorSystem): Transaction = {
|
||||
val fundingTxid = getChannelData(node, channelId).asInstanceOf[PersistentChannelData].commitments.commitInput.outPoint.txid
|
||||
val fundingTxid = getChannelData(node, channelId).asInstanceOf[PersistentChannelData].commitments.fundingTxId
|
||||
node.wallet.funded(fundingTxid)
|
||||
}
|
||||
|
||||
|
@ -196,7 +196,7 @@ object MinimalNodeFixture extends Assertions with Eventually with IntegrationPat
|
|||
def confirmChannelDeep(node1: MinimalNodeFixture, node2: MinimalNodeFixture, channelId: ByteVector32, blockHeight: BlockHeight, txIndex: Int)(implicit system: ActorSystem): RealScidStatus.Final = {
|
||||
assert(getChannelState(node1, channelId) == NORMAL)
|
||||
val data1Before = getChannelData(node1, channelId).asInstanceOf[DATA_NORMAL]
|
||||
val fundingTxid = data1Before.commitments.commitInput.outPoint.txid
|
||||
val fundingTxid = data1Before.commitments.fundingTxId
|
||||
val fundingTx = node1.wallet.funded(fundingTxid)
|
||||
|
||||
val watch1 = node1.watcher.fishForMessage() { case w: WatchFundingDeeplyBuried if w.txId == fundingTx.txid => true; case _ => false }.asInstanceOf[WatchFundingDeeplyBuried]
|
||||
|
|
|
@ -55,13 +55,17 @@ trait Channel {
|
|||
if (!channelTypeOk) {
|
||||
reject(MalformedFormFieldRejection("channelType", s"Channel type not supported: must be one of ${supportedChannelTypes.keys.mkString(",")}"))
|
||||
} else {
|
||||
complete {
|
||||
eclairApi.open(nodeId, fundingSatoshis, pushMsat, channelType_opt, fundingFeerateSatByte, announceChannel_opt, openTimeout_opt)
|
||||
}
|
||||
complete(eclairApi.open(nodeId, fundingSatoshis, pushMsat, channelType_opt, fundingFeerateSatByte, announceChannel_opt, openTimeout_opt))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
val rbfOpen: Route = postRequest("rbfopen") { implicit f =>
|
||||
formFields(channelIdFormParam, "targetFeerateSatByte".as[FeeratePerByte], "lockTime".as[Long] ?) {
|
||||
(channelId, targetFeerateSatByte, lockTime_opt) => complete(eclairApi.rbfOpen(channelId, FeeratePerKw(targetFeerateSatByte), lockTime_opt))
|
||||
}
|
||||
}
|
||||
|
||||
val close: Route = postRequest("close") { implicit t =>
|
||||
withChannelsIdentifier { channels =>
|
||||
formFields("scriptPubKey".as[ByteVector](binaryDataUnmarshaller).?, "preferredFeerateSatByte".as[FeeratePerByte].?, "minFeerateSatByte".as[FeeratePerByte].?, "maxFeerateSatByte".as[FeeratePerByte].?) {
|
||||
|
@ -119,6 +123,6 @@ trait Channel {
|
|||
complete(eclairApi.channelBalances())
|
||||
}
|
||||
|
||||
val channelRoutes: Route = open ~ close ~ forceClose ~ channel ~ channels ~ allChannels ~ allUpdates ~ channelStats ~ channelBalances
|
||||
val channelRoutes: Route = open ~ rbfOpen ~ close ~ forceClose ~ channel ~ channels ~ allChannels ~ allUpdates ~ channelStats ~ channelBalances
|
||||
|
||||
}
|
||||
|
|
Loading…
Add table
Reference in a new issue