1
0
Fork 0
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:
Bastien Teinturier 2023-04-05 14:52:16 +02:00 committed by GitHub
parent db15beb015
commit 3a95a7deb5
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
36 changed files with 1690 additions and 1190 deletions

View file

@ -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
}

View file

@ -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))

View file

@ -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

View file

@ -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)

View file

@ -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 &&

View file

@ -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 _ =>

View file

@ -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)
})

View file

@ -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 =>

View file

@ -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

View file

@ -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 _ => ()
}
}
}

View file

@ -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

View file

@ -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 {

View file

@ -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
}

View file

@ -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)
}

View file

@ -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

View file

@ -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)
}
}

View file

@ -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()

View file

@ -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 {

View file

@ -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)

View file

@ -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

View file

@ -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,

View file

@ -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))

View file

@ -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 = {

View file

@ -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]

View file

@ -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)
}
}

View file

@ -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)

View file

@ -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)

View file

@ -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)

View file

@ -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)
}

View file

@ -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 =>

View file

@ -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._

View file

@ -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))

View file

@ -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)
}
}
}

View file

@ -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)
}
}