mirror of
https://github.com/ACINQ/eclair.git
synced 2025-02-22 14:22:39 +01:00
Store channel state after sending commit_sig
(#2614)
We must store the channel state after sending our `tx_signatures`, because our peer may broadcast the transaction at that point. But it's useful to store it earlier than that, to allow resuming the signatures exchange if we get disconnected, otherwise we could end up in a state where one peer has forgotten the channel while the other has sent `tx_signatures` and must thus wait for the channel to be spent or double-spent. With that change, we can cleanly handle any disconnection: - if we get disconnected before any peer sent `commitment_signed`, the channel is forgotten by both peers - if we get disconnected after one peer send `commitment_signed` but the other peer didn't, the channel will be forgotten when reconnecting (this is safe because the peer that sent `commitment_signed` did *not* send `tx_signatures` since the other peer didn't send `commitment_signed`) - if we get disconnected after both peers sent `commitment_signed`, we simply resume the signatures exchange on reconnection We introduce a new TLV field to `channel_reestablish` that contains the `txid` of the funding transaction that is partially signed: this lets peers figure out which messages to send back on reconnection.
This commit is contained in:
parent
db15beb015
commit
3a95a7deb5
36 changed files with 1690 additions and 1190 deletions
|
@ -16,7 +16,7 @@
|
|||
|
||||
package fr.acinq.eclair
|
||||
|
||||
import fr.acinq.eclair.channel.PersistentChannelData
|
||||
import fr.acinq.eclair.channel.{ChannelDataWithCommitments, PersistentChannelData}
|
||||
import grizzled.slf4j.Logging
|
||||
|
||||
import scala.util.{Failure, Success, Try}
|
||||
|
@ -31,7 +31,10 @@ object DBChecker extends Logging {
|
|||
def checkChannelsDB(nodeParams: NodeParams): Seq[PersistentChannelData] = {
|
||||
Try(nodeParams.db.channels.listLocalChannels()) match {
|
||||
case Success(channels) =>
|
||||
channels.foreach(data => if (!data.commitments.validateSeed(nodeParams.channelKeyManager)) throw InvalidChannelSeedException(data.channelId))
|
||||
channels.foreach {
|
||||
case data: ChannelDataWithCommitments if !data.commitments.validateSeed(nodeParams.channelKeyManager) => throw InvalidChannelSeedException(data.channelId)
|
||||
case _ => ()
|
||||
}
|
||||
channels
|
||||
case Failure(_) => throw IncompatibleDBException
|
||||
}
|
||||
|
|
|
@ -207,6 +207,7 @@ object CheckBalance {
|
|||
.foldLeft(OffChainBalance()) {
|
||||
case (r, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) => r.modify(_.waitForFundingConfirmed).using(updateMainBalance(d.commitments.latest.localCommit))
|
||||
case (r, d: DATA_WAIT_FOR_CHANNEL_READY) => r.modify(_.waitForChannelReady).using(updateMainBalance(d.commitments.latest.localCommit))
|
||||
case (r, d: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED) => r // we ignore our balance from unsigned commitments
|
||||
case (r, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => r.modify(_.waitForFundingConfirmed).using(updateMainBalance(d.commitments.latest.localCommit))
|
||||
case (r, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) => r.modify(_.waitForChannelReady).using(updateMainBalance(d.commitments.latest.localCommit))
|
||||
case (r, d: DATA_NORMAL) => r.modify(_.normal).using(updateMainAndHtlcBalance(d.commitments, knownPreimages))
|
||||
|
|
|
@ -21,13 +21,13 @@ import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
|||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, DeterministicWallet, OutPoint, Satoshi, Transaction}
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.eclair.channel.LocalFundingStatus.DualFundedUnconfirmedFundingTx
|
||||
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder
|
||||
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder._
|
||||
import fr.acinq.eclair.io.Peer
|
||||
import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession}
|
||||
import fr.acinq.eclair.payment.OutgoingPaymentPacket.Upstream
|
||||
import fr.acinq.eclair.transactions.CommitmentSpec
|
||||
import fr.acinq.eclair.transactions.Transactions._
|
||||
import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelReady, ChannelReestablish, ChannelUpdate, ClosingSigned, FailureMessage, FundingCreated, FundingSigned, Init, OnionRoutingPacket, OpenChannel, OpenDualFundedChannel, Shutdown, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc}
|
||||
import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelReady, ChannelReestablish, ChannelUpdate, ClosingSigned, CommitSig, FailureMessage, FundingCreated, FundingSigned, Init, OnionRoutingPacket, OpenChannel, OpenDualFundedChannel, Shutdown, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc}
|
||||
import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Features, InitFeature, MilliSatoshi, RealShortChannelId, UInt64}
|
||||
import scodec.bits.ByteVector
|
||||
|
||||
|
@ -66,6 +66,7 @@ case object WAIT_FOR_INIT_DUAL_FUNDED_CHANNEL extends ChannelState
|
|||
case object WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL extends ChannelState
|
||||
case object WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL extends ChannelState
|
||||
case object WAIT_FOR_DUAL_FUNDING_CREATED extends ChannelState
|
||||
case object WAIT_FOR_DUAL_FUNDING_SIGNED extends ChannelState
|
||||
case object WAIT_FOR_DUAL_FUNDING_CONFIRMED extends ChannelState
|
||||
case object WAIT_FOR_DUAL_FUNDING_READY extends ChannelState
|
||||
// Channel opened:
|
||||
|
@ -429,7 +430,8 @@ sealed trait RbfStatus
|
|||
object RbfStatus {
|
||||
case object NoRbf extends RbfStatus
|
||||
case class RbfRequested(cmd: CMD_BUMP_FUNDING_FEE) extends RbfStatus
|
||||
case class RbfInProgress(cmd_opt: Option[CMD_BUMP_FUNDING_FEE], rbf: typed.ActorRef[InteractiveTxBuilder.Command]) extends RbfStatus
|
||||
case class RbfInProgress(cmd_opt: Option[CMD_BUMP_FUNDING_FEE], rbf: typed.ActorRef[InteractiveTxBuilder.Command], remoteCommitSig: Option[CommitSig]) extends RbfStatus
|
||||
case class RbfWaitingForSigs(signingSession: InteractiveTxSigningSession.WaitingForSigs) extends RbfStatus
|
||||
case object RbfAborted extends RbfStatus
|
||||
}
|
||||
|
||||
|
@ -444,7 +446,16 @@ case object Nothing extends TransientChannelData {
|
|||
}
|
||||
|
||||
sealed trait PersistentChannelData extends ChannelData {
|
||||
def remoteNodeId: PublicKey
|
||||
}
|
||||
sealed trait ChannelDataWithoutCommitments extends PersistentChannelData {
|
||||
val channelId: ByteVector32 = channelParams.channelId
|
||||
val remoteNodeId: PublicKey = channelParams.remoteNodeId
|
||||
def channelParams: ChannelParams
|
||||
}
|
||||
sealed trait ChannelDataWithCommitments extends PersistentChannelData {
|
||||
val channelId: ByteVector32 = commitments.channelId
|
||||
val remoteNodeId: PublicKey = commitments.remoteNodeId
|
||||
def commitments: Commitments
|
||||
}
|
||||
|
||||
|
@ -482,11 +493,10 @@ final case class DATA_WAIT_FOR_FUNDING_SIGNED(params: ChannelParams,
|
|||
final case class DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments: Commitments,
|
||||
waitingSince: BlockHeight, // how long have we been waiting for the funding tx to confirm
|
||||
deferred: Option[ChannelReady],
|
||||
lastSent: Either[FundingCreated, FundingSigned]) extends PersistentChannelData {
|
||||
lastSent: Either[FundingCreated, FundingSigned]) extends ChannelDataWithCommitments {
|
||||
def fundingTx_opt: Option[Transaction] = commitments.latest.localFundingStatus.signedTx_opt
|
||||
}
|
||||
final case class DATA_WAIT_FOR_CHANNEL_READY(commitments: Commitments,
|
||||
shortIds: ShortIds) extends PersistentChannelData
|
||||
final case class DATA_WAIT_FOR_CHANNEL_READY(commitments: Commitments, shortIds: ShortIds) extends ChannelDataWithCommitments
|
||||
|
||||
final case class DATA_WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL(init: INPUT_INIT_CHANNEL_NON_INITIATOR) extends TransientChannelData {
|
||||
val channelId: ByteVector32 = init.temporaryChannelId
|
||||
|
@ -500,21 +510,26 @@ final case class DATA_WAIT_FOR_DUAL_FUNDING_CREATED(channelId: ByteVector32,
|
|||
localPushAmount: MilliSatoshi,
|
||||
remotePushAmount: MilliSatoshi,
|
||||
txBuilder: typed.ActorRef[InteractiveTxBuilder.Command],
|
||||
deferred: Option[ChannelReady],
|
||||
deferred: Option[CommitSig],
|
||||
replyTo_opt: Option[akka.actor.typed.ActorRef[Peer.OpenChannelResponse]]) extends TransientChannelData
|
||||
final case class DATA_WAIT_FOR_DUAL_FUNDING_SIGNED(channelParams: ChannelParams,
|
||||
secondRemotePerCommitmentPoint: PublicKey,
|
||||
localPushAmount: MilliSatoshi,
|
||||
remotePushAmount: MilliSatoshi,
|
||||
signingSession: InteractiveTxSigningSession.WaitingForSigs,
|
||||
remoteChannelData_opt: Option[ByteVector]) extends ChannelDataWithoutCommitments
|
||||
final case class DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments: Commitments,
|
||||
localPushAmount: MilliSatoshi,
|
||||
remotePushAmount: MilliSatoshi,
|
||||
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
|
||||
rbfStatus: RbfStatus,
|
||||
deferred: Option[ChannelReady]) extends PersistentChannelData {
|
||||
deferred: Option[ChannelReady]) extends ChannelDataWithCommitments {
|
||||
def allFundingTxs: Seq[DualFundedUnconfirmedFundingTx] = commitments.active.map(_.localFundingStatus).collect { case fundingTx: DualFundedUnconfirmedFundingTx => fundingTx }
|
||||
def latestFundingTx: DualFundedUnconfirmedFundingTx = commitments.latest.localFundingStatus.asInstanceOf[DualFundedUnconfirmedFundingTx]
|
||||
def previousFundingTxs: Seq[DualFundedUnconfirmedFundingTx] = allFundingTxs diff Seq(latestFundingTx)
|
||||
}
|
||||
final case class DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments: Commitments,
|
||||
shortIds: ShortIds) extends PersistentChannelData
|
||||
final case class DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments: Commitments, shortIds: ShortIds) extends ChannelDataWithCommitments
|
||||
|
||||
final case class DATA_NORMAL(commitments: Commitments,
|
||||
shortIds: ShortIds,
|
||||
|
@ -522,12 +537,12 @@ final case class DATA_NORMAL(commitments: Commitments,
|
|||
channelUpdate: ChannelUpdate,
|
||||
localShutdown: Option[Shutdown],
|
||||
remoteShutdown: Option[Shutdown],
|
||||
closingFeerates: Option[ClosingFeerates]) extends PersistentChannelData
|
||||
final case class DATA_SHUTDOWN(commitments: Commitments, localShutdown: Shutdown, remoteShutdown: Shutdown, closingFeerates: Option[ClosingFeerates]) extends PersistentChannelData
|
||||
closingFeerates: Option[ClosingFeerates]) extends ChannelDataWithCommitments
|
||||
final case class DATA_SHUTDOWN(commitments: Commitments, localShutdown: Shutdown, remoteShutdown: Shutdown, closingFeerates: Option[ClosingFeerates]) extends ChannelDataWithCommitments
|
||||
final case class DATA_NEGOTIATING(commitments: Commitments,
|
||||
localShutdown: Shutdown, remoteShutdown: Shutdown,
|
||||
closingTxProposed: List[List[ClosingTxProposed]], // one list for every negotiation (there can be several in case of disconnection)
|
||||
bestUnpublishedClosingTx_opt: Option[ClosingTx]) extends PersistentChannelData {
|
||||
bestUnpublishedClosingTx_opt: Option[ClosingTx]) extends ChannelDataWithCommitments {
|
||||
require(closingTxProposed.nonEmpty, "there must always be a list for the current negotiation")
|
||||
require(!commitments.params.localParams.isInitiator || closingTxProposed.forall(_.nonEmpty), "initiator must have at least one closing signature for every negotiation attempt because it initiates the closing")
|
||||
}
|
||||
|
@ -540,12 +555,12 @@ final case class DATA_CLOSING(commitments: Commitments,
|
|||
remoteCommitPublished: Option[RemoteCommitPublished] = None,
|
||||
nextRemoteCommitPublished: Option[RemoteCommitPublished] = None,
|
||||
futureRemoteCommitPublished: Option[RemoteCommitPublished] = None,
|
||||
revokedCommitPublished: List[RevokedCommitPublished] = Nil) extends PersistentChannelData {
|
||||
revokedCommitPublished: List[RevokedCommitPublished] = Nil) extends ChannelDataWithCommitments {
|
||||
val spendingTxs: List[Transaction] = mutualClosePublished.map(_.tx) ::: localCommitPublished.map(_.commitTx).toList ::: remoteCommitPublished.map(_.commitTx).toList ::: nextRemoteCommitPublished.map(_.commitTx).toList ::: futureRemoteCommitPublished.map(_.commitTx).toList ::: revokedCommitPublished.map(_.commitTx)
|
||||
require(spendingTxs.nonEmpty, "there must be at least one tx published in this state")
|
||||
}
|
||||
|
||||
final case class DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT(commitments: Commitments, remoteChannelReestablish: ChannelReestablish) extends PersistentChannelData
|
||||
final case class DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT(commitments: Commitments, remoteChannelReestablish: ChannelReestablish) extends ChannelDataWithCommitments
|
||||
|
||||
/**
|
||||
* @param initFeatures current connection features, or last features used if the channel is disconnected. Note that these
|
||||
|
|
|
@ -147,7 +147,16 @@ case class CommitTxAndRemoteSig(commitTx: CommitTx, remoteSig: ByteVector64)
|
|||
case class LocalCommit(index: Long, spec: CommitmentSpec, commitTxAndRemoteSig: CommitTxAndRemoteSig, htlcTxsAndRemoteSigs: List[HtlcTxAndRemoteSig])
|
||||
|
||||
/** The remote commitment maps to a commitment transaction that only our peer can sign and broadcast. */
|
||||
case class RemoteCommit(index: Long, spec: CommitmentSpec, txid: ByteVector32, remotePerCommitmentPoint: PublicKey)
|
||||
case class RemoteCommit(index: Long, spec: CommitmentSpec, txid: ByteVector32, remotePerCommitmentPoint: PublicKey) {
|
||||
def sign(keyManager: ChannelKeyManager, params: ChannelParams, commitInput: InputInfo): CommitSig = {
|
||||
val (remoteCommitTx, htlcTxs) = Commitment.makeRemoteTxs(keyManager, params.channelConfig, params.channelFeatures, index, params.localParams, params.remoteParams, commitInput, remotePerCommitmentPoint, spec)
|
||||
val sig = keyManager.sign(remoteCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath), TxOwner.Remote, params.commitmentFormat)
|
||||
val channelKeyPath = keyManager.keyPath(params.localParams, params.channelConfig)
|
||||
val sortedHtlcTxs = htlcTxs.sortBy(_.input.outPoint.index)
|
||||
val htlcSigs = sortedHtlcTxs.map(keyManager.sign(_, keyManager.htlcPoint(channelKeyPath), remotePerCommitmentPoint, TxOwner.Remote, params.commitmentFormat))
|
||||
CommitSig(params.channelId, sig, htlcSigs.toList, TlvStream(CommitSigTlv.FundingTxIdTlv(commitInput.outPoint.txid)))
|
||||
}
|
||||
}
|
||||
|
||||
/** We have the next remote commit when we've sent our commit_sig but haven't yet received their revoke_and_ack. */
|
||||
case class NextRemoteCommit(sig: CommitSig, commit: RemoteCommit)
|
||||
|
|
|
@ -52,6 +52,7 @@ object Helpers {
|
|||
def updateFeatures(data: PersistentChannelData, localInit: Init, remoteInit: Init): PersistentChannelData = {
|
||||
data match {
|
||||
case d: DATA_WAIT_FOR_FUNDING_CONFIRMED => d.modify(_.commitments.params).using(_.updateFeatures(localInit, remoteInit))
|
||||
case d: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED => d.modify(_.channelParams).using(_.updateFeatures(localInit, remoteInit))
|
||||
case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => d.modify(_.commitments.params).using(_.updateFeatures(localInit, remoteInit))
|
||||
case d: DATA_WAIT_FOR_CHANNEL_READY => d.modify(_.commitments.params).using(_.updateFeatures(localInit, remoteInit))
|
||||
case d: DATA_WAIT_FOR_DUAL_FUNDING_READY => d.modify(_.commitments.params).using(_.updateFeatures(localInit, remoteInit))
|
||||
|
@ -581,7 +582,10 @@ object Helpers {
|
|||
*
|
||||
* @return true if channel was never open, or got closed immediately, had never any htlcs and local never had a positive balance
|
||||
*/
|
||||
def nothingAtStake(data: PersistentChannelData): Boolean = data.commitments.active.forall(nothingAtStake)
|
||||
def nothingAtStake(data: PersistentChannelData): Boolean = data match {
|
||||
case _: ChannelDataWithoutCommitments => true
|
||||
case data: ChannelDataWithCommitments => data.commitments.active.forall(nothingAtStake)
|
||||
}
|
||||
|
||||
def nothingAtStake(commitment: Commitment): Boolean =
|
||||
commitment.localCommit.index == 0 &&
|
||||
|
|
|
@ -35,6 +35,7 @@ import fr.acinq.eclair.channel.Helpers.{Closing, Syncing, getRelayFees, scidForC
|
|||
import fr.acinq.eclair.channel.Monitoring.Metrics.ProcessMessage
|
||||
import fr.acinq.eclair.channel.Monitoring.{Metrics, Tags}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder
|
||||
import fr.acinq.eclair.channel.publish.TxPublisher
|
||||
import fr.acinq.eclair.channel.publish.TxPublisher.{PublishFinalTx, SetChannelId}
|
||||
import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
|
||||
|
@ -248,33 +249,36 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
|
||||
// we watch all unconfirmed funding txs, whatever our state is
|
||||
// (there can be multiple funding txs due to rbf, and they can be unconfirmed in any state due to zero-conf)
|
||||
data.commitments.active.foreach { commitment =>
|
||||
commitment.localFundingStatus match {
|
||||
case _: LocalFundingStatus.SingleFundedUnconfirmedFundingTx =>
|
||||
// NB: in the case of legacy single-funded channels, the funding tx may actually be confirmed already (and
|
||||
// the channel fully operational). We could have set a specific Unknown status, but it would have forced
|
||||
// us to keep it forever. Instead, we just put a watch which, if the funding tx was indeed confirmed, will
|
||||
// trigger instantly, and the state will be updated and a watch-spent will be set. This will only happen
|
||||
// once, because at the next restore, the status of the funding tx will be "confirmed".
|
||||
blockchain ! GetTxWithMeta(self, commitment.fundingTxId)
|
||||
watchFundingConfirmed(commitment.fundingTxId, Some(singleFundingMinDepth(data)))
|
||||
case fundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx =>
|
||||
publishFundingTx(fundingTx)
|
||||
watchFundingConfirmed(fundingTx.sharedTx.txId, fundingTx.fundingParams.minDepth_opt)
|
||||
case fundingTx: LocalFundingStatus.ZeroconfPublishedFundingTx =>
|
||||
// those are zero-conf channels, the min-depth isn't critical, we use the default
|
||||
watchFundingConfirmed(fundingTx.tx.txid, Some(nodeParams.channelConf.minDepthBlocks.toLong))
|
||||
case _: LocalFundingStatus.ConfirmedFundingTx =>
|
||||
data match {
|
||||
case closing: DATA_CLOSING if Closing.nothingAtStake(closing) || Closing.isClosingTypeAlreadyKnown(closing).isDefined =>
|
||||
// no need to do anything
|
||||
()
|
||||
case closing: DATA_CLOSING =>
|
||||
// in all other cases we need to be ready for any type of closing
|
||||
watchFundingSpent(commitment, closing.spendingTxs.map(_.txid).toSet)
|
||||
case _ =>
|
||||
watchFundingSpent(commitment)
|
||||
}
|
||||
data match {
|
||||
case _: ChannelDataWithoutCommitments => ()
|
||||
case data: ChannelDataWithCommitments => data.commitments.active.foreach { commitment =>
|
||||
commitment.localFundingStatus match {
|
||||
case _: LocalFundingStatus.SingleFundedUnconfirmedFundingTx =>
|
||||
// NB: in the case of legacy single-funded channels, the funding tx may actually be confirmed already (and
|
||||
// the channel fully operational). We could have set a specific Unknown status, but it would have forced
|
||||
// us to keep it forever. Instead, we just put a watch which, if the funding tx was indeed confirmed, will
|
||||
// trigger instantly, and the state will be updated and a watch-spent will be set. This will only happen
|
||||
// once, because at the next restore, the status of the funding tx will be "confirmed".
|
||||
blockchain ! GetTxWithMeta(self, commitment.fundingTxId)
|
||||
watchFundingConfirmed(commitment.fundingTxId, Some(singleFundingMinDepth(data)))
|
||||
case fundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx =>
|
||||
publishFundingTx(fundingTx)
|
||||
watchFundingConfirmed(fundingTx.sharedTx.txId, fundingTx.fundingParams.minDepth_opt)
|
||||
case fundingTx: LocalFundingStatus.ZeroconfPublishedFundingTx =>
|
||||
// those are zero-conf channels, the min-depth isn't critical, we use the default
|
||||
watchFundingConfirmed(fundingTx.tx.txid, Some(nodeParams.channelConf.minDepthBlocks.toLong))
|
||||
case _: LocalFundingStatus.ConfirmedFundingTx =>
|
||||
data match {
|
||||
case closing: DATA_CLOSING if Closing.nothingAtStake(closing) || Closing.isClosingTypeAlreadyKnown(closing).isDefined =>
|
||||
// no need to do anything
|
||||
()
|
||||
case closing: DATA_CLOSING =>
|
||||
// in all other cases we need to be ready for any type of closing
|
||||
watchFundingSpent(commitment, closing.spendingTxs.map(_.txid).toSet)
|
||||
case _ =>
|
||||
watchFundingSpent(commitment)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -319,7 +323,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
context.system.eventStream.publish(ShortChannelIdAssigned(self, normal.channelId, normal.shortIds, remoteNodeId))
|
||||
|
||||
// we check the configuration because the values for channel_update may have changed while eclair was down
|
||||
val fees = getRelayFees(nodeParams, remoteNodeId, data.commitments.announceChannel)
|
||||
val fees = getRelayFees(nodeParams, remoteNodeId, normal.commitments.announceChannel)
|
||||
if (fees.feeBase != normal.channelUpdate.feeBaseMsat ||
|
||||
fees.feeProportionalMillionths != normal.channelUpdate.feeProportionalMillionths ||
|
||||
nodeParams.channelConf.expiryDelta != normal.channelUpdate.cltvExpiryDelta) {
|
||||
|
@ -1091,7 +1095,6 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
// Force-closing is our only option here, if we are in this state the channel was closing and it is too late
|
||||
// to negotiate a mutual close.
|
||||
log.info("channelId={} was confirmed at blockHeight={} txIndex={} with a previous funding txid={}", d.channelId, w.blockHeight, w.txIndex, w.tx.txid)
|
||||
val commitment = commitments1.latest
|
||||
val d1 = d.copy(commitments = commitments1)
|
||||
spendLocalCurrent(d1)
|
||||
}
|
||||
|
@ -1273,30 +1276,52 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
val d1 = Helpers.updateFeatures(d, localInit, remoteInit)
|
||||
goto(WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT) using d1 sending error
|
||||
|
||||
case Event(INPUT_RECONNECTED(r, localInit, remoteInit), d: PersistentChannelData) =>
|
||||
case Event(INPUT_RECONNECTED(r, localInit, remoteInit), d: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED) =>
|
||||
activeConnection = r
|
||||
val channelKeyPath = keyManager.keyPath(d.channelParams.localParams, d.channelParams.channelConfig)
|
||||
val myFirstPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 0)
|
||||
val channelReestablish = ChannelReestablish(
|
||||
channelId = d.channelId,
|
||||
nextLocalCommitmentNumber = 1,
|
||||
nextRemoteRevocationNumber = 0,
|
||||
yourLastPerCommitmentSecret = PrivateKey(ByteVector32.Zeroes),
|
||||
myCurrentPerCommitmentPoint = myFirstPerCommitmentPoint,
|
||||
TlvStream(ChannelReestablishTlv.NextFundingTlv(d.signingSession.fundingTx.txId.reverse)),
|
||||
)
|
||||
val d1 = Helpers.updateFeatures(d, localInit, remoteInit)
|
||||
goto(SYNCING) using d1 sending channelReestablish
|
||||
|
||||
case Event(INPUT_RECONNECTED(r, localInit, remoteInit), d: ChannelDataWithCommitments) =>
|
||||
activeConnection = r
|
||||
val remotePerCommitmentSecrets = d.commitments.remotePerCommitmentSecrets
|
||||
val yourLastPerCommitmentSecret = remotePerCommitmentSecrets.lastIndex.flatMap(remotePerCommitmentSecrets.getHash).getOrElse(ByteVector32.Zeroes)
|
||||
val channelKeyPath = keyManager.keyPath(d.commitments.params.localParams, d.commitments.params.channelConfig)
|
||||
val myCurrentPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, d.commitments.localCommitIndex)
|
||||
|
||||
val tlvs: TlvStream[ChannelReestablishTlv] = d match {
|
||||
case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => d.rbfStatus match {
|
||||
case RbfStatus.RbfWaitingForSigs(status) => TlvStream(ChannelReestablishTlv.NextFundingTlv(status.fundingTx.txId.reverse))
|
||||
case _ => d.latestFundingTx.sharedTx match {
|
||||
case _: InteractiveTxBuilder.PartiallySignedSharedTransaction => TlvStream(ChannelReestablishTlv.NextFundingTlv(d.latestFundingTx.sharedTx.txId.reverse))
|
||||
case _: InteractiveTxBuilder.FullySignedSharedTransaction => TlvStream.empty
|
||||
}
|
||||
}
|
||||
case _ => TlvStream.empty
|
||||
}
|
||||
val channelReestablish = ChannelReestablish(
|
||||
channelId = d.channelId,
|
||||
nextLocalCommitmentNumber = d.commitments.localCommitIndex + 1,
|
||||
nextRemoteRevocationNumber = d.commitments.remoteCommitIndex,
|
||||
yourLastPerCommitmentSecret = PrivateKey(yourLastPerCommitmentSecret),
|
||||
myCurrentPerCommitmentPoint = myCurrentPerCommitmentPoint
|
||||
myCurrentPerCommitmentPoint = myCurrentPerCommitmentPoint,
|
||||
tlvStream = tlvs
|
||||
)
|
||||
|
||||
// we update local/remote connection-local global/local features, we don't persist it right now
|
||||
val d1 = Helpers.updateFeatures(d, localInit, remoteInit)
|
||||
|
||||
goto(SYNCING) using d1 sending channelReestablish
|
||||
|
||||
case Event(ProcessCurrentBlockHeight(c), d: PersistentChannelData) => handleNewBlock(c, d)
|
||||
case Event(ProcessCurrentBlockHeight(c), d: ChannelDataWithCommitments) => handleNewBlock(c, d)
|
||||
|
||||
case Event(c: CurrentFeerates, d: PersistentChannelData) => handleCurrentFeerateDisconnected(c, d)
|
||||
case Event(c: CurrentFeerates, d: ChannelDataWithCommitments) => handleCurrentFeerateDisconnected(c, d)
|
||||
|
||||
case Event(c: CMD_ADD_HTLC, d: DATA_NORMAL) => handleAddDisconnected(c, d)
|
||||
|
||||
|
@ -1318,8 +1343,41 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
case Event(_: ChannelReestablish, _: DATA_WAIT_FOR_FUNDING_CONFIRMED) =>
|
||||
goto(WAIT_FOR_FUNDING_CONFIRMED)
|
||||
|
||||
case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) sending d.latestFundingTx.sharedTx.localSigs
|
||||
case Event(channelReestablish: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED) =>
|
||||
channelReestablish.nextFundingTxId_opt match {
|
||||
case Some(fundingTxId) if fundingTxId == d.signingSession.fundingTx.txId =>
|
||||
// We retransmit our commit_sig, and will send our tx_signatures once we've received their commit_sig.
|
||||
val commitSig = d.signingSession.remoteCommit.sign(keyManager, d.channelParams, d.signingSession.commitInput)
|
||||
goto(WAIT_FOR_DUAL_FUNDING_SIGNED) sending commitSig
|
||||
case _ => goto(WAIT_FOR_DUAL_FUNDING_SIGNED)
|
||||
}
|
||||
|
||||
case Event(channelReestablish: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
channelReestablish.nextFundingTxId_opt match {
|
||||
case Some(fundingTxId) =>
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.RbfWaitingForSigs(status) if status.fundingTx.txId == fundingTxId =>
|
||||
// We retransmit our commit_sig, and will send our tx_signatures once we've received their commit_sig.
|
||||
val commitSig = status.remoteCommit.sign(keyManager, d.commitments.params, status.commitInput)
|
||||
goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) sending commitSig
|
||||
case _ if d.latestFundingTx.sharedTx.txId == fundingTxId =>
|
||||
val toSend = d.latestFundingTx.sharedTx match {
|
||||
case fundingTx: InteractiveTxBuilder.PartiallySignedSharedTransaction =>
|
||||
// We have not received their tx_signatures: we retransmit our commit_sig because we don't know if they received it.
|
||||
val commitSig = d.commitments.latest.remoteCommit.sign(keyManager, d.commitments.params, d.commitments.latest.commitInput)
|
||||
Seq(commitSig, fundingTx.localSigs)
|
||||
case fundingTx: InteractiveTxBuilder.FullySignedSharedTransaction =>
|
||||
// We've already received their tx_signatures, which means they've received and stored our commit_sig, we only need to retransmit our tx_signatures.
|
||||
Seq(fundingTx.localSigs)
|
||||
}
|
||||
goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) sending toSend
|
||||
case _ =>
|
||||
// The fundingTxId must be for an RBF attempt that we didn't store (we got disconnected before receiving
|
||||
// their tx_complete): we tell them to abort that RBF attempt.
|
||||
goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage)
|
||||
}
|
||||
case None => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
}
|
||||
|
||||
case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_CHANNEL_READY) =>
|
||||
log.debug("re-sending channelReady")
|
||||
|
@ -1461,9 +1519,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
context.system.scheduler.scheduleOnce(5 seconds, self, remoteAnnSigs)
|
||||
stay() sending Warning(d.channelId, "spec violation: you sent announcement_signatures before channel_reestablish")
|
||||
|
||||
case Event(ProcessCurrentBlockHeight(c), d: PersistentChannelData) => handleNewBlock(c, d)
|
||||
case Event(ProcessCurrentBlockHeight(c), d: ChannelDataWithCommitments) => handleNewBlock(c, d)
|
||||
|
||||
case Event(c: CurrentFeerates, d: PersistentChannelData) => handleCurrentFeerateDisconnected(c, d)
|
||||
case Event(c: CurrentFeerates, d: ChannelDataWithCommitments) => handleCurrentFeerateDisconnected(c, d)
|
||||
|
||||
case Event(getTxResponse: GetTxWithMetaResponse, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) if getTxResponse.txid == d.commitments.latest.fundingTxId => handleGetFundingTx(getTxResponse, d.waitingSince, d.fundingTx_opt)
|
||||
|
||||
|
@ -1566,7 +1624,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
// peer doesn't cancel the timer
|
||||
case Event(TickChannelOpenTimeout, _) => stay()
|
||||
|
||||
case Event(w: WatchPublishedTriggered, d: PersistentChannelData) =>
|
||||
case Event(w: WatchPublishedTriggered, d: ChannelDataWithCommitments) =>
|
||||
// When our peer sends us channel_ready while we're still waiting for confirmations, we may opportunistically
|
||||
// switch to zero-conf, in which case we have both a WatchPublished and a WatchConfirmed pending. But it may not
|
||||
// actually be a real switch to zero-conf: maybe the transaction is confirmed, and they simply received the block
|
||||
|
@ -1605,7 +1663,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
}
|
||||
}
|
||||
|
||||
case Event(w: WatchFundingConfirmedTriggered, d: PersistentChannelData) =>
|
||||
case Event(w: WatchFundingConfirmedTriggered, d: ChannelDataWithCommitments) =>
|
||||
acceptFundingTxConfirmed(w, d) match {
|
||||
case Right((commitments1, commitment)) =>
|
||||
log.info(s"funding txid=${w.tx.txid} has been confirmed")
|
||||
|
@ -1642,7 +1700,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
|
||||
case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT) => handleRemoteSpentFuture(tx, d)
|
||||
|
||||
case Event(WatchFundingSpentTriggered(tx), d: PersistentChannelData) =>
|
||||
case Event(WatchFundingSpentTriggered(tx), d: ChannelDataWithCommitments) =>
|
||||
if (tx.txid == d.commitments.latest.remoteCommit.txid) {
|
||||
handleRemoteSpentCurrent(tx, d)
|
||||
} else if (d.commitments.latest.nextRemoteCommit_opt.exists(_.commit.txid == tx.txid)) {
|
||||
|
@ -1660,7 +1718,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
case state -> nextState =>
|
||||
if (state != nextState) {
|
||||
val commitments_opt = nextStateData match {
|
||||
case d: PersistentChannelData => Some(d.commitments)
|
||||
case d: ChannelDataWithCommitments => Some(d.commitments)
|
||||
case _: ChannelDataWithoutCommitments => None
|
||||
case _: TransientChannelData => None
|
||||
}
|
||||
context.system.eventStream.publish(ChannelStateChanged(self, nextStateData.channelId, peer, remoteNodeId, state, nextState, commitments_opt))
|
||||
|
@ -1731,7 +1790,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
(stateData, nextState) match {
|
||||
case (_: TransientChannelData, CLOSING | CLOSED) => context.system.eventStream.publish(ChannelAborted(self, remoteNodeId, stateData.channelId))
|
||||
case (_: DATA_WAIT_FOR_FUNDING_CONFIRMED | _: DATA_WAIT_FOR_CHANNEL_READY, CLOSING | CLOSED) => context.system.eventStream.publish(ChannelAborted(self, remoteNodeId, stateData.channelId))
|
||||
case (_: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED | _: DATA_WAIT_FOR_DUAL_FUNDING_READY, CLOSING | CLOSED) => context.system.eventStream.publish(ChannelAborted(self, remoteNodeId, stateData.channelId))
|
||||
case (_: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED | _: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED | _: DATA_WAIT_FOR_DUAL_FUNDING_READY, CLOSING | CLOSED) => context.system.eventStream.publish(ChannelAborted(self, remoteNodeId, stateData.channelId))
|
||||
case _ => ()
|
||||
}
|
||||
}
|
||||
|
@ -1786,7 +1845,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
888 888 d88P 888 888 Y888 8888888P" 88888888 8888888888 888 T88b "Y8888P"
|
||||
*/
|
||||
|
||||
private def handleCurrentFeerate(c: CurrentFeerates, d: PersistentChannelData) = {
|
||||
private def handleCurrentFeerate(c: CurrentFeerates, d: ChannelDataWithCommitments) = {
|
||||
// TODO: we should consider *all* commitments
|
||||
val commitments = d.commitments.latest
|
||||
val networkFeeratePerKw = nodeParams.onChainFeeConf.getCommitmentFeerate(remoteNodeId, d.commitments.params.channelType, commitments.capacity, Some(c))
|
||||
|
@ -1812,7 +1871,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
* @param d the channel commtiments
|
||||
* @return
|
||||
*/
|
||||
private def handleCurrentFeerateDisconnected(c: CurrentFeerates, d: PersistentChannelData) = {
|
||||
private def handleCurrentFeerateDisconnected(c: CurrentFeerates, d: ChannelDataWithCommitments) = {
|
||||
// TODO: we should consider *all* commitments
|
||||
val commitments = d.commitments.latest
|
||||
val networkFeeratePerKw = nodeParams.onChainFeeConf.getCommitmentFeerate(remoteNodeId, d.commitments.params.channelType, commitments.capacity, Some(c))
|
||||
|
@ -1864,7 +1923,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
stay()
|
||||
}
|
||||
|
||||
private def handleRevocationTimeout(revocationTimeout: RevocationTimeout, d: PersistentChannelData) = {
|
||||
private def handleRevocationTimeout(revocationTimeout: RevocationTimeout, d: ChannelDataWithCommitments) = {
|
||||
d.commitments.remoteNextCommitInfo match {
|
||||
case Left(_) if revocationTimeout.remoteCommitNumber + 1 == d.commitments.nextRemoteCommitIndex =>
|
||||
log.warning(s"waited for too long for a revocation to remoteCommitNumber=${revocationTimeout.remoteCommitNumber}, disconnecting")
|
||||
|
@ -1906,7 +1965,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
stay() using d.copy(channelUpdate = channelUpdate1) storing()
|
||||
}
|
||||
|
||||
private def handleSyncFailure(channelReestablish: ChannelReestablish, syncFailure: SyncResult.Failure, d: PersistentChannelData) = {
|
||||
private def handleSyncFailure(channelReestablish: ChannelReestablish, syncFailure: SyncResult.Failure, d: ChannelDataWithCommitments) = {
|
||||
syncFailure match {
|
||||
case res: SyncResult.LocalLateProven =>
|
||||
log.error(s"counterparty proved that we have an outdated (revoked) local commitment!!! ourLocalCommitmentNumber=${res.ourLocalCommitmentNumber} theirRemoteCommitmentNumber=${res.theirRemoteCommitmentNumber}")
|
||||
|
@ -1935,7 +1994,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
|||
}
|
||||
}
|
||||
|
||||
private def handleNewBlock(c: CurrentBlockHeight, d: PersistentChannelData) = {
|
||||
private def handleNewBlock(c: CurrentBlockHeight, d: ChannelDataWithCommitments) = {
|
||||
d match {
|
||||
case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => handleNewBlockDualFundingUnconfirmed(c, d)
|
||||
case _ =>
|
||||
|
|
|
@ -23,8 +23,8 @@ import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
|
|||
import fr.acinq.eclair.channel.Helpers.Funding
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.fsm.Channel._
|
||||
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder
|
||||
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTransaction, InteractiveTxParams, PartiallySignedSharedTransaction, RequireConfirmedInputs}
|
||||
import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession}
|
||||
import fr.acinq.eclair.channel.publish.TxPublisher.SetChannelId
|
||||
import fr.acinq.eclair.crypto.ShaChain
|
||||
import fr.acinq.eclair.io.Peer.OpenChannelResponse
|
||||
|
@ -57,7 +57,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
|-------------------------------->|
|
||||
| tx_complete |
|
||||
|<--------------------------------|
|
||||
| |
|
||||
WAIT_FOR_DUAL_FUNDING_SIGNED | | WAIT_FOR_DUAL_FUNDING_SIGNED
|
||||
| commitment_signed |
|
||||
|-------------------------------->|
|
||||
| commitment_signed |
|
||||
|
@ -311,16 +311,18 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
case Event(msg: InteractiveTxMessage, d: DATA_WAIT_FOR_DUAL_FUNDING_CREATED) =>
|
||||
msg match {
|
||||
case msg: InteractiveTxConstructionMessage =>
|
||||
d.txBuilder ! InteractiveTxBuilder.ReceiveTxMessage(msg)
|
||||
stay()
|
||||
case msg: TxSignatures =>
|
||||
d.txBuilder ! InteractiveTxBuilder.ReceiveTxSigs(msg)
|
||||
d.txBuilder ! InteractiveTxBuilder.ReceiveMessage(msg)
|
||||
stay()
|
||||
case msg: TxAbort =>
|
||||
log.info("our peer aborted the dual funding flow: ascii='{}' bin={}", msg.toAscii, msg.data)
|
||||
d.txBuilder ! InteractiveTxBuilder.Abort
|
||||
d.replyTo_opt.foreach(_ ! OpenChannelResponse.RemoteError(msg.toAscii))
|
||||
goto(CLOSED) sending TxAbort(d.channelId, DualFundingAborted(d.channelId).getMessage)
|
||||
case _: TxSignatures =>
|
||||
log.info("received unexpected tx_signatures")
|
||||
d.txBuilder ! InteractiveTxBuilder.Abort
|
||||
d.replyTo_opt.foreach(_ ! OpenChannelResponse.Rejected(UnexpectedFundingSignatures(d.channelId).getMessage))
|
||||
goto(CLOSED) sending TxAbort(d.channelId, UnexpectedFundingSignatures(d.channelId).getMessage)
|
||||
case _: TxInitRbf =>
|
||||
log.info("ignoring unexpected tx_init_rbf message")
|
||||
stay() sending Warning(d.channelId, InvalidRbfAttempt(d.channelId).getMessage)
|
||||
|
@ -330,32 +332,16 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
}
|
||||
|
||||
case Event(commitSig: CommitSig, d: DATA_WAIT_FOR_DUAL_FUNDING_CREATED) =>
|
||||
d.txBuilder ! InteractiveTxBuilder.ReceiveCommitSig(commitSig)
|
||||
stay()
|
||||
|
||||
case Event(channelReady: ChannelReady, d: DATA_WAIT_FOR_DUAL_FUNDING_CREATED) =>
|
||||
log.debug("received their channel_ready, deferring message")
|
||||
stay() using d.copy(deferred = Some(channelReady))
|
||||
log.debug("received their commit_sig, deferring message")
|
||||
stay() using d.copy(deferred = Some(commitSig))
|
||||
|
||||
case Event(msg: InteractiveTxBuilder.Response, d: DATA_WAIT_FOR_DUAL_FUNDING_CREATED) => msg match {
|
||||
case InteractiveTxBuilder.SendMessage(msg) => stay() sending msg
|
||||
case InteractiveTxBuilder.Succeeded(fundingTx, commitment) =>
|
||||
case InteractiveTxBuilder.Succeeded(status, commitSig) =>
|
||||
d.deferred.foreach(self ! _)
|
||||
watchFundingConfirmed(fundingTx.sharedTx.txId, fundingTx.fundingParams.minDepth_opt)
|
||||
d.replyTo_opt.foreach(_ ! OpenChannelResponse.Created(d.channelId, commitment.fundingTxId, fundingTx.sharedTx.tx.localFees))
|
||||
val commitments = Commitments(
|
||||
params = d.channelParams,
|
||||
changes = CommitmentChanges.init(),
|
||||
active = List(commitment),
|
||||
remoteNextCommitInfo = Right(d.secondRemotePerCommitmentPoint),
|
||||
remotePerCommitmentSecrets = ShaChain.init,
|
||||
originChannels = Map.empty
|
||||
)
|
||||
val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, d.localPushAmount, d.remotePushAmount, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, RbfStatus.NoRbf, None)
|
||||
fundingTx.sharedTx match {
|
||||
case sharedTx: PartiallySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d1 storing() sending sharedTx.localSigs
|
||||
case sharedTx: FullySignedSharedTransaction => goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d1 storing() sending sharedTx.localSigs calling publishFundingTx(fundingTx)
|
||||
}
|
||||
d.replyTo_opt.foreach(_ ! OpenChannelResponse.Created(d.channelId, status.fundingTx.txId, status.fundingTx.tx.localFees))
|
||||
val d1 = DATA_WAIT_FOR_DUAL_FUNDING_SIGNED(d.channelParams, d.secondRemotePerCommitmentPoint, d.localPushAmount, d.remotePushAmount, status, None)
|
||||
goto(WAIT_FOR_DUAL_FUNDING_SIGNED) using d1 storing() sending commitSig
|
||||
case f: InteractiveTxBuilder.Failed =>
|
||||
d.replyTo_opt.foreach(_ ! OpenChannelResponse.Rejected(f.cause.getMessage))
|
||||
goto(CLOSED) sending TxAbort(d.channelId, f.cause.getMessage)
|
||||
|
@ -382,10 +368,86 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
goto(CLOSED)
|
||||
})
|
||||
|
||||
when(WAIT_FOR_DUAL_FUNDING_SIGNED)(handleExceptions {
|
||||
case Event(commitSig: CommitSig, d: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED) =>
|
||||
d.signingSession.receiveCommitSig(nodeParams, d.channelParams, commitSig) match {
|
||||
case Left(f) =>
|
||||
rollbackFundingAttempt(d.signingSession.fundingTx.tx, Nil)
|
||||
goto(CLOSED) sending Error(d.channelId, f.getMessage)
|
||||
case Right(signingSession1) => signingSession1 match {
|
||||
case signingSession1: InteractiveTxSigningSession.WaitingForSigs =>
|
||||
// No need to store their commit_sig, they will re-send it if we disconnect.
|
||||
stay() using d.copy(signingSession = signingSession1)
|
||||
case signingSession1: InteractiveTxSigningSession.SendingSigs =>
|
||||
// We don't have their tx_sigs, but they have ours, and could publish the funding tx without telling us.
|
||||
// That's why we move on immediately to the next step, and will update our unsigned funding tx when we
|
||||
// receive their tx_sigs.
|
||||
watchFundingConfirmed(d.signingSession.fundingTx.txId, d.signingSession.fundingParams.minDepth_opt)
|
||||
val commitments = Commitments(
|
||||
params = d.channelParams,
|
||||
changes = CommitmentChanges.init(),
|
||||
active = List(signingSession1.commitment),
|
||||
remoteNextCommitInfo = Right(d.secondRemotePerCommitmentPoint),
|
||||
remotePerCommitmentSecrets = ShaChain.init,
|
||||
originChannels = Map.empty
|
||||
)
|
||||
val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, d.localPushAmount, d.remotePushAmount, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, RbfStatus.NoRbf, None)
|
||||
goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d1 storing() sending signingSession1.localSigs
|
||||
}
|
||||
}
|
||||
|
||||
case Event(msg: InteractiveTxMessage, d: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED) =>
|
||||
msg match {
|
||||
case txSigs: TxSignatures =>
|
||||
d.signingSession.receiveTxSigs(nodeParams, txSigs) match {
|
||||
case Left(f) =>
|
||||
rollbackFundingAttempt(d.signingSession.fundingTx.tx, Nil)
|
||||
goto(CLOSED) sending Error(d.channelId, f.getMessage)
|
||||
case Right(signingSession) =>
|
||||
watchFundingConfirmed(d.signingSession.fundingTx.txId, d.signingSession.fundingParams.minDepth_opt)
|
||||
val commitments = Commitments(
|
||||
params = d.channelParams,
|
||||
changes = CommitmentChanges.init(),
|
||||
active = List(signingSession.commitment),
|
||||
remoteNextCommitInfo = Right(d.secondRemotePerCommitmentPoint),
|
||||
remotePerCommitmentSecrets = ShaChain.init,
|
||||
originChannels = Map.empty
|
||||
)
|
||||
val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments, d.localPushAmount, d.remotePushAmount, nodeParams.currentBlockHeight, nodeParams.currentBlockHeight, RbfStatus.NoRbf, None)
|
||||
goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) using d1 storing() sending signingSession.localSigs calling publishFundingTx(signingSession.fundingTx)
|
||||
}
|
||||
case msg: TxAbort =>
|
||||
log.info("our peer aborted the dual funding flow: ascii='{}' bin={}", msg.toAscii, msg.data)
|
||||
rollbackFundingAttempt(d.signingSession.fundingTx.tx, Nil)
|
||||
goto(CLOSED) sending TxAbort(d.channelId, DualFundingAborted(d.channelId).getMessage)
|
||||
case msg: InteractiveTxConstructionMessage =>
|
||||
log.info("received unexpected interactive-tx message: {}", msg.getClass.getSimpleName)
|
||||
stay() sending Warning(d.channelId, UnexpectedInteractiveTxMessage(d.channelId, msg).getMessage)
|
||||
case _: TxInitRbf =>
|
||||
log.info("ignoring unexpected tx_init_rbf message")
|
||||
stay() sending Warning(d.channelId, InvalidRbfAttempt(d.channelId).getMessage)
|
||||
case _: TxAckRbf =>
|
||||
log.info("ignoring unexpected tx_ack_rbf message")
|
||||
stay() sending Warning(d.channelId, InvalidRbfAttempt(d.channelId).getMessage)
|
||||
}
|
||||
|
||||
case Event(c: CloseCommand, d: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED) =>
|
||||
rollbackFundingAttempt(d.signingSession.fundingTx.tx, Nil)
|
||||
handleFastClose(c, d.channelId) sending Error(d.channelId, DualFundingAborted(d.channelId).getMessage)
|
||||
|
||||
case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED) =>
|
||||
rollbackFundingAttempt(d.signingSession.fundingTx.tx, Nil)
|
||||
handleRemoteError(e, d)
|
||||
|
||||
case Event(INPUT_DISCONNECTED, d: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED) =>
|
||||
// We should be able to complete the channel open when reconnecting.
|
||||
goto(OFFLINE) using d
|
||||
})
|
||||
|
||||
when(WAIT_FOR_DUAL_FUNDING_CONFIRMED)(handleExceptions {
|
||||
case Event(txSigs: TxSignatures, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
d.latestFundingTx.sharedTx match {
|
||||
case fundingTx: PartiallySignedSharedTransaction => InteractiveTxBuilder.addRemoteSigs(d.latestFundingTx.fundingParams, fundingTx, txSigs) match {
|
||||
case fundingTx: PartiallySignedSharedTransaction => InteractiveTxSigningSession.addRemoteSigs(d.latestFundingTx.fundingParams, fundingTx, txSigs) match {
|
||||
case Left(cause) =>
|
||||
val unsignedFundingTx = fundingTx.tx.buildUnsignedTx()
|
||||
log.warning("received invalid tx_signatures for txid={} (current funding txid={}): {}", txSigs.txId, unsignedFundingTx.txid, cause.getMessage)
|
||||
|
@ -400,13 +462,21 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
}
|
||||
case _: FullySignedSharedTransaction =>
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.RbfInProgress(_, txBuilder) =>
|
||||
txBuilder ! InteractiveTxBuilder.ReceiveTxSigs(txSigs)
|
||||
stay()
|
||||
case RbfStatus.RbfWaitingForSigs(signingSession) =>
|
||||
signingSession.receiveTxSigs(nodeParams, txSigs) match {
|
||||
case Left(f) =>
|
||||
rollbackRbfAttempt(signingSession, d)
|
||||
stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, f.getMessage)
|
||||
case Right(signingSession1) =>
|
||||
watchFundingConfirmed(signingSession.fundingTx.txId, signingSession.fundingParams.minDepth_opt)
|
||||
val commitments1 = d.commitments.add(signingSession1.commitment)
|
||||
val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments1, d.localPushAmount, d.remotePushAmount, d.waitingSince, d.lastChecked, RbfStatus.NoRbf, d.deferred)
|
||||
stay() using d1 storing() sending signingSession1.localSigs calling publishFundingTx(signingSession1.fundingTx)
|
||||
}
|
||||
case _ =>
|
||||
// Signatures are retransmitted on reconnection, but we may have already received them.
|
||||
log.debug("ignoring duplicate tx_signatures for txid={}", txSigs.txId)
|
||||
stay()
|
||||
log.debug("rejecting unexpected tx_signatures for txId={}", txSigs.txId)
|
||||
reportRbfFailure(d.rbfStatus, UnexpectedFundingSignatures(d.channelId))
|
||||
stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, UnexpectedFundingSignatures(d.channelId).getMessage)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -480,12 +550,12 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
Some(TxAckRbf(d.channelId, fundingParams.localAmount)),
|
||||
if (remainingRbfAttempts <= 3) Some(Warning(d.channelId, s"will accept at most ${remainingRbfAttempts - 1} future rbf attempts")) else None,
|
||||
).flatten
|
||||
stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(cmd_opt = None, txBuilder)) sending toSend
|
||||
stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(cmd_opt = None, txBuilder, remoteCommitSig = None)) sending toSend
|
||||
}
|
||||
case RbfStatus.RbfAborted =>
|
||||
log.info("rejecting rbf attempt: our previous tx_abort was not acked")
|
||||
stay() sending Warning(d.channelId, InvalidRbfTxAbortNotAcked(d.channelId).getMessage)
|
||||
case _: RbfStatus.RbfRequested | _: RbfStatus.RbfInProgress =>
|
||||
case _: RbfStatus.RbfRequested | _: RbfStatus.RbfInProgress | _: RbfStatus.RbfWaitingForSigs =>
|
||||
log.info("rejecting rbf attempt: the current rbf attempt must be completed or aborted first")
|
||||
stay() sending Warning(d.channelId, InvalidRbfAlreadyInProgress(d.channelId).getMessage)
|
||||
}
|
||||
|
@ -515,7 +585,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
localPushAmount = d.localPushAmount, remotePushAmount = d.remotePushAmount,
|
||||
wallet))
|
||||
txBuilder ! InteractiveTxBuilder.Start(self)
|
||||
stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(cmd_opt = Some(cmd), txBuilder))
|
||||
stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(cmd_opt = Some(cmd), txBuilder, remoteCommitSig = None))
|
||||
case _ =>
|
||||
log.info("ignoring unexpected tx_ack_rbf")
|
||||
stay() sending Warning(d.channelId, UnexpectedInteractiveTxMessage(d.channelId, msg).getMessage)
|
||||
|
@ -523,8 +593,8 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
|
||||
case Event(msg: InteractiveTxConstructionMessage, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.RbfInProgress(_, txBuilder) =>
|
||||
txBuilder ! InteractiveTxBuilder.ReceiveTxMessage(msg)
|
||||
case RbfStatus.RbfInProgress(_, txBuilder, _) =>
|
||||
txBuilder ! InteractiveTxBuilder.ReceiveMessage(msg)
|
||||
stay()
|
||||
case _ =>
|
||||
log.info("ignoring unexpected interactive-tx message: {}", msg.getClass.getSimpleName)
|
||||
|
@ -533,21 +603,41 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
|
||||
case Event(commitSig: CommitSig, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.RbfInProgress(_, txBuilder) =>
|
||||
txBuilder ! InteractiveTxBuilder.ReceiveCommitSig(commitSig)
|
||||
stay()
|
||||
case RbfStatus.RbfInProgress(cmd_opt, txBuilder, _) =>
|
||||
log.debug("received their commit_sig, deferring message")
|
||||
stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(cmd_opt, txBuilder, Some(commitSig)))
|
||||
case RbfStatus.RbfWaitingForSigs(signingSession) =>
|
||||
signingSession.receiveCommitSig(nodeParams, d.commitments.params, commitSig) match {
|
||||
case Left(f) =>
|
||||
rollbackRbfAttempt(signingSession, d)
|
||||
stay() using d.copy(rbfStatus = RbfStatus.RbfAborted) sending TxAbort(d.channelId, f.getMessage)
|
||||
case Right(signingSession1) => signingSession1 match {
|
||||
case signingSession1: InteractiveTxSigningSession.WaitingForSigs =>
|
||||
// No need to store their commit_sig, they will re-send it if we disconnect.
|
||||
stay() using d.copy(rbfStatus = RbfStatus.RbfWaitingForSigs(signingSession1))
|
||||
case signingSession1: InteractiveTxSigningSession.SendingSigs =>
|
||||
watchFundingConfirmed(signingSession.fundingTx.txId, signingSession.fundingParams.minDepth_opt)
|
||||
val commitments1 = d.commitments.add(signingSession1.commitment)
|
||||
val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments1, d.localPushAmount, d.remotePushAmount, d.waitingSince, d.lastChecked, RbfStatus.NoRbf, d.deferred)
|
||||
stay() using d1 storing() sending signingSession1.localSigs
|
||||
}
|
||||
}
|
||||
case _ =>
|
||||
log.info("ignoring unexpected commit_sig")
|
||||
stay() sending Warning(d.channelId, UnexpectedCommitSig(d.channelId).getMessage)
|
||||
log.info("ignoring redundant commit_sig")
|
||||
stay()
|
||||
}
|
||||
|
||||
case Event(msg: TxAbort, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.RbfInProgress(cmd_opt, txBuilder) =>
|
||||
case RbfStatus.RbfInProgress(cmd_opt, txBuilder, _) =>
|
||||
log.info("our peer aborted the rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data)
|
||||
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, RbfAttemptAborted(d.channelId)))
|
||||
txBuilder ! InteractiveTxBuilder.Abort
|
||||
stay() using d.copy(rbfStatus = RbfStatus.NoRbf) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage)
|
||||
case RbfStatus.RbfWaitingForSigs(status) =>
|
||||
log.info("our peer aborted the rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data)
|
||||
rollbackRbfAttempt(status, d)
|
||||
stay() using d.copy(rbfStatus = RbfStatus.NoRbf) sending TxAbort(d.channelId, RbfAttemptAborted(d.channelId).getMessage)
|
||||
case RbfStatus.RbfRequested(cmd) =>
|
||||
log.info("our peer rejected our rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data)
|
||||
cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException(s"rbf attempt rejected by our peer: ${msg.toAscii}"))
|
||||
|
@ -563,19 +653,14 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
|
||||
case Event(msg: InteractiveTxBuilder.Response, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.RbfInProgress(cmd_opt, _) =>
|
||||
case RbfStatus.RbfInProgress(cmd_opt, _, remoteCommitSig_opt) =>
|
||||
msg match {
|
||||
case InteractiveTxBuilder.SendMessage(msg) => stay() sending msg
|
||||
case InteractiveTxBuilder.Succeeded(fundingTx, commitment) =>
|
||||
cmd_opt.foreach(cmd => cmd.replyTo ! RES_BUMP_FUNDING_FEE(rbfIndex = d.previousFundingTxs.length, commitment.fundingTxId, fundingTx.sharedTx.tx.fees))
|
||||
watchFundingConfirmed(fundingTx.sharedTx.txId, fundingTx.fundingParams.minDepth_opt)
|
||||
// we add the latest commitments to the list
|
||||
val commitments1 = d.commitments.add(commitment)
|
||||
val d1 = DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED(commitments1, d.localPushAmount, d.remotePushAmount, d.waitingSince, d.lastChecked, RbfStatus.NoRbf, d.deferred)
|
||||
fundingTx.sharedTx match {
|
||||
case sharedTx: PartiallySignedSharedTransaction => stay() using d1 storing() sending sharedTx.localSigs
|
||||
case sharedTx: FullySignedSharedTransaction => stay() using d1 storing() sending sharedTx.localSigs calling publishFundingTx(fundingTx)
|
||||
}
|
||||
case InteractiveTxBuilder.Succeeded(status, commitSig) =>
|
||||
cmd_opt.foreach(cmd => cmd.replyTo ! RES_BUMP_FUNDING_FEE(rbfIndex = d.previousFundingTxs.length, status.fundingTx.txId, status.fundingTx.tx.localFees))
|
||||
remoteCommitSig_opt.foreach(self ! _)
|
||||
val d1 = d.copy(rbfStatus = RbfStatus.RbfWaitingForSigs(status))
|
||||
stay() using d1 storing() sending commitSig
|
||||
case f: InteractiveTxBuilder.Failed =>
|
||||
log.info("rbf attempt failed: {}", f.cause.getMessage)
|
||||
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, f.cause))
|
||||
|
@ -608,16 +693,10 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
val realScidStatus = RealScidStatus.Temporary(RealShortChannelId(w.blockHeight, w.txIndex, commitment.commitInput.outPoint.index.toInt))
|
||||
val shortIds = createShortIds(d.channelId, realScidStatus)
|
||||
val channelReady = createChannelReady(shortIds, d.commitments.params)
|
||||
reportRbfFailure(d.rbfStatus, InvalidRbfTxConfirmed(d.channelId))
|
||||
val toSend = d.rbfStatus match {
|
||||
case RbfStatus.RbfInProgress(cmd_opt, txBuilder) =>
|
||||
txBuilder ! InteractiveTxBuilder.Abort
|
||||
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfTxConfirmed(d.channelId)))
|
||||
Seq(TxAbort(d.channelId, InvalidRbfTxConfirmed(d.channelId).getMessage), channelReady)
|
||||
case RbfStatus.RbfRequested(cmd) =>
|
||||
cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfTxConfirmed(d.channelId))
|
||||
Seq(TxAbort(d.channelId, InvalidRbfTxConfirmed(d.channelId).getMessage), channelReady)
|
||||
case RbfStatus.NoRbf | RbfStatus.RbfAborted =>
|
||||
Seq(channelReady)
|
||||
case RbfStatus.NoRbf | RbfStatus.RbfAborted => Seq(channelReady)
|
||||
case _: RbfStatus.RbfRequested | _: RbfStatus.RbfInProgress | _: RbfStatus.RbfWaitingForSigs => Seq(TxAbort(d.channelId, InvalidRbfTxConfirmed(d.channelId).getMessage), channelReady)
|
||||
}
|
||||
d.deferred.foreach(self ! _)
|
||||
goto(WAIT_FOR_DUAL_FUNDING_READY) using DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments1, shortIds) storing() sending toSend
|
||||
|
@ -627,13 +706,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
case Event(ProcessCurrentBlockHeight(c), d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) => handleNewBlockDualFundingUnconfirmed(c, d)
|
||||
|
||||
case Event(e: BITCOIN_FUNDING_DOUBLE_SPENT, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.RbfRequested(cmd) => cmd.replyTo ! RES_FAILURE(cmd, FundingTxDoubleSpent(d.channelId))
|
||||
case RbfStatus.RbfInProgress(cmd_opt, txBuilder) =>
|
||||
txBuilder ! InteractiveTxBuilder.Abort
|
||||
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, FundingTxDoubleSpent(d.channelId)))
|
||||
case _ => ()
|
||||
}
|
||||
reportRbfFailure(d.rbfStatus, FundingTxDoubleSpent(d.channelId))
|
||||
handleDualFundingDoubleSpent(e, d)
|
||||
|
||||
case Event(remoteChannelReady: ChannelReady, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
|
@ -649,24 +722,16 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
|||
stay()
|
||||
|
||||
case Event(INPUT_DISCONNECTED, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.RbfRequested(cmd) => cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException("rbf attempt failed: disconnected"))
|
||||
case RbfStatus.RbfInProgress(cmd_opt, txBuilder) =>
|
||||
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException("rbf attempt failed: disconnected")))
|
||||
txBuilder ! InteractiveTxBuilder.Abort
|
||||
case RbfStatus.RbfAborted => // nothing to do
|
||||
case RbfStatus.NoRbf => // nothing to do
|
||||
reportRbfFailure(d.rbfStatus, new RuntimeException("rbf attempt failed: disconnected"))
|
||||
val d1 = d.rbfStatus match {
|
||||
// We keep track of the RBF status: we should be able to complete the signature steps on reconnection.
|
||||
case _: RbfStatus.RbfWaitingForSigs => d
|
||||
case _ => d.copy(rbfStatus = RbfStatus.NoRbf)
|
||||
}
|
||||
goto(OFFLINE) using d.copy(rbfStatus = RbfStatus.NoRbf)
|
||||
goto(OFFLINE) using d1
|
||||
|
||||
case Event(e: Error, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
|
||||
d.rbfStatus match {
|
||||
case RbfStatus.RbfRequested(cmd) => cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException(s"remote error: ${e.toAscii}"))
|
||||
case RbfStatus.RbfInProgress(cmd_opt, txBuilder) =>
|
||||
txBuilder ! InteractiveTxBuilder.Abort
|
||||
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException(s"remote error: ${e.toAscii}")))
|
||||
case _ => ()
|
||||
}
|
||||
reportRbfFailure(d.rbfStatus, new RuntimeException(s"remote error: ${e.toAscii}"))
|
||||
handleRemoteError(e, d)
|
||||
})
|
||||
|
||||
|
|
|
@ -54,7 +54,7 @@ trait CommonFundingHandlers extends CommonHandlers {
|
|||
}
|
||||
}
|
||||
|
||||
def acceptFundingTxConfirmed(w: WatchFundingConfirmedTriggered, d: PersistentChannelData): Either[Commitments, (Commitments, Commitment)] = {
|
||||
def acceptFundingTxConfirmed(w: WatchFundingConfirmedTriggered, d: ChannelDataWithCommitments): Either[Commitments, (Commitments, Commitment)] = {
|
||||
log.info("funding txid={} was confirmed at blockHeight={} txIndex={}", w.tx.txid, w.blockHeight, w.txIndex)
|
||||
d.commitments.latest.localFundingStatus match {
|
||||
case _: SingleFundedUnconfirmedFundingTx =>
|
||||
|
|
|
@ -101,7 +101,7 @@ trait CommonHandlers {
|
|||
}
|
||||
|
||||
/** We don't regenerate the final address if we already have one. */
|
||||
def getOrGenerateFinalScriptPubKey(data: PersistentChannelData): ByteVector = data match {
|
||||
def getOrGenerateFinalScriptPubKey(data: ChannelDataWithCommitments): ByteVector = data match {
|
||||
case d: DATA_NORMAL if d.localShutdown.isDefined => d.localShutdown.get.scriptPubKey
|
||||
case d: DATA_SHUTDOWN => d.localShutdown.scriptPubKey
|
||||
case d: DATA_NEGOTIATING => d.localShutdown.scriptPubKey
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
|
||||
package fr.acinq.eclair.channel.fsm
|
||||
|
||||
import akka.actor.{ActorRef, Status}
|
||||
import fr.acinq.bitcoin.scalacompat.{Transaction, TxIn}
|
||||
import fr.acinq.eclair.NotificationsLogger
|
||||
import fr.acinq.eclair.NotificationsLogger.NotifyNodeOperator
|
||||
|
@ -26,6 +25,7 @@ import fr.acinq.eclair.channel.LocalFundingStatus.DualFundedUnconfirmedFundingTx
|
|||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.fsm.Channel.BITCOIN_FUNDING_DOUBLE_SPENT
|
||||
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder._
|
||||
import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession}
|
||||
import fr.acinq.eclair.wire.protocol.{ChannelReady, Error}
|
||||
|
||||
import scala.concurrent.Future
|
||||
|
@ -122,4 +122,28 @@ trait DualFundingHandlers extends CommonFundingHandlers {
|
|||
}
|
||||
}
|
||||
|
||||
def rollbackFundingAttempt(tx: SharedTransaction, previousTxs: Seq[SignedSharedTransaction]): Unit = {
|
||||
// We don't unlock previous inputs as the corresponding funding transaction may confirm.
|
||||
val previousInputs = previousTxs.flatMap(_.tx.localInputs.map(_.outPoint)).toSet
|
||||
val toUnlock = tx.localInputs.map(_.outPoint).toSet -- previousInputs
|
||||
if (toUnlock.nonEmpty) {
|
||||
val dummyTx = Transaction(2, toUnlock.toSeq.map(o => TxIn(o, Nil, 0)), Nil, 0)
|
||||
wallet.rollback(dummyTx)
|
||||
}
|
||||
}
|
||||
|
||||
def rollbackRbfAttempt(signingSession: InteractiveTxSigningSession.WaitingForSigs, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED): Unit = {
|
||||
rollbackFundingAttempt(signingSession.fundingTx.tx, d.allFundingTxs.map(_.sharedTx))
|
||||
}
|
||||
|
||||
def reportRbfFailure(rbfStatus: RbfStatus, f: Throwable): Unit = {
|
||||
rbfStatus match {
|
||||
case RbfStatus.RbfRequested(cmd) => cmd.replyTo ! RES_FAILURE(cmd, f)
|
||||
case RbfStatus.RbfInProgress(cmd_opt, txBuilder, _) =>
|
||||
txBuilder ! InteractiveTxBuilder.Abort
|
||||
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, f))
|
||||
case _ => ()
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -87,7 +87,7 @@ trait ErrorHandlers extends CommonHandlers {
|
|||
log.info(s"we have a valid closing tx, publishing it instead of our commitment: closingTxId=${bestUnpublishedClosingTx.tx.txid}")
|
||||
// if we were in the process of closing and already received a closing sig from the counterparty, it's always better to use that
|
||||
handleMutualClose(bestUnpublishedClosingTx, Left(negotiating))
|
||||
case dd: PersistentChannelData =>
|
||||
case dd: ChannelDataWithCommitments =>
|
||||
// We publish our commitment even if we have nothing at stake: it's a nice thing to do because it lets our peer
|
||||
// get their funds back without delays.
|
||||
cause match {
|
||||
|
@ -117,7 +117,9 @@ trait ErrorHandlers extends CommonHandlers {
|
|||
stop(FSM.Shutdown)
|
||||
}
|
||||
}
|
||||
case _: TransientChannelData => goto(CLOSED) sending error // when there is no commitment yet, we just send an error to our peer and go to CLOSED state
|
||||
// When there is no commitment yet, we just send an error to our peer and go to CLOSED state.
|
||||
case _: ChannelDataWithoutCommitments => goto(CLOSED) sending error
|
||||
case _: TransientChannelData => goto(CLOSED) sending error
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -132,8 +134,10 @@ trait ErrorHandlers extends CommonHandlers {
|
|||
// if we were in the process of closing and already received a closing sig from the counterparty, it's always better to use that
|
||||
handleMutualClose(bestUnpublishedClosingTx, Left(negotiating))
|
||||
// NB: we publish the commitment even if we have nothing at stake (in a dataloss situation our peer will send us an error just for that)
|
||||
case hasCommitments: PersistentChannelData => spendLocalCurrent(hasCommitments)
|
||||
case _: TransientChannelData => goto(CLOSED) // when there is no commitment yet, we just go to CLOSED state in case an error occurs
|
||||
case hasCommitments: ChannelDataWithCommitments => spendLocalCurrent(hasCommitments)
|
||||
// When there is no commitment yet, we just go to CLOSED state in case an error occurs.
|
||||
case _: ChannelDataWithoutCommitments => goto(CLOSED)
|
||||
case _: TransientChannelData => goto(CLOSED)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -170,7 +174,7 @@ trait ErrorHandlers extends CommonHandlers {
|
|||
skip.foreach(output => log.info(s"no need to watch output=${output.txid}:${output.index}, it has already been spent by txid=${irrevocablySpent.get(output).map(_.txid)}"))
|
||||
}
|
||||
|
||||
def spendLocalCurrent(d: PersistentChannelData) = {
|
||||
def spendLocalCurrent(d: ChannelDataWithCommitments) = {
|
||||
val outdatedCommitment = d match {
|
||||
case _: DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT => true
|
||||
case closing: DATA_CLOSING if closing.futureRemoteCommitPublished.isDefined => true
|
||||
|
@ -220,7 +224,7 @@ trait ErrorHandlers extends CommonHandlers {
|
|||
watchSpentIfNeeded(commitTx, watchSpentQueue, irrevocablySpent)
|
||||
}
|
||||
|
||||
def handleRemoteSpentCurrent(commitTx: Transaction, d: PersistentChannelData) = {
|
||||
def handleRemoteSpentCurrent(commitTx: Transaction, d: ChannelDataWithCommitments) = {
|
||||
val commitments = d.commitments.latest
|
||||
log.warning(s"they published their current commit in txid=${commitTx.txid}")
|
||||
require(commitTx.txid == commitments.remoteCommit.txid, "txid mismatch")
|
||||
|
@ -252,7 +256,7 @@ trait ErrorHandlers extends CommonHandlers {
|
|||
goto(CLOSING) using nextData storing() calling doPublish(remoteCommitPublished, commitments)
|
||||
}
|
||||
|
||||
def handleRemoteSpentNext(commitTx: Transaction, d: PersistentChannelData) = {
|
||||
def handleRemoteSpentNext(commitTx: Transaction, d: ChannelDataWithCommitments) = {
|
||||
val commitment = d.commitments.latest
|
||||
log.warning(s"they published their next commit in txid=${commitTx.txid}")
|
||||
require(commitment.nextRemoteCommit_opt.nonEmpty, "next remote commit must be defined")
|
||||
|
@ -289,7 +293,7 @@ trait ErrorHandlers extends CommonHandlers {
|
|||
watchSpentIfNeeded(commitTx, watchSpentQueue, irrevocablySpent)
|
||||
}
|
||||
|
||||
def handleRemoteSpentOther(tx: Transaction, d: PersistentChannelData) = {
|
||||
def handleRemoteSpentOther(tx: Transaction, d: ChannelDataWithCommitments) = {
|
||||
val commitments = d.commitments.latest
|
||||
log.warning(s"funding tx spent in txid=${tx.txid}")
|
||||
val finalScriptPubKey = getOrGenerateFinalScriptPubKey(d)
|
||||
|
@ -331,7 +335,7 @@ trait ErrorHandlers extends CommonHandlers {
|
|||
watchSpentIfNeeded(commitTx, watchSpentQueue, irrevocablySpent)
|
||||
}
|
||||
|
||||
def handleOutdatedCommitment(channelReestablish: ChannelReestablish, d: PersistentChannelData) = {
|
||||
def handleOutdatedCommitment(channelReestablish: ChannelReestablish, d: ChannelDataWithCommitments) = {
|
||||
val exc = PleasePublishYourCommitment(d.channelId)
|
||||
val error = Error(d.channelId, exc.getMessage)
|
||||
goto(WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT) using DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT(d.commitments, channelReestablish) storing() sending error
|
||||
|
|
|
@ -114,7 +114,7 @@ trait SingleFundingHandlers extends CommonFundingHandlers {
|
|||
goto(CLOSED) sending error
|
||||
}
|
||||
|
||||
def singleFundingMinDepth(d: PersistentChannelData) = {
|
||||
def singleFundingMinDepth(d: ChannelDataWithCommitments): Long = {
|
||||
val minDepth_opt = if (d.commitments.params.localParams.isInitiator) {
|
||||
Helpers.Funding.minDepthFunder(d.commitments.params.localParams.initFeatures)
|
||||
} else {
|
||||
|
|
|
@ -18,6 +18,7 @@ package fr.acinq.eclair.channel.fund
|
|||
|
||||
import akka.actor.typed.scaladsl.{ActorContext, Behaviors, StashBuffer}
|
||||
import akka.actor.typed.{ActorRef, Behavior}
|
||||
import akka.event.LoggingAdapter
|
||||
import fr.acinq.bitcoin.ScriptFlags
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, LexicographicalOrdering, OutPoint, Satoshi, SatoshiLong, Script, ScriptWitness, Transaction, TxIn, TxOut}
|
||||
|
@ -25,12 +26,12 @@ import fr.acinq.eclair.blockchain.OnChainChannelFunder
|
|||
import fr.acinq.eclair.blockchain.OnChainWallet.SignTransactionResponse
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.eclair.channel.Helpers.Funding
|
||||
import fr.acinq.eclair.channel.LocalFundingStatus.DualFundedUnconfirmedFundingTx
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.Purpose
|
||||
import fr.acinq.eclair.channel.fund.InteractiveTxSigningSession.UnsignedLocalCommit
|
||||
import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
|
||||
import fr.acinq.eclair.transactions.Transactions.{InputInfo, TxOwner}
|
||||
import fr.acinq.eclair.transactions.{Scripts, Transactions}
|
||||
import fr.acinq.eclair.transactions.Transactions.{CommitTx, HtlcTx, InputInfo, TxOwner}
|
||||
import fr.acinq.eclair.transactions.{CommitmentSpec, Scripts, Transactions}
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
import fr.acinq.eclair.{Logs, MilliSatoshi, NodeParams, UInt64}
|
||||
import scodec.bits.ByteVector
|
||||
|
@ -47,8 +48,9 @@ import scala.util.{Failure, Success, Try}
|
|||
* It allows two participants to collaborate to create a shared transaction.
|
||||
* This is a turn-based protocol: each participant sends one message and then waits for the other participant's response.
|
||||
*
|
||||
* This actor returns [[InteractiveTxBuilder.Succeeded]] once we're ready to send our signatures for the shared
|
||||
* transaction. Once they are sent, we must remember it because the transaction may confirm (unless it is double-spent).
|
||||
* This actor returns [[InteractiveTxBuilder.Succeeded]] once we're ready to send our signatures for the commitment and
|
||||
* the shared transaction. Once those are sent, we must remember it because the transaction may confirm (unless it is
|
||||
* double-spent).
|
||||
*
|
||||
* Note that this actor doesn't handle the RBF messages: the parent actor must decide whether they accept an RBF attempt
|
||||
* and how much they want to contribute.
|
||||
|
@ -71,32 +73,22 @@ object InteractiveTxBuilder {
|
|||
// | |<------- tx_add_output ------| |
|
||||
// | |-------- tx_complete ------->| |
|
||||
// | |<------- tx_complete --------| |
|
||||
// | |-------- commit_sig -------->| |
|
||||
// | |<------- commit_sig ---------| |
|
||||
// | |-------- tx_signatures ----->| |
|
||||
// | |<------- tx_signatures ------| |
|
||||
// +-------+ +-------+
|
||||
|
||||
// @formatter:off
|
||||
sealed trait Command
|
||||
case class Start(replyTo: ActorRef[Response]) extends Command
|
||||
sealed trait ReceiveMessage extends Command
|
||||
case class ReceiveTxMessage(msg: InteractiveTxConstructionMessage) extends ReceiveMessage
|
||||
case class ReceiveCommitSig(msg: CommitSig) extends ReceiveMessage
|
||||
case class ReceiveTxSigs(msg: TxSignatures) extends ReceiveMessage
|
||||
case class ReceiveMessage(msg: InteractiveTxConstructionMessage) extends Command
|
||||
case object Abort extends Command
|
||||
private case class FundTransactionResult(result: InteractiveTxFunder.Response) extends Command
|
||||
private case object ValidateSharedTx extends Command
|
||||
private case class SignTransactionResult(signedTx: PartiallySignedSharedTransaction, remoteSigs_opt: Option[TxSignatures]) extends Command
|
||||
private case class SignTransactionResult(signedTx: PartiallySignedSharedTransaction) extends Command
|
||||
private case class WalletFailure(t: Throwable) extends Command
|
||||
private case object UtxosUnlocked extends Command
|
||||
|
||||
sealed trait Response
|
||||
case class SendMessage(msg: LightningMessage) extends Response
|
||||
case class Succeeded(fundingTx: DualFundedUnconfirmedFundingTx, commitment: Commitment) extends Response {
|
||||
val fundingParams: InteractiveTxParams = fundingTx.fundingParams
|
||||
val sharedTx: SignedSharedTransaction = fundingTx.sharedTx
|
||||
}
|
||||
case class Succeeded(signingSession: InteractiveTxSigningSession.WaitingForSigs, commitSig: CommitSig) extends Response
|
||||
sealed trait Failed extends Response { def cause: ChannelException }
|
||||
case class LocalFailure(cause: ChannelException) extends Failed
|
||||
case class RemoteFailure(cause: ChannelException) extends Failed
|
||||
|
@ -254,9 +246,9 @@ object InteractiveTxBuilder {
|
|||
}
|
||||
|
||||
type OutgoingInput = Input with Outgoing
|
||||
type IncomingInput = Input with Incoming
|
||||
private type IncomingInput = Input with Incoming
|
||||
type OutgoingOutput = Output with Outgoing
|
||||
type IncomingOutput = Output with Incoming
|
||||
private type IncomingOutput = Output with Incoming
|
||||
// @formatter:on
|
||||
|
||||
private case class InteractiveTxSession(toSend: Seq[Outgoing],
|
||||
|
@ -354,41 +346,6 @@ object InteractiveTxBuilder {
|
|||
// We restrict the number of inputs / outputs that our peer can send us to ensure the protocol eventually ends.
|
||||
val MAX_INPUTS_OUTPUTS_RECEIVED = 4096
|
||||
|
||||
def addRemoteSigs(fundingParams: InteractiveTxParams, partiallySignedTx: PartiallySignedSharedTransaction, remoteSigs: TxSignatures): Either[ChannelException, FullySignedSharedTransaction] = {
|
||||
if (partiallySignedTx.tx.localInputs.length != partiallySignedTx.localSigs.witnesses.length) {
|
||||
return Left(InvalidFundingSignature(fundingParams.channelId, Some(partiallySignedTx.txId)))
|
||||
}
|
||||
if (partiallySignedTx.tx.remoteInputs.length != remoteSigs.witnesses.length) {
|
||||
return Left(InvalidFundingSignature(fundingParams.channelId, Some(partiallySignedTx.txId)))
|
||||
}
|
||||
val sharedSigs_opt = fundingParams.sharedInput_opt match {
|
||||
case Some(sharedInput: Multisig2of2Input) =>
|
||||
(partiallySignedTx.localSigs.previousFundingTxSig_opt, remoteSigs.previousFundingTxSig_opt) match {
|
||||
case (Some(localSig), Some(remoteSig)) => Some(Scripts.witness2of2(localSig, remoteSig, sharedInput.localFundingPubkey, sharedInput.remoteFundingPubkey))
|
||||
case _ => return Left(InvalidFundingSignature(fundingParams.channelId, Some(partiallySignedTx.txId)))
|
||||
}
|
||||
case None => None
|
||||
}
|
||||
val txWithSigs = FullySignedSharedTransaction(partiallySignedTx.tx, partiallySignedTx.localSigs, remoteSigs, sharedSigs_opt)
|
||||
if (remoteSigs.txId != txWithSigs.signedTx.txid) {
|
||||
return Left(InvalidFundingSignature(fundingParams.channelId, Some(partiallySignedTx.txId)))
|
||||
}
|
||||
// We allow a 5% error margin since witness size prediction could be inaccurate.
|
||||
if (fundingParams.localAmount > 0.sat && txWithSigs.feerate < fundingParams.targetFeerate * 0.95) {
|
||||
return Left(InvalidFundingFeerate(fundingParams.channelId, fundingParams.targetFeerate, txWithSigs.feerate))
|
||||
}
|
||||
val previousOutputs = {
|
||||
val sharedOutput = fundingParams.sharedInput_opt.map(sharedInput => sharedInput.info.outPoint -> sharedInput.info.txOut).toMap
|
||||
val localOutputs = txWithSigs.tx.localInputs.map(i => i.outPoint -> i.previousTx.txOut(i.previousTxOutput.toInt)).toMap
|
||||
val remoteOutputs = txWithSigs.tx.remoteInputs.map(i => i.outPoint -> i.txOut).toMap
|
||||
sharedOutput ++ localOutputs ++ remoteOutputs
|
||||
}
|
||||
Try(Transaction.correctlySpends(txWithSigs.signedTx, previousOutputs, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) match {
|
||||
case Failure(_) => Left(InvalidFundingSignature(fundingParams.channelId, Some(partiallySignedTx.txId)))
|
||||
case Success(_) => Right(txWithSigs)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Response],
|
||||
|
@ -532,7 +489,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
|
|||
|
||||
private def receive(session: InteractiveTxSession): Behavior[Command] = {
|
||||
Behaviors.receiveMessagePartial {
|
||||
case ReceiveTxMessage(msg) => msg match {
|
||||
case ReceiveMessage(msg) => msg match {
|
||||
case msg: HasSerialId if msg.serialId.toByteVector.bits.last != fundingParams.isInitiator =>
|
||||
replyTo ! RemoteFailure(InvalidSerialId(fundingParams.channelId, msg.serialId))
|
||||
unlockAndStop(session)
|
||||
|
@ -594,12 +551,6 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
|
|||
send(next)
|
||||
}
|
||||
}
|
||||
case _: ReceiveCommitSig =>
|
||||
replyTo ! RemoteFailure(UnexpectedCommitSig(fundingParams.channelId))
|
||||
unlockAndStop(session)
|
||||
case _: ReceiveTxSigs =>
|
||||
replyTo ! RemoteFailure(UnexpectedFundingSignatures(fundingParams.channelId))
|
||||
unlockAndStop(session)
|
||||
case Abort =>
|
||||
unlockAndStop(session)
|
||||
}
|
||||
|
@ -622,19 +573,13 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
|
|||
case Left(cause) =>
|
||||
replyTo ! RemoteFailure(cause)
|
||||
unlockAndStop(session)
|
||||
case Right((completeTx, fundingOutputIndex)) =>
|
||||
stash.unstashAll(signCommitTx(completeTx, fundingOutputIndex))
|
||||
case Right(completeTx) =>
|
||||
signCommitTx(completeTx)
|
||||
}
|
||||
case _: WalletFailure =>
|
||||
replyTo ! RemoteFailure(UnconfirmedInteractiveTxInputs(fundingParams.channelId))
|
||||
unlockAndStop(session)
|
||||
case msg: ReceiveCommitSig =>
|
||||
stash.stash(msg)
|
||||
Behaviors.same
|
||||
case ReceiveTxSigs(_) =>
|
||||
replyTo ! RemoteFailure(UnexpectedFundingSignatures(fundingParams.channelId))
|
||||
unlockAndStop(session)
|
||||
case ReceiveTxMessage(msg) =>
|
||||
case ReceiveMessage(msg) =>
|
||||
replyTo ! RemoteFailure(UnexpectedInteractiveTxMessage(fundingParams.channelId, msg))
|
||||
unlockAndStop(session)
|
||||
case Abort =>
|
||||
|
@ -656,7 +601,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
|
|||
}
|
||||
}
|
||||
|
||||
private def validateTx(session: InteractiveTxSession): Either[ChannelException, (SharedTransaction, Int)] = {
|
||||
private def validateTx(session: InteractiveTxSession): Either[ChannelException, SharedTransaction] = {
|
||||
if (session.localInputs.length + session.remoteInputs.length > 252 || session.localOutputs.length + session.remoteOutputs.length > 252) {
|
||||
log.warn("invalid interactive tx ({} local inputs, {} remote inputs, {} local outputs and {} remote outputs)", session.localInputs.length, session.remoteInputs.length, session.localOutputs.length, session.remoteOutputs.length)
|
||||
return Left(InvalidCompleteInteractiveTx(fundingParams.channelId))
|
||||
|
@ -700,8 +645,6 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
|
|||
|
||||
val sharedTx = SharedTransaction(sharedInput_opt, sharedOutput, localInputs.toList, remoteInputs.toList, localOutputs.toList, remoteOutputs.toList, fundingParams.lockTime)
|
||||
val tx = sharedTx.buildUnsignedTx()
|
||||
val sharedOutputIndex = tx.txOut.indexWhere(_.publicKeyScript == fundingParams.fundingPubkeyScript)
|
||||
|
||||
if (sharedTx.localAmountIn < sharedTx.localAmountOut || sharedTx.remoteAmountIn < sharedTx.remoteAmountOut) {
|
||||
log.warn("invalid interactive tx: input amount is too small (localIn={}, localOut={}, remoteIn={}, remoteOut={})", sharedTx.localAmountIn, sharedTx.localAmountOut, sharedTx.remoteAmountIn, sharedTx.remoteAmountOut)
|
||||
return Left(InvalidCompleteInteractiveTx(fundingParams.channelId))
|
||||
|
@ -749,11 +692,12 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
|
|||
return Left(InvalidCompleteInteractiveTx(fundingParams.channelId))
|
||||
}
|
||||
|
||||
Right(sharedTx, sharedOutputIndex)
|
||||
Right(sharedTx)
|
||||
}
|
||||
|
||||
private def signCommitTx(completeTx: SharedTransaction, fundingOutputIndex: Int): Behavior[Command] = {
|
||||
private def signCommitTx(completeTx: SharedTransaction): Behavior[Command] = {
|
||||
val fundingTx = completeTx.buildUnsignedTx()
|
||||
val fundingOutputIndex = fundingTx.txOut.indexWhere(_.publicKeyScript == fundingParams.fundingPubkeyScript)
|
||||
Funding.makeCommitTxsWithoutHtlcs(keyManager, channelParams,
|
||||
fundingAmount = fundingParams.fundingAmount,
|
||||
toLocal = fundingParams.localAmount - localPushAmount + remotePushAmount,
|
||||
|
@ -765,95 +709,27 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
|
|||
case Right((localSpec, localCommitTx, remoteSpec, remoteCommitTx)) =>
|
||||
require(fundingTx.txOut(fundingOutputIndex).publicKeyScript == localCommitTx.input.txOut.publicKeyScript, "pubkey script mismatch!")
|
||||
val fundingPubKey = keyManager.fundingPublicKey(channelParams.localParams.fundingKeyPath)
|
||||
val localSigOfLocalTx = keyManager.sign(localCommitTx, fundingPubKey, TxOwner.Local, channelParams.channelFeatures.commitmentFormat)
|
||||
val localSigOfRemoteTx = keyManager.sign(remoteCommitTx, fundingPubKey, TxOwner.Remote, channelParams.channelFeatures.commitmentFormat)
|
||||
val localCommitSig = CommitSig(fundingParams.channelId, localSigOfRemoteTx, Nil)
|
||||
replyTo ! SendMessage(localCommitSig)
|
||||
Behaviors.receiveMessagePartial {
|
||||
case ReceiveCommitSig(remoteCommitSig) =>
|
||||
val signedLocalCommitTx = Transactions.addSigs(localCommitTx, fundingPubKey.publicKey, channelParams.remoteParams.fundingPubKey, localSigOfLocalTx, remoteCommitSig.signature)
|
||||
Transactions.checkSpendable(signedLocalCommitTx) match {
|
||||
case Failure(_) =>
|
||||
replyTo ! RemoteFailure(InvalidCommitmentSignature(fundingParams.channelId, signedLocalCommitTx.tx.txid))
|
||||
unlockAndStop(completeTx)
|
||||
case Success(_) =>
|
||||
val localCommit = LocalCommit(purpose.localCommitIndex, localSpec, CommitTxAndRemoteSig(localCommitTx, remoteCommitSig.signature), htlcTxsAndRemoteSigs = Nil)
|
||||
val remoteCommit = RemoteCommit(purpose.remoteCommitIndex, remoteSpec, remoteCommitTx.tx.txid, purpose.remotePerCommitmentPoint)
|
||||
signFundingTx(completeTx, localCommit, remoteCommit)
|
||||
}
|
||||
case ReceiveTxSigs(_) =>
|
||||
replyTo ! RemoteFailure(UnexpectedFundingSignatures(fundingParams.channelId))
|
||||
unlockAndStop(completeTx)
|
||||
case ReceiveTxMessage(msg) =>
|
||||
replyTo ! RemoteFailure(UnexpectedInteractiveTxMessage(fundingParams.channelId, msg))
|
||||
unlockAndStop(completeTx)
|
||||
case Abort =>
|
||||
unlockAndStop(completeTx)
|
||||
}
|
||||
val localCommit = UnsignedLocalCommit(purpose.localCommitIndex, localSpec, localCommitTx, htlcTxs = Nil)
|
||||
val remoteCommit = RemoteCommit(purpose.remoteCommitIndex, remoteSpec, remoteCommitTx.tx.txid, purpose.remotePerCommitmentPoint)
|
||||
signFundingTx(completeTx, localCommitSig, localCommit, remoteCommit)
|
||||
}
|
||||
}
|
||||
|
||||
private def createCommitment(fundingTx: DualFundedUnconfirmedFundingTx, localCommit: LocalCommit, remoteCommit: RemoteCommit): Commitment = {
|
||||
Commitment(fundingTx, RemoteFundingStatus.NotLocked, localCommit, remoteCommit, nextRemoteCommit_opt = None)
|
||||
}
|
||||
|
||||
private def signFundingTx(completeTx: SharedTransaction, localCommit: LocalCommit, remoteCommit: RemoteCommit): Behavior[Command] = {
|
||||
val shouldSignFirst = if (completeTx.localAmountIn == completeTx.remoteAmountIn) {
|
||||
// When both peers contribute the same amount, the peer with the lowest pubkey must transmit its `tx_signatures` first.
|
||||
LexicographicalOrdering.isLessThan(channelParams.localNodeId.value, channelParams.remoteNodeId.value)
|
||||
} else if (fundingParams.sharedInput_opt.isEmpty && completeTx.remoteAmountIn == 0.sat) {
|
||||
// If our peer didn't contribute to the transaction, we don't need to wait for their `tx_signatures`, they will be
|
||||
// empty anyway.
|
||||
true
|
||||
} else {
|
||||
// Otherwise, the peer with the lowest total of input amount must transmit its `tx_signatures` first.
|
||||
completeTx.localAmountIn < completeTx.remoteAmountIn
|
||||
}
|
||||
if (shouldSignFirst) {
|
||||
signTx(completeTx, None)
|
||||
}
|
||||
private def signFundingTx(completeTx: SharedTransaction, commitSig: CommitSig, localCommit: UnsignedLocalCommit, remoteCommit: RemoteCommit): Behavior[Command] = {
|
||||
signTx(completeTx)
|
||||
Behaviors.receiveMessagePartial {
|
||||
case SignTransactionResult(signedTx, Some(remoteSigs)) =>
|
||||
addRemoteSigs(fundingParams, signedTx, remoteSigs) match {
|
||||
case Left(cause) =>
|
||||
replyTo ! RemoteFailure(cause)
|
||||
unlockAndStop(completeTx)
|
||||
case Right(fullySignedTx) =>
|
||||
log.info("interactive-tx fully signed with {} local inputs, {} remote inputs, {} local outputs and {} remote outputs", fullySignedTx.tx.localInputs.length, fullySignedTx.tx.remoteInputs.length, fullySignedTx.tx.localOutputs.length, fullySignedTx.tx.remoteOutputs.length)
|
||||
val fundingTx = DualFundedUnconfirmedFundingTx(fullySignedTx, nodeParams.currentBlockHeight, fundingParams)
|
||||
val commitment = createCommitment(fundingTx, localCommit, remoteCommit)
|
||||
replyTo ! Succeeded(fundingTx, commitment)
|
||||
Behaviors.stopped
|
||||
}
|
||||
case SignTransactionResult(signedTx, None) =>
|
||||
// We return as soon as we sign the tx, because we need to be able to handle the case where remote publishes the
|
||||
// tx right away without properly sending us their signatures.
|
||||
if (fundingParams.sharedInput_opt.isEmpty && completeTx.remoteAmountIn == 0.sat) {
|
||||
log.info("interactive-tx fully signed with {} local inputs, {} remote inputs, {} local outputs and {} remote outputs", signedTx.tx.localInputs.length, signedTx.tx.remoteInputs.length, signedTx.tx.localOutputs.length, signedTx.tx.remoteOutputs.length)
|
||||
val remoteSigs = TxSignatures(signedTx.localSigs.channelId, signedTx.localSigs.txHash, Nil)
|
||||
val signedTx1 = FullySignedSharedTransaction(signedTx.tx, signedTx.localSigs, remoteSigs, None)
|
||||
val fundingTx = DualFundedUnconfirmedFundingTx(signedTx1, nodeParams.currentBlockHeight, fundingParams)
|
||||
val commitment = createCommitment(fundingTx, localCommit, remoteCommit)
|
||||
replyTo ! Succeeded(fundingTx, commitment)
|
||||
} else {
|
||||
log.info("interactive-tx partially signed with {} local inputs, {} remote inputs, {} local outputs and {} remote outputs", signedTx.tx.localInputs.length, signedTx.tx.remoteInputs.length, signedTx.tx.localOutputs.length, signedTx.tx.remoteOutputs.length)
|
||||
val fundingTx = DualFundedUnconfirmedFundingTx(signedTx, nodeParams.currentBlockHeight, fundingParams)
|
||||
val commitment = createCommitment(fundingTx, localCommit, remoteCommit)
|
||||
replyTo ! Succeeded(fundingTx, commitment)
|
||||
}
|
||||
case SignTransactionResult(signedTx) =>
|
||||
log.info("interactive-tx partially signed with {} local inputs, {} remote inputs, {} local outputs and {} remote outputs", signedTx.tx.localInputs.length, signedTx.tx.remoteInputs.length, signedTx.tx.localOutputs.length, signedTx.tx.remoteOutputs.length)
|
||||
replyTo ! Succeeded(InteractiveTxSigningSession.WaitingForSigs(fundingParams, signedTx, Left(localCommit), remoteCommit), commitSig)
|
||||
Behaviors.stopped
|
||||
case ReceiveTxSigs(remoteSigs) =>
|
||||
signTx(completeTx, Some(remoteSigs))
|
||||
Behaviors.same
|
||||
case WalletFailure(t) =>
|
||||
log.error("could not sign funding transaction: ", t)
|
||||
// We use a generic exception and don't send the internal error to the peer.
|
||||
replyTo ! LocalFailure(ChannelFundingError(fundingParams.channelId))
|
||||
unlockAndStop(completeTx)
|
||||
case ReceiveCommitSig(_) =>
|
||||
replyTo ! RemoteFailure(UnexpectedCommitSig(fundingParams.channelId))
|
||||
unlockAndStop(completeTx)
|
||||
case ReceiveTxMessage(msg) =>
|
||||
case ReceiveMessage(msg) =>
|
||||
replyTo ! RemoteFailure(UnexpectedInteractiveTxMessage(fundingParams.channelId, msg))
|
||||
unlockAndStop(completeTx)
|
||||
case Abort =>
|
||||
|
@ -861,11 +737,11 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
|
|||
}
|
||||
}
|
||||
|
||||
private def signTx(unsignedTx: SharedTransaction, remoteSigs_opt: Option[TxSignatures]): Unit = {
|
||||
private def signTx(unsignedTx: SharedTransaction): Unit = {
|
||||
val tx = unsignedTx.buildUnsignedTx()
|
||||
val sharedSig_opt = fundingParams.sharedInput_opt.map(_.sign(keyManager, channelParams, tx))
|
||||
if (unsignedTx.localInputs.isEmpty) {
|
||||
context.self ! SignTransactionResult(PartiallySignedSharedTransaction(unsignedTx, TxSignatures(fundingParams.channelId, tx, Nil, sharedSig_opt)), remoteSigs_opt)
|
||||
context.self ! SignTransactionResult(PartiallySignedSharedTransaction(unsignedTx, TxSignatures(fundingParams.channelId, tx, Nil, sharedSig_opt)))
|
||||
} else {
|
||||
context.pipeToSelf(wallet.signTransaction(tx, allowIncomplete = true).map {
|
||||
case SignTransactionResponse(signedTx, _) =>
|
||||
|
@ -874,7 +750,7 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
|
|||
PartiallySignedSharedTransaction(unsignedTx, TxSignatures(fundingParams.channelId, tx, sigs, sharedSig_opt))
|
||||
}) {
|
||||
case Failure(t) => WalletFailure(t)
|
||||
case Success(signedTx) => SignTransactionResult(signedTx, remoteSigs_opt)
|
||||
case Success(signedTx) => SignTransactionResult(signedTx)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -909,4 +785,135 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
|
|||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Once a shared transaction has been created, peers exchange signatures for the commitment and the shared transaction.
|
||||
* We store the channel state once we reach that step. Once we've sent tx_signatures, we cannot forget the channel
|
||||
* until it has been spent or double-spent.
|
||||
*/
|
||||
sealed trait InteractiveTxSigningSession
|
||||
|
||||
object InteractiveTxSigningSession {
|
||||
|
||||
import InteractiveTxBuilder._
|
||||
|
||||
// Example flow:
|
||||
// +-------+ +-------+
|
||||
// | |-------- commit_sig -------->| |
|
||||
// | A |<------- commit_sig ---------| B |
|
||||
// | |-------- tx_signatures ----->| |
|
||||
// | |<------- tx_signatures ------| |
|
||||
// +-------+ +-------+
|
||||
|
||||
/** A local commitment for which we haven't received our peer's signatures. */
|
||||
case class UnsignedLocalCommit(index: Long, spec: CommitmentSpec, commitTx: CommitTx, htlcTxs: List[HtlcTx])
|
||||
|
||||
private def shouldSignFirst(channelParams: ChannelParams, tx: SharedTransaction): Boolean = {
|
||||
if (tx.localAmountIn == tx.remoteAmountIn) {
|
||||
// When both peers contribute the same amount, the peer with the lowest pubkey must transmit its `tx_signatures` first.
|
||||
LexicographicalOrdering.isLessThan(channelParams.localNodeId.value, channelParams.remoteNodeId.value)
|
||||
} else {
|
||||
// Otherwise, the peer with the lowest total of input amount must transmit its `tx_signatures` first.
|
||||
tx.localAmountIn < tx.remoteAmountIn
|
||||
}
|
||||
}
|
||||
|
||||
def addRemoteSigs(fundingParams: InteractiveTxParams, partiallySignedTx: PartiallySignedSharedTransaction, remoteSigs: TxSignatures)(implicit log: LoggingAdapter): Either[ChannelException, FullySignedSharedTransaction] = {
|
||||
if (partiallySignedTx.tx.localInputs.length != partiallySignedTx.localSigs.witnesses.length) {
|
||||
return Left(InvalidFundingSignature(fundingParams.channelId, Some(partiallySignedTx.txId)))
|
||||
}
|
||||
if (partiallySignedTx.tx.remoteInputs.length != remoteSigs.witnesses.length) {
|
||||
log.info("invalid tx_signatures: witness count mismatch (expected={}, got={})", partiallySignedTx.tx.remoteInputs.length, remoteSigs.witnesses.length)
|
||||
return Left(InvalidFundingSignature(fundingParams.channelId, Some(partiallySignedTx.txId)))
|
||||
}
|
||||
val sharedSigs_opt = fundingParams.sharedInput_opt match {
|
||||
case Some(sharedInput: Multisig2of2Input) =>
|
||||
(partiallySignedTx.localSigs.previousFundingTxSig_opt, remoteSigs.previousFundingTxSig_opt) match {
|
||||
case (Some(localSig), Some(remoteSig)) => Some(Scripts.witness2of2(localSig, remoteSig, sharedInput.localFundingPubkey, sharedInput.remoteFundingPubkey))
|
||||
case _ =>
|
||||
log.info("invalid tx_signatures: missing shared input signatures")
|
||||
return Left(InvalidFundingSignature(fundingParams.channelId, Some(partiallySignedTx.txId)))
|
||||
}
|
||||
case None => None
|
||||
}
|
||||
val txWithSigs = FullySignedSharedTransaction(partiallySignedTx.tx, partiallySignedTx.localSigs, remoteSigs, sharedSigs_opt)
|
||||
if (remoteSigs.txId != txWithSigs.signedTx.txid) {
|
||||
log.info("invalid tx_signatures: txId mismatch (expected={}, got={})", txWithSigs.signedTx.txid, remoteSigs.txId)
|
||||
return Left(InvalidFundingSignature(fundingParams.channelId, Some(partiallySignedTx.txId)))
|
||||
}
|
||||
// We allow a 5% error margin since witness size prediction could be inaccurate.
|
||||
if (fundingParams.localAmount > 0.sat && txWithSigs.feerate < fundingParams.targetFeerate * 0.95) {
|
||||
return Left(InvalidFundingFeerate(fundingParams.channelId, fundingParams.targetFeerate, txWithSigs.feerate))
|
||||
}
|
||||
val previousOutputs = {
|
||||
val sharedOutput = fundingParams.sharedInput_opt.map(sharedInput => sharedInput.info.outPoint -> sharedInput.info.txOut).toMap
|
||||
val localOutputs = txWithSigs.tx.localInputs.map(i => i.outPoint -> i.previousTx.txOut(i.previousTxOutput.toInt)).toMap
|
||||
val remoteOutputs = txWithSigs.tx.remoteInputs.map(i => i.outPoint -> i.txOut).toMap
|
||||
sharedOutput ++ localOutputs ++ remoteOutputs
|
||||
}
|
||||
Try(Transaction.correctlySpends(txWithSigs.signedTx, previousOutputs, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) match {
|
||||
case Failure(f) =>
|
||||
log.info("invalid tx_signatures: {}", f.getMessage)
|
||||
Left(InvalidFundingSignature(fundingParams.channelId, Some(partiallySignedTx.txId)))
|
||||
case Success(_) => Right(txWithSigs)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* We haven't sent our tx_signatures: we store the channel state, but we can safely forget it if we discover that our
|
||||
* peer has forgotten that channel (which may happen if we disconnected before they received our tx_complete).
|
||||
*/
|
||||
case class WaitingForSigs(fundingParams: InteractiveTxParams,
|
||||
fundingTx: PartiallySignedSharedTransaction,
|
||||
localCommit: Either[UnsignedLocalCommit, LocalCommit],
|
||||
remoteCommit: RemoteCommit) extends InteractiveTxSigningSession {
|
||||
val commitInput = localCommit.fold(_.commitTx.input, _.commitTxAndRemoteSig.commitTx.input)
|
||||
|
||||
def receiveCommitSig(nodeParams: NodeParams, channelParams: ChannelParams, remoteCommitSig: CommitSig)(implicit log: LoggingAdapter): Either[ChannelException, InteractiveTxSigningSession] = {
|
||||
localCommit match {
|
||||
case Left(unsignedLocalCommit) =>
|
||||
val fundingPubKey = nodeParams.channelKeyManager.fundingPublicKey(channelParams.localParams.fundingKeyPath)
|
||||
val localSigOfLocalTx = nodeParams.channelKeyManager.sign(unsignedLocalCommit.commitTx, fundingPubKey, TxOwner.Local, channelParams.channelFeatures.commitmentFormat)
|
||||
val signedLocalCommitTx = Transactions.addSigs(unsignedLocalCommit.commitTx, fundingPubKey.publicKey, channelParams.remoteParams.fundingPubKey, localSigOfLocalTx, remoteCommitSig.signature)
|
||||
Transactions.checkSpendable(signedLocalCommitTx) match {
|
||||
case Failure(_) => Left(InvalidCommitmentSignature(fundingParams.channelId, signedLocalCommitTx.tx.txid))
|
||||
case Success(_) =>
|
||||
val signedLocalCommit = LocalCommit(unsignedLocalCommit.index, unsignedLocalCommit.spec, CommitTxAndRemoteSig(unsignedLocalCommit.commitTx, remoteCommitSig.signature), htlcTxsAndRemoteSigs = Nil)
|
||||
if (shouldSignFirst(channelParams, fundingTx.tx)) {
|
||||
val fundingStatus = LocalFundingStatus.DualFundedUnconfirmedFundingTx(fundingTx, nodeParams.currentBlockHeight, fundingParams)
|
||||
val commitment = Commitment(fundingStatus, RemoteFundingStatus.NotLocked, signedLocalCommit, remoteCommit, None)
|
||||
Right(SendingSigs(fundingStatus, commitment, fundingTx.localSigs))
|
||||
} else {
|
||||
Right(this.copy(localCommit = Right(signedLocalCommit)))
|
||||
}
|
||||
}
|
||||
case Right(_) =>
|
||||
log.info("ignoring duplicate commit_sig")
|
||||
Right(this)
|
||||
}
|
||||
}
|
||||
|
||||
def receiveTxSigs(nodeParams: NodeParams, remoteTxSigs: TxSignatures)(implicit log: LoggingAdapter): Either[ChannelException, SendingSigs] = {
|
||||
localCommit match {
|
||||
case Left(_) =>
|
||||
log.info("received tx_signatures before commit_sig")
|
||||
Left(UnexpectedFundingSignatures(fundingParams.channelId))
|
||||
case Right(signedLocalCommit) =>
|
||||
addRemoteSigs(fundingParams, fundingTx, remoteTxSigs) match {
|
||||
case Left(f) =>
|
||||
log.info("received invalid tx_signatures")
|
||||
Left(f)
|
||||
case Right(fullySignedTx) =>
|
||||
log.info("interactive-tx fully signed with {} local inputs, {} remote inputs, {} local outputs and {} remote outputs", fullySignedTx.tx.localInputs.length, fullySignedTx.tx.remoteInputs.length, fullySignedTx.tx.localOutputs.length, fullySignedTx.tx.remoteOutputs.length)
|
||||
val fundingStatus = LocalFundingStatus.DualFundedUnconfirmedFundingTx(fullySignedTx, nodeParams.currentBlockHeight, fundingParams)
|
||||
val commitment = Commitment(fundingStatus, RemoteFundingStatus.NotLocked, signedLocalCommit, remoteCommit, None)
|
||||
Right(SendingSigs(fundingStatus, commitment, fullySignedTx.localSigs))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** We send our tx_signatures: we cannot forget the channel until it has been spent or double-spent. */
|
||||
case class SendingSigs(fundingTx: LocalFundingStatus.DualFundedUnconfirmedFundingTx, commitment: Commitment, localSigs: TxSignatures) extends InteractiveTxSigningSession
|
||||
}
|
|
@ -392,7 +392,7 @@ class Peer(val nodeParams: NodeParams, remoteNodeId: PublicKey, wallet: OnchainP
|
|||
}
|
||||
|
||||
def replyUnknownChannel(peerConnection: ActorRef, unknownChannelId: ByteVector32): Unit = {
|
||||
val msg = Warning(unknownChannelId, "unknown channel")
|
||||
val msg = Error(unknownChannelId, "unknown channel")
|
||||
self ! Peer.OutgoingMessage(msg, peerConnection)
|
||||
}
|
||||
|
||||
|
|
|
@ -170,6 +170,7 @@ object PeerReadyNotifier {
|
|||
case channel.WAIT_FOR_OPEN_DUAL_FUNDED_CHANNEL => true
|
||||
case channel.WAIT_FOR_ACCEPT_DUAL_FUNDED_CHANNEL => true
|
||||
case channel.WAIT_FOR_DUAL_FUNDING_CREATED => true
|
||||
case channel.WAIT_FOR_DUAL_FUNDING_SIGNED => true
|
||||
case channel.WAIT_FOR_DUAL_FUNDING_CONFIRMED => true
|
||||
case channel.WAIT_FOR_DUAL_FUNDING_READY => true
|
||||
case channel.NORMAL => true
|
||||
|
|
|
@ -46,11 +46,12 @@ object PendingChannelsRateLimiter {
|
|||
private[io] def filterPendingChannels(channels: Seq[PersistentChannelData]): Map[PublicKey, Seq[PersistentChannelData]] = {
|
||||
channels.filter {
|
||||
case _: DATA_WAIT_FOR_FUNDING_CONFIRMED => true
|
||||
case _: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED => true
|
||||
case _: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => true
|
||||
case _: DATA_WAIT_FOR_CHANNEL_READY => true
|
||||
case _: DATA_WAIT_FOR_DUAL_FUNDING_READY => true
|
||||
case _ => false
|
||||
}.groupBy(_.commitments.remoteNodeId)
|
||||
}.groupBy(_.remoteNodeId)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -56,8 +56,7 @@ class Switchboard(nodeParams: NodeParams, peerFactory: Switchboard.PeerFactory)
|
|||
log.info(s"closing channel ${c.channelId}")
|
||||
nodeParams.db.channels.removeChannel(c.channelId)
|
||||
})
|
||||
|
||||
val peerChannels = channels.groupBy(_.commitments.params.remoteParams.nodeId)
|
||||
val peerChannels = channels.groupBy(_.remoteNodeId)
|
||||
peerChannels.foreach { case (remoteNodeId, states) => createOrGetPeer(remoteNodeId, offlineChannels = states.toSet) }
|
||||
log.info("restoring {} peer(s) with {} channel(s)", peerChannels.size, channels.size)
|
||||
unstashAll()
|
||||
|
|
|
@ -349,6 +349,7 @@ object PostRestartHtlcCleaner {
|
|||
// They signed it first, so the HTLC will first appear in our commitment tx, and later on in their commitment when
|
||||
// we subsequently sign it. That's why we need to look in *their* commitment with direction=OUT.
|
||||
channels
|
||||
.collect { case c: ChannelDataWithCommitments => c }
|
||||
.flatMap(_.commitments.latest.remoteCommit.spec.htlcs)
|
||||
.collect(outgoing)
|
||||
.map(IncomingPaymentPacket.decrypt(_, privateKey, features))
|
||||
|
@ -378,6 +379,7 @@ object PostRestartHtlcCleaner {
|
|||
/** @return pending outgoing HTLCs, grouped by their upstream origin. */
|
||||
private def getHtlcsRelayedOut(channels: Seq[PersistentChannelData], htlcsIn: Seq[IncomingHtlc])(implicit log: LoggingAdapter): Map[Origin, Set[(ByteVector32, Long)]] = {
|
||||
val htlcsOut = channels
|
||||
.collect { case c: ChannelDataWithCommitments => c }
|
||||
.flatMap { c =>
|
||||
// Filter out HTLCs that will never reach the blockchain or have already been timed-out on-chain.
|
||||
val htlcsToIgnore: Set[Long] = c match {
|
||||
|
|
|
@ -5,7 +5,8 @@ import fr.acinq.bitcoin.scalacompat.DeterministicWallet.KeyPath
|
|||
import fr.acinq.bitcoin.scalacompat.{OutPoint, ScriptWitness, Transaction, TxOut}
|
||||
import fr.acinq.eclair.channel.LocalFundingStatus._
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder
|
||||
import fr.acinq.eclair.channel.fund.InteractiveTxSigningSession.UnsignedLocalCommit
|
||||
import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession}
|
||||
import fr.acinq.eclair.crypto.ShaChain
|
||||
import fr.acinq.eclair.transactions.Transactions._
|
||||
import fr.acinq.eclair.transactions.{CommitmentSpec, DirectedHtlc, IncomingHtlc, OutgoingHtlc}
|
||||
|
@ -88,13 +89,18 @@ private[channel] object ChannelCodecs4 {
|
|||
.typecase(true, minimalHtlcCodec(htlcs.collect(DirectedHtlc.incoming)).as[IncomingHtlc])
|
||||
.typecase(false, minimalHtlcCodec(htlcs.collect(DirectedHtlc.outgoing)).as[OutgoingHtlc])
|
||||
|
||||
/** HTLCs are stored separately to avoid duplicating data. */
|
||||
def commitmentSpecCodec(htlcs: Set[DirectedHtlc]): Codec[CommitmentSpec] = (
|
||||
("htlcs" | setCodec(minimalDirectedHtlcCodec(htlcs))) ::
|
||||
private def baseCommitmentSpecCodec(directedHtlcCodec: Codec[DirectedHtlc]): Codec[CommitmentSpec] = (
|
||||
("htlcs" | setCodec(directedHtlcCodec)) ::
|
||||
("feeratePerKw" | feeratePerKw) ::
|
||||
("toLocal" | millisatoshi) ::
|
||||
("toRemote" | millisatoshi)).as[CommitmentSpec]
|
||||
|
||||
/** HTLCs are stored separately to avoid duplicating data. */
|
||||
def minimalCommitmentSpecCodec(htlcs: Set[DirectedHtlc]): Codec[CommitmentSpec] = baseCommitmentSpecCodec(minimalDirectedHtlcCodec(htlcs))
|
||||
|
||||
/** HTLCs are stored in full, the codec is stateless but creates duplication between local/remote commitment, and across commitments. */
|
||||
val commitmentSpecCodec: Codec[CommitmentSpec] = baseCommitmentSpecCodec(htlcCodec)
|
||||
|
||||
val outPointCodec: Codec[OutPoint] = lengthDelimited(bytes.xmap(d => OutPoint.read(d.toArray), d => OutPoint.write(d)))
|
||||
|
||||
val txOutCodec: Codec[TxOut] = lengthDelimited(bytes.xmap(d => TxOut.read(d.toArray), d => TxOut.write(d)))
|
||||
|
@ -330,33 +336,28 @@ private[channel] object ChannelCodecs4 {
|
|||
("localNextHtlcId" | uint64overflow) ::
|
||||
("remoteNextHtlcId" | uint64overflow)).as[CommitmentChanges]
|
||||
|
||||
def commitmentCodec(htlcs: Set[DirectedHtlc]): Codec[Commitment] = {
|
||||
val localCommitCodec: Codec[LocalCommit] = (
|
||||
("index" | uint64overflow) ::
|
||||
("spec" | commitmentSpecCodec(htlcs)) ::
|
||||
("commitTxAndRemoteSig" | commitTxAndRemoteSigCodec) ::
|
||||
("htlcTxsAndRemoteSigs" | listOfN(uint16, htlcTxsAndRemoteSigsCodec))).as[LocalCommit]
|
||||
private def localCommitCodec(commitmentSpecCodec: Codec[CommitmentSpec]): Codec[LocalCommit] = (
|
||||
("index" | uint64overflow) ::
|
||||
("spec" | commitmentSpecCodec) ::
|
||||
("commitTxAndRemoteSig" | commitTxAndRemoteSigCodec) ::
|
||||
("htlcTxsAndRemoteSigs" | listOfN(uint16, htlcTxsAndRemoteSigsCodec))).as[LocalCommit]
|
||||
|
||||
val remoteCommitCodec: Codec[RemoteCommit] = (
|
||||
("index" | uint64overflow) ::
|
||||
("spec" | commitmentSpecCodec(htlcs.map(_.opposite))) ::
|
||||
("txid" | bytes32) ::
|
||||
("remotePerCommitmentPoint" | publicKey)).as[RemoteCommit]
|
||||
private def remoteCommitCodec(commitmentSpecCodec: Codec[CommitmentSpec]): Codec[RemoteCommit] = (
|
||||
("index" | uint64overflow) ::
|
||||
("spec" | commitmentSpecCodec) ::
|
||||
("txid" | bytes32) ::
|
||||
("remotePerCommitmentPoint" | publicKey)).as[RemoteCommit]
|
||||
|
||||
val nextRemoteCommitCodec: Codec[NextRemoteCommit] = (
|
||||
("sig" | lengthDelimited(commitSigCodec)) ::
|
||||
("commit" | remoteCommitCodec)).as[NextRemoteCommit]
|
||||
private def nextRemoteCommitCodec(commitmentSpecCodec: Codec[CommitmentSpec]): Codec[NextRemoteCommit] = (
|
||||
("sig" | lengthDelimited(commitSigCodec)) ::
|
||||
("commit" | remoteCommitCodec(commitmentSpecCodec))).as[NextRemoteCommit]
|
||||
|
||||
val commitmentCodec: Codec[Commitment] = (
|
||||
("fundingTxStatus" | fundingTxStatusCodec) ::
|
||||
("remoteFundingStatus" | remoteFundingStatusCodec) ::
|
||||
("localCommit" | localCommitCodec) ::
|
||||
("remoteCommit" | remoteCommitCodec) ::
|
||||
("nextRemoteCommit_opt" | optional(bool8, nextRemoteCommitCodec))
|
||||
).as[Commitment]
|
||||
|
||||
commitmentCodec
|
||||
}
|
||||
private def commitmentCodec(htlcs: Set[DirectedHtlc]): Codec[Commitment] = (
|
||||
("fundingTxStatus" | fundingTxStatusCodec) ::
|
||||
("remoteFundingStatus" | remoteFundingStatusCodec) ::
|
||||
("localCommit" | localCommitCodec(minimalCommitmentSpecCodec(htlcs))) ::
|
||||
("remoteCommit" | remoteCommitCodec(minimalCommitmentSpecCodec(htlcs.map(_.opposite)))) ::
|
||||
("nextRemoteCommit_opt" | optional(bool8, nextRemoteCommitCodec(minimalCommitmentSpecCodec(htlcs.map(_.opposite)))))).as[Commitment]
|
||||
|
||||
/**
|
||||
* When multiple commitments are active, htlcs are shared between all of these commitments.
|
||||
|
@ -452,6 +453,28 @@ private[channel] object ChannelCodecs4 {
|
|||
("claimHtlcDelayedPenaltyTxs" | listOfN(uint16, claimHtlcDelayedOutputPenaltyTxCodec)) ::
|
||||
("spent" | spentMapCodec)).as[RevokedCommitPublished]
|
||||
|
||||
// We don't bother removing the duplication across HTLCs: this is a short-lived state during which the channel
|
||||
// cannot be used for payments.
|
||||
private val interactiveTxWaitingForSigsCodec: Codec[InteractiveTxSigningSession.WaitingForSigs] = {
|
||||
val unsignedLocalCommitCodec: Codec[UnsignedLocalCommit] = (
|
||||
("index" | uint64overflow) ::
|
||||
("spec" | commitmentSpecCodec) ::
|
||||
("commitTx" | commitTxCodec) ::
|
||||
("htlcTxs" | listOfN(uint16, htlcTxCodec))).as[UnsignedLocalCommit]
|
||||
|
||||
val waitingForSigsCodec: Codec[InteractiveTxSigningSession.WaitingForSigs] = (
|
||||
("fundingParams" | fundingParamsCodec) ::
|
||||
("fundingTx" | partiallySignedSharedTransactionCodec) ::
|
||||
("localCommit" | either(bool8, unsignedLocalCommitCodec, localCommitCodec(commitmentSpecCodec))) ::
|
||||
("remoteCommit" | remoteCommitCodec(commitmentSpecCodec))).as[InteractiveTxSigningSession.WaitingForSigs]
|
||||
|
||||
waitingForSigsCodec
|
||||
}
|
||||
|
||||
val rbfStatusCodec: Codec[RbfStatus] = discriminated[RbfStatus].by(uint8)
|
||||
.\(0x01) { case status: RbfStatus if !status.isInstanceOf[RbfStatus.RbfWaitingForSigs] => RbfStatus.NoRbf }(provide(RbfStatus.NoRbf))
|
||||
.\(0x02) { case status: RbfStatus.RbfWaitingForSigs => status }(interactiveTxWaitingForSigsCodec.as[RbfStatus.RbfWaitingForSigs])
|
||||
|
||||
val DATA_WAIT_FOR_FUNDING_CONFIRMED_00_Codec: Codec[DATA_WAIT_FOR_FUNDING_CONFIRMED] = (
|
||||
("commitments" | commitmentsCodec) ::
|
||||
("waitingSince" | blockHeight) ::
|
||||
|
@ -462,13 +485,21 @@ private[channel] object ChannelCodecs4 {
|
|||
("commitments" | commitmentsCodec) ::
|
||||
("shortIds" | shortids)).as[DATA_WAIT_FOR_CHANNEL_READY]
|
||||
|
||||
val DATA_WAIT_FOR_DUAL_FUNDING_SIGNED_09_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED] = (
|
||||
("channelParams" | paramsCodec) ::
|
||||
("secondRemotePerCommitmentPoint" | publicKey) ::
|
||||
("localPushAmount" | millisatoshi) ::
|
||||
("remotePushAmount" | millisatoshi) ::
|
||||
("status" | interactiveTxWaitingForSigsCodec) ::
|
||||
("remoteChannelData_opt" | optional(bool8, varsizebinarydata))).as[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED]
|
||||
|
||||
val DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED_02_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED] = (
|
||||
("commitments" | commitmentsCodec) ::
|
||||
("localPushAmount" | millisatoshi) ::
|
||||
("remotePushAmount" | millisatoshi) ::
|
||||
("waitingSince" | blockHeight) ::
|
||||
("lastChecked" | blockHeight) ::
|
||||
("rbfStatus" | provide[RbfStatus](RbfStatus.NoRbf)) ::
|
||||
("rbfStatus" | rbfStatusCodec) ::
|
||||
("deferred" | optional(bool8, lengthDelimited(channelReadyCodec)))).as[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
|
||||
val DATA_WAIT_FOR_DUAL_FUNDING_READY_03_Codec: Codec[DATA_WAIT_FOR_DUAL_FUNDING_READY] = (
|
||||
|
@ -516,6 +547,7 @@ private[channel] object ChannelCodecs4 {
|
|||
|
||||
// Order matters!
|
||||
val channelDataCodec: Codec[PersistentChannelData] = discriminated[PersistentChannelData].by(uint16)
|
||||
.typecase(0x09, Codecs.DATA_WAIT_FOR_DUAL_FUNDING_SIGNED_09_Codec)
|
||||
.typecase(0x08, Codecs.DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT_08_Codec)
|
||||
.typecase(0x07, Codecs.DATA_CLOSING_07_Codec)
|
||||
.typecase(0x06, Codecs.DATA_NEGOTIATING_06_Codec)
|
||||
|
|
|
@ -16,7 +16,7 @@
|
|||
|
||||
package fr.acinq.eclair.wire.protocol
|
||||
|
||||
import fr.acinq.bitcoin.scalacompat.Satoshi
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi}
|
||||
import fr.acinq.eclair.channel.{ChannelType, ChannelTypes}
|
||||
import fr.acinq.eclair.wire.protocol.CommonCodecs._
|
||||
import fr.acinq.eclair.wire.protocol.TlvCodecs.{tlvField, tlvStream, tmillisatoshi}
|
||||
|
@ -135,7 +135,16 @@ object ChannelReadyTlv {
|
|||
sealed trait ChannelReestablishTlv extends Tlv
|
||||
|
||||
object ChannelReestablishTlv {
|
||||
val channelReestablishTlvCodec: Codec[TlvStream[ChannelReestablishTlv]] = tlvStream(discriminated[ChannelReestablishTlv].by(varint))
|
||||
|
||||
case class NextFundingTlv(txHash: ByteVector32) extends ChannelReestablishTlv
|
||||
|
||||
object NextFundingTlv {
|
||||
val codec: Codec[NextFundingTlv] = tlvField("funding_tx_hash" | bytes32)
|
||||
}
|
||||
|
||||
val channelReestablishTlvCodec: Codec[TlvStream[ChannelReestablishTlv]] = tlvStream(discriminated[ChannelReestablishTlv].by(varint)
|
||||
.typecase(UInt64(333), NextFundingTlv.codec)
|
||||
)
|
||||
}
|
||||
|
||||
sealed trait UpdateFeeTlv extends Tlv
|
||||
|
|
|
@ -165,7 +165,9 @@ case class ChannelReestablish(channelId: ByteVector32,
|
|||
nextRemoteRevocationNumber: Long,
|
||||
yourLastPerCommitmentSecret: PrivateKey,
|
||||
myCurrentPerCommitmentPoint: PublicKey,
|
||||
tlvStream: TlvStream[ChannelReestablishTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId
|
||||
tlvStream: TlvStream[ChannelReestablishTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId {
|
||||
val nextFundingTxId_opt: Option[ByteVector32] = tlvStream.get[ChannelReestablishTlv.NextFundingTlv].map(_.txHash.reverse)
|
||||
}
|
||||
|
||||
case class OpenChannel(chainHash: ByteVector32,
|
||||
temporaryChannelId: ByteVector32,
|
||||
|
|
|
@ -74,6 +74,7 @@ object TestDatabases {
|
|||
def freeze3(input: PersistentChannelData): PersistentChannelData = input match {
|
||||
case d: DATA_WAIT_FOR_FUNDING_CONFIRMED => d.copy(commitments = freeze2(d.commitments))
|
||||
case d: DATA_WAIT_FOR_CHANNEL_READY => d.copy(commitments = freeze2(d.commitments))
|
||||
case d: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED => d
|
||||
case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => d.copy(commitments = freeze2(d.commitments))
|
||||
case d: DATA_WAIT_FOR_DUAL_FUNDING_READY => d.copy(commitments = freeze2(d.commitments))
|
||||
case d: DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT => d.copy(commitments = freeze2(d.commitments))
|
||||
|
|
File diff suppressed because it is too large
Load diff
|
@ -404,7 +404,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
|
|||
s ! cmdAdd
|
||||
val htlc = s2r.expectMsgType[UpdateAddHtlc]
|
||||
s2r.forward(r)
|
||||
eventually(assert(r.stateData.asInstanceOf[PersistentChannelData].commitments.changes.remoteChanges.proposed.contains(htlc)))
|
||||
eventually(assert(r.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.changes.remoteChanges.proposed.contains(htlc)))
|
||||
htlc
|
||||
}
|
||||
|
||||
|
@ -412,21 +412,21 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
|
|||
s ! CMD_FULFILL_HTLC(id, preimage)
|
||||
val fulfill = s2r.expectMsgType[UpdateFulfillHtlc]
|
||||
s2r.forward(r)
|
||||
eventually(assert(r.stateData.asInstanceOf[PersistentChannelData].commitments.changes.remoteChanges.proposed.contains(fulfill)))
|
||||
eventually(assert(r.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.changes.remoteChanges.proposed.contains(fulfill)))
|
||||
}
|
||||
|
||||
def failHtlc(id: Long, s: TestFSMRef[ChannelState, ChannelData, Channel], r: TestFSMRef[ChannelState, ChannelData, Channel], s2r: TestProbe, r2s: TestProbe): Unit = {
|
||||
s ! CMD_FAIL_HTLC(id, Right(TemporaryNodeFailure()))
|
||||
val fail = s2r.expectMsgType[UpdateFailHtlc]
|
||||
s2r.forward(r)
|
||||
eventually(assert(r.stateData.asInstanceOf[PersistentChannelData].commitments.changes.remoteChanges.proposed.contains(fail)))
|
||||
eventually(assert(r.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.changes.remoteChanges.proposed.contains(fail)))
|
||||
}
|
||||
|
||||
def crossSign(s: TestFSMRef[ChannelState, ChannelData, Channel], r: TestFSMRef[ChannelState, ChannelData, Channel], s2r: TestProbe, r2s: TestProbe): Unit = {
|
||||
val sender = TestProbe()
|
||||
val sCommitIndex = s.stateData.asInstanceOf[PersistentChannelData].commitments.localCommitIndex
|
||||
val rCommitIndex = r.stateData.asInstanceOf[PersistentChannelData].commitments.localCommitIndex
|
||||
val rHasChanges = r.stateData.asInstanceOf[PersistentChannelData].commitments.changes.localHasChanges
|
||||
val sCommitIndex = s.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.localCommitIndex
|
||||
val rCommitIndex = r.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.localCommitIndex
|
||||
val rHasChanges = r.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.changes.localHasChanges
|
||||
s ! CMD_SIGN(Some(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_SIGN]]
|
||||
s2r.expectMsgType[CommitSig]
|
||||
|
@ -443,17 +443,17 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
|
|||
r2s.expectMsgType[RevokeAndAck]
|
||||
r2s.forward(s)
|
||||
eventually {
|
||||
assert(s.stateData.asInstanceOf[PersistentChannelData].commitments.localCommitIndex == sCommitIndex + 1)
|
||||
assert(s.stateData.asInstanceOf[PersistentChannelData].commitments.remoteCommitIndex == sCommitIndex + 2)
|
||||
assert(r.stateData.asInstanceOf[PersistentChannelData].commitments.localCommitIndex == rCommitIndex + 2)
|
||||
assert(r.stateData.asInstanceOf[PersistentChannelData].commitments.remoteCommitIndex == rCommitIndex + 1)
|
||||
assert(s.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.localCommitIndex == sCommitIndex + 1)
|
||||
assert(s.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.remoteCommitIndex == sCommitIndex + 2)
|
||||
assert(r.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.localCommitIndex == rCommitIndex + 2)
|
||||
assert(r.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.remoteCommitIndex == rCommitIndex + 1)
|
||||
}
|
||||
} else {
|
||||
eventually {
|
||||
assert(s.stateData.asInstanceOf[PersistentChannelData].commitments.localCommitIndex == sCommitIndex + 1)
|
||||
assert(s.stateData.asInstanceOf[PersistentChannelData].commitments.remoteCommitIndex == sCommitIndex + 1)
|
||||
assert(r.stateData.asInstanceOf[PersistentChannelData].commitments.localCommitIndex == rCommitIndex + 1)
|
||||
assert(r.stateData.asInstanceOf[PersistentChannelData].commitments.remoteCommitIndex == rCommitIndex + 1)
|
||||
assert(s.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.localCommitIndex == sCommitIndex + 1)
|
||||
assert(s.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.remoteCommitIndex == sCommitIndex + 1)
|
||||
assert(r.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.localCommitIndex == rCommitIndex + 1)
|
||||
assert(r.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.remoteCommitIndex == rCommitIndex + 1)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -470,7 +470,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
|
|||
r2s.forward(s)
|
||||
s2r.expectMsgType[RevokeAndAck]
|
||||
s2r.forward(r)
|
||||
eventually(assert(s.stateData.asInstanceOf[PersistentChannelData].commitments.latest.localCommit.spec.commitTxFeerate == feerate))
|
||||
eventually(assert(s.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.latest.localCommit.spec.commitTxFeerate == feerate))
|
||||
}
|
||||
|
||||
def mutualClose(s: TestFSMRef[ChannelState, ChannelData, Channel], r: TestFSMRef[ChannelState, ChannelData, Channel], s2r: TestProbe, r2s: TestProbe, s2blockchain: TestProbe, r2blockchain: TestProbe): Unit = {
|
||||
|
|
|
@ -18,19 +18,17 @@ package fr.acinq.eclair.channel.states.b
|
|||
|
||||
import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, SatoshiLong, Script}
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong, Script}
|
||||
import fr.acinq.eclair.blockchain.SingleKeyOnChainWallet
|
||||
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchPublished}
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.fsm.Channel
|
||||
import fr.acinq.eclair.channel.fsm.Channel.TickChannelOpenTimeout
|
||||
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTransaction, PartiallySignedSharedTransaction}
|
||||
import fr.acinq.eclair.channel.publish.TxPublisher
|
||||
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
|
||||
import fr.acinq.eclair.io.Peer.OpenChannelResponse
|
||||
import fr.acinq.eclair.wire.protocol.{AcceptDualFundedChannel, CommitSig, Error, Init, OpenDualFundedChannel, TxAbort, TxAckRbf, TxAddInput, TxAddOutput, TxComplete, TxInitRbf, TxSignatures, Warning}
|
||||
import fr.acinq.eclair.{Features, MilliSatoshiLong, TestConstants, TestKitBaseClass, ToMilliSatoshiConversion, UInt64, randomBytes32, randomKey}
|
||||
import fr.acinq.eclair.wire.protocol.{AcceptDualFundedChannel, ChannelReestablish, CommitSig, Error, Init, OpenDualFundedChannel, TxAbort, TxAckRbf, TxAddInput, TxAddOutput, TxComplete, TxInitRbf, Warning}
|
||||
import fr.acinq.eclair.{TestConstants, TestKitBaseClass, UInt64, randomKey}
|
||||
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
|
||||
import org.scalatest.{Outcome, Tag}
|
||||
import scodec.bits.HexStringSyntax
|
||||
|
@ -50,15 +48,13 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
|
|||
val (aliceParams, bobParams, channelType) = computeFeatures(setup, test.tags, channelFlags)
|
||||
val aliceInit = Init(aliceParams.initFeatures)
|
||||
val bobInit = Init(bobParams.initFeatures)
|
||||
val bobContribution = if (channelType.features.contains(Features.ZeroConf)) None else Some(TestConstants.nonInitiatorFundingSatoshis)
|
||||
val (initiatorPushAmount, nonInitiatorPushAmount) = if (test.tags.contains("both_push_amount")) (Some(TestConstants.initiatorPushAmount), Some(TestConstants.nonInitiatorPushAmount)) else (None, None)
|
||||
val aliceListener = TestProbe()
|
||||
val bobListener = TestProbe()
|
||||
within(30 seconds) {
|
||||
alice.underlying.system.eventStream.subscribe(aliceListener.ref, classOf[ChannelAborted])
|
||||
bob.underlying.system.eventStream.subscribe(bobListener.ref, classOf[ChannelAborted])
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.feeratePerKw, TestConstants.feeratePerKw, initiatorPushAmount, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, bobContribution, dualFunded = true, nonInitiatorPushAmount, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.feeratePerKw, TestConstants.feeratePerKw, None, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, Some(TestConstants.nonInitiatorFundingSatoshis), dualFunded = true, None, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id
|
||||
alice2bob.expectMsgType[OpenDualFundedChannel]
|
||||
|
@ -76,9 +72,6 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
|
|||
test("complete interactive-tx protocol", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val listener = TestProbe()
|
||||
alice.underlyingActor.context.system.eventStream.subscribe(listener.ref, classOf[TransactionPublished])
|
||||
|
||||
// The initiator sends the first interactive-tx message.
|
||||
bob2alice.expectNoMessage(100 millis)
|
||||
alice2bob.expectMsgType[TxAddInput]
|
||||
|
@ -96,126 +89,9 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
|
|||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxComplete]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[CommitSig]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
alice2bob.forward(bob)
|
||||
|
||||
// Bob sends its signatures first as he contributed less than Alice.
|
||||
bob2alice.expectMsgType[TxSignatures]
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
assert(bobData.commitments.params.channelFeatures.hasFeature(Features.DualFunding))
|
||||
assert(bobData.latestFundingTx.sharedTx.isInstanceOf[PartiallySignedSharedTransaction])
|
||||
val fundingTxId = bobData.latestFundingTx.sharedTx.asInstanceOf[PartiallySignedSharedTransaction].txId
|
||||
assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTxId)
|
||||
|
||||
// Alice receives Bob's signatures and sends her own signatures.
|
||||
bob2alice.forward(alice)
|
||||
assert(listener.expectMsgType[TransactionPublished].tx.txid == fundingTxId)
|
||||
assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTxId)
|
||||
alice2bob.expectMsgType[TxSignatures]
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
assert(aliceData.commitments.params.channelFeatures.hasFeature(Features.DualFunding))
|
||||
assert(aliceData.latestFundingTx.sharedTx.isInstanceOf[FullySignedSharedTransaction])
|
||||
assert(aliceData.latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction].signedTx.txid == fundingTxId)
|
||||
}
|
||||
|
||||
test("complete interactive-tx protocol (zero-conf)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.ZeroConf), Tag(ChannelStateTestsTags.ScidAlias), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
|
||||
import f._
|
||||
|
||||
val listener = TestProbe()
|
||||
alice.underlyingActor.context.system.eventStream.subscribe(listener.ref, classOf[TransactionPublished])
|
||||
|
||||
alice2bob.expectMsgType[TxAddInput]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[TxComplete]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxAddOutput]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[TxComplete]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxAddOutput]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[TxComplete]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxComplete]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[CommitSig]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
alice2bob.forward(bob)
|
||||
|
||||
// Bob sends its signatures first as he contributed less than Alice.
|
||||
bob2alice.expectMsgType[TxSignatures]
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
assert(bobData.commitments.params.channelFeatures.hasFeature(Features.DualFunding))
|
||||
assert(bobData.latestFundingTx.sharedTx.isInstanceOf[PartiallySignedSharedTransaction])
|
||||
val fundingTxId = bobData.latestFundingTx.sharedTx.asInstanceOf[PartiallySignedSharedTransaction].tx.buildUnsignedTx().txid
|
||||
assert(bob2blockchain.expectMsgType[WatchPublished].txId == fundingTxId)
|
||||
bob2blockchain.expectNoMessage(100 millis)
|
||||
|
||||
// Alice receives Bob's signatures and sends her own signatures.
|
||||
bob2alice.forward(alice)
|
||||
assert(listener.expectMsgType[TransactionPublished].tx.txid == fundingTxId)
|
||||
assert(alice2blockchain.expectMsgType[WatchPublished].txId == fundingTxId)
|
||||
alice2bob.expectMsgType[TxSignatures]
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
assert(aliceData.commitments.params.channelFeatures.hasFeature(Features.DualFunding))
|
||||
assert(aliceData.latestFundingTx.sharedTx.isInstanceOf[FullySignedSharedTransaction])
|
||||
assert(aliceData.latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction].signedTx.txid == fundingTxId)
|
||||
}
|
||||
|
||||
test("complete interactive-tx protocol (with push amount)", Tag(ChannelStateTestsTags.DualFunding), Tag("both_push_amount")) { f =>
|
||||
import f._
|
||||
|
||||
val listener = TestProbe()
|
||||
alice.underlyingActor.context.system.eventStream.subscribe(listener.ref, classOf[TransactionPublished])
|
||||
|
||||
// The initiator sends the first interactive-tx message.
|
||||
bob2alice.expectNoMessage(100 millis)
|
||||
alice2bob.expectMsgType[TxAddInput]
|
||||
alice2bob.expectNoMessage(100 millis)
|
||||
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)
|
||||
|
||||
val expectedBalanceAlice = TestConstants.fundingSatoshis.toMilliSatoshi + TestConstants.nonInitiatorPushAmount - TestConstants.initiatorPushAmount
|
||||
assert(expectedBalanceAlice == 900_000_000.msat)
|
||||
val expectedBalanceBob = TestConstants.nonInitiatorFundingSatoshis.toMilliSatoshi + TestConstants.initiatorPushAmount - TestConstants.nonInitiatorPushAmount
|
||||
assert(expectedBalanceBob == 600_000_000.msat)
|
||||
|
||||
// Bob sends its signatures first as he contributed less than Alice.
|
||||
bob2alice.expectMsgType[TxSignatures]
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
assert(bobData.commitments.latest.localCommit.spec.toLocal == expectedBalanceBob)
|
||||
assert(bobData.commitments.latest.localCommit.spec.toRemote == expectedBalanceAlice)
|
||||
|
||||
// Alice receives Bob's signatures and sends her own signatures.
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxSignatures]
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
assert(aliceData.commitments.latest.localCommit.spec.toLocal == expectedBalanceAlice)
|
||||
assert(aliceData.commitments.latest.localCommit.spec.toRemote == expectedBalanceBob)
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
|
||||
}
|
||||
|
||||
test("recv invalid interactive-tx message", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
|
@ -239,78 +115,6 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
|
|||
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
|
||||
}
|
||||
|
||||
test("recv invalid CommitSig", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
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)
|
||||
val bobCommitSig = bob2alice.expectMsgType[CommitSig]
|
||||
val aliceCommitSig = alice2bob.expectMsgType[CommitSig]
|
||||
|
||||
bob2alice.forward(alice, bobCommitSig.copy(signature = ByteVector64.Zeroes))
|
||||
alice2bob.expectMsgType[TxAbort]
|
||||
awaitCond(wallet.rolledback.length == 1)
|
||||
aliceListener.expectMsgType[ChannelAborted]
|
||||
awaitCond(alice.stateName == CLOSED)
|
||||
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
|
||||
|
||||
alice2bob.forward(bob, aliceCommitSig.copy(signature = ByteVector64.Zeroes))
|
||||
bob2alice.expectMsgType[TxAbort]
|
||||
awaitCond(wallet.rolledback.length == 2)
|
||||
bobListener.expectMsgType[ChannelAborted]
|
||||
awaitCond(bob.stateName == CLOSED)
|
||||
}
|
||||
|
||||
test("recv invalid TxSignatures", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
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)
|
||||
|
||||
val bobSigs = bob2alice.expectMsgType[TxSignatures]
|
||||
bob2blockchain.expectMsgType[WatchFundingConfirmed]
|
||||
bob2alice.forward(alice, bobSigs.copy(txHash = randomBytes32(), witnesses = Nil))
|
||||
alice2bob.expectMsgType[TxAbort]
|
||||
awaitCond(wallet.rolledback.size == 1)
|
||||
aliceListener.expectMsgType[ChannelAborted]
|
||||
awaitCond(alice.stateName == CLOSED)
|
||||
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Rejected]
|
||||
|
||||
// Bob has sent his signatures already, so he cannot close the channel yet.
|
||||
alice2bob.forward(bob, TxSignatures(channelId(alice), randomBytes32(), Nil))
|
||||
bob2alice.expectMsgType[Error]
|
||||
bob2blockchain.expectNoMessage(100 millis)
|
||||
assert(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
}
|
||||
|
||||
test("recv TxAbort", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
|
@ -410,8 +214,46 @@ class WaitForDualFundingCreatedStateSpec extends TestKitBaseClass with FixtureAn
|
|||
awaitCond(bob.stateName == CLOSED)
|
||||
}
|
||||
|
||||
test("recv INPUT_DISCONNECTED (alice sent commit_sig)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
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] // bob doesn't receive alice's tx_complete
|
||||
alice2bob.expectMsgType[CommitSig] // bob doesn't receive alice's commit_sig
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CREATED)
|
||||
|
||||
val channelId = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED].channelId
|
||||
val fundingTxId = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED].signingSession.fundingTx.txId
|
||||
|
||||
alice ! INPUT_DISCONNECTED
|
||||
awaitCond(alice.stateName == OFFLINE)
|
||||
bob ! INPUT_DISCONNECTED
|
||||
awaitCond(bob.stateName == CLOSED)
|
||||
|
||||
val aliceInit = Init(alice.underlyingActor.nodeParams.features.initFeatures())
|
||||
val bobInit = Init(bob.underlyingActor.nodeParams.features.initFeatures())
|
||||
alice ! INPUT_RECONNECTED(bob, aliceInit, bobInit)
|
||||
assert(alice2bob.expectMsgType[ChannelReestablish].nextFundingTxId_opt.contains(fundingTxId))
|
||||
alice ! Error(channelId, "unknown channel")
|
||||
aliceListener.expectMsgType[ChannelAborted]
|
||||
awaitCond(alice.stateName == CLOSED)
|
||||
}
|
||||
|
||||
test("recv TickChannelOpenTimeout", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
alice ! TickChannelOpenTimeout
|
||||
awaitCond(wallet.rolledback.size == 1)
|
||||
aliceListener.expectMsgType[ChannelAborted]
|
||||
|
|
|
@ -0,0 +1,423 @@
|
|||
/*
|
||||
* Copyright 2023 ACINQ SAS
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package fr.acinq.eclair.channel.states.b
|
||||
|
||||
import akka.actor.typed.scaladsl.adapter.ClassicActorRefOps
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, SatoshiLong}
|
||||
import fr.acinq.eclair.blockchain.SingleKeyOnChainWallet
|
||||
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchPublished}
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.fsm.Channel
|
||||
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTransaction, PartiallySignedSharedTransaction}
|
||||
import fr.acinq.eclair.channel.publish.TxPublisher
|
||||
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
|
||||
import fr.acinq.eclair.io.Peer.OpenChannelResponse
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
import fr.acinq.eclair.{Features, MilliSatoshiLong, TestConstants, TestKitBaseClass, ToMilliSatoshiConversion, randomBytes32}
|
||||
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
|
||||
import org.scalatest.{Outcome, Tag}
|
||||
|
||||
import scala.concurrent.duration.DurationInt
|
||||
|
||||
class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with ChannelStateTestsBase {
|
||||
|
||||
case class FixtureParam(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel], alice2bob: TestProbe, bob2alice: TestProbe, alice2blockchain: TestProbe, bob2blockchain: TestProbe, wallet: SingleKeyOnChainWallet, aliceListener: TestProbe, bobListener: TestProbe)
|
||||
|
||||
override def withFixture(test: OneArgTest): Outcome = {
|
||||
val wallet = new SingleKeyOnChainWallet()
|
||||
val setup = init(wallet_opt = Some(wallet), tags = test.tags)
|
||||
import setup._
|
||||
val channelConfig = ChannelConfig.standard
|
||||
val channelFlags = ChannelFlags.Private
|
||||
val (aliceParams, bobParams, channelType) = computeFeatures(setup, test.tags, channelFlags)
|
||||
val aliceInit = Init(aliceParams.initFeatures)
|
||||
val bobInit = Init(bobParams.initFeatures)
|
||||
val bobContribution = if (channelType.features.contains(Features.ZeroConf)) None else Some(TestConstants.nonInitiatorFundingSatoshis)
|
||||
val (initiatorPushAmount, nonInitiatorPushAmount) = if (test.tags.contains("both_push_amount")) (Some(TestConstants.initiatorPushAmount), Some(TestConstants.nonInitiatorPushAmount)) else (None, None)
|
||||
val aliceListener = TestProbe()
|
||||
val bobListener = TestProbe()
|
||||
within(30 seconds) {
|
||||
alice.underlying.system.eventStream.subscribe(aliceListener.ref, classOf[ChannelAborted])
|
||||
bob.underlying.system.eventStream.subscribe(bobListener.ref, classOf[ChannelAborted])
|
||||
alice ! INPUT_INIT_CHANNEL_INITIATOR(ByteVector32.Zeroes, TestConstants.fundingSatoshis, dualFunded = true, TestConstants.feeratePerKw, TestConstants.feeratePerKw, initiatorPushAmount, requireConfirmedInputs = false, aliceParams, alice2bob.ref, bobInit, channelFlags, channelConfig, channelType, replyTo = aliceOpenReplyTo.ref.toTyped)
|
||||
bob ! INPUT_INIT_CHANNEL_NON_INITIATOR(ByteVector32.Zeroes, bobContribution, dualFunded = true, nonInitiatorPushAmount, bobParams, bob2alice.ref, aliceInit, channelConfig, channelType)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId] // temporary channel id
|
||||
alice2bob.expectMsgType[OpenDualFundedChannel]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[AcceptDualFundedChannel]
|
||||
bob2alice.forward(alice)
|
||||
alice2blockchain.expectMsgType[TxPublisher.SetChannelId] // final channel id
|
||||
bob2blockchain.expectMsgType[TxPublisher.SetChannelId] // final channel id
|
||||
// Alice and Bob complete the interactive-tx protocol.
|
||||
alice2bob.expectMsgType[TxAddInput]
|
||||
alice2bob.forward(bob)
|
||||
bobContribution match {
|
||||
case Some(_) => bob2alice.expectMsgType[TxAddInput]
|
||||
case None => bob2alice.expectMsgType[TxComplete]
|
||||
}
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxAddOutput]
|
||||
alice2bob.forward(bob)
|
||||
bobContribution match {
|
||||
case Some(_) => bob2alice.expectMsgType[TxAddOutput]
|
||||
case None => bob2alice.expectMsgType[TxComplete]
|
||||
}
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxAddOutput]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[TxComplete]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxComplete]
|
||||
alice2bob.forward(bob)
|
||||
aliceOpenReplyTo.expectMsgType[OpenChannelResponse.Created]
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
|
||||
withFixture(test.toNoArgTest(FixtureParam(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain, wallet, aliceListener, bobListener)))
|
||||
}
|
||||
}
|
||||
|
||||
test("complete interactive-tx protocol", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val listener = TestProbe()
|
||||
alice.underlyingActor.context.system.eventStream.subscribe(listener.ref, classOf[TransactionPublished])
|
||||
|
||||
bob2alice.expectMsgType[CommitSig]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
alice2bob.forward(bob)
|
||||
|
||||
// Bob sends its signatures first as he contributed less than Alice.
|
||||
bob2alice.expectMsgType[TxSignatures]
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
assert(bobData.commitments.params.channelFeatures.hasFeature(Features.DualFunding))
|
||||
assert(bobData.latestFundingTx.sharedTx.isInstanceOf[PartiallySignedSharedTransaction])
|
||||
val fundingTxId = bobData.latestFundingTx.sharedTx.asInstanceOf[PartiallySignedSharedTransaction].txId
|
||||
assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTxId)
|
||||
|
||||
// Alice receives Bob's signatures and sends her own signatures.
|
||||
bob2alice.forward(alice)
|
||||
assert(listener.expectMsgType[TransactionPublished].tx.txid == fundingTxId)
|
||||
assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTxId)
|
||||
alice2bob.expectMsgType[TxSignatures]
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
assert(aliceData.commitments.params.channelFeatures.hasFeature(Features.DualFunding))
|
||||
assert(aliceData.latestFundingTx.sharedTx.isInstanceOf[FullySignedSharedTransaction])
|
||||
assert(aliceData.latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction].signedTx.txid == fundingTxId)
|
||||
}
|
||||
|
||||
test("complete interactive-tx protocol (zero-conf)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.ZeroConf), Tag(ChannelStateTestsTags.ScidAlias), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
|
||||
import f._
|
||||
|
||||
val listener = TestProbe()
|
||||
alice.underlyingActor.context.system.eventStream.subscribe(listener.ref, classOf[TransactionPublished])
|
||||
|
||||
bob2alice.expectMsgType[CommitSig]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
alice2bob.forward(bob)
|
||||
|
||||
// Bob sends its signatures first as he contributed less than Alice.
|
||||
bob2alice.expectMsgType[TxSignatures]
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
assert(bobData.commitments.params.channelFeatures.hasFeature(Features.DualFunding))
|
||||
assert(bobData.latestFundingTx.sharedTx.isInstanceOf[PartiallySignedSharedTransaction])
|
||||
val fundingTxId = bobData.latestFundingTx.sharedTx.asInstanceOf[PartiallySignedSharedTransaction].tx.buildUnsignedTx().txid
|
||||
assert(bob2blockchain.expectMsgType[WatchPublished].txId == fundingTxId)
|
||||
bob2blockchain.expectNoMessage(100 millis)
|
||||
|
||||
// Alice receives Bob's signatures and sends her own signatures.
|
||||
bob2alice.forward(alice)
|
||||
assert(listener.expectMsgType[TransactionPublished].tx.txid == fundingTxId)
|
||||
assert(alice2blockchain.expectMsgType[WatchPublished].txId == fundingTxId)
|
||||
alice2bob.expectMsgType[TxSignatures]
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
assert(aliceData.commitments.params.channelFeatures.hasFeature(Features.DualFunding))
|
||||
assert(aliceData.latestFundingTx.sharedTx.isInstanceOf[FullySignedSharedTransaction])
|
||||
assert(aliceData.latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction].signedTx.txid == fundingTxId)
|
||||
}
|
||||
|
||||
test("complete interactive-tx protocol (with push amount)", Tag(ChannelStateTestsTags.DualFunding), Tag("both_push_amount")) { f =>
|
||||
import f._
|
||||
|
||||
val listener = TestProbe()
|
||||
alice.underlyingActor.context.system.eventStream.subscribe(listener.ref, classOf[TransactionPublished])
|
||||
|
||||
bob2alice.expectMsgType[CommitSig]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
alice2bob.forward(bob)
|
||||
|
||||
val expectedBalanceAlice = TestConstants.fundingSatoshis.toMilliSatoshi + TestConstants.nonInitiatorPushAmount - TestConstants.initiatorPushAmount
|
||||
assert(expectedBalanceAlice == 900_000_000.msat)
|
||||
val expectedBalanceBob = TestConstants.nonInitiatorFundingSatoshis.toMilliSatoshi + TestConstants.initiatorPushAmount - TestConstants.nonInitiatorPushAmount
|
||||
assert(expectedBalanceBob == 600_000_000.msat)
|
||||
|
||||
// Bob sends its signatures first as he contributed less than Alice.
|
||||
bob2alice.expectMsgType[TxSignatures]
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
val bobData = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
assert(bobData.commitments.latest.localCommit.spec.toLocal == expectedBalanceBob)
|
||||
assert(bobData.commitments.latest.localCommit.spec.toRemote == expectedBalanceAlice)
|
||||
|
||||
// Alice receives Bob's signatures and sends her own signatures.
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxSignatures]
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
val aliceData = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED]
|
||||
assert(aliceData.commitments.latest.localCommit.spec.toLocal == expectedBalanceAlice)
|
||||
assert(aliceData.commitments.latest.localCommit.spec.toRemote == expectedBalanceBob)
|
||||
}
|
||||
|
||||
test("recv invalid CommitSig", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val bobCommitSig = bob2alice.expectMsgType[CommitSig]
|
||||
val aliceCommitSig = alice2bob.expectMsgType[CommitSig]
|
||||
|
||||
bob2alice.forward(alice, bobCommitSig.copy(signature = ByteVector64.Zeroes))
|
||||
alice2bob.expectMsgType[Error]
|
||||
awaitCond(wallet.rolledback.length == 1)
|
||||
aliceListener.expectMsgType[ChannelAborted]
|
||||
awaitCond(alice.stateName == CLOSED)
|
||||
|
||||
alice2bob.forward(bob, aliceCommitSig.copy(signature = ByteVector64.Zeroes))
|
||||
bob2alice.expectMsgType[Error]
|
||||
awaitCond(wallet.rolledback.length == 2)
|
||||
bobListener.expectMsgType[ChannelAborted]
|
||||
awaitCond(bob.stateName == CLOSED)
|
||||
}
|
||||
|
||||
test("recv invalid TxSignatures", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
bob2alice.expectMsgType[CommitSig]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
alice2bob.forward(bob)
|
||||
|
||||
val bobSigs = bob2alice.expectMsgType[TxSignatures]
|
||||
bob2blockchain.expectMsgType[WatchFundingConfirmed]
|
||||
bob2alice.forward(alice, bobSigs.copy(txHash = randomBytes32(), witnesses = Nil))
|
||||
alice2bob.expectMsgType[Error]
|
||||
awaitCond(wallet.rolledback.size == 1)
|
||||
aliceListener.expectMsgType[ChannelAborted]
|
||||
awaitCond(alice.stateName == CLOSED)
|
||||
|
||||
// Bob has sent his signatures already, so he cannot close the channel yet.
|
||||
alice2bob.forward(bob, TxSignatures(channelId(alice), randomBytes32(), Nil))
|
||||
bob2alice.expectMsgType[Error]
|
||||
bob2blockchain.expectNoMessage(100 millis)
|
||||
assert(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
}
|
||||
|
||||
test("recv TxInitRbf", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
bob2alice.expectMsgType[CommitSig]
|
||||
|
||||
alice2bob.forward(bob, TxInitRbf(channelId(alice), 0, FeeratePerKw(15_000 sat)))
|
||||
bob2alice.expectMsgType[Warning]
|
||||
assert(bob.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
|
||||
|
||||
bob2alice.forward(alice, TxInitRbf(channelId(bob), 0, FeeratePerKw(15_000 sat)))
|
||||
alice2bob.expectMsgType[Warning]
|
||||
assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
|
||||
assert(wallet.rolledback.isEmpty)
|
||||
}
|
||||
|
||||
test("recv TxAckRbf", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
bob2alice.expectMsgType[CommitSig]
|
||||
|
||||
alice2bob.forward(bob, TxAckRbf(channelId(alice)))
|
||||
bob2alice.expectMsgType[Warning]
|
||||
assert(bob.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
|
||||
|
||||
bob2alice.forward(alice, TxAckRbf(channelId(bob)))
|
||||
alice2bob.expectMsgType[Warning]
|
||||
assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
|
||||
assert(wallet.rolledback.isEmpty)
|
||||
}
|
||||
|
||||
test("recv Error", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val finalChannelId = channelId(alice)
|
||||
alice ! Error(finalChannelId, "oops")
|
||||
awaitCond(wallet.rolledback.size == 1)
|
||||
aliceListener.expectMsgType[ChannelAborted]
|
||||
awaitCond(alice.stateName == CLOSED)
|
||||
|
||||
bob ! Error(finalChannelId, "oops")
|
||||
awaitCond(wallet.rolledback.size == 2)
|
||||
bobListener.expectMsgType[ChannelAborted]
|
||||
awaitCond(bob.stateName == CLOSED)
|
||||
}
|
||||
|
||||
test("recv CMD_CLOSE", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val finalChannelId = channelId(alice)
|
||||
val sender = TestProbe()
|
||||
val c = CMD_CLOSE(sender.ref, None, None)
|
||||
|
||||
alice ! c
|
||||
sender.expectMsg(RES_SUCCESS(c, finalChannelId))
|
||||
awaitCond(wallet.rolledback.size == 1)
|
||||
aliceListener.expectMsgType[ChannelAborted]
|
||||
awaitCond(alice.stateName == CLOSED)
|
||||
|
||||
bob ! c
|
||||
sender.expectMsg(RES_SUCCESS(c, finalChannelId))
|
||||
awaitCond(wallet.rolledback.size == 2)
|
||||
bobListener.expectMsgType[ChannelAborted]
|
||||
awaitCond(bob.stateName == CLOSED)
|
||||
}
|
||||
|
||||
test("recv INPUT_DISCONNECTED", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val aliceData = alice.stateData
|
||||
alice ! INPUT_DISCONNECTED
|
||||
awaitCond(alice.stateName == OFFLINE)
|
||||
assert(alice.stateData == aliceData)
|
||||
aliceListener.expectNoMessage(100 millis)
|
||||
|
||||
val bobData = bob.stateData
|
||||
bob ! INPUT_DISCONNECTED
|
||||
awaitCond(bob.stateName == OFFLINE)
|
||||
assert(bob.stateData == bobData)
|
||||
bobListener.expectNoMessage(100 millis)
|
||||
assert(wallet.rolledback.isEmpty)
|
||||
}
|
||||
|
||||
test("recv INPUT_DISCONNECTED (commit_sig not received)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val fundingTxId = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED].signingSession.fundingTx.txId
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
bob2alice.expectMsgType[CommitSig]
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
|
||||
|
||||
alice ! INPUT_DISCONNECTED
|
||||
awaitCond(alice.stateName == OFFLINE)
|
||||
bob ! INPUT_DISCONNECTED
|
||||
awaitCond(bob.stateName == OFFLINE)
|
||||
|
||||
reconnect(f, fundingTxId)
|
||||
}
|
||||
|
||||
test("recv INPUT_DISCONNECTED (commit_sig received)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val fundingTxId = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED].signingSession.fundingTx.txId
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[CommitSig]
|
||||
bob2alice.forward(alice)
|
||||
bob2alice.expectMsgType[TxSignatures]
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
|
||||
alice ! INPUT_DISCONNECTED
|
||||
awaitCond(alice.stateName == OFFLINE)
|
||||
bob ! INPUT_DISCONNECTED
|
||||
awaitCond(bob.stateName == OFFLINE)
|
||||
|
||||
reconnect(f, fundingTxId)
|
||||
}
|
||||
|
||||
test("recv INPUT_DISCONNECTED (tx_signatures received)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val listener = TestProbe()
|
||||
bob.underlyingActor.context.system.eventStream.subscribe(listener.ref, classOf[TransactionPublished])
|
||||
|
||||
val fundingTxId = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED].signingSession.fundingTx.txId
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[CommitSig]
|
||||
bob2alice.forward(alice)
|
||||
bob2alice.expectMsgType[TxSignatures]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxSignatures]
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
|
||||
alice ! INPUT_DISCONNECTED
|
||||
awaitCond(alice.stateName == OFFLINE)
|
||||
bob ! INPUT_DISCONNECTED
|
||||
awaitCond(bob.stateName == OFFLINE)
|
||||
|
||||
val aliceInit = Init(alice.underlyingActor.nodeParams.features.initFeatures())
|
||||
val bobInit = Init(bob.underlyingActor.nodeParams.features.initFeatures())
|
||||
alice ! INPUT_RECONNECTED(bob, aliceInit, bobInit)
|
||||
bob ! INPUT_RECONNECTED(alice, bobInit, aliceInit)
|
||||
|
||||
assert(alice2bob.expectMsgType[ChannelReestablish].nextFundingTxId_opt.isEmpty)
|
||||
alice2bob.forward(bob)
|
||||
assert(bob2alice.expectMsgType[ChannelReestablish].nextFundingTxId_opt.contains(fundingTxId))
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxSignatures]
|
||||
alice2bob.forward(bob)
|
||||
assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTxId)
|
||||
assert(listener.expectMsgType[TransactionPublished].tx.txid == fundingTxId)
|
||||
}
|
||||
|
||||
private def reconnect(f: FixtureParam, fundingTxId: ByteVector32): Unit = {
|
||||
import f._
|
||||
|
||||
val listener = TestProbe()
|
||||
alice.underlyingActor.context.system.eventStream.subscribe(listener.ref, classOf[TransactionPublished])
|
||||
|
||||
val aliceInit = Init(alice.underlyingActor.nodeParams.features.initFeatures())
|
||||
val bobInit = Init(bob.underlyingActor.nodeParams.features.initFeatures())
|
||||
alice ! INPUT_RECONNECTED(bob, aliceInit, bobInit)
|
||||
bob ! INPUT_RECONNECTED(alice, bobInit, aliceInit)
|
||||
assert(alice2bob.expectMsgType[ChannelReestablish].nextFundingTxId_opt.contains(fundingTxId))
|
||||
alice2bob.forward(bob)
|
||||
assert(bob2alice.expectMsgType[ChannelReestablish].nextFundingTxId_opt.contains(fundingTxId))
|
||||
bob2alice.forward(alice)
|
||||
bob2alice.expectMsgType[CommitSig]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[TxSignatures]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxSignatures]
|
||||
alice2bob.forward(bob)
|
||||
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTxId)
|
||||
assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTxId)
|
||||
assert(listener.expectMsgType[TransactionPublished].tx.txid == fundingTxId)
|
||||
}
|
||||
|
||||
}
|
|
@ -126,40 +126,17 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
|
|||
import f._
|
||||
|
||||
val aliceSigs = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.localSigs
|
||||
val bobSigs = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.localSigs
|
||||
alice2bob.forward(bob, aliceSigs)
|
||||
bob2alice.expectMsgType[TxAbort]
|
||||
|
||||
val bobSigs = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.localSigs
|
||||
bob2alice.forward(alice, bobSigs)
|
||||
alice2bob.expectNoMessage(100 millis)
|
||||
bob2alice.expectNoMessage(100 millis)
|
||||
alice2bob.expectMsgType[TxAbort]
|
||||
|
||||
assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
assert(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
}
|
||||
|
||||
test("re-transmit TxSignatures on reconnection", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val aliceInit = Init(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.params.localParams.initFeatures)
|
||||
val bobInit = Init(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.params.localParams.initFeatures)
|
||||
val aliceSigs = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.localSigs
|
||||
val bobSigs = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.localSigs
|
||||
|
||||
alice ! INPUT_DISCONNECTED
|
||||
awaitCond(alice.stateName == OFFLINE)
|
||||
bob ! INPUT_DISCONNECTED
|
||||
awaitCond(bob.stateName == OFFLINE)
|
||||
alice ! INPUT_RECONNECTED(alice2bob.ref, aliceInit, bobInit)
|
||||
bob ! INPUT_RECONNECTED(bob2alice.ref, bobInit, aliceInit)
|
||||
alice2bob.expectMsgType[ChannelReestablish]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[ChannelReestablish]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsg(aliceSigs)
|
||||
bob2alice.expectMsg(bobSigs)
|
||||
|
||||
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
}
|
||||
|
||||
test("recv WatchPublishedTriggered (initiator)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.ZeroConf), Tag(ChannelStateTestsTags.ScidAlias), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
|
||||
import f._
|
||||
val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction].signedTx
|
||||
|
@ -717,6 +694,133 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
|
|||
awaitCond(alice.stateName == ERR_INFORMATION_LEAK)
|
||||
}
|
||||
|
||||
test("recv INPUT_DISCONNECTED (unsigned rbf attempt)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
alice ! CMD_BUMP_FUNDING_FEE(TestProbe().ref, TestConstants.feeratePerKw * 1.1, 0)
|
||||
alice2bob.expectMsgType[TxInitRbf]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[TxAckRbf]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxAddInput]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[TxAddInput]
|
||||
bob2alice.forward(alice)
|
||||
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] // bob doesn't receive alice's tx_complete
|
||||
alice2bob.expectMsgType[CommitSig] // bob doesn't receive alice's commit_sig
|
||||
|
||||
awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs])
|
||||
val rbfTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.asInstanceOf[RbfStatus.RbfWaitingForSigs].signingSession.fundingTx
|
||||
assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfInProgress])
|
||||
|
||||
alice ! INPUT_DISCONNECTED
|
||||
awaitCond(alice.stateName == OFFLINE)
|
||||
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs])
|
||||
bob ! INPUT_DISCONNECTED
|
||||
awaitCond(bob.stateName == OFFLINE)
|
||||
assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf)
|
||||
|
||||
val aliceInit = Init(alice.underlyingActor.nodeParams.features.initFeatures())
|
||||
val bobInit = Init(bob.underlyingActor.nodeParams.features.initFeatures())
|
||||
alice ! INPUT_RECONNECTED(bob, aliceInit, bobInit)
|
||||
bob ! INPUT_RECONNECTED(alice, bobInit, aliceInit)
|
||||
assert(alice2bob.expectMsgType[ChannelReestablish].nextFundingTxId_opt.contains(rbfTx.txId))
|
||||
alice2bob.forward(bob)
|
||||
assert(bob2alice.expectMsgType[ChannelReestablish].nextFundingTxId_opt.isEmpty)
|
||||
bob2alice.forward(alice)
|
||||
|
||||
// Bob detects that Alice stored an old RBF attempt and tells her to abort.
|
||||
bob2alice.expectMsgType[TxAbort]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxAbort]
|
||||
alice2bob.forward(bob)
|
||||
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf)
|
||||
assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf)
|
||||
alice2bob.expectNoMessage(100 millis)
|
||||
bob2alice.expectNoMessage(100 millis)
|
||||
}
|
||||
|
||||
test("recv INPUT_DISCONNECTED (signed rbf attempt)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
|
||||
val currentFundingTxId = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.txId
|
||||
alice ! CMD_BUMP_FUNDING_FEE(TestProbe().ref, TestConstants.feeratePerKw * 1.1, 0)
|
||||
alice2bob.expectMsgType[TxInitRbf]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[TxAckRbf]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxAddInput]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[TxAddInput]
|
||||
bob2alice.forward(alice)
|
||||
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] // alice doesn't receive bob's commit_sig
|
||||
alice2bob.expectMsgType[CommitSig] // bob doesn't receive alice's commit_sig
|
||||
|
||||
awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs])
|
||||
awaitCond(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs])
|
||||
val rbfTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.asInstanceOf[RbfStatus.RbfWaitingForSigs].signingSession.fundingTx
|
||||
|
||||
alice ! INPUT_DISCONNECTED
|
||||
awaitCond(alice.stateName == OFFLINE)
|
||||
assert(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs])
|
||||
bob ! INPUT_DISCONNECTED
|
||||
awaitCond(bob.stateName == OFFLINE)
|
||||
assert(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus.isInstanceOf[RbfStatus.RbfWaitingForSigs])
|
||||
|
||||
val aliceInit = Init(alice.underlyingActor.nodeParams.features.initFeatures())
|
||||
val bobInit = Init(bob.underlyingActor.nodeParams.features.initFeatures())
|
||||
alice ! INPUT_RECONNECTED(bob, aliceInit, bobInit)
|
||||
bob ! INPUT_RECONNECTED(alice, bobInit, aliceInit)
|
||||
assert(alice2bob.expectMsgType[ChannelReestablish].nextFundingTxId_opt.contains(rbfTx.txId))
|
||||
alice2bob.forward(bob)
|
||||
assert(bob2alice.expectMsgType[ChannelReestablish].nextFundingTxId_opt.contains(rbfTx.txId))
|
||||
bob2alice.forward(alice)
|
||||
|
||||
// Alice and Bob exchange signatures and complete the RBF attempt.
|
||||
alice2bob.expectMsgType[CommitSig]
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[CommitSig]
|
||||
bob2alice.forward(alice)
|
||||
bob2alice.expectMsgType[TxSignatures]
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[TxSignatures]
|
||||
alice2bob.forward(bob)
|
||||
val nextFundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.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(currentFundingTxId != nextFundingTx.txId)
|
||||
awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf)
|
||||
awaitCond(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].rbfStatus == RbfStatus.NoRbf)
|
||||
}
|
||||
|
||||
test("recv Error", Tag(ChannelStateTestsTags.DualFunding)) { f =>
|
||||
import f._
|
||||
val tx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx
|
||||
|
@ -862,12 +966,12 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
|
|||
val bobInit = Init(bob2.underlyingActor.nodeParams.features.initFeatures())
|
||||
alice2 ! INPUT_RECONNECTED(alice2bob.ref, aliceInit, bobInit)
|
||||
val aliceChannelReestablish = alice2bob.expectMsgType[ChannelReestablish]
|
||||
assert(aliceChannelReestablish.nextFundingTxId_opt.isEmpty)
|
||||
bob2 ! INPUT_RECONNECTED(bob2alice.ref, bobInit, aliceInit)
|
||||
val bobChannelReestablish = bob2alice.expectMsgType[ChannelReestablish]
|
||||
assert(bobChannelReestablish.nextFundingTxId_opt.isEmpty)
|
||||
alice2 ! bobChannelReestablish
|
||||
alice2bob.expectMsgType[TxSignatures]
|
||||
bob2 ! aliceChannelReestablish
|
||||
bob2alice.expectMsgType[TxSignatures]
|
||||
|
||||
awaitCond(alice2.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
awaitCond(bob2.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
|
||||
|
|
|
@ -816,12 +816,12 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
alice ! INPUT_RECONNECTED(alice2bob.ref, aliceInit, bobInit)
|
||||
bob ! INPUT_RECONNECTED(bob2alice.ref, bobInit, aliceInit)
|
||||
|
||||
val aliceCommitments = alice.stateData.asInstanceOf[PersistentChannelData].commitments
|
||||
val aliceCommitments = alice.stateData.asInstanceOf[ChannelDataWithCommitments].commitments
|
||||
val aliceCurrentPerCommitmentPoint = TestConstants.Alice.channelKeyManager.commitmentPoint(
|
||||
TestConstants.Alice.channelKeyManager.keyPath(aliceCommitments.params.localParams, aliceCommitments.params.channelConfig),
|
||||
aliceCommitments.localCommitIndex)
|
||||
|
||||
val bobCommitments = bob.stateData.asInstanceOf[PersistentChannelData].commitments
|
||||
val bobCommitments = bob.stateData.asInstanceOf[ChannelDataWithCommitments].commitments
|
||||
val bobCurrentPerCommitmentPoint = TestConstants.Bob.channelKeyManager.commitmentPoint(
|
||||
TestConstants.Bob.channelKeyManager.keyPath(bobCommitments.params.localParams, bobCommitments.params.channelConfig),
|
||||
bobCommitments.localCommitIndex)
|
||||
|
|
|
@ -1627,7 +1627,7 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
|
|||
import f._
|
||||
mutualClose(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain)
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_CLOSING]
|
||||
val bobCommitments = bob.stateData.asInstanceOf[PersistentChannelData].commitments
|
||||
val bobCommitments = bob.stateData.asInstanceOf[DATA_CLOSING].commitments
|
||||
val bobCurrentPerCommitmentPoint = TestConstants.Bob.channelKeyManager.commitmentPoint(
|
||||
TestConstants.Bob.channelKeyManager.keyPath(bobCommitments.params.localParams, bobCommitments.params.channelConfig),
|
||||
bobCommitments.localCommitIndex)
|
||||
|
|
|
@ -184,7 +184,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.latest.fundingTxId
|
||||
val fundingTxid = getChannelData(node, channelId).asInstanceOf[ChannelDataWithCommitments].commitments.latest.fundingTxId
|
||||
node.wallet.funded(fundingTxid)
|
||||
}
|
||||
|
||||
|
|
|
@ -5,7 +5,7 @@ import fr.acinq.bitcoin.scalacompat.{ByteVector32, SatoshiLong}
|
|||
import fr.acinq.eclair.FeatureSupport.Optional
|
||||
import fr.acinq.eclair.Features.ZeroConf
|
||||
import fr.acinq.eclair.channel.ChannelTypes.AnchorOutputsZeroFeeHtlcTx
|
||||
import fr.acinq.eclair.channel.{DATA_NORMAL, NORMAL, PersistentChannelData, SupportedChannelType}
|
||||
import fr.acinq.eclair.channel.{ChannelDataWithCommitments, DATA_NORMAL, NORMAL, SupportedChannelType}
|
||||
import fr.acinq.eclair.integration.basic.fixtures.composite.TwoNodesFixture
|
||||
import fr.acinq.eclair.testutils.FixtureSpec
|
||||
import org.scalatest.concurrent.IntegrationPatience
|
||||
|
@ -60,8 +60,8 @@ class ZeroConfActivationSpec extends FixtureSpec with IntegrationPatience {
|
|||
assert(!bob.nodeParams.features.activated.contains(ZeroConf))
|
||||
|
||||
val channelId = createChannel(f)
|
||||
assert(!getChannelData(alice, channelId).asInstanceOf[PersistentChannelData].commitments.params.channelFeatures.hasFeature(ZeroConf))
|
||||
assert(!getChannelData(bob, channelId).asInstanceOf[PersistentChannelData].commitments.params.channelFeatures.hasFeature(ZeroConf))
|
||||
assert(!getChannelData(alice, channelId).asInstanceOf[ChannelDataWithCommitments].commitments.params.channelFeatures.hasFeature(ZeroConf))
|
||||
assert(!getChannelData(bob, channelId).asInstanceOf[ChannelDataWithCommitments].commitments.params.channelFeatures.hasFeature(ZeroConf))
|
||||
}
|
||||
|
||||
test("open a channel alice-bob (zero-conf disabled on both sides, requested via channel type by alice)") { f =>
|
||||
|
|
|
@ -20,7 +20,7 @@ import akka.actor.{ActorContext, ActorRef, ActorSystem, FSM, PoisonPill, Status}
|
|||
import akka.testkit.TestActor.KeepRunning
|
||||
import akka.testkit.{TestFSMRef, TestKit, TestProbe}
|
||||
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.scalacompat.{Block, Btc, ByteVector32, SatoshiLong}
|
||||
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, SatoshiLong}
|
||||
import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional}
|
||||
import fr.acinq.eclair.Features._
|
||||
import fr.acinq.eclair.TestConstants._
|
||||
|
@ -141,7 +141,7 @@ class PeerSpec extends FixtureSpec {
|
|||
probe.expectMsg(PeerConnection.ConnectionResult.NoAddressFound)
|
||||
}
|
||||
|
||||
/** We need to be careful to avoir race conditions due to event stream asynchronous nature */
|
||||
/** We need to be careful to avoid race conditions due to event stream asynchronous nature */
|
||||
def spawnClientSpawner(f: FixtureParam): Unit = {
|
||||
import f._
|
||||
val readyListener = TestProbe("ready-listener")
|
||||
|
@ -345,6 +345,15 @@ class PeerSpec extends FixtureSpec {
|
|||
assert(peer.stateData.channels.size == 1)
|
||||
}
|
||||
|
||||
test("send error when receiving message for unknown channel") { f =>
|
||||
import f._
|
||||
|
||||
connect(remoteNodeId, peer, peerConnection, switchboard)
|
||||
val channelId = randomBytes32()
|
||||
peerConnection.send(peer, ChannelReestablish(channelId, 1, 0, randomKey(), randomKey().publicKey))
|
||||
peerConnection.expectMsg(Error(channelId, "unknown channel"))
|
||||
}
|
||||
|
||||
test("handle OpenChannelInterceptor spawning a user initiated open channel request ") { f =>
|
||||
import f._
|
||||
|
||||
|
|
|
@ -49,7 +49,7 @@ class ChannelCodecsSpec extends AnyFunSuite {
|
|||
|
||||
test("nonreg for der/bin64 signatures") {
|
||||
val bin = ByteVector.fromValidHex(Source.fromInputStream(getClass.getResourceAsStream("/normal_data_htlcs.bin")).mkString)
|
||||
val c = ChannelCodecs.channelDataCodec.decode(bin.toBitVector).require.value
|
||||
val c = ChannelCodecs.channelDataCodec.decode(bin.toBitVector).require.value.asInstanceOf[ChannelDataWithCommitments]
|
||||
|
||||
val ref = Seq(
|
||||
hex"304502210097fcda40b22916b5d61badedf6126658c2b5927d5002cc2c3e5f88a78ba5f45b02204a74bcf8827d894cab153fc051f39d8e2aeb660162a6a05797f7140587a6133301",
|
||||
|
@ -232,11 +232,11 @@ class ChannelCodecsSpec extends AnyFunSuite {
|
|||
|
||||
oldbins.foreach { oldbin =>
|
||||
// we decode with compat codec
|
||||
val oldnormal = channelDataCodec.decode(oldbin.bits).require.value
|
||||
val oldnormal = channelDataCodec.decode(oldbin.bits).require.value.asInstanceOf[ChannelDataWithCommitments]
|
||||
// and we encode with new codec
|
||||
val newbin = channelDataCodec.encode(oldnormal).require.bytes
|
||||
// make sure that round-trip yields the same data
|
||||
val newnormal = channelDataCodec.decode(newbin.bits).require.value
|
||||
val newnormal = channelDataCodec.decode(newbin.bits).require.value.asInstanceOf[ChannelDataWithCommitments]
|
||||
assert(newnormal == oldnormal)
|
||||
// make sure that we have stripped sigs from the transactions
|
||||
assert(newnormal.commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx.txIn.forall(_.witness.stack.isEmpty))
|
||||
|
|
File diff suppressed because one or more lines are too long
|
@ -1,14 +1,21 @@
|
|||
package fr.acinq.eclair.wire.internal.channel.version4
|
||||
|
||||
import com.softwaremill.quicklens.ModifyPimp
|
||||
import fr.acinq.bitcoin.scalacompat.{DeterministicWallet, Satoshi}
|
||||
import fr.acinq.bitcoin.scalacompat.{DeterministicWallet, OutPoint, Satoshi, SatoshiLong, Script, Transaction, TxIn, TxOut}
|
||||
import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional}
|
||||
import fr.acinq.eclair.Features.{ChannelRangeQueries, PaymentSecret, VariableLengthOnion}
|
||||
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{InteractiveTxParams, PartiallySignedSharedTransaction, RequireConfirmedInputs, SharedTransaction}
|
||||
import fr.acinq.eclair.channel.fund.InteractiveTxSigningSession.UnsignedLocalCommit
|
||||
import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningSession}
|
||||
import fr.acinq.eclair.transactions.CommitmentSpec
|
||||
import fr.acinq.eclair.transactions.Transactions.{CommitTx, InputInfo}
|
||||
import fr.acinq.eclair.wire.internal.channel.ChannelCodecsSpec.normal
|
||||
import fr.acinq.eclair.wire.internal.channel.version4.ChannelCodecs4.Codecs.{channelConfigCodec, localParamsCodec, remoteParamsCodec}
|
||||
import fr.acinq.eclair.wire.internal.channel.version4.ChannelCodecs4.Codecs.{channelConfigCodec, localParamsCodec, rbfStatusCodec, remoteParamsCodec}
|
||||
import fr.acinq.eclair.wire.internal.channel.version4.ChannelCodecs4.channelDataCodec
|
||||
import fr.acinq.eclair.{CltvExpiryDelta, Features, MilliSatoshi, UInt64, randomKey}
|
||||
import fr.acinq.eclair.wire.protocol.TxSignatures
|
||||
import fr.acinq.eclair.{CltvExpiryDelta, Features, MilliSatoshi, MilliSatoshiLong, UInt64, randomBytes32, randomKey}
|
||||
import org.scalatest.funsuite.AnyFunSuite
|
||||
import scodec.bits._
|
||||
|
||||
|
@ -19,7 +26,7 @@ class ChannelCodecs4Spec extends AnyFunSuite {
|
|||
test("basic serialization test (NORMAL)") {
|
||||
val data = normal
|
||||
val bin = channelDataCodec.encode(data).require
|
||||
val check = channelDataCodec.decodeValue(bin).require
|
||||
val check = channelDataCodec.decodeValue(bin).require.asInstanceOf[ChannelDataWithCommitments]
|
||||
assert(data.commitments.latest.localCommit.spec == check.commitments.latest.localCommit.spec)
|
||||
assert(data == check)
|
||||
}
|
||||
|
@ -113,4 +120,38 @@ class ChannelCodecs4Spec extends AnyFunSuite {
|
|||
assert(channelDataCodec.decode(channelDataCodec.encode(dataWithRemoteShutdownScript).require).require.value == dataWithRemoteShutdownScript)
|
||||
}
|
||||
|
||||
test("encode/decode rbf status") {
|
||||
val channelId = randomBytes32()
|
||||
val fundingInput = InputInfo(OutPoint(randomBytes32(), 3), TxOut(175_000 sat, Script.pay2wpkh(randomKey().publicKey)), Nil)
|
||||
val fundingTx = SharedTransaction(
|
||||
sharedInput_opt = None,
|
||||
sharedOutput = InteractiveTxBuilder.Output.Shared(UInt64(8), ByteVector.empty, 100_000 sat, 75_000 sat),
|
||||
localInputs = Nil, remoteInputs = Nil,
|
||||
localOutputs = Nil, remoteOutputs = Nil,
|
||||
lockTime = 0
|
||||
)
|
||||
val commitTx = CommitTx(
|
||||
fundingInput,
|
||||
Transaction(2, Seq(TxIn(fundingInput.outPoint, Nil, 0)), Seq(TxOut(150_000 sat, Script.pay2wpkh(randomKey().publicKey))), 0),
|
||||
)
|
||||
val waitingForSigs = InteractiveTxSigningSession.WaitingForSigs(
|
||||
InteractiveTxParams(channelId, isInitiator = true, 100_000 sat, 75_000 sat, None, ByteVector.empty, Nil, 0, 330 sat, FeeratePerKw(500 sat), None, RequireConfirmedInputs(forLocal = false, forRemote = false)),
|
||||
PartiallySignedSharedTransaction(fundingTx, TxSignatures(channelId, randomBytes32(), Nil)),
|
||||
Left(UnsignedLocalCommit(0, CommitmentSpec(Set.empty, FeeratePerKw(1000 sat), 100_000_000 msat, 75_000_000 msat), commitTx, Nil)),
|
||||
RemoteCommit(0, CommitmentSpec(Set.empty, FeeratePerKw(1000 sat), 75_000_000 msat, 100_000_000 msat), randomBytes32(), randomKey().publicKey)
|
||||
)
|
||||
val testCases = Map(
|
||||
RbfStatus.NoRbf -> RbfStatus.NoRbf,
|
||||
RbfStatus.RbfRequested(CMD_BUMP_FUNDING_FEE(null, FeeratePerKw(750 sat), 0)) -> RbfStatus.NoRbf,
|
||||
RbfStatus.RbfInProgress(None, null, None) -> RbfStatus.NoRbf,
|
||||
RbfStatus.RbfWaitingForSigs(waitingForSigs) -> RbfStatus.RbfWaitingForSigs(waitingForSigs),
|
||||
RbfStatus.RbfAborted -> RbfStatus.NoRbf,
|
||||
)
|
||||
testCases.foreach { case (status, expected) =>
|
||||
val encoded = rbfStatusCodec.encode(status).require
|
||||
val decoded = rbfStatusCodec.decode(encoded).require.value
|
||||
assert(decoded == expected)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -33,7 +33,7 @@ import fr.acinq.eclair.wire.protocol.TxRbfTlv.SharedOutputContributionTlv
|
|||
import org.json4s.jackson.Serialization
|
||||
import org.scalatest.funsuite.AnyFunSuite
|
||||
import scodec.DecodeResult
|
||||
import scodec.bits.{BinStringSyntax, ByteVector, HexStringSyntax}
|
||||
import scodec.bits.{BinStringSyntax, BitVector, ByteVector, HexStringSyntax}
|
||||
|
||||
import java.net.{Inet4Address, Inet6Address, InetAddress}
|
||||
|
||||
|
@ -112,37 +112,39 @@ class LightningMessageCodecsSpec extends AnyFunSuite {
|
|||
val signature = randomBytes64()
|
||||
val key = randomKey()
|
||||
val point = randomKey().publicKey
|
||||
val txHash = randomBytes32()
|
||||
val randomData = randomBytes(42)
|
||||
val tlvTag = UInt64(hex"47010000")
|
||||
|
||||
val refs = Map(
|
||||
(hex"0023" ++ channelId ++ signature, hex"") -> FundingSigned(channelId, signature),
|
||||
(hex"0023" ++ channelId ++ signature ++ hex"fe47010000 00", hex"") -> FundingSigned(channelId, signature, TlvStream[FundingSignedTlv](Set.empty[FundingSignedTlv], Set(GenericTlv(tlvTag, ByteVector.empty)))),
|
||||
(hex"0023" ++ channelId ++ signature ++ hex"fe47010000 07 cccccccccccccc", hex"") -> FundingSigned(channelId, signature, TlvStream[FundingSignedTlv](Set.empty[FundingSignedTlv], Set(GenericTlv(tlvTag, hex"cccccccccccccc")))),
|
||||
hex"0023" ++ channelId ++ signature -> FundingSigned(channelId, signature),
|
||||
hex"0023" ++ channelId ++ signature ++ hex"fe47010000 00" -> FundingSigned(channelId, signature, TlvStream[FundingSignedTlv](Set.empty[FundingSignedTlv], Set(GenericTlv(tlvTag, ByteVector.empty)))),
|
||||
hex"0023" ++ channelId ++ signature ++ hex"fe47010000 07 cccccccccccccc" -> FundingSigned(channelId, signature, TlvStream[FundingSignedTlv](Set.empty[FundingSignedTlv], Set(GenericTlv(tlvTag, hex"cccccccccccccc")))),
|
||||
|
||||
(hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value, hex"") -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point),
|
||||
(hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"fe47010000 00", hex"") -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream[ChannelReestablishTlv](Set.empty[ChannelReestablishTlv], Set(GenericTlv(tlvTag, ByteVector.empty)))),
|
||||
(hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"fe47010000 07 bbbbbbbbbbbbbb", hex"") -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream[ChannelReestablishTlv](Set.empty[ChannelReestablishTlv], Set(GenericTlv(tlvTag, hex"bbbbbbbbbbbbbb")))),
|
||||
hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point),
|
||||
hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"fd014d 20" ++ txHash.bytes -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream(ChannelReestablishTlv.NextFundingTlv(txHash))),
|
||||
hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"fe47010000 00" -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream[ChannelReestablishTlv](Set.empty[ChannelReestablishTlv], Set(GenericTlv(tlvTag, ByteVector.empty)))),
|
||||
hex"0088" ++ channelId ++ hex"0001020304050607 0809aabbccddeeff" ++ key.value ++ point.value ++ hex"fe47010000 07 bbbbbbbbbbbbbb" -> ChannelReestablish(channelId, 0x01020304050607L, 0x0809aabbccddeeffL, key, point, TlvStream[ChannelReestablishTlv](Set.empty[ChannelReestablishTlv], Set(GenericTlv(tlvTag, hex"bbbbbbbbbbbbbb")))),
|
||||
|
||||
(hex"0084" ++ channelId ++ signature ++ hex"0000", hex"") -> CommitSig(channelId, signature, Nil),
|
||||
(hex"0084" ++ channelId ++ signature ++ hex"0000 fe47010000 00", hex"") -> CommitSig(channelId, signature, Nil, TlvStream[CommitSigTlv](Set.empty[CommitSigTlv], Set(GenericTlv(tlvTag, ByteVector.empty)))),
|
||||
(hex"0084" ++ channelId ++ signature ++ hex"0000 fe47010000 07 cccccccccccccc", hex"") -> CommitSig(channelId, signature, Nil, TlvStream[CommitSigTlv](Set.empty[CommitSigTlv], Set(GenericTlv(tlvTag, hex"cccccccccccccc")))),
|
||||
hex"0084" ++ channelId ++ signature ++ hex"0000" -> CommitSig(channelId, signature, Nil),
|
||||
hex"0084" ++ channelId ++ signature ++ hex"0000 fe47010000 00" -> CommitSig(channelId, signature, Nil, TlvStream[CommitSigTlv](Set.empty[CommitSigTlv], Set(GenericTlv(tlvTag, ByteVector.empty)))),
|
||||
hex"0084" ++ channelId ++ signature ++ hex"0000 fe47010000 07 cccccccccccccc" -> CommitSig(channelId, signature, Nil, TlvStream[CommitSigTlv](Set.empty[CommitSigTlv], Set(GenericTlv(tlvTag, hex"cccccccccccccc")))),
|
||||
|
||||
(hex"0085" ++ channelId ++ key.value ++ point.value, hex"") -> RevokeAndAck(channelId, key, point),
|
||||
(hex"0085" ++ channelId ++ key.value ++ point.value ++ hex" fe47010000 00", hex"") -> RevokeAndAck(channelId, key, point, TlvStream[RevokeAndAckTlv](Set.empty[RevokeAndAckTlv], Set(GenericTlv(tlvTag, ByteVector.empty)))),
|
||||
(hex"0085" ++ channelId ++ key.value ++ point.value ++ hex" fe47010000 07 cccccccccccccc", hex"") -> RevokeAndAck(channelId, key, point, TlvStream[RevokeAndAckTlv](Set.empty[RevokeAndAckTlv], Set(GenericTlv(tlvTag, hex"cccccccccccccc")))),
|
||||
hex"0085" ++ channelId ++ key.value ++ point.value -> RevokeAndAck(channelId, key, point),
|
||||
hex"0085" ++ channelId ++ key.value ++ point.value ++ hex" fe47010000 00" -> RevokeAndAck(channelId, key, point, TlvStream[RevokeAndAckTlv](Set.empty[RevokeAndAckTlv], Set(GenericTlv(tlvTag, ByteVector.empty)))),
|
||||
hex"0085" ++ channelId ++ key.value ++ point.value ++ hex" fe47010000 07 cccccccccccccc" -> RevokeAndAck(channelId, key, point, TlvStream[RevokeAndAckTlv](Set.empty[RevokeAndAckTlv], Set(GenericTlv(tlvTag, hex"cccccccccccccc")))),
|
||||
|
||||
(hex"0026" ++ channelId ++ hex"002a" ++ randomData, hex"") -> Shutdown(channelId, randomData),
|
||||
(hex"0026" ++ channelId ++ hex"002a" ++ randomData ++ hex"fe47010000 00", hex"") -> Shutdown(channelId, randomData, TlvStream[ShutdownTlv](Set.empty[ShutdownTlv], Set(GenericTlv(tlvTag, ByteVector.empty)))),
|
||||
(hex"0026" ++ channelId ++ hex"002a" ++ randomData ++ hex"fe47010000 07 cccccccccccccc", hex"") -> Shutdown(channelId, randomData, TlvStream[ShutdownTlv](Set.empty[ShutdownTlv], Set(GenericTlv(tlvTag, hex"cccccccccccccc")))),
|
||||
hex"0026" ++ channelId ++ hex"002a" ++ randomData -> Shutdown(channelId, randomData),
|
||||
hex"0026" ++ channelId ++ hex"002a" ++ randomData ++ hex"fe47010000 00" -> Shutdown(channelId, randomData, TlvStream[ShutdownTlv](Set.empty[ShutdownTlv], Set(GenericTlv(tlvTag, ByteVector.empty)))),
|
||||
hex"0026" ++ channelId ++ hex"002a" ++ randomData ++ hex"fe47010000 07 cccccccccccccc" -> Shutdown(channelId, randomData, TlvStream[ShutdownTlv](Set.empty[ShutdownTlv], Set(GenericTlv(tlvTag, hex"cccccccccccccc")))),
|
||||
|
||||
(hex"0027" ++ channelId ++ hex"00000000075bcd15" ++ signature, hex"") -> ClosingSigned(channelId, 123456789.sat, signature),
|
||||
(hex"0027" ++ channelId ++ hex"00000000075bcd15" ++ signature ++ hex"fe47010000 00", hex"") -> ClosingSigned(channelId, 123456789.sat, signature, TlvStream[ClosingSignedTlv](Set.empty[ClosingSignedTlv], Set(GenericTlv(tlvTag, ByteVector.empty)))),
|
||||
(hex"0027" ++ channelId ++ hex"00000000075bcd15" ++ signature ++ hex"fe47010000 07 cccccccccccccc", hex"") -> ClosingSigned(channelId, 123456789.sat, signature, TlvStream[ClosingSignedTlv](Set.empty[ClosingSignedTlv], Set(GenericTlv(tlvTag, hex"cccccccccccccc")))),
|
||||
hex"0027" ++ channelId ++ hex"00000000075bcd15" ++ signature -> ClosingSigned(channelId, 123456789.sat, signature),
|
||||
hex"0027" ++ channelId ++ hex"00000000075bcd15" ++ signature ++ hex"fe47010000 00" -> ClosingSigned(channelId, 123456789.sat, signature, TlvStream[ClosingSignedTlv](Set.empty[ClosingSignedTlv], Set(GenericTlv(tlvTag, ByteVector.empty)))),
|
||||
hex"0027" ++ channelId ++ hex"00000000075bcd15" ++ signature ++ hex"fe47010000 07 cccccccccccccc" -> ClosingSigned(channelId, 123456789.sat, signature, TlvStream[ClosingSignedTlv](Set.empty[ClosingSignedTlv], Set(GenericTlv(tlvTag, hex"cccccccccccccc")))),
|
||||
)
|
||||
|
||||
refs.foreach { case ((bin, remainder), msg) =>
|
||||
assert(lightningMessageCodec.decode(bin.bits ++ remainder.bits).require == DecodeResult(msg, remainder.bits))
|
||||
refs.foreach { case (bin, msg) =>
|
||||
assert(lightningMessageCodec.decode(bin.bits).require == DecodeResult(msg, BitVector.empty))
|
||||
assert(lightningMessageCodec.encode(msg).require == bin.bits)
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Add table
Reference in a new issue