1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-02-22 14:22:39 +01:00

Add support for splices (#2584)

Add support for both splice-in and splice-out in Eclair. Mixing concurrent local/remote splice-in/splice-out is wired, although not supported in the API.

The implementation differs from the current wip BOLT proposal on at least the following points:
- we use a poor man's _quiescence_ protocol which just rejects the splice if the channel is not idle
- splice txs always _spend_ the previous funding/splice tx, even if it isn't confirmed yet and could theoretically be RBFed. This is done to be compatible with zero-conf splices
- the persistence/reconnection follows the logic described in https://gist.github.com/t-bast/1ac31f4e27734a10c5b9847d06db8d86.

We add a new `fundingTxIndex` to `Commitment`, which has two nice advantages:
- making debug much easier compared to dealing with txid:
  `splice=1 is now active, removed=0 remaining=2,1`
- allowing to discriminate between initial funding, splices, rbf, and
  combinations thereof.

We closely mimick RBFing the initial funding tx (e.g. `RbfStatus` vs `SpliceStatus`).

---------

Co-authored-by: Bastien Teinturier <31281497+t-bast@users.noreply.github.com>
This commit is contained in:
Pierre-Marie Padiou 2023-04-13 19:14:02 +02:00 committed by GitHub
parent daf947fb60
commit de6d3c1709
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
61 changed files with 2448 additions and 463 deletions

View file

@ -39,6 +39,8 @@ and COMMAND is one of the available commands:
=== Channel ===
- open
- rbfopen
- splicein
- spliceout
- close
- forceclose
- channel
@ -143,7 +145,7 @@ jq_filter='if type=="object" and .error != null then .error else .';
# apply special jq filter if we are in "short" ouput mode -- only for specific commands such as 'channels'
if [ "$short" = true ]; then
jq_channel_filter="{ nodeId, shortChannelId: .data.shortIds.real.realScid, channelId, state, balanceSat: (try (.data.commitments.localCommit.spec.toLocal / 1000 | floor) catch null), capacitySat: .data.commitments.commitInput.amountSatoshis, channelPoint: .data.commitments.commitInput.outPoint }";
jq_channel_filter="{ nodeId, shortChannelId: .data.shortIds.real.realScid, channelId, state, commitments: (.data.commitments.active | map({balanceSat: (try (.localCommit.spec.toLocal / 1000 | floor) catch null), capacitySat: .fundingTx.amountSatoshis, fundingTxIndex: .fundingTxIndex, channelPoint: .fundingTx.outPoint})) }";
case $api_endpoint in
"channels") jq_filter="$jq_filter | map( $jq_channel_filter )" ;;
"channel") jq_filter="$jq_filter | $jq_channel_filter" ;;

View file

@ -24,7 +24,7 @@ import akka.pattern._
import akka.util.Timeout
import com.softwaremill.quicklens.ModifyPimp
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Crypto, Satoshi}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Crypto, Satoshi, Script, addressToPublicKeyScript}
import fr.acinq.eclair.ApiTypes.ChannelNotFound
import fr.acinq.eclair.balance.CheckBalance.GlobalBalance
import fr.acinq.eclair.balance.{BalanceActor, ChannelsListener}
@ -91,6 +91,10 @@ trait Eclair {
def rbfOpen(channelId: ByteVector32, targetFeerate: FeeratePerKw, lockTime_opt: Option[Long])(implicit timeout: Timeout): Future[CommandResponse[CMD_BUMP_FUNDING_FEE]]
def spliceIn(channelId: ByteVector32, amountIn: Satoshi, pushAmount_opt: Option[MilliSatoshi])(implicit timeout: Timeout): Future[CommandResponse[CMD_SPLICE]]
def spliceOut(channelId: ByteVector32, amountOut: Satoshi, scriptOrAddress: Either[ByteVector, String])(implicit timeout: Timeout): Future[CommandResponse[CMD_SPLICE]]
def close(channels: List[ApiTypes.ChannelIdentifier], scriptPubKey_opt: Option[ByteVector], closingFeerates_opt: Option[ClosingFeerates])(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_CLOSE]]]]
def forceClose(channels: List[ApiTypes.ChannelIdentifier])(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_FORCECLOSE]]]]
@ -209,7 +213,27 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
}
override def rbfOpen(channelId: ByteVector32, targetFeerate: FeeratePerKw, lockTime_opt: Option[Long])(implicit timeout: Timeout): Future[CommandResponse[CMD_BUMP_FUNDING_FEE]] = {
sendToChannelTyped(Left(channelId), CMD_BUMP_FUNDING_FEE(_, targetFeerate, lockTime_opt.getOrElse(appKit.nodeParams.currentBlockHeight.toLong)))
sendToChannelTyped(channel = Left(channelId),
cmdBuilder = CMD_BUMP_FUNDING_FEE(_, targetFeerate, lockTime_opt.getOrElse(appKit.nodeParams.currentBlockHeight.toLong)))
}
override def spliceIn(channelId: ByteVector32, amountIn: Satoshi, pushAmount_opt: Option[MilliSatoshi])(implicit timeout: Timeout): Future[CommandResponse[CMD_SPLICE]] = {
sendToChannelTyped(channel = Left(channelId),
cmdBuilder = CMD_SPLICE(_,
spliceIn_opt = Some(SpliceIn(additionalLocalFunding = amountIn, pushAmount = pushAmount_opt.getOrElse(0.msat))),
spliceOut_opt = None
))
}
override def spliceOut(channelId: ByteVector32, amountOut: Satoshi, scriptOrAddress: Either[ByteVector, String])(implicit timeout: Timeout): Future[CommandResponse[CMD_SPLICE]] = {
sendToChannelTyped(channel = Left(channelId),
cmdBuilder = CMD_SPLICE(_,
spliceIn_opt = None,
spliceOut_opt = Some(SpliceOut(amount = amountOut, scriptPubKey = scriptOrAddress match {
case Left(script) => script
case Right(address) => Script.write(addressToPublicKeyScript(appKit.nodeParams.chainHash, address))
}))
))
}
override def close(channels: List[ApiTypes.ChannelIdentifier], scriptPubKey_opt: Option[ByteVector], closingFeerates_opt: Option[ClosingFeerates])(implicit timeout: Timeout): Future[Map[ApiTypes.ChannelIdentifier, Either[Throwable, CommandResponse[CMD_CLOSE]]]] = {

View file

@ -294,6 +294,12 @@ object Features {
val mandatory = 152
}
// TODO: @pm47 custom splices implementation for phoenix, to be replaced once splices is spec-ed (currently reserved here: https://github.com/lightning/bolts/issues/605)
case object SplicePrototype extends Feature with InitFeature {
val rfcName = "splice_prototype"
val mandatory = 154
}
val knownFeatures: Set[Feature] = Set(
DataLossProtect,
InitialRoutingSync,
@ -317,7 +323,8 @@ object Features {
ZeroConf,
KeySend,
TrampolinePaymentPrototype,
AsyncPaymentPrototype
AsyncPaymentPrototype,
SplicePrototype,
)
// Features may depend on other features, as specified in Bolt 9.

View file

@ -98,7 +98,7 @@ class Setup(val datadir: File,
val Seeds(nodeSeed, channelSeed) = seeds_opt.getOrElse(NodeParams.getSeeds(datadir))
val chain = config.getString("chain")
if (chain != "regtest") {
if (chain != "regtest" && chain != "testnet") {
// TODO: database format is WIP, we want to be able to squash changes and not support intermediate unreleased versions
throw new RuntimeException("this unreleased version of Eclair only works on regtest")
}

View file

@ -161,6 +161,9 @@ object ZmqWatcher {
case class WatchParentTxConfirmed(replyTo: ActorRef[WatchParentTxConfirmedTriggered], txId: ByteVector32, minDepth: Long) extends WatchConfirmed[WatchParentTxConfirmedTriggered]
case class WatchParentTxConfirmedTriggered(blockHeight: BlockHeight, txIndex: Int, tx: Transaction) extends WatchConfirmedTriggered
case class WatchAlternativeCommitTxConfirmed(replyTo: ActorRef[WatchAlternativeCommitTxConfirmedTriggered], txId: ByteVector32, minDepth: Long) extends WatchConfirmed[WatchAlternativeCommitTxConfirmedTriggered]
case class WatchAlternativeCommitTxConfirmedTriggered(blockHeight: BlockHeight, txIndex: Int, tx: Transaction) extends WatchConfirmedTriggered
private sealed trait AddWatchResult
private case object Keep extends AddWatchResult
private case object Ignore extends AddWatchResult
@ -423,6 +426,7 @@ private class ZmqWatcher(nodeParams: NodeParams, blockHeight: AtomicLong, client
case w: WatchFundingDeeplyBuried => context.self ! TriggerEvent(w.replyTo, w, WatchFundingDeeplyBuriedTriggered(height, index, tx))
case w: WatchTxConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchTxConfirmedTriggered(height, index, tx))
case w: WatchParentTxConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchParentTxConfirmedTriggered(height, index, tx))
case w: WatchAlternativeCommitTxConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchAlternativeCommitTxConfirmedTriggered(height, index, tx))
}
}
}

View file

@ -18,7 +18,7 @@ package fr.acinq.eclair.channel
import akka.actor.{ActorRef, PossiblyHarmful, typed}
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.{ByteVector32, DeterministicWallet, OutPoint, Satoshi, Transaction}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, DeterministicWallet, OutPoint, Satoshi, SatoshiLong, Transaction, TxOut}
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel.LocalFundingStatus.DualFundedUnconfirmedFundingTx
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder._
@ -27,6 +27,8 @@ import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxSigningS
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, SpliceInit, UpdateAddHtlc, UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc}
import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Features, InitFeature, MilliSatoshi, MilliSatoshiLong, RealShortChannelId, UInt64}
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
@ -186,13 +188,15 @@ sealed trait Command extends PossiblyHarmful
sealed trait HasReplyToCommand extends Command { def replyTo: ActorRef }
sealed trait HasOptionalReplyToCommand extends Command { def replyTo_opt: Option[ActorRef] }
final case class CMD_ADD_HTLC(replyTo: ActorRef, amount: MilliSatoshi, paymentHash: ByteVector32, cltvExpiry: CltvExpiry, onion: OnionRoutingPacket, nextBlindingKey_opt: Option[PublicKey], origin: Origin.Hot, commit: Boolean = false) extends HasReplyToCommand
sealed trait HtlcSettlementCommand extends HasOptionalReplyToCommand { def id: Long }
sealed trait ForbiddenCommandDuringSplice extends Command
final case class CMD_ADD_HTLC(replyTo: ActorRef, amount: MilliSatoshi, paymentHash: ByteVector32, cltvExpiry: CltvExpiry, onion: OnionRoutingPacket, nextBlindingKey_opt: Option[PublicKey], origin: Origin.Hot, commit: Boolean = false) extends HasReplyToCommand with ForbiddenCommandDuringSplice
sealed trait HtlcSettlementCommand extends HasOptionalReplyToCommand with ForbiddenCommandDuringSplice { def id: Long }
final case class CMD_FULFILL_HTLC(id: Long, r: ByteVector32, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand
final case class CMD_FAIL_HTLC(id: Long, reason: Either[ByteVector, FailureMessage], delay_opt: Option[FiniteDuration] = None, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand
final case class CMD_FAIL_MALFORMED_HTLC(id: Long, onionHash: ByteVector32, failureCode: Int, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HtlcSettlementCommand
final case class CMD_UPDATE_FEE(feeratePerKw: FeeratePerKw, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HasOptionalReplyToCommand
final case class CMD_SIGN(replyTo_opt: Option[ActorRef] = None) extends HasOptionalReplyToCommand
final case class CMD_UPDATE_FEE(feeratePerKw: FeeratePerKw, commit: Boolean = false, replyTo_opt: Option[ActorRef] = None) extends HasOptionalReplyToCommand with ForbiddenCommandDuringSplice
final case class CMD_SIGN(replyTo_opt: Option[ActorRef] = None) extends HasOptionalReplyToCommand with ForbiddenCommandDuringSplice
final case class ClosingFees(preferred: Satoshi, min: Satoshi, max: Satoshi)
final case class ClosingFeerates(preferred: FeeratePerKw, min: FeeratePerKw, max: FeeratePerKw) {
@ -200,10 +204,18 @@ final case class ClosingFeerates(preferred: FeeratePerKw, min: FeeratePerKw, max
}
sealed trait CloseCommand extends HasReplyToCommand
final case class CMD_CLOSE(replyTo: ActorRef, scriptPubKey: Option[ByteVector], feerates: Option[ClosingFeerates]) extends CloseCommand
final case class CMD_CLOSE(replyTo: ActorRef, scriptPubKey: Option[ByteVector], feerates: Option[ClosingFeerates]) extends CloseCommand with ForbiddenCommandDuringSplice
final case class CMD_FORCECLOSE(replyTo: ActorRef) extends CloseCommand
final case class CMD_BUMP_FUNDING_FEE(replyTo: akka.actor.typed.ActorRef[CommandResponse[CMD_BUMP_FUNDING_FEE]], targetFeerate: FeeratePerKw, lockTime: Long) extends Command
case class SpliceIn(additionalLocalFunding: Satoshi, pushAmount: MilliSatoshi = 0 msat)
case class SpliceOut(amount: Satoshi, scriptPubKey: ByteVector)
final case class CMD_SPLICE(replyTo: akka.actor.typed.ActorRef[CommandResponse[CMD_SPLICE]], spliceIn_opt: Option[SpliceIn], spliceOut_opt: Option[SpliceOut]) extends Command {
require(spliceIn_opt.isDefined || spliceOut_opt.isDefined, "there must be a splice-in or a splice-out")
val additionalLocalFunding: Satoshi = spliceIn_opt.map(_.additionalLocalFunding).getOrElse(0 sat)
val pushAmount: MilliSatoshi = spliceIn_opt.map(_.pushAmount).getOrElse(0 msat)
val spliceOutputs: List[TxOut] = spliceOut_opt.toList.map(s => TxOut(s.amount, s.scriptPubKey))
}
final case class CMD_UPDATE_RELAY_FEE(replyTo: ActorRef, feeBase: MilliSatoshi, feeProportionalMillionths: Long, cltvExpiryDelta_opt: Option[CltvExpiryDelta]) extends HasReplyToCommand
final case class CMD_GET_CHANNEL_STATE(replyTo: ActorRef) extends HasReplyToCommand
final case class CMD_GET_CHANNEL_DATA(replyTo: ActorRef) extends HasReplyToCommand
@ -252,6 +264,7 @@ final case class RES_ADD_SETTLED[+O <: Origin, +R <: HtlcResult](origin: O, htlc
/** other specific responses */
final case class RES_BUMP_FUNDING_FEE(rbfIndex: Int, fundingTxId: ByteVector32, fee: Satoshi) extends CommandSuccess[CMD_BUMP_FUNDING_FEE]
final case class RES_SPLICE(fundingTxIndex: Long, fundingTxId: ByteVector32, capacity: Satoshi, balance: MilliSatoshi) extends CommandSuccess[CMD_SPLICE]
final case class RES_GET_CHANNEL_STATE(state: ChannelState) extends CommandSuccess[CMD_GET_CHANNEL_STATE]
final case class RES_GET_CHANNEL_DATA[+D <: ChannelData](data: D) extends CommandSuccess[CMD_GET_CHANNEL_DATA]
final case class RES_GET_CHANNEL_INFO(nodeId: PublicKey, channelId: ByteVector32, state: ChannelState, data: ChannelData) extends CommandSuccess[CMD_GET_CHANNEL_INFO]
@ -401,6 +414,9 @@ case class ShortIds(real: RealScidStatus, localAlias: Alias, remoteAlias_opt: Op
sealed trait LocalFundingStatus { def signedTx_opt: Option[Transaction] }
object LocalFundingStatus {
sealed trait NotLocked extends LocalFundingStatus
sealed trait Locked extends LocalFundingStatus
sealed trait UnconfirmedFundingTx extends LocalFundingStatus
/**
* In single-funding, fundees only know the funding txid.
@ -408,14 +424,14 @@ object LocalFundingStatus {
* didn't keep the funding tx at all, even as funder (e.g. NORMAL). However, right after restoring those channels we
* retrieve the funding tx and update the funding status immediately.
*/
case class SingleFundedUnconfirmedFundingTx(signedTx_opt: Option[Transaction]) extends UnconfirmedFundingTx
case class DualFundedUnconfirmedFundingTx(sharedTx: SignedSharedTransaction, createdAt: BlockHeight, fundingParams: InteractiveTxParams) extends UnconfirmedFundingTx {
case class SingleFundedUnconfirmedFundingTx(signedTx_opt: Option[Transaction]) extends UnconfirmedFundingTx with NotLocked
case class DualFundedUnconfirmedFundingTx(sharedTx: SignedSharedTransaction, createdAt: BlockHeight, fundingParams: InteractiveTxParams) extends UnconfirmedFundingTx with NotLocked {
override def signedTx_opt: Option[Transaction] = sharedTx.signedTx_opt
}
case class ZeroconfPublishedFundingTx(tx: Transaction) extends UnconfirmedFundingTx {
case class ZeroconfPublishedFundingTx(tx: Transaction) extends UnconfirmedFundingTx with Locked {
override val signedTx_opt: Option[Transaction] = Some(tx)
}
case class ConfirmedFundingTx(tx: Transaction) extends LocalFundingStatus {
case class ConfirmedFundingTx(tx: Transaction) extends LocalFundingStatus with Locked {
override val signedTx_opt: Option[Transaction] = Some(tx)
}
}
@ -435,6 +451,15 @@ object RbfStatus {
case object RbfAborted extends RbfStatus
}
sealed trait SpliceStatus
object SpliceStatus {
case object NoSplice extends SpliceStatus
case class SpliceRequested(cmd: CMD_SPLICE, init: SpliceInit) extends SpliceStatus
case class SpliceInProgress(cmd_opt: Option[CMD_SPLICE], splice: typed.ActorRef[InteractiveTxBuilder.Command], remoteCommitSig: Option[CommitSig]) extends SpliceStatus
case class SpliceWaitingForSigs(signingSession: InteractiveTxSigningSession.WaitingForSigs) extends SpliceStatus
case object SpliceAborted extends SpliceStatus
}
sealed trait ChannelData extends PossiblyHarmful {
def channelId: ByteVector32
}
@ -537,7 +562,8 @@ final case class DATA_NORMAL(commitments: Commitments,
channelUpdate: ChannelUpdate,
localShutdown: Option[Shutdown],
remoteShutdown: Option[Shutdown],
closingFeerates: Option[ClosingFeerates]) extends ChannelDataWithCommitments
closingFeerates: Option[ClosingFeerates],
spliceStatus: SpliceStatus) 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,

View file

@ -22,6 +22,7 @@ import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.wire.protocol
import fr.acinq.eclair.wire.protocol.{AnnouncementSignatures, InteractiveTxMessage, UpdateAddHtlc}
import fr.acinq.eclair.{BlockHeight, CltvExpiry, CltvExpiryDelta, MilliSatoshi, UInt64}
import scodec.bits.ByteVector
/**
* Created by PM on 11/04/2017.
@ -64,20 +65,25 @@ case class PreviousTxMissing (override val channelId: Byte
case class InvalidSharedInput (override val channelId: ByteVector32, serialId: UInt64) extends ChannelException(channelId, s"invalid shared tx_add_input (serial_id=${serialId.toByteVector.toHex})")
case class OutputBelowDust (override val channelId: ByteVector32, serialId: UInt64, amount: Satoshi, dustLimit: Satoshi) extends ChannelException(channelId, s"invalid output amount=$amount below dust=$dustLimit (serial_id=${serialId.toByteVector.toHex})")
case class InvalidSharedOutputAmount (override val channelId: ByteVector32, serialId: UInt64, amount: Satoshi, expected: Satoshi) extends ChannelException(channelId, s"invalid shared output amount=$amount expected=$expected (serial_id=${serialId.toByteVector.toHex})")
case class InvalidSpliceOutputScript (override val channelId: ByteVector32, serialId: UInt64, publicKeyScript: ByteVector) extends ChannelException(channelId, s"invalid splice output publicKeyScript=$publicKeyScript (serial_id=${serialId.toByteVector.toHex})")
case class UnconfirmedInteractiveTxInputs (override val channelId: ByteVector32) extends ChannelException(channelId, "the completed interactive tx contains unconfirmed inputs")
case class InvalidCompleteInteractiveTx (override val channelId: ByteVector32) extends ChannelException(channelId, "the completed interactive tx is invalid")
case class TooManyInteractiveTxRounds (override val channelId: ByteVector32) extends ChannelException(channelId, "too many messages exchanged during interactive tx construction")
case class RbfAttemptAborted (override val channelId: ByteVector32) extends ChannelException(channelId, "rbf attempt aborted")
case class SpliceAttemptAborted (override val channelId: ByteVector32) extends ChannelException(channelId, "splice attempt aborted")
case class DualFundingAborted (override val channelId: ByteVector32) extends ChannelException(channelId, "dual funding aborted")
case class UnexpectedInteractiveTxMessage (override val channelId: ByteVector32, msg: InteractiveTxMessage) extends ChannelException(channelId, s"unexpected interactive-tx message (${msg.getClass.getSimpleName})")
case class UnexpectedFundingSignatures (override val channelId: ByteVector32) extends ChannelException(channelId, "unexpected funding signatures (tx_signatures)")
case class InvalidFundingFeerate (override val channelId: ByteVector32, targetFeerate: FeeratePerKw, actualFeerate: FeeratePerKw) extends ChannelException(channelId, s"invalid funding feerate: target=$targetFeerate actual=$actualFeerate")
case class InvalidFundingSignature (override val channelId: ByteVector32, txId_opt: Option[ByteVector32]) extends ChannelException(channelId, s"invalid funding signature: txId=${txId_opt.map(_.toHex).getOrElse("n/a")}")
case class InvalidRbfFeerate (override val channelId: ByteVector32, proposed: FeeratePerKw, expected: FeeratePerKw) extends ChannelException(channelId, s"invalid rbf attempt: the feerate must be at least $expected, you proposed $proposed")
case class InvalidSpliceRequest (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid splice request")
case class InvalidRbfAlreadyInProgress (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid rbf attempt: the current rbf attempt must be completed or aborted first")
case class InvalidSpliceAlreadyInProgress (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid splice attempt: the current splice attempt must be completed or aborted first")
case class InvalidRbfTxAbortNotAcked (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid rbf attempt: our previous tx_abort has not been acked")
case class InvalidRbfAttemptsExhausted (override val channelId: ByteVector32, maxAttempts: Int) extends ChannelException(channelId, s"invalid rbf attempt: $maxAttempts/$maxAttempts attempts already published")
case class InvalidRbfAttemptTooSoon (override val channelId: ByteVector32, previousAttempt: BlockHeight, nextAttempt: BlockHeight) extends ChannelException(channelId, s"invalid rbf attempt: last attempt made at block=$previousAttempt, next attempt available after block=$nextAttempt")
case class InvalidSpliceTxAbortNotAcked (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid splice attempt: our previous tx_abort has not been acked")
case class InvalidRbfTxConfirmed (override val channelId: ByteVector32) extends ChannelException(channelId, "no need to rbf, transaction is already confirmed")
case class InvalidRbfNonInitiator (override val channelId: ByteVector32) extends ChannelException(channelId, "cannot initiate rbf: we're not the initiator of this interactive-tx attempt")
case class InvalidRbfZeroConf (override val channelId: ByteVector32) extends ChannelException(channelId, "cannot initiate rbf: we're using zero-conf for this interactive-tx attempt")
@ -104,7 +110,7 @@ case class InvalidCloseSignature (override val channelId: Byte
case class InvalidCloseFee (override val channelId: ByteVector32, fee: Satoshi) extends ChannelException(channelId, s"invalid close fee: fee_satoshis=$fee")
case class InvalidCloseAmountBelowDust (override val channelId: ByteVector32, txId: ByteVector32) extends ChannelException(channelId, s"invalid closing tx: some outputs are below dust: txId=$txId")
case class CommitSigCountMismatch (override val channelId: ByteVector32, expected: Int, actual: Int) extends ChannelException(channelId, s"commit sig count mismatch: expected=$expected actual=$actual")
case class HtlcSigCountMismatch (override val channelId: ByteVector32, expected: Int, actual: Int) extends ChannelException(channelId, s"htlc sig count mismatch: expected=$expected actual: $actual")
case class HtlcSigCountMismatch (override val channelId: ByteVector32, expected: Int, actual: Int) extends ChannelException(channelId, s"htlc sig count mismatch: expected=$expected actual=$actual")
case class ForcedLocalCommit (override val channelId: ByteVector32) extends ChannelException(channelId, s"forced local commit")
case class UnexpectedHtlcId (override val channelId: ByteVector32, expected: Long, actual: Long) extends ChannelException(channelId, s"unexpected htlc id: expected=$expected actual=$actual")
case class ExpiryTooSmall (override val channelId: ByteVector32, minimum: CltvExpiry, actual: CltvExpiry, blockHeight: BlockHeight) extends ChannelException(channelId, s"expiry too small: minimum=$minimum actual=$actual blockHeight=$blockHeight")
@ -126,8 +132,8 @@ case class CannotSignWithoutChanges (override val channelId: Byte
case class CannotSignBeforeRevocation (override val channelId: ByteVector32) extends ChannelException(channelId, "cannot sign until next revocation hash is received")
case class UnexpectedRevocation (override val channelId: ByteVector32) extends ChannelException(channelId, "received unexpected RevokeAndAck message")
case class InvalidRevocation (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid revocation")
case class InvalidRevokedCommitProof (override val channelId: ByteVector32, ourLocalCommitmentNumber: Long, theirRemoteCommitmentNumber: Long, invalidPerCommitmentSecret: PrivateKey) extends ChannelException(channelId, s"counterparty claimed that we have a revoked commit but their proof doesn't check out: ourCommitmentNumber=$ourLocalCommitmentNumber theirCommitmentNumber=$theirRemoteCommitmentNumber perCommitmentSecret=$invalidPerCommitmentSecret")
case class InvalidFailureCode (override val channelId: ByteVector32) extends ChannelException(channelId, "UpdateFailMalformedHtlc message doesn't have BADONION bit set")
case class PleasePublishYourCommitment (override val channelId: ByteVector32) extends ChannelException(channelId, "please publish your local commitment")
case class CommandUnavailableInThisState (override val channelId: ByteVector32, command: String, state: ChannelState) extends ChannelException(channelId, s"cannot execute command=$command in state=$state")
case class ForbiddenDuringSplice (override val channelId: ByteVector32, command: String) extends ChannelException(channelId, s"cannot process $command while splicing")
// @formatter:on

View file

@ -1,8 +1,9 @@
package fr.acinq.eclair.channel
import akka.event.LoggingAdapter
import com.softwaremill.quicklens.ModifyPimp
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Crypto, Satoshi, SatoshiLong, Script}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Crypto, Satoshi, SatoshiLong, Script, Transaction}
import fr.acinq.eclair.blockchain.fee.{FeeratePerKw, OnChainFeeConf}
import fr.acinq.eclair.channel.Helpers.Closing
import fr.acinq.eclair.channel.Monitoring.Metrics
@ -45,6 +46,50 @@ case class ChannelParams(channelId: ByteVector32,
remoteParams = remoteParams.copy(initFeatures = remoteInit.features)
)
/**
* As funder we trust ourselves to not double spend funding txs: we could always use a zero-confirmation watch,
* but we need a scid to send the initial channel_update and remote may not provide an alias. That's why we always
* wait for one conf, except if the channel has the zero-conf feature (because presumably the peer will send an
* alias in that case).
*/
def minDepthFunder: Option[Long] = {
if (localParams.initFeatures.hasFeature(Features.ZeroConf)) {
None
} else {
Some(1)
}
}
/**
* Returns the number of confirmations needed to safely handle a funding transaction with remote inputs. We make sure
* the cumulative block reward largely exceeds the channel size, because an attacker that could create a reorg would
* be able to steal the entire channel funding, but would likely miss block rewards during that process, making it
* economically irrational for them.
*
* @param fundingSatoshis funding amount of the channel
* @return number of confirmations needed, if any
*/
def minDepthFundee(defaultMinDepth: Int, fundingSatoshis: Satoshi): Option[Long] = fundingSatoshis match {
case _ if localParams.initFeatures.hasFeature(Features.ZeroConf) => None // zero-conf stay zero-conf, whatever the funding amount is
case funding if funding <= Channel.MAX_FUNDING_WITHOUT_WUMBO => Some(defaultMinDepth)
case funding => Some(ChannelParams.minDepthScaled(defaultMinDepth, funding))
}
/**
* When using dual funding or splices, we wait for multiple confirmations even if we're the initiator because:
* - our peer may also contribute to the funding transaction, even if they don't contribute to the channel funding amount
* - even if they don't, we may RBF the transaction and don't want to handle reorgs
*
* @param fundingAmount the total target channel funding amount, including local and remote contributions.
*/
def minDepthDualFunding(defaultMinDepth: Int, fundingAmount: Satoshi): Option[Long] = {
if (localParams.initFeatures.hasFeature(Features.ZeroConf)) {
None
} else {
minDepthFundee(defaultMinDepth, fundingAmount)
}
}
/**
*
* @param localScriptPubKey local script pubkey (provided in CMD_CLOSE, as an upfront shutdown script, or set to the current final onchain script)
@ -75,6 +120,15 @@ case class ChannelParams(channelId: ByteVector32,
}
object ChannelParams {
def minDepthScaled(defaultMinDepth: Int, amount: Satoshi): Int = {
val blockReward = 6.25 // this is true as of ~May 2020, but will be too large after 2024
val scalingFactor = 15
val blocksToReachFunding = (((scalingFactor * amount.toBtc.toDouble) / blockReward).ceil + 1).toInt
defaultMinDepth.max(blocksToReachFunding)
}
}
// @formatter:off
case class LocalChanges(proposed: List[UpdateMessage], signed: List[UpdateMessage], acked: List[UpdateMessage]) {
def all: List[UpdateMessage] = proposed ++ signed ++ acked
@ -154,15 +208,23 @@ case class RemoteCommit(index: Long, spec: CommitmentSpec, txid: ByteVector32, r
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)))
CommitSig(params.channelId, sig, htlcSigs.toList)
}
}
/** 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)
/** A minimal commitment for a given funding tx. */
case class Commitment(localFundingStatus: LocalFundingStatus, remoteFundingStatus: RemoteFundingStatus,
/**
* A minimal commitment for a given funding tx.
*
* @param fundingTxIndex index of the funding tx in the life of the channel:
* - initial funding tx has index 0
* - splice txs have index 1, 2, ...
* - commitments that share the same index are rbfed
*/
case class Commitment(fundingTxIndex: Long,
localFundingStatus: LocalFundingStatus, remoteFundingStatus: RemoteFundingStatus,
localCommit: LocalCommit, remoteCommit: RemoteCommit, nextRemoteCommit_opt: Option[NextRemoteCommit]) {
val commitInput: InputInfo = localCommit.commitTxAndRemoteSig.commitTx.input
val fundingTxId: ByteVector32 = commitInput.outPoint.txid
@ -266,16 +328,15 @@ case class Commitment(localFundingStatus: LocalFundingStatus, remoteFundingStatu
private def hasNoPendingHtlcs: Boolean = localCommit.spec.htlcs.isEmpty && remoteCommit.spec.htlcs.isEmpty && nextRemoteCommit_opt.isEmpty
def hasNoPendingHtlcsOrFeeUpdate(changes: CommitmentChanges): Boolean =
nextRemoteCommit_opt.isEmpty &&
localCommit.spec.htlcs.isEmpty &&
remoteCommit.spec.htlcs.isEmpty &&
(changes.localChanges.signed ++ changes.localChanges.acked ++ changes.remoteChanges.signed ++ changes.remoteChanges.acked).collectFirst { case _: UpdateFee => true }.isEmpty
def hasNoPendingHtlcsOrFeeUpdate(changes: CommitmentChanges): Boolean = hasNoPendingHtlcs &&
(changes.localChanges.signed ++ changes.localChanges.acked ++ changes.remoteChanges.signed ++ changes.remoteChanges.acked).collectFirst { case _: UpdateFee => true }.isEmpty
def hasPendingOrProposedHtlcs(changes: CommitmentChanges): Boolean = !hasNoPendingHtlcs ||
changes.localChanges.all.exists(_.isInstanceOf[UpdateAddHtlc]) ||
changes.remoteChanges.all.exists(_.isInstanceOf[UpdateAddHtlc])
def isIdle(changes: CommitmentChanges): Boolean = hasNoPendingHtlcs && changes.localChanges.all.isEmpty && changes.remoteChanges.all.isEmpty
def timedOutOutgoingHtlcs(currentHeight: BlockHeight): Set[UpdateAddHtlc] = {
def expired(add: UpdateAddHtlc): Boolean = currentHeight >= add.cltvExpiry.blockHeight
@ -514,7 +575,7 @@ case class Commitment(localFundingStatus: LocalFundingStatus, remoteFundingStatu
log.info(s"built remote commit number=${remoteCommit.index + 1} toLocalMsat=${spec.toLocal.toLong} toRemoteMsat=${spec.toRemote.toLong} htlc_in={} htlc_out={} feeratePerKw=${spec.commitTxFeerate} txid=${remoteCommitTx.tx.txid} fundingTxId=$fundingTxId", spec.htlcs.collect(DirectedHtlc.outgoing).map(_.id).mkString(","), spec.htlcs.collect(DirectedHtlc.incoming).map(_.id).mkString(","))
Metrics.recordHtlcsInFlight(spec, remoteCommit.spec)
val commitSig = CommitSig(params.channelId, sig, htlcSigs.toList, TlvStream(CommitSigTlv.FundingTxIdTlv(fundingTxId)))
val commitSig = CommitSig(params.channelId, sig, htlcSigs.toList)
val nextRemoteCommit = NextRemoteCommit(commitSig, RemoteCommit(remoteCommit.index + 1, spec, remoteCommitTx.tx.txid, remoteNextPerCommitmentPoint))
(copy(nextRemoteCommit_opt = Some(nextRemoteCommit)), commitSig)
}
@ -622,6 +683,7 @@ object Commitment {
/** Subset of Commitments when we want to work with a single, specific commitment. */
case class FullCommitment(params: ChannelParams, changes: CommitmentChanges,
fundingTxIndex: Long,
localFundingStatus: LocalFundingStatus, remoteFundingStatus: RemoteFundingStatus,
localCommit: LocalCommit, remoteCommit: RemoteCommit, nextRemoteCommit_opt: Option[NextRemoteCommit]) {
val channelId = params.channelId
@ -630,7 +692,7 @@ case class FullCommitment(params: ChannelParams, changes: CommitmentChanges,
val commitInput = localCommit.commitTxAndRemoteSig.commitTx.input
val fundingTxId = commitInput.outPoint.txid
val capacity = commitInput.txOut.amount
val commitment = Commitment(localFundingStatus, remoteFundingStatus, localCommit, remoteCommit, nextRemoteCommit_opt)
val commitment = Commitment(fundingTxIndex, localFundingStatus, remoteFundingStatus, localCommit, remoteCommit, nextRemoteCommit_opt)
def localChannelReserve: Satoshi = commitment.localChannelReserve(params)
@ -662,11 +724,15 @@ case class WaitForRev(sentAfterLocalCommitIndex: Long)
/**
* @param active all currently valid commitments
* @param inactive commitments that can potentially end up on-chain, but shouldn't be taken into account
* when updating the channel state; they are zero-conf and have been superseded by a newer
* commitment, which funding tx is not yet confirmed, and will be pruned when it confirms
* @param remoteChannelData_opt peer backup
*/
case class Commitments(params: ChannelParams,
changes: CommitmentChanges,
active: Seq[Commitment],
inactive: Seq[Commitment] = Nil,
remoteNextCommitInfo: Either[WaitForRev, PublicKey], // this one is tricky, it must be kept in sync with Commitment.nextRemoteCommit_opt
remotePerCommitmentSecrets: ShaChain,
originChannels: Map[Long, Origin], // for outgoing htlcs relayed through us, details about the corresponding incoming htlcs
@ -690,12 +756,15 @@ case class Commitments(params: ChannelParams,
lazy val availableBalanceForReceive: MilliSatoshi = active.map(_.availableBalanceForReceive(params, changes)).min
// We always use the last commitment that was created, to make sure we never go back in time.
val latest = FullCommitment(params, changes, active.head.localFundingStatus, active.head.remoteFundingStatus, active.head.localCommit, active.head.remoteCommit, active.head.nextRemoteCommit_opt)
val latest = FullCommitment(params, changes, active.head.fundingTxIndex, active.head.localFundingStatus, active.head.remoteFundingStatus, active.head.localCommit, active.head.remoteCommit, active.head.nextRemoteCommit_opt)
val all: Seq[Commitment] = active ++ inactive
def add(commitment: Commitment): Commitments = copy(active = commitment +: active)
// @formatter:off
// HTLCs and pending changes are the same for all active commitments, so we don't need to loop through all of them.
def isIdle: Boolean = active.head.isIdle(changes)
def hasNoPendingHtlcsOrFeeUpdate: Boolean = active.head.hasNoPendingHtlcsOrFeeUpdate(changes)
def hasPendingOrProposedHtlcs: Boolean = active.head.hasPendingOrProposedHtlcs(changes)
def timedOutOutgoingHtlcs(currentHeight: BlockHeight): Set[UpdateAddHtlc] = active.head.timedOutOutgoingHtlcs(currentHeight)
@ -875,18 +944,27 @@ case class Commitments(params: ChannelParams,
active = active1,
remoteNextCommitInfo = Left(WaitForRev(localCommitIndex))
)
Right(commitments1, sigs)
val sigs1 = if (sigs.size > 1) {
// if there are more than one sig, we add a tlv to tell the receiver how many sigs are to be expected
sigs.map { sig => sig.modify(_.tlvStream.records).using(_ + CommitSigTlv.BatchTlv(sigs.size)) }
} else {
sigs
}
Right(commitments1, sigs1)
case Left(_) => Left(CannotSignBeforeRevocation(channelId))
}
}
def receiveCommit(commits: Seq[CommitSig], keyManager: ChannelKeyManager)(implicit log: LoggingAdapter): Either[ChannelException, (Commitments, RevokeAndAck)] = {
// first we make sure that we have exactly one commit_sig for each active commitment
if (commits.size != active.size) {
// We may receive more commit_sig than the number of active commitments, because there can be a race where we send
// splice_locked while our peer is sending us a batch of commit_sig. When that happens, we simply need to discard
// the commit_sig that belong to commitments we deactivated.
if (commits.size < active.size) {
return Left(CommitSigCountMismatch(channelId, active.size, commits.size))
}
val channelKeyPath = keyManager.keyPath(params.localParams, params.channelConfig)
val localPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, localCommitIndex + 1)
// Signatures are sent in order (most recent first), calling `zip` will drop trailing sigs that are for deactivated/pruned commitments.
val active1 = active.zip(commits).map { case (commitment, commit) =>
commitment.receiveCommit(keyManager, params, changes, localPerCommitmentPoint, commit) match {
case Left(f) => return Left(f)
@ -1010,38 +1088,113 @@ case class Commitments(params: ChannelParams,
active.forall(_.commitInput.redeemScript == fundingScript)
}
def updateLocalFundingStatus(txId: ByteVector32, status: LocalFundingStatus)(implicit log: LoggingAdapter): Either[Commitments, (Commitments, Commitment)] = {
if (!this.active.exists(_.fundingTxId == txId)) {
log.error(s"funding txid=$txId doesn't match any of our funding txs")
Left(this)
} else {
val commitments1 = copy(active = active.map {
case c if c.fundingTxId == txId =>
log.info(s"setting localFundingStatus=${status.getClass.getSimpleName} for funding txid=$txId")
c.copy(localFundingStatus = status)
case c => c
}).pruneCommitments()
val commitment = commitments1.active.find(_.fundingTxId == txId).get
Right(commitments1, commitment)
/**
* Update the local/remote funding status
*
* @param updateMethod This method is tricky: it passes the fundingTxIndex of the commitment corresponding to the
* fundingTxId, because in the remote case we may update several commitments.
*/
private def updateFundingStatus(fundingTxId: ByteVector32, updateMethod: Long => PartialFunction[Commitment, Commitment])(implicit log: LoggingAdapter): Either[Commitments, (Commitments, Commitment)] = {
all.find(_.fundingTxId == fundingTxId) match {
case Some(commitment) =>
val commitments1 = copy(
active = active.map(updateMethod(commitment.fundingTxIndex)),
inactive = inactive.map(updateMethod(commitment.fundingTxIndex))
)
val commitment1 = commitments1.all.find(_.fundingTxId == fundingTxId).get // NB: this commitment might be pruned at the next line
val commitments2 = commitments1.deactivateCommitments().pruneCommitments()
Right(commitments2, commitment1)
case None =>
log.warning(s"fundingTxId=$fundingTxId doesn't match any of our funding txs")
Left(this)
}
}
def updateLocalFundingStatus(fundingTxId: ByteVector32, status: LocalFundingStatus)(implicit log: LoggingAdapter): Either[Commitments, (Commitments, Commitment)] =
updateFundingStatus(fundingTxId, _ => {
case c if c.fundingTxId == fundingTxId =>
log.info(s"setting localFundingStatus=${status.getClass.getSimpleName} for fundingTxId=$fundingTxId fundingTxIndex=${c.fundingTxIndex}")
c.copy(localFundingStatus = status)
case c => c
})
def updateRemoteFundingStatus(fundingTxId: ByteVector32)(implicit log: LoggingAdapter): Either[Commitments, (Commitments, Commitment)] =
updateFundingStatus(fundingTxId, fundingTxIndex => {
// all funding older than this one are considered locked
case c if c.fundingTxId == fundingTxId || c.fundingTxIndex < fundingTxIndex =>
log.info(s"setting remoteFundingStatus=${RemoteFundingStatus.Locked.getClass.getSimpleName} for fundingTxId=$fundingTxId fundingTxIndex=${c.fundingTxIndex}")
c.copy(remoteFundingStatus = RemoteFundingStatus.Locked)
case c => c
})
/**
* Commitments are considered inactive when they have been superseded by a newer commitment, but can still potentially
* end up on-chain. This is a consequence of using zero-conf. Inactive commitments will be cleaned up by
* [[pruneCommitments()]], when the next funding tx confirms.
*/
private def deactivateCommitments()(implicit log: LoggingAdapter): Commitments = {
// When a commitment is locked, it implicitly locks all previous commitments.
// This ensures that we only have to send splice_locked for the latest commitment instead of sending it for every commitment.
// A side-effect is that previous commitments that are implicitly locked don't necessarily have their status correctly set.
// That's why we look at locked commitments separately and then select the one with the oldest fundingTxIndex.
val lastLocalLocked_opt = active.find(_.localFundingStatus.isInstanceOf[LocalFundingStatus.Locked])
val lastRemoteLocked_opt = active.find(_.remoteFundingStatus == RemoteFundingStatus.Locked)
val lastLocked_opt = (lastLocalLocked_opt, lastRemoteLocked_opt) match {
// We select the locked commitment with the smaller value for fundingTxIndex, but both have to be defined.
// If both have the same fundingTxIndex, they must actually be the same commitment, because:
// - we only allow RBF attempts when we're not using zero-conf
// - transactions with the same fundingTxIndex double-spend each other, so only one of them can confirm
// - we don't allow creating a splice on top of an unconfirmed transaction that has RBF attempts (because it
// would become invalid if another of the RBF attempts end up being confirmed)
case (Some(lastLocalLocked), Some(lastRemoteLocked)) => Some(Seq(lastLocalLocked, lastRemoteLocked).minBy(_.fundingTxIndex))
// Special case for the initial funding tx, we only require a local lock because channel_ready doesn't explicitly reference a funding tx.
case (Some(lastLocalLocked), None) if lastLocalLocked.fundingTxIndex == 0 => Some(lastLocalLocked)
case _ => None
}
lastLocked_opt match {
case Some(lastLocked) =>
// all commitments older than this one are inactive
val inactive1 = active.filter(c => c.fundingTxId != lastLocked.fundingTxId && c.fundingTxIndex <= lastLocked.fundingTxIndex)
inactive1.foreach(c => log.info("deactivating commitment fundingTxIndex={} fundingTxId={}", c.fundingTxIndex, c.fundingTxId))
copy(
active = active diff inactive1,
inactive = inactive1 ++ inactive
)
case _ =>
this
}
}
/**
* Current (pre-splice) implementation prune initial commitments. There can be several of them with RBF, but they all
* double-spend each other and can be pruned once one of them confirms.
* We can prune commitments in two cases:
* - their funding tx has been permanently double-spent by the funding tx of a concurrent commitment (happens when using RBF)
* - their funding tx has been permanently spent by a splice tx
*/
def pruneCommitments()(implicit log: LoggingAdapter): Commitments = {
active.find(_.localFundingStatus.isInstanceOf[LocalFundingStatus.ConfirmedFundingTx]) match {
private def pruneCommitments()(implicit log: LoggingAdapter): Commitments = {
all
.filter(_.localFundingStatus.isInstanceOf[LocalFundingStatus.ConfirmedFundingTx])
.sortBy(_.fundingTxIndex)
.lastOption match {
case Some(lastConfirmed) =>
// we can prune all other commitments with the same or lower funding index
val pruned = active.filter(c => c.fundingTxId != lastConfirmed.fundingTxId)
val active1 = active diff pruned
pruned.foreach(c => log.info("pruning commitment fundingTxid={}", c.fundingTxId))
copy(active = active1)
// We can prune all other commitments with the same or lower funding index.
// NB: we cannot prune active commitments, even if we know that they have been double-spent, because our peer
// may not yet be aware of it, and will expect us to send commit_sig.
val pruned = inactive.filter(c => c.fundingTxId != lastConfirmed.fundingTxId && c.fundingTxIndex <= lastConfirmed.fundingTxIndex)
pruned.foreach(c => log.info("pruning commitment fundingTxIndex={} fundingTxId={}", c.fundingTxIndex, c.fundingTxId))
copy(inactive = inactive diff pruned)
case _ =>
this
}
}
/**
* Find the corresponding commitment, based on a spending transaction.
*
* @param spendingTx A transaction that may spend a current or former funding tx
*/
def resolveCommitment(spendingTx: Transaction): Option[Commitment] = {
all.find(c => spendingTx.txIn.map(_.outPoint).contains(c.commitInput.outPoint))
}
}
object Commitments {

View file

@ -25,7 +25,7 @@ import fr.acinq.bitcoin.scalacompat._
import fr.acinq.eclair._
import fr.acinq.eclair.blockchain.fee.{FeeEstimator, FeeTargets, FeeratePerKw, OnChainFeeConf}
import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.channel.fsm.Channel.{ChannelConf, REFRESH_CHANNEL_UPDATE_INTERVAL}
import fr.acinq.eclair.channel.fsm.Channel.REFRESH_CHANNEL_UPDATE_INTERVAL
import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
import fr.acinq.eclair.crypto.{Generators, ShaChain}
import fr.acinq.eclair.db.ChannelsDb
@ -250,8 +250,8 @@ object Helpers {
}
/** Compute the channelId of a dual-funded channel. */
def computeChannelId(open: OpenDualFundedChannel, accept: AcceptDualFundedChannel): ByteVector32 = {
val bin = Seq(open.revocationBasepoint.value, accept.revocationBasepoint.value)
def computeChannelId(openRevocationBasepoint: PublicKey, acceptRevocationBasepoint: PublicKey): ByteVector32 = {
val bin = Seq(openRevocationBasepoint.value, acceptRevocationBasepoint.value)
.sortWith(LexicographicalOrdering.isLessThan)
.reduce(_ ++ _)
Crypto.sha256(bin)
@ -348,54 +348,6 @@ object Helpers {
object Funding {
/**
* As funder we trust ourselves to not double spend funding txs: we could always use a zero-confirmation watch,
* but we need a scid to send the initial channel_update and remote may not provide an alias. That's why we always
* wait for one conf, except if the channel has the zero-conf feature (because presumably the peer will send an
* alias in that case).
*/
def minDepthFunder(localFeatures: Features[InitFeature]): Option[Long] = {
if (localFeatures.hasFeature(Features.ZeroConf)) {
None
} else {
Some(1)
}
}
/**
* Returns the number of confirmations needed to safely handle the funding transaction,
* we make sure the cumulative block reward largely exceeds the channel size.
*
* @param fundingSatoshis funding amount of the channel
* @return number of confirmations needed, if any
*/
def minDepthFundee(channelConf: ChannelConf, localFeatures: Features[InitFeature], fundingSatoshis: Satoshi): Option[Long] = fundingSatoshis match {
case _ if localFeatures.hasFeature(Features.ZeroConf) => None // zero-conf stay zero-conf, whatever the funding amount is
case funding if funding <= Channel.MAX_FUNDING_WITHOUT_WUMBO => Some(channelConf.minDepthBlocks)
case funding =>
val blockReward = 6.25 // this is true as of ~May 2020, but will be too large after 2024
val scalingFactor = 15
val blocksToReachFunding = (((scalingFactor * funding.toBtc.toDouble) / blockReward).ceil + 1).toInt
Some(channelConf.minDepthBlocks.max(blocksToReachFunding))
}
/**
* When using dual funding, we wait for multiple confirmations even if we're the initiator because:
* - our peer may also contribute to the funding transaction, even if they don't contribute to the channel funding amount
* - even if they don't, we may RBF the transaction and don't want to handle reorgs
*/
def minDepthDualFunding(channelConf: ChannelConf, localFeatures: Features[InitFeature], isInitiator: Boolean, localAmount: Satoshi, remoteAmount: Satoshi): Option[Long] = {
if (isInitiator && remoteAmount == 0.sat) {
if (localFeatures.hasFeature(Features.ZeroConf)) {
None
} else {
Some(channelConf.minDepthBlocks)
}
} else {
minDepthFundee(channelConf, localFeatures, localAmount + remoteAmount)
}
}
def makeFundingInputInfo(fundingTxId: ByteVector32, fundingTxOutputIndex: Int, fundingSatoshis: Satoshi, fundingPubkey1: PublicKey, fundingPubkey2: PublicKey): InputInfo = {
val fundingScript = multiSig2of2(fundingPubkey1, fundingPubkey2)
val fundingTxOut = TxOut(fundingSatoshis, pay2wsh(fundingScript))
@ -1028,15 +980,17 @@ object Helpers {
object RevokedClose {
/**
* When an unexpected transaction spending the funding tx is detected:
* 1) we find out if the published transaction is one of remote's revoked txs
* 2) and then:
* a) if it is a revoked tx we build a set of transactions that will punish them by stealing all their funds
* b) otherwise there is nothing we can do
* When an unexpected transaction spending the funding tx is detected, we must be in one of the following scenarios:
*
* @return a [[RevokedCommitPublished]] object containing penalty transactions if the tx is a revoked commitment
* - it is a revoked commitment: we then extract the remote per-commitment secret and publish penalty transactions
* - it is a future commitment: if we lost future state, our peer could publish a future commitment (which may be
* revoked, but we won't be able to know because we lost the corresponding state)
* - it is not a valid commitment transaction: if our peer was able to steal our funding private key, they can
* spend the funding transaction however they want, and we won't be able to do anything about it
*
* This function returns the per-commitment secret in the first case, and None in the other cases.
*/
def claimCommitTxOutputs(keyManager: ChannelKeyManager, params: ChannelParams, remotePerCommitmentSecrets: ShaChain, commitTx: Transaction, db: ChannelsDb, feeEstimator: FeeEstimator, feeTargets: FeeTargets, finalScriptPubKey: ByteVector)(implicit log: LoggingAdapter): Option[RevokedCommitPublished] = {
def getRemotePerCommitmentSecret(keyManager: ChannelKeyManager, params: ChannelParams, remotePerCommitmentSecrets: ShaChain, commitTx: Transaction)(implicit log: LoggingAdapter): Option[(Long, PrivateKey)] = {
import params._
// a valid tx will always have at least one input, but this ensures we don't throw in tests
val sequence = commitTx.txIn.headOption.map(_.sequence).getOrElse(0L)
@ -1050,84 +1004,91 @@ object Helpers {
None
} else {
// now we know what commit number this tx is referring to, we can derive the commitment point from the shachain
remotePerCommitmentSecrets.getHash(0xFFFFFFFFFFFFL - txNumber)
.map(d => PrivateKey(d))
.map(remotePerCommitmentSecret => {
log.warning(s"a revoked commit has been published with txnumber=$txNumber")
val remotePerCommitmentPoint = remotePerCommitmentSecret.publicKey
val remoteDelayedPaymentPubkey = Generators.derivePubKey(remoteParams.delayedPaymentBasepoint, remotePerCommitmentPoint)
val remoteRevocationPubkey = Generators.revocationPubKey(keyManager.revocationPoint(channelKeyPath).publicKey, remotePerCommitmentPoint)
val remoteHtlcPubkey = Generators.derivePubKey(remoteParams.htlcBasepoint, remotePerCommitmentPoint)
val localPaymentPubkey = Generators.derivePubKey(keyManager.paymentPoint(channelKeyPath).publicKey, remotePerCommitmentPoint)
val localHtlcPubkey = Generators.derivePubKey(keyManager.htlcPoint(channelKeyPath).publicKey, remotePerCommitmentPoint)
val feeratePerKwMain = feeEstimator.getFeeratePerKw(feeTargets.claimMainBlockTarget)
// we need to use a high fee here for punishment txs because after a delay they can be spent by the counterparty
val feeratePerKwPenalty = feeEstimator.getFeeratePerKw(target = 2)
// first we will claim our main output right away
val mainTx = channelFeatures match {
case ct if ct.paysDirectlyToWallet =>
log.info(s"channel uses option_static_remotekey to pay directly to our wallet, there is nothing to do")
None
case ct => ct.commitmentFormat match {
case DefaultCommitmentFormat => withTxGenerationLog("claim-p2wpkh-output") {
Transactions.makeClaimP2WPKHOutputTx(commitTx, localParams.dustLimit, localPaymentPubkey, finalScriptPubKey, feeratePerKwMain).map(claimMain => {
val sig = keyManager.sign(claimMain, keyManager.paymentPoint(channelKeyPath), remotePerCommitmentPoint, TxOwner.Local, commitmentFormat)
Transactions.addSigs(claimMain, localPaymentPubkey, sig)
})
}
case _: AnchorOutputsCommitmentFormat => withTxGenerationLog("remote-main-delayed") {
Transactions.makeClaimRemoteDelayedOutputTx(commitTx, localParams.dustLimit, localPaymentPoint, finalScriptPubKey, feeratePerKwMain).map(claimMain => {
val sig = keyManager.sign(claimMain, keyManager.paymentPoint(channelKeyPath), TxOwner.Local, commitmentFormat)
Transactions.addSigs(claimMain, sig)
})
}
}
}
// then we punish them by stealing their main output
val mainPenaltyTx = withTxGenerationLog("main-penalty") {
Transactions.makeMainPenaltyTx(commitTx, localParams.dustLimit, remoteRevocationPubkey, finalScriptPubKey, localParams.toSelfDelay, remoteDelayedPaymentPubkey, feeratePerKwPenalty).map(txinfo => {
val sig = keyManager.sign(txinfo, keyManager.revocationPoint(channelKeyPath), remotePerCommitmentSecret, TxOwner.Local, commitmentFormat)
Transactions.addSigs(txinfo, sig)
})
}
// we retrieve the information needed to rebuild htlc scripts
val htlcInfos = db.listHtlcInfos(channelId, txNumber)
log.info(s"got htlcs=${htlcInfos.size} for txnumber=$txNumber")
val htlcsRedeemScripts = (
htlcInfos.map { case (paymentHash, cltvExpiry) => Scripts.htlcReceived(remoteHtlcPubkey, localHtlcPubkey, remoteRevocationPubkey, Crypto.ripemd160(paymentHash), cltvExpiry, commitmentFormat) } ++
htlcInfos.map { case (paymentHash, _) => Scripts.htlcOffered(remoteHtlcPubkey, localHtlcPubkey, remoteRevocationPubkey, Crypto.ripemd160(paymentHash), commitmentFormat) }
)
.map(redeemScript => Script.write(pay2wsh(redeemScript)) -> Script.write(redeemScript))
.toMap
// and finally we steal the htlc outputs
val htlcPenaltyTxs = commitTx.txOut.zipWithIndex.collect { case (txOut, outputIndex) if htlcsRedeemScripts.contains(txOut.publicKeyScript) =>
val htlcRedeemScript = htlcsRedeemScripts(txOut.publicKeyScript)
withTxGenerationLog("htlc-penalty") {
Transactions.makeHtlcPenaltyTx(commitTx, outputIndex, htlcRedeemScript, localParams.dustLimit, finalScriptPubKey, feeratePerKwPenalty).map(htlcPenalty => {
val sig = keyManager.sign(htlcPenalty, keyManager.revocationPoint(channelKeyPath), remotePerCommitmentSecret, TxOwner.Local, commitmentFormat)
Transactions.addSigs(htlcPenalty, sig, remoteRevocationPubkey)
})
}
}.toList.flatten
RevokedCommitPublished(
commitTx = commitTx,
claimMainOutputTx = mainTx,
mainPenaltyTx = mainPenaltyTx,
htlcPenaltyTxs = htlcPenaltyTxs,
claimHtlcDelayedPenaltyTxs = Nil, // we will generate and spend those if they publish their HtlcSuccessTx or HtlcTimeoutTx
irrevocablySpent = Map.empty
)
})
remotePerCommitmentSecrets.getHash(0xFFFFFFFFFFFFL - txNumber).map(d => (txNumber, PrivateKey(d)))
}
}
/**
* When a revoked commitment transaction spending the funding tx is detected, we build a set of transactions that
* will punish our peer by stealing all their funds.
*/
def claimCommitTxOutputs(keyManager: ChannelKeyManager, params: ChannelParams, commitTx: Transaction, commitmentNumber: Long, remotePerCommitmentSecret: PrivateKey, db: ChannelsDb, feeEstimator: FeeEstimator, feeTargets: FeeTargets, finalScriptPubKey: ByteVector)(implicit log: LoggingAdapter): RevokedCommitPublished = {
import params._
log.warning("a revoked commit has been published with commitmentNumber={}", commitmentNumber)
val channelKeyPath = keyManager.keyPath(localParams, channelConfig)
val localPaymentPoint = localParams.walletStaticPaymentBasepoint.getOrElse(keyManager.paymentPoint(channelKeyPath).publicKey)
val remotePerCommitmentPoint = remotePerCommitmentSecret.publicKey
val remoteDelayedPaymentPubkey = Generators.derivePubKey(remoteParams.delayedPaymentBasepoint, remotePerCommitmentPoint)
val remoteRevocationPubkey = Generators.revocationPubKey(keyManager.revocationPoint(channelKeyPath).publicKey, remotePerCommitmentPoint)
val remoteHtlcPubkey = Generators.derivePubKey(remoteParams.htlcBasepoint, remotePerCommitmentPoint)
val localPaymentPubkey = Generators.derivePubKey(keyManager.paymentPoint(channelKeyPath).publicKey, remotePerCommitmentPoint)
val localHtlcPubkey = Generators.derivePubKey(keyManager.htlcPoint(channelKeyPath).publicKey, remotePerCommitmentPoint)
val feeratePerKwMain = feeEstimator.getFeeratePerKw(feeTargets.claimMainBlockTarget)
// we need to use a high fee here for punishment txs because after a delay they can be spent by the counterparty
val feeratePerKwPenalty = feeEstimator.getFeeratePerKw(target = 2)
// first we will claim our main output right away
val mainTx = channelFeatures match {
case ct if ct.paysDirectlyToWallet =>
log.info(s"channel uses option_static_remotekey to pay directly to our wallet, there is nothing to do")
None
case ct => ct.commitmentFormat match {
case DefaultCommitmentFormat => withTxGenerationLog("remote-main") {
Transactions.makeClaimP2WPKHOutputTx(commitTx, localParams.dustLimit, localPaymentPubkey, finalScriptPubKey, feeratePerKwMain).map(claimMain => {
val sig = keyManager.sign(claimMain, keyManager.paymentPoint(channelKeyPath), remotePerCommitmentPoint, TxOwner.Local, commitmentFormat)
Transactions.addSigs(claimMain, localPaymentPubkey, sig)
})
}
case _: AnchorOutputsCommitmentFormat => withTxGenerationLog("remote-main-delayed") {
Transactions.makeClaimRemoteDelayedOutputTx(commitTx, localParams.dustLimit, localPaymentPoint, finalScriptPubKey, feeratePerKwMain).map(claimMain => {
val sig = keyManager.sign(claimMain, keyManager.paymentPoint(channelKeyPath), TxOwner.Local, commitmentFormat)
Transactions.addSigs(claimMain, sig)
})
}
}
}
// then we punish them by stealing their main output
val mainPenaltyTx = withTxGenerationLog("main-penalty") {
Transactions.makeMainPenaltyTx(commitTx, localParams.dustLimit, remoteRevocationPubkey, finalScriptPubKey, localParams.toSelfDelay, remoteDelayedPaymentPubkey, feeratePerKwPenalty).map(txinfo => {
val sig = keyManager.sign(txinfo, keyManager.revocationPoint(channelKeyPath), remotePerCommitmentSecret, TxOwner.Local, commitmentFormat)
Transactions.addSigs(txinfo, sig)
})
}
// we retrieve the information needed to rebuild htlc scripts
val htlcInfos = db.listHtlcInfos(channelId, commitmentNumber)
log.info("got {} htlcs for commitmentNumber={}", htlcInfos.size, commitmentNumber)
val htlcsRedeemScripts = (
htlcInfos.map { case (paymentHash, cltvExpiry) => Scripts.htlcReceived(remoteHtlcPubkey, localHtlcPubkey, remoteRevocationPubkey, Crypto.ripemd160(paymentHash), cltvExpiry, commitmentFormat) } ++
htlcInfos.map { case (paymentHash, _) => Scripts.htlcOffered(remoteHtlcPubkey, localHtlcPubkey, remoteRevocationPubkey, Crypto.ripemd160(paymentHash), commitmentFormat) }
)
.map(redeemScript => Script.write(pay2wsh(redeemScript)) -> Script.write(redeemScript))
.toMap
// and finally we steal the htlc outputs
val htlcPenaltyTxs = commitTx.txOut.zipWithIndex.collect { case (txOut, outputIndex) if htlcsRedeemScripts.contains(txOut.publicKeyScript) =>
val htlcRedeemScript = htlcsRedeemScripts(txOut.publicKeyScript)
withTxGenerationLog("htlc-penalty") {
Transactions.makeHtlcPenaltyTx(commitTx, outputIndex, htlcRedeemScript, localParams.dustLimit, finalScriptPubKey, feeratePerKwPenalty).map(htlcPenalty => {
val sig = keyManager.sign(htlcPenalty, keyManager.revocationPoint(channelKeyPath), remotePerCommitmentSecret, TxOwner.Local, commitmentFormat)
Transactions.addSigs(htlcPenalty, sig, remoteRevocationPubkey)
})
}
}.toList.flatten
RevokedCommitPublished(
commitTx = commitTx,
claimMainOutputTx = mainTx,
mainPenaltyTx = mainPenaltyTx,
htlcPenaltyTxs = htlcPenaltyTxs,
claimHtlcDelayedPenaltyTxs = Nil, // we will generate and spend those if they publish their HtlcSuccessTx or HtlcTimeoutTx
irrevocablySpent = Map.empty
)
}
/**
* Claims the output of an [[HtlcSuccessTx]] or [[HtlcTimeoutTx]] transaction using a revocation key.
*

View file

@ -21,7 +21,8 @@ import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapte
import akka.actor.{Actor, ActorContext, ActorRef, FSM, OneForOneStrategy, PossiblyHarmful, Props, SupervisorStrategy, typed}
import akka.event.Logging.MDC
import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, SatoshiLong, Transaction}
import fr.acinq.bitcoin.scalacompat.{Block, ByteVector32, Satoshi, SatoshiLong, Transaction}
import fr.acinq.eclair.Features.SplicePrototype
import fr.acinq.eclair.Logs.LogCategory
import fr.acinq.eclair._
import fr.acinq.eclair.blockchain.OnChainWallet.MakeFundingTxResponse
@ -30,12 +31,14 @@ import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient
import fr.acinq.eclair.channel.Commitments.PostRevocationAction
import fr.acinq.eclair.channel.Helpers.Closing.MutualClose
import fr.acinq.eclair.channel.Helpers.Syncing.SyncResult
import fr.acinq.eclair.channel.Helpers.{Closing, Syncing, getRelayFees, scidForChannelUpdate}
import fr.acinq.eclair.channel.Helpers._
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.fund.InteractiveTxBuilder._
import fr.acinq.eclair.channel.fund.{InteractiveTxBuilder, InteractiveTxFunder, InteractiveTxSigningSession}
import fr.acinq.eclair.channel.publish.TxPublisher
import fr.acinq.eclair.channel.publish.TxPublisher.{PublishFinalTx, SetChannelId}
import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
@ -196,6 +199,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
// choose to not make this an Option (that would be None before the first connection), and instead embrace the fact
// that the active connection may point to dead letters at all time
var activeConnection = context.system.deadLetters
// we aggregate sigs for splices before processing
var sigStash = Seq.empty[CommitSig]
val txPublisher = txPublisherFactory.spawnTxPublisher(context, remoteNodeId)
@ -250,7 +255,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
// (there can be multiple funding txs due to rbf, and they can be unconfirmed in any state due to zero-conf)
data match {
case _: ChannelDataWithoutCommitments => ()
case data: ChannelDataWithCommitments => data.commitments.active.foreach { commitment =>
case data: ChannelDataWithCommitments => data.commitments.all.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
@ -353,6 +358,20 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
*/
when(NORMAL)(handleExceptions {
case Event(c: ForbiddenCommandDuringSplice, d: DATA_NORMAL) if d.spliceStatus != SpliceStatus.NoSplice =>
val error = ForbiddenDuringSplice(d.channelId, c.getClass.getSimpleName)
c match {
case c: CMD_ADD_HTLC => handleAddHtlcCommandError(c, error, Some(d.channelUpdate))
// NB: the command cannot be an htlc settlement (fail/fulfill), because if we are splicing it means the channel is idle and has no htlcs
case _ => handleCommandError(error, c)
}
case Event(msg: ForbiddenMessageDuringSplice, d: DATA_NORMAL) if d.spliceStatus != SpliceStatus.NoSplice && !d.spliceStatus.isInstanceOf[SpliceStatus.SpliceRequested] =>
// In case of a race between our splice_init and a forbidden message from our peer, we accept their message, because
// we know they are going to reject our splice attempt
val error = ForbiddenDuringSplice(d.channelId, msg.getClass.getSimpleName)
handleLocalError(error, d, Some(msg))
case Event(c: CMD_ADD_HTLC, d: DATA_NORMAL) if d.localShutdown.isDefined || d.remoteShutdown.isDefined =>
// note: spec would allow us to keep sending new htlcs after having received their shutdown (and not sent ours)
// but we want to converge as fast as possible and they would probably not route them anyway
@ -485,20 +504,58 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
}
case Event(commit: CommitSig, d: DATA_NORMAL) =>
d.commitments.receiveCommit(Seq(commit), keyManager) match {
case Right((commitments1, revocation)) =>
log.debug("received a new sig, spec:\n{}", commitments1.latest.specs2String)
if (commitments1.changes.localHasChanges) {
// if we have newly acknowledged changes let's sign them
self ! CMD_SIGN()
aggregateSigs(commit) match {
case Some(sigs) =>
d.spliceStatus match {
case s: SpliceStatus.SpliceInProgress =>
log.debug("received their commit_sig, deferring message")
stay() using d.copy(spliceStatus = s.copy(remoteCommitSig = Some(commit)))
case SpliceStatus.SpliceWaitingForSigs(signingSession) =>
signingSession.receiveCommitSig(nodeParams, d.commitments.params, commit) match {
case Left(f) =>
rollbackFundingAttempt(signingSession.fundingTx.tx, Nil)
stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) 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(spliceStatus = SpliceStatus.SpliceWaitingForSigs(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(signingSession.fundingTx.txId, signingSession.fundingParams.minDepth_opt)
val commitments1 = d.commitments.add(signingSession1.commitment)
val d1 = d.copy(commitments = commitments1, spliceStatus = SpliceStatus.NoSplice)
stay() using d1 storing() sending signingSession1.localSigs
}
}
case _ if d.commitments.latest.localFundingStatus.signedTx_opt.isEmpty && commit.batchSize == 1 =>
// The latest funding transaction is unconfirmed and we're missing our peer's tx_signatures: any commit_sig
// that we receive before that should be ignored, it's either a retransmission of a commit_sig we've already
// received or a bug that will eventually lead to a force-close anyway.
log.info("ignoring commit_sig, we're still waiting for tx_signatures")
stay()
case _ =>
// NB: in all other cases we process the commit_sig normally. We could do a full pattern matching on all
// splice statuses, but it would force us to handle corner cases like race condition between splice_init
// and a non-splice commit_sig
d.commitments.receiveCommit(sigs, keyManager) match {
case Right((commitments1, revocation)) =>
log.debug("received a new sig, spec:\n{}", commitments1.latest.specs2String)
if (commitments1.changes.localHasChanges) {
// if we have newly acknowledged changes let's sign them
self ! CMD_SIGN()
}
if (d.commitments.availableBalanceForSend != commitments1.availableBalanceForSend) {
// we send this event only when our balance changes
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.shortIds, commitments1))
}
context.system.eventStream.publish(ChannelSignatureReceived(self, commitments1))
stay() using d.copy(commitments = commitments1) storing() sending revocation
case Left(cause) => handleLocalError(cause, d, Some(commit))
}
}
if (d.commitments.availableBalanceForSend != commitments1.availableBalanceForSend) {
// we send this event only when our balance changes
context.system.eventStream.publish(AvailableBalanceChanged(self, d.channelId, d.shortIds, commitments1))
}
context.system.eventStream.publish(ChannelSignatureReceived(self, commitments1))
stay() using d.copy(commitments = commitments1) storing() sending revocation
case Left(cause) => handleLocalError(cause, d, Some(commit))
case None => stay()
}
case Event(revocation: RevokeAndAck, d: DATA_NORMAL) =>
@ -715,9 +772,270 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
goto(NORMAL) using d.copy(channelUpdate = channelUpdate1) storing()
}
case Event(cmd: CMD_SPLICE, d: DATA_NORMAL) =>
d.spliceStatus match {
case SpliceStatus.NoSplice =>
if (d.commitments.isIdle && d.commitments.params.remoteParams.initFeatures.hasFeature(SplicePrototype)) {
val targetFeerate = nodeParams.onChainFeeConf.feeEstimator.getFeeratePerKw(target = nodeParams.onChainFeeConf.feeTargets.fundingBlockTarget)
val fundingContribution = InteractiveTxFunder.computeSpliceContribution(
isInitiator = true,
sharedInput = Multisig2of2Input(keyManager, d.commitments.params, d.commitments.active.head),
spliceInAmount = cmd.additionalLocalFunding,
spliceOut = cmd.spliceOutputs,
targetFeerate = targetFeerate)
if (d.commitments.latest.localCommit.spec.toLocal + fundingContribution < d.commitments.latest.localChannelReserve) {
log.warning("cannot do splice: insufficient funds")
cmd.replyTo ! RES_FAILURE(cmd, InvalidSpliceRequest(d.channelId))
stay()
} else if (cmd.spliceOut_opt.map(_.scriptPubKey).exists(!MutualClose.isValidFinalScriptPubkey(_, allowAnySegwit = true))) {
log.warning("cannot do splice: invalid splice-out script")
cmd.replyTo ! RES_FAILURE(cmd, InvalidSpliceRequest(d.channelId))
stay()
} else {
log.info(s"initiating splice with local.in.amount=${cmd.additionalLocalFunding} local.in.push=${cmd.pushAmount} local.out.amount=${cmd.spliceOut_opt.map(_.amount).sum}")
val spliceInit = SpliceInit(d.channelId,
fundingContribution = fundingContribution,
lockTime = nodeParams.currentBlockHeight.toLong,
feerate = targetFeerate,
pushAmount = cmd.pushAmount,
requireConfirmedInputs = nodeParams.channelConf.requireConfirmedInputsForDualFunding
)
stay() using d.copy(spliceStatus = SpliceStatus.SpliceRequested(cmd, spliceInit)) sending spliceInit
}
} else {
log.warning("cannot initiate splice, channel is not idle or peer doesn't support splices")
cmd.replyTo ! RES_FAILURE(cmd, CommandUnavailableInThisState(d.channelId, "splice", NORMAL))
stay()
}
case _ =>
log.warning("cannot initiate splice, another one is already in progress")
cmd.replyTo ! RES_FAILURE(cmd, InvalidSpliceAlreadyInProgress(d.channelId))
stay()
}
// NB: we only accept splices on regtest and testnet
case Event(msg: SpliceInit, d: DATA_NORMAL) if nodeParams.chainHash == Block.RegtestGenesisBlock.hash || nodeParams.chainHash == Block.TestnetGenesisBlock.hash =>
d.spliceStatus match {
case SpliceStatus.NoSplice =>
if (d.commitments.isIdle && d.commitments.params.localParams.initFeatures.hasFeature(SplicePrototype)) {
log.info(s"accepting splice with remote.in.amount=${msg.fundingContribution} remote.in.push=${msg.pushAmount}")
val spliceAck = SpliceAck(d.channelId,
fundingContribution = 0.sat, // only remote contributes to the splice
pushAmount = 0.msat,
requireConfirmedInputs = nodeParams.channelConf.requireConfirmedInputsForDualFunding
)
val parentCommitment = d.commitments.latest.commitment
val nextFundingAmount = parentCommitment.capacity + spliceAck.fundingContribution + msg.fundingContribution
val fundingParams = InteractiveTxParams(
channelId = d.channelId,
isInitiator = false,
localContribution = spliceAck.fundingContribution,
remoteContribution = msg.fundingContribution,
sharedInput_opt = Some(Multisig2of2Input(keyManager, d.commitments.params, parentCommitment)),
fundingPubkeyScript = parentCommitment.commitInput.txOut.publicKeyScript, // same pubkey script as before
localOutputs = Nil,
lockTime = nodeParams.currentBlockHeight.toLong,
dustLimit = d.commitments.params.localParams.dustLimit.max(d.commitments.params.remoteParams.dustLimit),
targetFeerate = msg.feerate,
minDepth_opt = d.commitments.params.minDepthDualFunding(nodeParams.channelConf.minDepthBlocks, nextFundingAmount),
requireConfirmedInputs = RequireConfirmedInputs(forLocal = msg.requireConfirmedInputs, forRemote = spliceAck.requireConfirmedInputs)
)
val txBuilder = context.spawnAnonymous(InteractiveTxBuilder(
nodeParams, fundingParams,
channelParams = d.commitments.params,
purpose = InteractiveTxBuilder.SpliceTx(parentCommitment),
localPushAmount = spliceAck.pushAmount, remotePushAmount = msg.pushAmount,
wallet
))
txBuilder ! InteractiveTxBuilder.Start(self)
stay() using d.copy(spliceStatus = SpliceStatus.SpliceInProgress(cmd_opt = None, splice = txBuilder, remoteCommitSig = None)) sending spliceAck
} else {
log.info("rejecting splice request, channel not idle or not compatible")
stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceRequest(d.channelId).getMessage)
}
case SpliceStatus.SpliceAborted =>
log.info("rejecting splice attempt: our previous tx_abort was not acked")
stay() sending Warning(d.channelId, InvalidSpliceTxAbortNotAcked(d.channelId).getMessage)
case _: SpliceStatus.SpliceRequested | _: SpliceStatus.SpliceInProgress | _: SpliceStatus.SpliceWaitingForSigs =>
log.info("rejecting splice attempt: the current splice attempt must be completed or aborted first")
stay() sending Warning(d.channelId, InvalidSpliceAlreadyInProgress(d.channelId).getMessage)
}
case Event(msg: SpliceAck, d: DATA_NORMAL) =>
d.spliceStatus match {
case SpliceStatus.SpliceRequested(cmd, spliceInit) =>
log.info("our peer accepted our splice request and will contribute {} to the funding transaction", msg.fundingContribution)
val parentCommitment = d.commitments.latest.commitment
val nextFundingAmount = parentCommitment.capacity + spliceInit.fundingContribution + msg.fundingContribution
val fundingParams = InteractiveTxParams(
channelId = d.channelId,
isInitiator = true,
localContribution = spliceInit.fundingContribution,
remoteContribution = msg.fundingContribution,
sharedInput_opt = Some(Multisig2of2Input(keyManager, d.commitments.params, parentCommitment)),
fundingPubkeyScript = parentCommitment.commitInput.txOut.publicKeyScript, // same pubkey script as before
localOutputs = cmd.spliceOutputs,
lockTime = spliceInit.lockTime,
dustLimit = d.commitments.params.localParams.dustLimit.max(d.commitments.params.remoteParams.dustLimit),
targetFeerate = spliceInit.feerate,
minDepth_opt = d.commitments.params.minDepthDualFunding(nodeParams.channelConf.minDepthBlocks, nextFundingAmount),
requireConfirmedInputs = RequireConfirmedInputs(forLocal = msg.requireConfirmedInputs, forRemote = spliceInit.requireConfirmedInputs)
)
val txBuilder = context.spawnAnonymous(InteractiveTxBuilder(
nodeParams, fundingParams,
channelParams = d.commitments.params,
purpose = InteractiveTxBuilder.SpliceTx(parentCommitment),
localPushAmount = cmd.pushAmount, remotePushAmount = msg.pushAmount,
wallet
))
txBuilder ! InteractiveTxBuilder.Start(self)
stay() using d.copy(spliceStatus = SpliceStatus.SpliceInProgress(cmd_opt = Some(cmd), splice = txBuilder, remoteCommitSig = None))
case _ =>
log.info(s"ignoring unexpected splice_ack=$msg")
stay()
}
case Event(msg: InteractiveTxConstructionMessage, d: DATA_NORMAL) =>
d.spliceStatus match {
case SpliceStatus.SpliceInProgress(_, txBuilder, _) =>
txBuilder ! InteractiveTxBuilder.ReceiveMessage(msg)
stay()
case _ =>
log.info("ignoring unexpected interactive-tx message: {}", msg.getClass.getSimpleName)
stay() sending Warning(d.channelId, UnexpectedInteractiveTxMessage(d.channelId, msg).getMessage)
}
case Event(msg: TxAbort, d: DATA_NORMAL) =>
d.spliceStatus match {
case SpliceStatus.SpliceInProgress(cmd_opt, txBuilder, _) =>
log.info("our peer aborted the splice attempt: ascii='{}' bin={}", msg.toAscii, msg.data)
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, SpliceAttemptAborted(d.channelId)))
txBuilder ! InteractiveTxBuilder.Abort
stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage)
case SpliceStatus.SpliceWaitingForSigs(signingSession) =>
log.info("our peer aborted the splice attempt: ascii='{}' bin={}", msg.toAscii, msg.data)
rollbackFundingAttempt(signingSession.fundingTx.tx, previousTxs = Seq.empty) // no splice rbf yet
stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage)
case SpliceStatus.SpliceRequested(cmd, _) =>
log.info("our peer rejected our splice attempt: ascii='{}' bin={}", msg.toAscii, msg.data)
cmd.replyTo ! RES_FAILURE(cmd, new RuntimeException(s"splice attempt rejected by our peer: ${msg.toAscii}"))
stay() using d.copy(spliceStatus = SpliceStatus.NoSplice) sending TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage)
case SpliceStatus.SpliceAborted =>
log.debug("our peer acked our previous tx_abort")
stay() using d.copy(spliceStatus = SpliceStatus.NoSplice)
case SpliceStatus.NoSplice =>
log.info("our peer wants to abort the splice, but we've already negotiated a splice transaction: ascii='{}' bin={}", msg.toAscii, msg.data)
// We ack their tx_abort but we keep monitoring the funding transaction until it's confirmed or double-spent.
stay() sending TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage)
}
case Event(msg: InteractiveTxBuilder.Response, d: DATA_NORMAL) =>
d.spliceStatus match {
case SpliceStatus.SpliceInProgress(cmd_opt, _, remoteCommitSig_opt) =>
msg match {
case InteractiveTxBuilder.SendMessage(msg) => stay() sending msg
case InteractiveTxBuilder.Succeeded(signingSession, commitSig) =>
log.info(s"splice tx created with fundingTxIndex=${signingSession.fundingTxIndex} fundingTxId=${signingSession.fundingTx.txId}")
cmd_opt.foreach(cmd => cmd.replyTo ! RES_SPLICE(fundingTxIndex = signingSession.fundingTxIndex, signingSession.fundingTx.txId, signingSession.fundingParams.fundingAmount, signingSession.localCommit.fold(_.spec, _.spec).toLocal))
remoteCommitSig_opt.foreach(self ! _)
val d1 = d.copy(spliceStatus = SpliceStatus.SpliceWaitingForSigs(signingSession))
stay() using d1 storing() sending commitSig
case f: InteractiveTxBuilder.Failed =>
log.info("splice attempt failed: {}", f.cause.getMessage)
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, f.cause))
stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, f.cause.getMessage)
}
case _ =>
// This can happen if we received a tx_abort right before receiving the interactive-tx result.
log.warning("ignoring interactive-tx result with spliceStatus={}", d.spliceStatus.getClass.getSimpleName)
stay()
}
case Event(msg: TxSignatures, d: DATA_NORMAL) =>
d.commitments.latest.localFundingStatus match {
case dfu@LocalFundingStatus.DualFundedUnconfirmedFundingTx(fundingTx: PartiallySignedSharedTransaction, _, _) if fundingTx.txId == msg.txId =>
// we already sent our tx_signatures
InteractiveTxSigningSession.addRemoteSigs(dfu.fundingParams, fundingTx, msg) match {
case Left(cause) =>
log.warning("received invalid tx_signatures for fundingTxId={}: {}", msg.txId, cause.getMessage)
// The funding transaction may still confirm (since our peer should be able to generate valid signatures),
// so we cannot close the channel yet.
stay() sending Error(d.channelId, InvalidFundingSignature(d.channelId, Some(fundingTx.txId)).getMessage)
case Right(fundingTx) =>
val dfu1 = dfu.copy(sharedTx = fundingTx)
d.commitments.updateLocalFundingStatus(msg.txId, dfu1) match {
case Right((commitments1, _)) =>
log.info("publishing funding tx for channelId={} fundingTxId={}", d.channelId, fundingTx.signedTx.txid)
stay() using d.copy(commitments = commitments1) storing() calling publishFundingTx(dfu1)
case Left(_) =>
stay()
}
}
case _ =>
d.spliceStatus match {
case SpliceStatus.SpliceWaitingForSigs(signingSession) =>
// we have not yet sent our tx_signatures
signingSession.receiveTxSigs(nodeParams, msg) match {
case Left(f) =>
rollbackFundingAttempt(signingSession.fundingTx.tx, previousTxs = Seq.empty) // no splice rbf yet
stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) 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 = d.copy(commitments = commitments1, spliceStatus = SpliceStatus.NoSplice)
log.info("publishing funding tx for channelId={} fundingTxId={}", d.channelId, signingSession1.fundingTx.sharedTx.txId)
stay() using d1 storing() sending signingSession1.localSigs calling publishFundingTx(signingSession1.fundingTx)
}
case _ =>
// We may receive an outdated tx_signatures if the transaction is already confirmed.
log.warning("ignoring unexpected tx_signatures for txId={}", msg.txId)
stay()
}
}
case Event(w: WatchPublishedTriggered, d: DATA_NORMAL) =>
val fundingStatus = LocalFundingStatus.ZeroconfPublishedFundingTx(w.tx)
d.commitments.updateLocalFundingStatus(w.tx.txid, fundingStatus) match {
case Right((commitments1, _)) =>
watchFundingConfirmed(w.tx.txid, Some(nodeParams.channelConf.minDepthBlocks))
maybeEmitEventsPostSplice(d.shortIds, d.commitments, commitments1)
stay() using d.copy(commitments = commitments1) storing() sending SpliceLocked(d.channelId, w.tx.hash)
case Left(_) => stay()
}
case Event(w: WatchFundingConfirmedTriggered, d: DATA_NORMAL) =>
acceptFundingTxConfirmed(w, d) match {
case Right((commitments1, commitment)) =>
val toSend = if (d.commitments.all.exists(c => c.fundingTxId == commitment.fundingTxId && c.localFundingStatus.isInstanceOf[LocalFundingStatus.NotLocked])) {
// this commitment just moved from NotLocked to Locked
Some(SpliceLocked(d.channelId, w.tx.hash))
} else {
// this was a zero-conf splice and we already sent our splice_locked
None
}
maybeEmitEventsPostSplice(d.shortIds, d.commitments, commitments1)
stay() using d.copy(commitments = commitments1) storing() sending toSend.toSeq
case Left(_) => stay()
}
case Event(msg: SpliceLocked, d: DATA_NORMAL) =>
d.commitments.updateRemoteFundingStatus(msg.fundingTxid) match {
case Right((commitments1, _)) =>
maybeEmitEventsPostSplice(d.shortIds, d.commitments, commitments1)
stay() using d.copy(commitments = commitments1) storing()
case Left(_) => stay()
}
case Event(INPUT_DISCONNECTED, d: DATA_NORMAL) =>
// we cancel the timer that would have made us send the enabled update after reconnection (flappy channel protection)
cancelTimer(Reconnected.toString)
// if we are splicing, we need to cancel it
reportSpliceFailure(d.spliceStatus, new RuntimeException("splice attempt failed: disconnected"))
val d1 = d.spliceStatus match {
// We keep track of the RBF status: we should be able to complete the signature steps on reconnection.
case _: SpliceStatus.SpliceWaitingForSigs => d
case _ => d.copy(spliceStatus = SpliceStatus.NoSplice)
}
// if we have pending unsigned htlcs, then we cancel them and generate an update with the disabled flag set, that will be returned to the sender in a temporary channel failure
if (d.commitments.changes.localChanges.proposed.collectFirst { case add: UpdateAddHtlc => add }.isDefined) {
log.debug("updating channel_update announcement (reason=disabled)")
@ -726,9 +1044,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
d.commitments.changes.localChanges.proposed.collect {
case add: UpdateAddHtlc => relayer ! RES_ADD_SETTLED(d.commitments.originChannels(add.id), add, HtlcResult.DisconnectedBeforeSigned(channelUpdate1))
}
goto(OFFLINE) using d.copy(channelUpdate = channelUpdate1) storing()
goto(OFFLINE) using d1.copy(channelUpdate = channelUpdate1) storing()
} else {
goto(OFFLINE) using d
goto(OFFLINE) using d1
}
case Event(e: Error, d: DATA_NORMAL) => handleRemoteError(e, d)
@ -846,28 +1164,32 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
}
case Event(commit: CommitSig, d@DATA_SHUTDOWN(_, localShutdown, remoteShutdown, closingFeerates)) =>
d.commitments.receiveCommit(Seq(commit), keyManager) match {
case Right((commitments1, revocation)) =>
// we always reply with a revocation
log.debug("received a new sig:\n{}", commitments1.latest.specs2String)
context.system.eventStream.publish(ChannelSignatureReceived(self, commitments1))
if (commitments1.hasNoPendingHtlcsOrFeeUpdate) {
if (d.commitments.params.localParams.isInitiator) {
// we are the channel initiator, need to initiate the negotiation by sending the first closing_signed
val (closingTx, closingSigned) = Closing.MutualClose.makeFirstClosingTx(keyManager, commitments1.latest, localShutdown.scriptPubKey, remoteShutdown.scriptPubKey, nodeParams.onChainFeeConf.feeEstimator, nodeParams.onChainFeeConf.feeTargets, closingFeerates)
goto(NEGOTIATING) using DATA_NEGOTIATING(commitments1, localShutdown, remoteShutdown, List(List(ClosingTxProposed(closingTx, closingSigned))), bestUnpublishedClosingTx_opt = None) storing() sending revocation :: closingSigned :: Nil
} else {
// we are not the channel initiator, will wait for their closing_signed
goto(NEGOTIATING) using DATA_NEGOTIATING(commitments1, localShutdown, remoteShutdown, closingTxProposed = List(List()), bestUnpublishedClosingTx_opt = None) storing() sending revocation
}
} else {
if (commitments1.changes.localHasChanges) {
// if we have newly acknowledged changes let's sign them
self ! CMD_SIGN()
}
stay() using d.copy(commitments = commitments1) storing() sending revocation
aggregateSigs(commit) match {
case Some(sigs) =>
d.commitments.receiveCommit(sigs, keyManager) match {
case Right((commitments1, revocation)) =>
// we always reply with a revocation
log.debug("received a new sig:\n{}", commitments1.latest.specs2String)
context.system.eventStream.publish(ChannelSignatureReceived(self, commitments1))
if (commitments1.hasNoPendingHtlcsOrFeeUpdate) {
if (d.commitments.params.localParams.isInitiator) {
// we are the channel initiator, need to initiate the negotiation by sending the first closing_signed
val (closingTx, closingSigned) = Closing.MutualClose.makeFirstClosingTx(keyManager, commitments1.latest, localShutdown.scriptPubKey, remoteShutdown.scriptPubKey, nodeParams.onChainFeeConf.feeEstimator, nodeParams.onChainFeeConf.feeTargets, closingFeerates)
goto(NEGOTIATING) using DATA_NEGOTIATING(commitments1, localShutdown, remoteShutdown, List(List(ClosingTxProposed(closingTx, closingSigned))), bestUnpublishedClosingTx_opt = None) storing() sending revocation :: closingSigned :: Nil
} else {
// we are not the channel initiator, will wait for their closing_signed
goto(NEGOTIATING) using DATA_NEGOTIATING(commitments1, localShutdown, remoteShutdown, closingTxProposed = List(List()), bestUnpublishedClosingTx_opt = None) storing() sending revocation
}
} else {
if (commitments1.changes.localHasChanges) {
// if we have newly acknowledged changes let's sign them
self ! CMD_SIGN()
}
stay() using d.copy(commitments = commitments1) storing() sending revocation
}
case Left(cause) => handleLocalError(cause, d, Some(commit))
}
case Left(cause) => handleLocalError(cause, d, Some(commit))
case None => stay()
}
case Event(revocation: RevokeAndAck, d@DATA_SHUTDOWN(_, localShutdown, remoteShutdown, closingFeerates)) =>
@ -1059,7 +1381,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
remoteCommitPublished1.foreach(rcp => doPublish(rcp, commitment))
nextRemoteCommitPublished1.foreach(rcp => doPublish(rcp, commitment))
}
// TODO: when using splices we should be updating all competing commitments
handleCommandSuccess(c, d.copy(commitments = commitments1, localCommitPublished = localCommitPublished1, remoteCommitPublished = remoteCommitPublished1, nextRemoteCommitPublished = nextRemoteCommitPublished1)) storing() calling republish()
case Left(cause) => handleCommandError(cause, c)
}
@ -1076,11 +1398,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(w: WatchFundingConfirmedTriggered, d: DATA_CLOSING) =>
acceptFundingTxConfirmed(w, d) match {
case Right((commitments1, _)) =>
if (d.commitments.latest.fundingTxId == w.tx.txid) {
// The best funding tx candidate has been confirmed, alternative commitments have been pruned
stay() using d.copy(commitments = commitments1) storing()
} else {
case Right((commitments1, commitment)) =>
if (d.commitments.latest.fundingTxIndex == commitment.fundingTxIndex && d.commitments.latest.fundingTxId != commitment.fundingTxId) {
// This is a corner case where:
// - we are using dual funding
// - *and* the funding tx was RBF-ed
@ -1094,14 +1413,24 @@ 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 d1 = d.copy(commitments = commitments1)
val commitments2 = commitments1.copy(
active = commitment +: Nil,
inactive = Nil
)
val d1 = d.copy(commitments = commitments2)
spendLocalCurrent(d1)
} else {
// We're still on the same splice history, nothing to do
stay() using d.copy(commitments = commitments1) storing()
}
case Left(_) => stay()
}
case Event(WatchFundingSpentTriggered(tx), d: DATA_CLOSING) =>
if (d.mutualClosePublished.exists(_.tx.txid == tx.txid)) {
if (d.commitments.all.map(_.fundingTxId).contains(tx.txid)) {
// if the spending tx is itself a funding tx, this is a splice and there is nothing to do
stay()
} else if (d.mutualClosePublished.exists(_.tx.txid == tx.txid)) {
// we already know about this tx, probably because we have published it ourselves after successful negotiation
stay()
} else if (d.mutualCloseProposed.exists(_.tx.txid == tx.txid)) {
@ -1130,9 +1459,45 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
// counterparty may attempt to spend a revoked commit tx at any time
handleRemoteSpentOther(tx, d)
} else {
log.warning(s"unrecognized tx=${tx.txid}")
// this was for another commitments
stay()
d.commitments.resolveCommitment(tx) match {
case Some(commitment) =>
log.warning(s"a commit tx for an older commitment has been published fundingTxId=${tx.txid} fundingTxIndex=${commitment.fundingTxIndex}")
blockchain ! WatchAlternativeCommitTxConfirmed(self, tx.txid, nodeParams.channelConf.minDepthBlocks)
stay()
case None =>
// This must be a former funding tx that has already been pruned, because watches are unordered.
log.warning(s"ignoring unrecognized tx=${tx.txid}")
stay()
}
}
case Event(WatchAlternativeCommitTxConfirmedTriggered(_, _, tx), d: DATA_CLOSING) =>
d.commitments.resolveCommitment(tx) match {
case Some(commitment) =>
log.warning("a commit tx for fundingTxIndex={} fundingTxId={} has been confirmed", commitment.fundingTxIndex, commitment.fundingTxId)
val commitments1 = d.commitments.copy(
active = commitment +: Nil,
inactive = Nil
)
// we reset the state
val d1 = d.copy(commitments = commitments1)
// This commitment may be revoked: we need to verify that its index matches our latest known index before overwriting our previous commitments.
if (commitment.localCommit.commitTxAndRemoteSig.commitTx.tx.txid == tx.txid) {
// our local commit has been published from the outside, it's unexpected but let's deal with it anyway
spendLocalCurrent(d1)
} else if (commitment.remoteCommit.txid == tx.txid && commitment.remoteCommit.index == d.commitments.remoteCommitIndex) {
// counterparty may attempt to spend its last commit tx at any time
handleRemoteSpentCurrent(tx, d1)
} else if (commitment.nextRemoteCommit_opt.exists(_.commit.txid == tx.txid) && commitment.remoteCommit.index == d.commitments.remoteCommitIndex && d.commitments.remoteNextCommitInfo.isLeft) {
// counterparty may attempt to spend its last commit tx at any time
handleRemoteSpentNext(tx, d1)
} else {
// counterparty may attempt to spend a revoked commit tx at any time
handleRemoteSpentOther(tx, d1)
}
case None =>
log.warning(s"ignoring unrecognized alternative commit tx=${tx.txid}")
stay()
}
case Event(WatchOutputSpentTriggered(tx), d: DATA_CLOSING) =>
@ -1355,9 +1720,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
channelReestablish.nextFundingTxId_opt match {
case Some(fundingTxId) =>
d.rbfStatus match {
case RbfStatus.RbfWaitingForSigs(status) if status.fundingTx.txId == fundingTxId =>
case RbfStatus.RbfWaitingForSigs(signingSession) if signingSession.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)
val commitSig = signingSession.remoteCommit.sign(keyManager, d.commitments.params, signingSession.commitInput)
goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) sending commitSig
case _ if d.latestFundingTx.sharedTx.txId == fundingTxId =>
val toSend = d.latestFundingTx.sharedTx match {
@ -1396,13 +1761,66 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
var sendQueue = Queue.empty[LightningMessage]
// normal case, our data is up-to-date
if (channelReestablish.nextLocalCommitmentNumber == 1 && d.commitments.localCommitIndex == 0) {
// re-send channel_ready or splice_locked
if (d.commitments.latest.fundingTxIndex == 0 && channelReestablish.nextLocalCommitmentNumber == 1 && d.commitments.localCommitIndex == 0) {
// If next_local_commitment_number is 1 in both the channel_reestablish it sent and received, then the node MUST retransmit channel_ready, otherwise it MUST NOT
log.debug("re-sending channelReady")
val channelKeyPath = keyManager.keyPath(d.commitments.params.localParams, d.commitments.params.channelConfig)
val nextPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1)
val channelReady = ChannelReady(d.commitments.channelId, nextPerCommitmentPoint)
sendQueue = sendQueue :+ channelReady
} else {
// NB: there is a key difference between channel_ready and splice_confirmed:
// - channel_ready: a non-zero commitment index implies that both sides have seen the channel_ready
// - splice_confirmed: the commitment index can be updated as long as it is compatible with all splices, so
// we must keep sending our most recent splice_locked at each reconnection
val spliceLocked = d.commitments.active
.filter(c => c.fundingTxIndex > 0) // only consider splice txs
.collectFirst { case c if c.localFundingStatus.isInstanceOf[LocalFundingStatus.Locked] =>
log.debug(s"re-sending splice_locked for fundingTxId=${c.fundingTxId}")
SpliceLocked(d.channelId, c.fundingTxId.reverse)
}
sendQueue = sendQueue ++ spliceLocked
}
// resume splice signing session if any
val spliceStatus1 = channelReestablish.nextFundingTxId_opt match {
case Some(fundingTxId) =>
d.spliceStatus match {
case SpliceStatus.SpliceWaitingForSigs(signingSession) if signingSession.fundingTx.txId == fundingTxId =>
// We retransmit our commit_sig, and will send our tx_signatures once we've received their commit_sig.
log.info(s"re-sending commit_sig for splice attempt with fundingTxIndex=${signingSession.fundingTxIndex} fundingTxId=${signingSession.fundingTx.txId}")
val commitSig = signingSession.remoteCommit.sign(keyManager, d.commitments.params, signingSession.commitInput)
sendQueue = sendQueue :+ commitSig
d.spliceStatus
case _ if d.commitments.latest.fundingTxId == fundingTxId =>
d.commitments.latest.localFundingStatus match {
case dfu: LocalFundingStatus.DualFundedUnconfirmedFundingTx =>
dfu.sharedTx match {
case fundingTx: InteractiveTxBuilder.PartiallySignedSharedTransaction =>
// If we have not received their tx_signatures, we can't tell whether they had received our commit_sig, so we need to retransmit it
log.info(s"re-sending commit_sig and tx_signatures for fundingTxIndex=${d.commitments.latest.fundingTxIndex} fundingTxId=${d.commitments.latest.fundingTxId}")
val commitSig = d.commitments.latest.remoteCommit.sign(keyManager, d.commitments.params, d.commitments.latest.commitInput)
sendQueue = sendQueue :+ commitSig :+ fundingTx.localSigs
case fundingTx: InteractiveTxBuilder.FullySignedSharedTransaction =>
log.info(s"re-sending tx_signatures for fundingTxIndex=${d.commitments.latest.fundingTxIndex} fundingTxId=${d.commitments.latest.fundingTxId}")
sendQueue = sendQueue :+ fundingTx.localSigs
}
case _ =>
// The funding tx is published or confirmed, and they have not received our tx_signatures, but they must have received our commit_sig, otherwise they
// would not have sent their tx_signatures and we would not have been able to publish the funding tx in the first place. We could in theory
// recompute our tx_signatures, but instead we do nothing: they will be notified that the funding tx has confirmed.
log.warning("cannot re-send tx_signatures for fundingTxId={}, transaction is already published or confirmed", fundingTxId)
}
d.spliceStatus
case _ =>
// The fundingTxId must be for a splice attempt that we didn't store (we got disconnected before receiving
// their tx_complete): we tell them to abort that splice attempt.
log.info(s"aborting obsolete splice attempt for fundingTxId=$fundingTxId")
sendQueue = sendQueue :+ TxAbort(d.channelId, SpliceAttemptAborted(d.channelId).getMessage)
SpliceStatus.SpliceAborted
}
case None => d.spliceStatus
}
// we may need to retransmit updates and/or commit_sig and/or revocation
@ -1459,7 +1877,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
// we send it (if needed) when reconnected.
val shutdownInProgress = d.localShutdown.nonEmpty || d.remoteShutdown.nonEmpty
if (d.commitments.params.localParams.isInitiator && !shutdownInProgress) {
// TODO: what should we do here if we have multiple commitments using different feerates?
// TODO: all active commitments use the same feerate, but may have a different channel capacity: how should we compute networkFeeratePerKw?
val currentFeeratePerKw = d.commitments.latest.localCommit.spec.commitTxFeerate
val networkFeeratePerKw = nodeParams.onChainFeeConf.getCommitmentFeerate(remoteNodeId, d.commitments.params.channelType, d.commitments.latest.capacity, None)
if (nodeParams.onChainFeeConf.shouldUpdateFee(currentFeeratePerKw, networkFeeratePerKw)) {
@ -1467,7 +1885,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
}
}
goto(NORMAL) using d.copy(commitments = commitments1) sending sendQueue
goto(NORMAL) using d.copy(commitments = commitments1, spliceStatus = spliceStatus1) sending sendQueue
}
case Event(c: CMD_ADD_HTLC, d: DATA_NORMAL) => handleAddDisconnected(c, d)
@ -1536,15 +1954,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(e: Error, d: PersistentChannelData) => handleRemoteError(e, d)
})
when(WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT)(handleExceptions {
case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT) => handleRemoteSpentFuture(tx, d)
})
when(WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT)(PartialFunction.empty[Event, State])
private def errorStateHandler: StateFunction = {
case Event(Symbol("nevermatches"), _) => stay() // we can't define a state with no event handler, so we put a dummy one here
}
when(ERR_INFORMATION_LEAK)(errorStateHandler)
when(ERR_INFORMATION_LEAK)(PartialFunction.empty[Event, State])
whenUnhandled {
@ -1594,6 +2006,10 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
c.replyTo ! RES_FAILURE(c, CommandUnavailableInThisState(d.channelId, "rbf", stateName))
stay()
case Event(c: CMD_SPLICE, d) =>
c.replyTo ! RES_FAILURE(c, CommandUnavailableInThisState(d.channelId, "splice", stateName))
stay()
// at restore, if the configuration has changed, the channel will send a command to itself to update the relay fees
case Event(RES_SUCCESS(_: CMD_UPDATE_RELAY_FEE, channelId), d: DATA_NORMAL) if channelId == d.channelId => stay()
@ -1697,18 +2113,31 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
// 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(d.bestUnpublishedClosingTx_opt.get, Left(d))
case Event(WatchFundingSpentTriggered(tx), d: DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT) => handleRemoteSpentFuture(tx, d)
case Event(WatchFundingSpentTriggered(tx), d: ChannelDataWithCommitments) =>
if (tx.txid == d.commitments.latest.remoteCommit.txid) {
if (d.commitments.all.map(_.fundingTxId).contains(tx.txid)) {
// if the spending tx is itself a funding tx, this is a splice and there is nothing to do
stay()
} else if (tx.txid == d.commitments.latest.remoteCommit.txid) {
handleRemoteSpentCurrent(tx, d)
} else if (d.commitments.latest.nextRemoteCommit_opt.exists(_.commit.txid == tx.txid)) {
handleRemoteSpentNext(tx, d)
} else if (tx.txid == d.commitments.latest.localCommit.commitTxAndRemoteSig.commitTx.tx.txid) {
log.warning(s"processing local commit spent from the outside")
spendLocalCurrent(d)
} else {
} else if (tx.txIn.map(_.outPoint.txid).contains(d.commitments.latest.fundingTxId)) {
handleRemoteSpentOther(tx, d)
} else {
d.commitments.resolveCommitment(tx) match {
case Some(commitment) =>
log.warning(s"a commit tx for an older commitment has been published fundingTxId=${tx.txid} fundingTxIndex=${commitment.fundingTxIndex}")
// we watch the commitment tx, in the meantime we force close using the latest commitment
blockchain ! WatchAlternativeCommitTxConfirmed(self, tx.txid, nodeParams.channelConf.minDepthBlocks)
spendLocalCurrent(d)
case None =>
// This must be a former funding tx that has already been pruned, because watches are unordered.
log.warning(s"ignoring unrecognized tx=${tx.txid}")
stay()
}
}
}
@ -1833,6 +2262,12 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
}
}
/** On disconnection we clear up the sig stash */
onTransition {
case _ -> OFFLINE =>
sigStash = Nil
}
/*
888 888 d8888 888b 888 8888888b. 888 8888888888 8888888b. .d8888b.
888 888 d88888 8888b 888 888 "Y88b 888 888 888 Y88b d88P Y88b
@ -1844,8 +2279,21 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
888 888 d88P 888 888 Y888 8888888P" 88888888 8888888888 888 T88b "Y8888P"
*/
/** For splices we will send one commit_sig per active commitments. */
private def aggregateSigs(commit: CommitSig): Option[Seq[CommitSig]] = {
sigStash = sigStash :+ commit
log.debug("received sig for batch of size={}", commit.batchSize)
if (sigStash.size == commit.batchSize) {
val sigs = sigStash
sigStash = Nil
Some(sigs)
} else {
None
}
}
private def handleCurrentFeerate(c: CurrentFeerates, d: ChannelDataWithCommitments) = {
// TODO: we should consider *all* commitments
// TODO: all active commitments use the same feerate, but may have a different channel capacity: how should we compute networkFeeratePerKw?
val commitments = d.commitments.latest
val networkFeeratePerKw = nodeParams.onChainFeeConf.getCommitmentFeerate(remoteNodeId, d.commitments.params.channelType, commitments.capacity, Some(c))
val currentFeeratePerKw = commitments.localCommit.spec.commitTxFeerate
@ -1871,7 +2319,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
* @return
*/
private def handleCurrentFeerateDisconnected(c: CurrentFeerates, d: ChannelDataWithCommitments) = {
// TODO: we should consider *all* commitments
// TODO: all active commitments use the same feerate, but may have a different channel capacity: how should we compute networkFeeratePerKw?
val commitments = d.commitments.latest
val networkFeeratePerKw = nodeParams.onChainFeeConf.getCommitmentFeerate(remoteNodeId, d.commitments.params.channelType, commitments.capacity, Some(c))
val currentFeeratePerKw = commitments.localCommit.spec.commitTxFeerate
@ -1978,9 +2426,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
// we need to remember their commitment point in order to be able to claim our outputs
handleOutdatedCommitment(channelReestablish, d)
case res: Syncing.SyncResult.RemoteLying =>
log.error(s"counterparty is lying about us having an outdated commitment!!! ourLocalCommitmentNumber=${res.ourLocalCommitmentNumber} theirRemoteCommitmentNumber=${res.theirRemoteCommitmentNumber}")
// they are deliberately trying to fool us into thinking we have a late commitment
handleLocalError(InvalidRevokedCommitProof(d.channelId, res.ourLocalCommitmentNumber, res.theirRemoteCommitmentNumber, res.invalidPerCommitmentSecret), d, Some(channelReestablish))
log.error(s"counterparty claims that we have an outdated commitment, but they sent an invalid proof, so our commitment may or may not be revoked: ourLocalCommitmentNumber=${res.ourLocalCommitmentNumber} theirRemoteCommitmentNumber=${res.theirRemoteCommitmentNumber}")
// they are deliberately trying to fool us into thinking we have a late commitment, but we cannot risk publishing it ourselves, because it may really be revoked!
handleOutdatedCommitment(channelReestablish, d)
case SyncResult.RemoteLate =>
log.error("counterparty appears to be using an outdated commitment, they may request a force-close, standing by...")
stay()
@ -1993,6 +2441,19 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
}
}
/** Splices change balances and capacity, we send events to notify other actors (router, relayer) */
private def maybeEmitEventsPostSplice(shortIds: ShortIds, oldCommitments: Commitments, newCommitments: Commitments): Unit = {
// NB: we consider the send and receive balance, because router tracks both
if (oldCommitments.availableBalanceForSend != newCommitments.availableBalanceForSend || oldCommitments.availableBalanceForReceive != newCommitments.availableBalanceForReceive) {
context.system.eventStream.publish(AvailableBalanceChanged(self, newCommitments.channelId, shortIds, newCommitments))
}
if (!Helpers.aboveReserve(oldCommitments) && Helpers.aboveReserve(newCommitments)) {
// we just went above reserve (can't go below), let's refresh our channel_update to enable/disable it accordingly
log.debug("updating channel_update aboveReserve={}", Helpers.aboveReserve(newCommitments))
self ! BroadcastChannelUpdate(AboveReserve)
}
}
private def handleNewBlock(c: CurrentBlockHeight, d: ChannelDataWithCommitments) = {
d match {
case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => handleNewBlockDualFundingUnconfirmed(c, d)

View file

@ -20,7 +20,6 @@ import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapte
import com.softwaremill.quicklens.{ModifyPimp, QuicklensAt}
import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Script}
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._
import fr.acinq.eclair.channel.Helpers.Funding
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel._
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.{FullySignedSharedTransaction, InteractiveTxParams, PartiallySignedSharedTransaction, RequireConfirmedInputs}
@ -146,11 +145,31 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case Left(t) => handleLocalError(t, d, Some(open))
case Right((channelFeatures, remoteShutdownScript)) =>
context.system.eventStream.publish(ChannelCreated(self, peer, remoteNodeId, isInitiator = false, open.temporaryChannelId, open.commitmentFeerate, Some(open.fundingFeerate)))
val remoteParams = RemoteParams(
nodeId = remoteNodeId,
dustLimit = open.dustLimit,
maxHtlcValueInFlightMsat = open.maxHtlcValueInFlightMsat,
requestedChannelReserve_opt = None, // channel reserve will be computed based on channel capacity
htlcMinimum = open.htlcMinimum,
toSelfDelay = open.toSelfDelay,
maxAcceptedHtlcs = open.maxAcceptedHtlcs,
fundingPubKey = open.fundingPubkey,
revocationBasepoint = open.revocationBasepoint,
paymentBasepoint = open.paymentBasepoint,
delayedPaymentBasepoint = open.delayedPaymentBasepoint,
htlcBasepoint = open.htlcBasepoint,
initFeatures = remoteInit.features,
upfrontShutdownScript_opt = remoteShutdownScript)
log.debug("remote params: {}", remoteParams)
val localFundingPubkey = keyManager.fundingPublicKey(localParams.fundingKeyPath).publicKey
val channelKeyPath = keyManager.keyPath(localParams, d.init.channelConfig)
val revocationBasePoint = keyManager.revocationPoint(channelKeyPath).publicKey
// We've exchanged open_channel2 and accept_channel2, we now know the final channelId.
val channelId = Helpers.computeChannelId(open.revocationBasepoint, revocationBasePoint)
val channelParams = ChannelParams(channelId, d.init.channelConfig, channelFeatures, localParams, remoteParams, open.channelFlags)
val localAmount = d.init.fundingContribution_opt.getOrElse(0 sat)
val remoteAmount = open.fundingAmount
val minDepth_opt = Funding.minDepthDualFunding(nodeParams.channelConf, d.init.localParams.initFeatures, isInitiator = localParams.isInitiator, localAmount = localAmount, remoteAmount = remoteAmount)
val minDepth_opt = channelParams.minDepthDualFunding(nodeParams.channelConf.minDepthBlocks, localAmount + remoteAmount)
val upfrontShutdownScript_opt = localParams.upfrontShutdownScript_opt.map(scriptPubKey => ChannelTlv.UpfrontShutdownScriptTlv(scriptPubKey))
val tlvs: Set[AcceptDualFundedChannelTlv] = Set(
upfrontShutdownScript_opt,
@ -168,31 +187,13 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
toSelfDelay = localParams.toSelfDelay,
maxAcceptedHtlcs = localParams.maxAcceptedHtlcs,
fundingPubkey = localFundingPubkey,
revocationBasepoint = keyManager.revocationPoint(channelKeyPath).publicKey,
revocationBasepoint = revocationBasePoint,
paymentBasepoint = localParams.walletStaticPaymentBasepoint.getOrElse(keyManager.paymentPoint(channelKeyPath).publicKey),
delayedPaymentBasepoint = keyManager.delayedPaymentPoint(channelKeyPath).publicKey,
htlcBasepoint = keyManager.htlcPoint(channelKeyPath).publicKey,
firstPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 0),
secondPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 1),
tlvStream = TlvStream(tlvs))
val remoteParams = RemoteParams(
nodeId = remoteNodeId,
dustLimit = open.dustLimit,
maxHtlcValueInFlightMsat = open.maxHtlcValueInFlightMsat,
requestedChannelReserve_opt = None, // channel reserve will be computed based on channel capacity
htlcMinimum = open.htlcMinimum,
toSelfDelay = open.toSelfDelay,
maxAcceptedHtlcs = open.maxAcceptedHtlcs,
fundingPubKey = open.fundingPubkey,
revocationBasepoint = open.revocationBasepoint,
paymentBasepoint = open.paymentBasepoint,
delayedPaymentBasepoint = open.delayedPaymentBasepoint,
htlcBasepoint = open.htlcBasepoint,
initFeatures = remoteInit.features,
upfrontShutdownScript_opt = remoteShutdownScript)
log.debug("remote params: {}", remoteParams)
// We've exchanged open_channel2 and accept_channel2, we now know the final channelId.
val channelId = Helpers.computeChannelId(open, accept)
peer ! ChannelIdAssigned(self, remoteNodeId, accept.temporaryChannelId, channelId) // we notify the peer asap so it knows how to route messages
txPublisher ! SetChannelId(remoteNodeId, channelId)
context.system.eventStream.publish(ChannelIdAssigned(self, remoteNodeId, accept.temporaryChannelId, channelId))
@ -212,7 +213,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
minDepth_opt = minDepth_opt,
requireConfirmedInputs = RequireConfirmedInputs(forLocal = open.requireConfirmedInputs, forRemote = accept.requireConfirmedInputs)
)
val channelParams = ChannelParams(channelId, d.init.channelConfig, channelFeatures, localParams, remoteParams, open.channelFlags)
val purpose = InteractiveTxBuilder.FundingTx(open.commitmentFeerate, open.firstPerCommitmentPoint)
val txBuilder = context.spawnAnonymous(InteractiveTxBuilder(
nodeParams, fundingParams,
@ -239,7 +240,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
handleLocalError(t, d, Some(accept))
case Right((channelFeatures, remoteShutdownScript)) =>
// We've exchanged open_channel2 and accept_channel2, we now know the final channelId.
val channelId = Helpers.computeChannelId(d.lastSent, accept)
val channelId = Helpers.computeChannelId(d.lastSent.revocationBasepoint, accept.revocationBasepoint)
peer ! ChannelIdAssigned(self, remoteNodeId, accept.temporaryChannelId, channelId) // we notify the peer asap so it knows how to route messages
txPublisher ! SetChannelId(remoteNodeId, channelId)
context.system.eventStream.publish(ChannelIdAssigned(self, remoteNodeId, accept.temporaryChannelId, channelId))
@ -262,9 +263,10 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
// We start the interactive-tx funding protocol.
val localFundingPubkey = keyManager.fundingPublicKey(localParams.fundingKeyPath)
val fundingPubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingPubkey.publicKey, remoteParams.fundingPubKey)))
val channelParams = ChannelParams(channelId, d.init.channelConfig, channelFeatures, localParams, remoteParams, d.lastSent.channelFlags)
val localAmount = d.lastSent.fundingAmount
val remoteAmount = accept.fundingAmount
val minDepth_opt = Funding.minDepthDualFunding(nodeParams.channelConf, d.init.localParams.initFeatures, isInitiator = localParams.isInitiator, localAmount = localAmount, remoteAmount = remoteAmount)
val minDepth_opt = channelParams.minDepthDualFunding(nodeParams.channelConf.minDepthBlocks, localAmount + remoteAmount)
val fundingParams = InteractiveTxParams(
channelId = channelId,
isInitiator = localParams.isInitiator,
@ -279,7 +281,6 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
minDepth_opt = minDepth_opt,
requireConfirmedInputs = RequireConfirmedInputs(forLocal = accept.requireConfirmedInputs, forRemote = d.lastSent.requireConfirmedInputs)
)
val channelParams = ChannelParams(channelId, d.init.channelConfig, channelFeatures, localParams, remoteParams, d.lastSent.channelFlags)
val purpose = InteractiveTxBuilder.FundingTx(d.lastSent.commitmentFeerate, accept.firstPerCommitmentPoint)
val txBuilder = context.spawnAnonymous(InteractiveTxBuilder(
nodeParams, fundingParams,
@ -603,9 +604,9 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case Event(commitSig: CommitSig, d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED) =>
d.rbfStatus match {
case RbfStatus.RbfInProgress(cmd_opt, txBuilder, _) =>
case s: RbfStatus.RbfInProgress =>
log.debug("received their commit_sig, deferring message")
stay() using d.copy(rbfStatus = RbfStatus.RbfInProgress(cmd_opt, txBuilder, Some(commitSig)))
stay() using d.copy(rbfStatus = s.copy(remoteCommitSig = Some(commitSig)))
case RbfStatus.RbfWaitingForSigs(signingSession) =>
signingSession.receiveCommitSig(nodeParams, d.commitments.params, commitSig) match {
case Left(f) =>
@ -634,9 +635,9 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
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) =>
case RbfStatus.RbfWaitingForSigs(signingSession) =>
log.info("our peer aborted the rbf attempt: ascii='{}' bin={}", msg.toAscii, msg.data)
rollbackRbfAttempt(status, d)
rollbackRbfAttempt(signingSession, 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)
@ -656,10 +657,10 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
case RbfStatus.RbfInProgress(cmd_opt, _, remoteCommitSig_opt) =>
msg match {
case InteractiveTxBuilder.SendMessage(msg) => stay() sending msg
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.truncateToSatoshi))
case InteractiveTxBuilder.Succeeded(signingSession, commitSig) =>
cmd_opt.foreach(cmd => cmd.replyTo ! RES_BUMP_FUNDING_FEE(rbfIndex = d.previousFundingTxs.length, signingSession.fundingTx.txId, signingSession.fundingTx.tx.localFees.truncateToSatoshi))
remoteCommitSig_opt.foreach(self ! _)
val d1 = d.copy(rbfStatus = RbfStatus.RbfWaitingForSigs(status))
val d1 = d.copy(rbfStatus = RbfStatus.RbfWaitingForSigs(signingSession))
stay() using d1 storing() sending commitSig
case f: InteractiveTxBuilder.Failed =>
log.info("rbf attempt failed: {}", f.cause.getMessage)

View file

@ -110,9 +110,26 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
case Left(t) => handleLocalError(t, d, Some(open))
case Right((channelFeatures, remoteShutdownScript)) =>
context.system.eventStream.publish(ChannelCreated(self, peer, remoteNodeId, isInitiator = false, open.temporaryChannelId, open.feeratePerKw, None))
val remoteParams = RemoteParams(
nodeId = remoteNodeId,
dustLimit = open.dustLimitSatoshis,
maxHtlcValueInFlightMsat = open.maxHtlcValueInFlightMsat,
requestedChannelReserve_opt = Some(open.channelReserveSatoshis), // our peer requires us to always have at least that much satoshis in our balance
htlcMinimum = open.htlcMinimumMsat,
toSelfDelay = open.toSelfDelay,
maxAcceptedHtlcs = open.maxAcceptedHtlcs,
fundingPubKey = open.fundingPubkey,
revocationBasepoint = open.revocationBasepoint,
paymentBasepoint = open.paymentBasepoint,
delayedPaymentBasepoint = open.delayedPaymentBasepoint,
htlcBasepoint = open.htlcBasepoint,
initFeatures = d.initFundee.remoteInit.features,
upfrontShutdownScript_opt = remoteShutdownScript)
log.debug("remote params: {}", remoteParams)
val fundingPubkey = keyManager.fundingPublicKey(d.initFundee.localParams.fundingKeyPath).publicKey
val channelKeyPath = keyManager.keyPath(d.initFundee.localParams, d.initFundee.channelConfig)
val minimumDepth = Funding.minDepthFundee(nodeParams.channelConf, d.initFundee.localParams.initFeatures, open.fundingSatoshis)
val params = ChannelParams(d.initFundee.temporaryChannelId, d.initFundee.channelConfig, channelFeatures, d.initFundee.localParams, remoteParams, open.channelFlags)
val minimumDepth = params.minDepthFundee(nodeParams.channelConf.minDepthBlocks, open.fundingSatoshis)
log.info("will use fundingMinDepth={}", minimumDepth)
// In order to allow TLV extensions and keep backwards-compatibility, we include an empty upfront_shutdown_script if this feature is not used.
// See https://github.com/lightningnetwork/lightning-rfc/pull/714.
@ -135,23 +152,6 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
ChannelTlv.UpfrontShutdownScriptTlv(localShutdownScript),
ChannelTlv.ChannelTypeTlv(d.initFundee.channelType)
))
val remoteParams = RemoteParams(
nodeId = remoteNodeId,
dustLimit = open.dustLimitSatoshis,
maxHtlcValueInFlightMsat = open.maxHtlcValueInFlightMsat,
requestedChannelReserve_opt = Some(open.channelReserveSatoshis), // our peer requires us to always have at least that much satoshis in our balance
htlcMinimum = open.htlcMinimumMsat,
toSelfDelay = open.toSelfDelay,
maxAcceptedHtlcs = open.maxAcceptedHtlcs,
fundingPubKey = open.fundingPubkey,
revocationBasepoint = open.revocationBasepoint,
paymentBasepoint = open.paymentBasepoint,
delayedPaymentBasepoint = open.delayedPaymentBasepoint,
htlcBasepoint = open.htlcBasepoint,
initFeatures = d.initFundee.remoteInit.features,
upfrontShutdownScript_opt = remoteShutdownScript)
log.debug("remote params: {}", remoteParams)
val params = ChannelParams(d.initFundee.temporaryChannelId, d.initFundee.channelConfig, channelFeatures, d.initFundee.localParams, remoteParams, open.channelFlags)
goto(WAIT_FOR_FUNDING_CREATED) using DATA_WAIT_FOR_FUNDING_CREATED(params, open.fundingSatoshis, open.pushMsat, open.feeratePerKw, open.firstPerCommitmentPoint) sending accept
}
@ -278,6 +278,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
signature = localSigOfRemoteTx
)
val commitment = Commitment(
fundingTxIndex = 0,
localFundingStatus = SingleFundedUnconfirmedFundingTx(None),
remoteFundingStatus = RemoteFundingStatus.NotLocked,
localCommit = LocalCommit(0, localSpec, CommitTxAndRemoteSig(localCommitTx, remoteSig), htlcTxsAndRemoteSigs = Nil),
@ -296,7 +297,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
context.system.eventStream.publish(ChannelSignatureReceived(self, commitments))
// NB: we don't send a ChannelSignatureSent for the first commit
log.info(s"waiting for them to publish the funding tx for channelId=$channelId fundingTxid=${commitment.fundingTxId}")
watchFundingConfirmed(commitment.fundingTxId, Funding.minDepthFundee(nodeParams.channelConf, params.localParams.initFeatures, fundingAmount))
watchFundingConfirmed(commitment.fundingTxId, params.minDepthFundee(nodeParams.channelConf.minDepthBlocks, fundingAmount))
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, nodeParams.currentBlockHeight, None, Right(fundingSigned)) storing() sending fundingSigned
}
}
@ -322,6 +323,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
handleLocalError(InvalidCommitmentSignature(d.channelId, signedLocalCommitTx.tx.txid), d, Some(msg))
case Success(_) =>
val commitment = Commitment(
fundingTxIndex = 0,
localFundingStatus = SingleFundedUnconfirmedFundingTx(Some(fundingTx)),
remoteFundingStatus = RemoteFundingStatus.NotLocked,
localCommit = LocalCommit(0, localSpec, CommitTxAndRemoteSig(localCommitTx, remoteSig), htlcTxsAndRemoteSigs = Nil),
@ -338,7 +340,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
val blockHeight = nodeParams.currentBlockHeight
context.system.eventStream.publish(ChannelSignatureReceived(self, commitments))
log.info(s"publishing funding tx fundingTxid=${commitment.fundingTxId}")
watchFundingConfirmed(commitment.fundingTxId, Funding.minDepthFunder(params.localParams.initFeatures))
watchFundingConfirmed(commitment.fundingTxId, params.minDepthFunder)
// we will publish the funding tx only after the channel state has been written to disk because we want to
// make sure we first persist the commitment that returns back the funds to us in case of problem
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, blockHeight, None, Left(fundingCreated)) storing() calling publishFundingTx(d.channelId, fundingTx, fundingTxFee, d.replyTo)

View file

@ -17,7 +17,7 @@
package fr.acinq.eclair.channel.fsm
import akka.actor.typed.scaladsl.adapter.actorRefAdapter
import com.softwaremill.quicklens.{ModifyPimp, QuicklensAt}
import com.softwaremill.quicklens.ModifyPimp
import fr.acinq.bitcoin.ScriptFlags
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Transaction}
import fr.acinq.eclair.ShortChannelId
@ -72,14 +72,12 @@ trait CommonFundingHandlers extends CommonHandlers {
context.system.eventStream.publish(TransactionConfirmed(d.channelId, remoteNodeId, w.tx))
d.commitments.updateLocalFundingStatus(w.tx.txid, fundingStatus).map {
case (commitments1, commitment) =>
require(commitments1.active.size == 1 && commitment.fundingTxId == w.tx.txid, "there must be exactly one commitment after an initial funding tx is confirmed")
// first of all, we watch the funding tx that is now confirmed
watchFundingSpent(commitment)
// in the dual-funding case we can forget all other transactions, they have been double spent by the tx that just confirmed
val otherFundingTxs = d.commitments.active // note how we use the unpruned original commitments
.filter(c => c.fundingTxId != commitment.fundingTxId)
.map(_.localFundingStatus).collect { case fundingTx: DualFundedUnconfirmedFundingTx => fundingTx.sharedTx }
rollbackDualFundingTxs(otherFundingTxs)
rollbackDualFundingTxs(d.commitments.active // note how we use the unpruned original commitments
.filter(c => c.fundingTxIndex == commitment.fundingTxIndex && c.fundingTxId != commitment.fundingTxId)
.map(_.localFundingStatus).collect { case fundingTx: DualFundedUnconfirmedFundingTx => fundingTx.sharedTx })
(commitments1, commitment)
}
}
@ -114,10 +112,8 @@ trait CommonFundingHandlers extends CommonHandlers {
context.system.scheduler.scheduleWithFixedDelay(initialDelay = REFRESH_CHANNEL_UPDATE_INTERVAL, delay = REFRESH_CHANNEL_UPDATE_INTERVAL, receiver = self, message = BroadcastChannelUpdate(PeriodicRefresh))
// used to get the final shortChannelId, used in announcements (if minDepth >= ANNOUNCEMENTS_MINCONF this event will fire instantly)
blockchain ! WatchFundingDeeplyBuried(self, commitments.latest.fundingTxId, ANNOUNCEMENTS_MINCONF)
val commitments1 = commitments
.modify(_.remoteNextCommitInfo).setTo(Right(channelReady.nextPerCommitmentPoint))
.modify(_.active.at(0).remoteFundingStatus).setTo(RemoteFundingStatus.Locked)
DATA_NORMAL(commitments1, shortIds1, None, initialChannelUpdate, None, None, None)
val commitments1 = commitments.modify(_.remoteNextCommitInfo).setTo(Right(channelReady.nextPerCommitmentPoint))
DATA_NORMAL(commitments1, shortIds1, None, initialChannelUpdate, None, None, None, SpliceStatus.NoSplice)
}
def delayEarlyAnnouncementSigs(remoteAnnSigs: AnnouncementSignatures): Unit = {

View file

@ -146,4 +146,14 @@ trait DualFundingHandlers extends CommonFundingHandlers {
}
}
def reportSpliceFailure(spliceStatus: SpliceStatus, f: Throwable): Unit = {
spliceStatus match {
case SpliceStatus.SpliceRequested(cmd, _) => cmd.replyTo ! RES_FAILURE(cmd, f)
case SpliceStatus.SpliceInProgress(cmd_opt, txBuilder, _) =>
txBuilder ! InteractiveTxBuilder.Abort
cmd_opt.foreach(cmd => cmd.replyTo ! RES_FAILURE(cmd, f))
case _ => ()
}
}
}

View file

@ -185,14 +185,16 @@ trait ErrorHandlers extends CommonHandlers {
stay()
} else {
val finalScriptPubKey = getOrGenerateFinalScriptPubKey(d)
val commitTx = d.commitments.latest.fullySignedLocalCommitTx(keyManager).tx
val localCommitPublished = Closing.LocalClose.claimCommitTxOutputs(keyManager, d.commitments.latest, commitTx, nodeParams.currentBlockHeight, nodeParams.onChainFeeConf, finalScriptPubKey)
val commitment = d.commitments.latest
log.error(s"force-closing with fundingIndex=${commitment.fundingTxIndex}")
val commitTx = commitment.fullySignedLocalCommitTx(keyManager).tx
val localCommitPublished = Closing.LocalClose.claimCommitTxOutputs(keyManager, commitment, commitTx, nodeParams.currentBlockHeight, nodeParams.onChainFeeConf, finalScriptPubKey)
val nextData = d match {
case closing: DATA_CLOSING => closing.copy(localCommitPublished = Some(localCommitPublished))
case negotiating: DATA_NEGOTIATING => DATA_CLOSING(d.commitments, waitingSince = nodeParams.currentBlockHeight, finalScriptPubKey = finalScriptPubKey, negotiating.closingTxProposed.flatten.map(_.unsignedTx), localCommitPublished = Some(localCommitPublished))
case _ => DATA_CLOSING(d.commitments, waitingSince = nodeParams.currentBlockHeight, finalScriptPubKey = finalScriptPubKey, mutualCloseProposed = Nil, localCommitPublished = Some(localCommitPublished))
}
goto(CLOSING) using nextData storing() calling doPublish(localCommitPublished, d.commitments.latest)
goto(CLOSING) using nextData storing() calling doPublish(localCommitPublished, commitment)
}
}
@ -239,23 +241,6 @@ trait ErrorHandlers extends CommonHandlers {
goto(CLOSING) using nextData storing() calling doPublish(remoteCommitPublished, commitments)
}
def handleRemoteSpentFuture(commitTx: Transaction, d: DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT) = {
// It doesn't matter which commitment we use here, we'll only be able to claim our main outputs which is independent of the commitment.
val commitments = d.commitments.latest
log.warning(s"they published their future commit (because we asked them to) in txid=${commitTx.txid}")
context.system.eventStream.publish(TransactionPublished(d.channelId, remoteNodeId, commitTx, Closing.commitTxFee(commitments.commitInput, commitTx, d.commitments.params.localParams.isInitiator), "future-remote-commit"))
val finalScriptPubKey = getOrGenerateFinalScriptPubKey(d)
val remotePerCommitmentPoint = d.remoteChannelReestablish.myCurrentPerCommitmentPoint
val remoteCommitPublished = RemoteCommitPublished(
commitTx = commitTx,
claimMainOutputTx = Closing.RemoteClose.claimMainOutput(keyManager, d.commitments.params, remotePerCommitmentPoint, commitTx, nodeParams.onChainFeeConf.feeEstimator, nodeParams.onChainFeeConf.feeTargets, finalScriptPubKey),
claimHtlcTxs = Map.empty,
claimAnchorTxs = List.empty,
irrevocablySpent = Map.empty)
val nextData = DATA_CLOSING(d.commitments, waitingSince = nodeParams.currentBlockHeight, finalScriptPubKey = finalScriptPubKey, mutualCloseProposed = Nil, futureRemoteCommitPublished = Some(remoteCommitPublished))
goto(CLOSING) using nextData storing() calling doPublish(remoteCommitPublished, commitments)
}
def handleRemoteSpentNext(commitTx: Transaction, d: ChannelDataWithCommitments) = {
val commitment = d.commitments.latest
log.warning(s"they published their next commit in txid=${commitTx.txid}")
@ -294,13 +279,14 @@ trait ErrorHandlers extends CommonHandlers {
}
def handleRemoteSpentOther(tx: Transaction, d: ChannelDataWithCommitments) = {
val commitments = d.commitments.latest
val commitment = d.commitments.latest
log.warning(s"funding tx spent in txid=${tx.txid}")
val finalScriptPubKey = getOrGenerateFinalScriptPubKey(d)
Closing.RevokedClose.claimCommitTxOutputs(keyManager, d.commitments.params, d.commitments.remotePerCommitmentSecrets, tx, nodeParams.db.channels, nodeParams.onChainFeeConf.feeEstimator, nodeParams.onChainFeeConf.feeTargets, finalScriptPubKey) match {
case Some(revokedCommitPublished) =>
Closing.RevokedClose.getRemotePerCommitmentSecret(keyManager, d.commitments.params, d.commitments.remotePerCommitmentSecrets, tx) match {
case Some((commitmentNumber, remotePerCommitmentSecret)) =>
val revokedCommitPublished = Closing.RevokedClose.claimCommitTxOutputs(keyManager, d.commitments.params, tx, commitmentNumber, remotePerCommitmentSecret, nodeParams.db.channels, nodeParams.onChainFeeConf.feeEstimator, nodeParams.onChainFeeConf.feeTargets, finalScriptPubKey)
log.warning(s"txid=${tx.txid} was a revoked commitment, publishing the penalty tx")
context.system.eventStream.publish(TransactionPublished(d.channelId, remoteNodeId, tx, Closing.commitTxFee(commitments.commitInput, tx, d.commitments.params.localParams.isInitiator), "revoked-commit"))
context.system.eventStream.publish(TransactionPublished(d.channelId, remoteNodeId, tx, Closing.commitTxFee(commitment.commitInput, tx, d.commitments.params.localParams.isInitiator), "revoked-commit"))
val exc = FundingTxSpent(d.channelId, tx.txid)
val error = Error(d.channelId, exc.getMessage)
val nextData = d match {
@ -310,11 +296,25 @@ trait ErrorHandlers extends CommonHandlers {
case _ => DATA_CLOSING(d.commitments, waitingSince = nodeParams.currentBlockHeight, finalScriptPubKey = finalScriptPubKey, mutualCloseProposed = Nil, revokedCommitPublished = revokedCommitPublished :: Nil)
}
goto(CLOSING) using nextData storing() calling doPublish(revokedCommitPublished) sending error
case None =>
// the published tx was neither their current commitment nor a revoked one
log.error(s"couldn't identify txid=${tx.txid}, something very bad is going on!!!")
context.system.eventStream.publish(NotifyNodeOperator(NotificationsLogger.Error, s"funding tx ${commitments.fundingTxId} of channel ${d.channelId} was spent by an unknown transaction, indicating that your DB has lost data or your node has been breached: please contact the dev team."))
goto(ERR_INFORMATION_LEAK)
case None => d match {
case d: DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT =>
log.warning(s"they published a future commit (because we asked them to) in txid=${tx.txid}")
context.system.eventStream.publish(TransactionPublished(d.channelId, remoteNodeId, tx, Closing.commitTxFee(d.commitments.latest.commitInput, tx, d.commitments.latest.localParams.isInitiator), "future-remote-commit"))
val remotePerCommitmentPoint = d.remoteChannelReestablish.myCurrentPerCommitmentPoint
val remoteCommitPublished = RemoteCommitPublished(
commitTx = tx,
claimMainOutputTx = Closing.RemoteClose.claimMainOutput(keyManager, d.commitments.params, remotePerCommitmentPoint, tx, nodeParams.onChainFeeConf.feeEstimator, nodeParams.onChainFeeConf.feeTargets, finalScriptPubKey),
claimHtlcTxs = Map.empty,
claimAnchorTxs = List.empty,
irrevocablySpent = Map.empty)
val nextData = DATA_CLOSING(d.commitments, waitingSince = nodeParams.currentBlockHeight, finalScriptPubKey = finalScriptPubKey, mutualCloseProposed = Nil, futureRemoteCommitPublished = Some(remoteCommitPublished))
goto(CLOSING) using nextData storing() calling doPublish(remoteCommitPublished, d.commitments.latest)
case _ =>
// the published tx doesn't seem to be a valid commitment transaction
log.error(s"couldn't identify txid=${tx.txid}, something very bad is going on!!!")
context.system.eventStream.publish(NotifyNodeOperator(NotificationsLogger.Error, s"funding tx ${commitment.fundingTxId} of channel ${d.channelId} was spent by an unknown transaction, indicating that your DB has lost data or your node has been breached: please contact the dev team."))
goto(ERR_INFORMATION_LEAK)
}
}
}

View file

@ -44,7 +44,7 @@ trait SingleFundingHandlers extends CommonFundingHandlers {
wallet.commit(fundingTx).onComplete {
case Success(true) =>
context.system.eventStream.publish(TransactionPublished(channelId, remoteNodeId, fundingTx, fundingTxFee, "funding"))
replyTo ! OpenChannelResponse.Created(channelId, fundingTx.txid, fundingTxFee)
replyTo ! OpenChannelResponse.Created(channelId, fundingTxId = fundingTx.txid, fundingTxFee)
case Success(false) =>
replyTo ! OpenChannelResponse.Rejected("couldn't publish funding tx")
self ! BITCOIN_FUNDING_PUBLISH_FAILED // fail-fast: this should be returned only when we are really sure the tx has *not* been published
@ -116,10 +116,10 @@ trait SingleFundingHandlers extends CommonFundingHandlers {
def singleFundingMinDepth(d: ChannelDataWithCommitments): Long = {
val minDepth_opt = if (d.commitments.params.localParams.isInitiator) {
Helpers.Funding.minDepthFunder(d.commitments.params.localParams.initFeatures)
d.commitments.params.minDepthFunder
} else {
// when we're not the channel initiator we scale the min_depth confirmations depending on the funding amount
Helpers.Funding.minDepthFundee(nodeParams.channelConf, d.commitments.params.localParams.initFeatures, d.commitments.latest.commitInput.txOut.amount)
d.commitments.params.minDepthFundee(nodeParams.channelConf.minDepthBlocks, d.commitments.latest.commitInput.txOut.amount)
}
val minDepth = minDepth_opt.getOrElse {
val defaultMinDepth = nodeParams.channelConf.minDepthBlocks

View file

@ -25,6 +25,7 @@ import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, Lexicographical
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.Closing.MutualClose
import fr.acinq.eclair.channel.Helpers.Funding
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder.Purpose
@ -100,14 +101,13 @@ object InteractiveTxBuilder {
sealed trait SharedFundingInput {
// @formatter:off
def info: InputInfo
def weight: Long
def weight: Int
def sign(keyManager: ChannelKeyManager, params: ChannelParams, tx: Transaction): ByteVector64
// @formatter:on
}
case class Multisig2of2Input(info: InputInfo, localFundingPubkey: PublicKey, remoteFundingPubkey: PublicKey) extends SharedFundingInput {
// This value was computed assuming 73 bytes signatures (worst-case scenario).
override val weight: Long = 388
override val weight: Int = 388
override def sign(keyManager: ChannelKeyManager, params: ChannelParams, tx: Transaction): ByteVector64 = {
val fundingPubkey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath)
@ -153,7 +153,7 @@ object InteractiveTxBuilder {
// BOLT 2: MUST set `feerate` greater than or equal to 25/24 times the `feerate` of the previously constructed transaction, rounded down.
val minNextFeerate: FeeratePerKw = targetFeerate * 25 / 24
// BOLT 2: the initiator's serial IDs MUST use even values and the non-initiator odd values.
val serialIdParity = if (isInitiator) 0 else 1
val serialIdParity: Int = if (isInitiator) 0 else 1
}
// @formatter:off
@ -497,6 +497,8 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
Left(OutputBelowDust(fundingParams.channelId, addOutput.serialId, addOutput.amount, fundingParams.dustLimit))
} else if (addOutput.pubkeyScript == fundingParams.fundingPubkeyScript && addOutput.amount != fundingParams.fundingAmount) {
Left(InvalidSharedOutputAmount(fundingParams.channelId, addOutput.serialId, addOutput.amount, fundingParams.fundingAmount))
} else if (!MutualClose.isValidFinalScriptPubkey(addOutput.pubkeyScript, allowAnySegwit = true)) {
Left(InvalidSpliceOutputScript(fundingParams.channelId, addOutput.serialId, addOutput.pubkeyScript))
} else if (addOutput.pubkeyScript == fundingParams.fundingPubkeyScript) {
Right(Output.Shared(addOutput.serialId, addOutput.pubkeyScript, purpose.previousLocalBalance + fundingParams.localContribution, purpose.previousRemoteBalance + fundingParams.remoteContribution))
} else {
@ -644,7 +646,8 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
val sharedInput_opt = fundingParams.sharedInput_opt.map(_ => {
val remoteReserve = (fundingParams.fundingAmount / 100).max(channelParams.localParams.dustLimit)
if (sharedOutput.remoteAmount < remoteReserve && remoteOutputs.nonEmpty) {
// We ignore the reserve requirement if we are splicing funds into the channel, which increases the size of the reserve.
if (sharedOutput.remoteAmount < remoteReserve && remoteOutputs.nonEmpty && localInputs.isEmpty) {
log.warn("invalid interactive tx: peer takes too much funds out and falls below the channel reserve ({} < {})", sharedOutput.remoteAmount, remoteReserve)
return Left(InvalidCompleteInteractiveTx(fundingParams.channelId))
}
@ -739,7 +742,12 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
Behaviors.receiveMessagePartial {
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)
val fundingTxIndex = purpose match {
case _: FundingTx => 0
case r: PreviousTxRbf => r.commitment.fundingTxIndex
case s: SpliceTx => s.commitment.fundingTxIndex + 1
}
replyTo ! Succeeded(InteractiveTxSigningSession.WaitingForSigs(fundingParams, fundingTxIndex, signedTx, Left(localCommit), remoteCommit), commitSig)
Behaviors.stopped
case WalletFailure(t) =>
log.error("could not sign funding transaction: ", t)
@ -882,10 +890,11 @@ object InteractiveTxSigningSession {
* peer has forgotten that channel (which may happen if we disconnected before they received our tx_complete).
*/
case class WaitingForSigs(fundingParams: InteractiveTxParams,
fundingTxIndex: Long,
fundingTx: PartiallySignedSharedTransaction,
localCommit: Either[UnsignedLocalCommit, LocalCommit],
remoteCommit: RemoteCommit) extends InteractiveTxSigningSession {
val commitInput = localCommit.fold(_.commitTx.input, _.commitTxAndRemoteSig.commitTx.input)
val commitInput: InputInfo = localCommit.fold(_.commitTx.input, _.commitTxAndRemoteSig.commitTx.input)
def receiveCommitSig(nodeParams: NodeParams, channelParams: ChannelParams, remoteCommitSig: CommitSig)(implicit log: LoggingAdapter): Either[ChannelException, InteractiveTxSigningSession] = {
localCommit match {
@ -899,7 +908,7 @@ object InteractiveTxSigningSession {
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)
val commitment = Commitment(fundingTxIndex, fundingStatus, RemoteFundingStatus.NotLocked, signedLocalCommit, remoteCommit, None)
Right(SendingSigs(fundingStatus, commitment, fundingTx.localSigs))
} else {
Right(this.copy(localCommit = Right(signedLocalCommit)))
@ -924,7 +933,7 @@ object InteractiveTxSigningSession {
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)
val commitment = Commitment(fundingTxIndex, fundingStatus, RemoteFundingStatus.NotLocked, signedLocalCommit, remoteCommit, None)
Right(SendingSigs(fundingStatus, commitment, fullySignedTx.localSigs))
}
}

View file

@ -19,8 +19,9 @@ package fr.acinq.eclair.channel.fund
import akka.actor.typed.scaladsl.{ActorContext, Behaviors}
import akka.actor.typed.{ActorRef, Behavior}
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat.{OutPoint, SatoshiLong, Script, Transaction, TxIn, TxOut}
import fr.acinq.bitcoin.scalacompat.{KotlinUtils, OutPoint, Satoshi, SatoshiLong, Script, Transaction, TxIn, TxOut}
import fr.acinq.eclair.blockchain.OnChainChannelFunder
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel.fund.InteractiveTxBuilder._
import fr.acinq.eclair.transactions.Transactions
import fr.acinq.eclair.wire.protocol.TxAddInput
@ -68,6 +69,30 @@ object InteractiveTxFunder {
/** A wallet input that doesn't match interactive-tx construction requirements. */
private case class UnusableInput(outpoint: OutPoint)
/**
* Compute the funding contribution we're making to the channel output, by aggregating splice-in and splice-out and
* paying on-chain fees either from our wallet inputs or our current channel balance.
*/
def computeSpliceContribution(isInitiator: Boolean, sharedInput: SharedFundingInput, spliceInAmount: Satoshi, spliceOut: Seq[TxOut], targetFeerate: FeeratePerKw): Satoshi = {
val fees = if (spliceInAmount == 0.sat) {
val spliceOutputsWeight = spliceOut.map(KotlinUtils.scala2kmp).map(_.weight()).sum
val weight = if (isInitiator) {
// The initiator must add the shared input, the shared output and pay for the fees of the common transaction fields.
val dummyTx = Transaction(2, Nil, Seq(sharedInput.info.txOut), 0)
sharedInput.weight + dummyTx.weight() + spliceOutputsWeight
} else {
// The non-initiator only pays for the weights of their own inputs and outputs.
spliceOutputsWeight
}
Transactions.weight2fee(targetFeerate, weight)
} else {
// If we're splicing some funds into the channel, bitcoind will be responsible for adding more funds to pay the
// fees, so we don't need to pay them from our channel balance.
0 sat
}
spliceInAmount - spliceOut.map(_.amount).sum - fees
}
private def canUseInput(fundingParams: InteractiveTxParams, txIn: TxIn, previousTx: Transaction, confirmations: Int): Boolean = {
// Wallet input transaction must fit inside the tx_add_input message.
val previousTxSizeOk = Transaction.write(previousTx).length <= 65000
@ -161,7 +186,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response
* inputs.
*/
private def fund(txNotFunded: Transaction, currentInputs: Seq[OutgoingInput], unusableInputs: Set[UnusableInput]): Behavior[Command] = {
val sharedInputWeight = fundingParams.sharedInput_opt.toSeq.map(i => i.info.outPoint -> i.weight).toMap
val sharedInputWeight = fundingParams.sharedInput_opt.toSeq.map(i => i.info.outPoint -> i.weight.toLong).toMap
context.pipeToSelf(wallet.fundTransaction(txNotFunded, fundingParams.targetFeerate, replaceable = true, externalInputsWeight = sharedInputWeight)) {
case Failure(t) => WalletFailure(t)
case Success(result) => FundTransactionResult(result.tx, result.changePosition)
@ -222,7 +247,7 @@ private class InteractiveTxFunder(replyTo: ActorRef[InteractiveTxFunder.Response
case Some(changeOutput) =>
val txWeightWithoutInput = Transaction(2, Nil, Seq(TxOut(fundingParams.fundingAmount, fundingParams.fundingPubkeyScript)), 0).weight()
val commonWeight = fundingParams.sharedInput_opt match {
case Some(sharedInput) => sharedInput.weight.toInt + txWeightWithoutInput
case Some(sharedInput) => sharedInput.weight + txWeightWithoutInput
case None => txWeightWithoutInput
}
val overpaidFees = Transactions.weight2fee(fundingParams.targetFeerate, commonWeight)

View file

@ -41,7 +41,7 @@ import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, Feature
import org.json4s
import org.json4s.JsonAST._
import org.json4s.jackson.Serialization
import org.json4s.{DefaultFormats, Extraction, Formats, JDecimal, JValue, KeySerializer, Serializer, ShortTypeHints, TypeHints, jackson}
import org.json4s.{CustomSerializer, DefaultFormats, Extraction, Formats, JDecimal, JValue, KeySerializer, Serializer, ShortTypeHints, TypeHints, jackson}
import scodec.bits.ByteVector
import java.net.InetSocketAddress
@ -488,8 +488,8 @@ object OriginSerializer extends MinimalSerializer({
})
// @formatter:off
case class CommitmentJson(fundingTx: InputInfo, localFunding: LocalFundingStatus, remoteFunding: RemoteFundingStatus, localCommit: LocalCommit, remoteCommit: RemoteCommit, nextRemoteCommit: Option[RemoteCommit])
object CommitmentSerializer extends ConvertClassSerializer[Commitment](c => CommitmentJson(c.commitInput, c.localFundingStatus, c.remoteFundingStatus, c.localCommit, c.remoteCommit, c.nextRemoteCommit_opt.map(_.commit)))
case class CommitmentJson(fundingTxIndex: Long, fundingTx: InputInfo, localFunding: LocalFundingStatus, remoteFunding: RemoteFundingStatus, localCommit: LocalCommit, remoteCommit: RemoteCommit, nextRemoteCommit: Option[RemoteCommit])
object CommitmentSerializer extends ConvertClassSerializer[Commitment](c => CommitmentJson(c.fundingTxIndex, c.commitInput, c.localFundingStatus, c.remoteFundingStatus, c.localCommit, c.remoteCommit, c.nextRemoteCommit_opt.map(_.commit)))
// @formatter:on
// @formatter:off
@ -656,6 +656,11 @@ object JsonSerializers {
OnionMessageReceivedSerializer +
ShortIdsSerializer +
FundingTxStatusSerializer +
CommitmentSerializer
CommitmentSerializer +
new CustomSerializer[SpliceStatus](_ => (
PartialFunction.empty, {
case _: SpliceStatus => JNothing
}
))
}

View file

@ -403,7 +403,7 @@ private[channel] object ChannelCodecs0 {
("remoteShutdown" | optional(bool, shutdownCodec)) ::
("closingFeerates" | provide(Option.empty[ClosingFeerates]))).map {
case commitments :: shortChannelId :: buried :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates)
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
}.decodeOnly
val DATA_NORMAL_10_Codec: Codec[DATA_NORMAL] = (
@ -416,7 +416,7 @@ private[channel] object ChannelCodecs0 {
("remoteShutdown" | optional(bool, shutdownCodec)) ::
("closingFeerates" | provide(Option.empty[ClosingFeerates]))).map {
case commitments :: shortChannelId :: buried :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates)
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
}.decodeOnly
val DATA_SHUTDOWN_04_Codec: Codec[DATA_SHUTDOWN] = (

View file

@ -206,6 +206,7 @@ private[channel] object ChannelTypes0 {
ChannelFeatures()
}
val commitment = Commitment(
fundingTxIndex = 0,
// We set an empty funding tx, even if it may 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. We will retrieve the
// funding tx when the channel is instantiated, and update the status (possibly immediately if it was confirmed).
@ -216,6 +217,7 @@ private[channel] object ChannelTypes0 {
ChannelParams(channelId, channelConfig, channelFeatures, localParams, remoteParams, channelFlags),
CommitmentChanges(localChanges, remoteChanges, localNextHtlcId, remoteNextHtlcId),
Seq(commitment),
inactive = Nil,
remoteNextCommitInfo.fold(w => Left(WaitForRev(w.sentAfterLocalCommitIndex)), remotePerCommitmentPoint => Right(remotePerCommitmentPoint)),
remotePerCommitmentSecrets,
originChannels

View file

@ -263,7 +263,7 @@ private[channel] object ChannelCodecs1 {
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | provide(Option.empty[ClosingFeerates]))).map {
case commitments :: shortChannelId :: buried :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates)
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
}.decodeOnly
val DATA_SHUTDOWN_23_Codec: Codec[DATA_SHUTDOWN] = (

View file

@ -298,7 +298,7 @@ private[channel] object ChannelCodecs2 {
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | provide(Option.empty[ClosingFeerates]))).map {
case commitments :: shortChannelId :: buried :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates)
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
}.decodeOnly
val DATA_SHUTDOWN_03_Codec: Codec[DATA_SHUTDOWN] = (

View file

@ -355,7 +355,7 @@ private[channel] object ChannelCodecs3 {
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | provide(Option.empty[ClosingFeerates]))).map {
case commitments :: shortChannelId :: buried :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates)
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
}.decodeOnly
val DATA_NORMAL_07_Codec: Codec[DATA_NORMAL] = (
@ -368,7 +368,7 @@ private[channel] object ChannelCodecs3 {
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | optional(bool8, closingFeeratesCodec))).map {
case commitments :: shortChannelId :: buried :: channelAnnouncement :: channelUpdate :: localShutdown :: remoteShutdown :: closingFeerates :: HNil =>
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates)
DATA_NORMAL(commitments, shortIds = ShortIds(real = if (buried) RealScidStatus.Final(shortChannelId) else RealScidStatus.Temporary(shortChannelId), localAlias = Alias(shortChannelId.toLong), remoteAlias_opt = None), channelAnnouncement, channelUpdate, localShutdown, remoteShutdown, closingFeerates, SpliceStatus.NoSplice)
}.decodeOnly
val DATA_NORMAL_09_Codec: Codec[DATA_NORMAL] = (
@ -378,7 +378,8 @@ private[channel] object ChannelCodecs3 {
("channelUpdate" | lengthDelimited(channelUpdateCodec)) ::
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | optional(bool8, closingFeeratesCodec))).as[DATA_NORMAL]
("closingFeerates" | optional(bool8, closingFeeratesCodec)) ::
("spliceStatus" | provide[SpliceStatus](SpliceStatus.NoSplice))).as[DATA_NORMAL]
val DATA_SHUTDOWN_03_Codec: Codec[DATA_SHUTDOWN] = (
("commitments" | commitmentsCodec) ::

View file

@ -44,7 +44,8 @@ private[channel] object ChannelTypes3 {
def migrate(): channel.Commitments = channel.Commitments(
ChannelParams(channelId, channelConfig, channelFeatures, localParams, remoteParams, channelFlags),
CommitmentChanges(localChanges, remoteChanges, localNextHtlcId, remoteNextHtlcId),
Seq(Commitment(localFundingStatus, remoteFundingStatus, localCommit, remoteCommit, remoteNextCommitInfo.left.toOption.map(w => NextRemoteCommit(w.sent, w.nextRemoteCommit)))),
Seq(Commitment(fundingTxIndex = 0, localFundingStatus, remoteFundingStatus, localCommit, remoteCommit, remoteNextCommitInfo.left.toOption.map(w => NextRemoteCommit(w.sent, w.nextRemoteCommit)))),
inactive = Nil,
remoteNextCommitInfo.fold(w => Left(WaitForRev(w.sentAfterLocalCommitIndex)), remotePerCommitmentPoint => Right(remotePerCommitmentPoint)),
remotePerCommitmentSecrets,
originChannels

View file

@ -13,7 +13,7 @@ import fr.acinq.eclair.transactions.{CommitmentSpec, DirectedHtlc, IncomingHtlc,
import fr.acinq.eclair.wire.protocol.CommonCodecs._
import fr.acinq.eclair.wire.protocol.LightningMessageCodecs._
import fr.acinq.eclair.wire.protocol.{UpdateAddHtlc, UpdateMessage}
import fr.acinq.eclair.{BlockHeight, FeatureSupport, Features, PermanentChannelFeature}
import fr.acinq.eclair.{BlockHeight, FeatureSupport, Features, PermanentChannelFeature, channel}
import scodec.bits.{BitVector, ByteVector}
import scodec.codecs._
import scodec.{Attempt, Codec}
@ -224,8 +224,8 @@ private[channel] object ChannelCodecs4 {
private val fundingParamsCodec: Codec[InteractiveTxBuilder.InteractiveTxParams] = (
("channelId" | bytes32) ::
("isInitiator" | bool8) ::
("localAmount" | satoshi) ::
("remoteAmount" | satoshi) ::
("localContribution" | satoshiSigned) ::
("remoteContribution" | satoshiSigned) ::
("sharedInput_opt" | optional(bool8, sharedFundingInputCodec)) ::
("fundingPubkeyScript" | lengthDelimited(bytes)) ::
("localOutputs" | listOfN(uint16, txOutCodec)) ::
@ -353,7 +353,8 @@ private[channel] object ChannelCodecs4 {
("commit" | remoteCommitCodec(commitmentSpecCodec))).as[NextRemoteCommit]
private def commitmentCodec(htlcs: Set[DirectedHtlc]): Codec[Commitment] = (
("fundingTxStatus" | fundingTxStatusCodec) ::
("fundingTxIndex" | uint32) ::
("fundingTxStatus" | fundingTxStatusCodec) ::
("remoteFundingStatus" | remoteFundingStatusCodec) ::
("localCommit" | localCommitCodec(minimalCommitmentSpecCodec(htlcs))) ::
("remoteCommit" | remoteCommitCodec(minimalCommitmentSpecCodec(htlcs.map(_.opposite)))) ::
@ -370,6 +371,7 @@ private[channel] object ChannelCodecs4 {
// The direction we use is from our local point of view.
htlcs: Set[DirectedHtlc],
active: List[Commitment],
inactive: List[Commitment],
remoteNextCommitInfo: Either[WaitForRev, PublicKey],
remotePerCommitmentSecrets: ShaChain,
originChannels: Map[Long, Origin],
@ -379,6 +381,7 @@ private[channel] object ChannelCodecs4 {
params = params,
changes = changes,
active = active,
inactive = inactive,
remoteNextCommitInfo,
remotePerCommitmentSecrets,
originChannels,
@ -391,14 +394,17 @@ private[channel] object ChannelCodecs4 {
def apply(commitments: Commitments): EncodedCommitments = {
// The direction we use is from our local point of view: we use sets, which deduplicates htlcs that are in both
// local and remote commitments.
val htlcs = commitments.active.head.localCommit.spec.htlcs ++
commitments.active.head.remoteCommit.spec.htlcs.map(_.opposite) ++
commitments.active.head.nextRemoteCommit_opt.map(_.commit.spec.htlcs.map(_.opposite)).getOrElse(Set.empty)
// All active commitments have the same htlc set, but each inactive commitment may have a distinct htlc set
val commitmentsSet = (commitments.active.head +: commitments.inactive).toSet
val htlcs = commitmentsSet.flatMap(_.localCommit.spec.htlcs) ++
commitmentsSet.flatMap(_.remoteCommit.spec.htlcs.map(_.opposite)) ++
commitmentsSet.flatMap(_.nextRemoteCommit_opt.toList.flatMap(_.commit.spec.htlcs.map(_.opposite)))
EncodedCommitments(
params = commitments.params,
changes = commitments.changes,
htlcs = htlcs,
active = commitments.active.toList,
inactive = commitments.inactive.toList,
remoteNextCommitInfo = commitments.remoteNextCommitInfo,
remotePerCommitmentSecrets = commitments.remotePerCommitmentSecrets,
originChannels = commitments.originChannels,
@ -412,6 +418,7 @@ private[channel] object ChannelCodecs4 {
("changes" | changesCodec) ::
(("htlcs" | setCodec(htlcCodec)) >>:~ { htlcs =>
("active" | listOfN(uint16, commitmentCodec(htlcs))) ::
("inactive" | listOfN(uint16, commitmentCodec(htlcs))) ::
("remoteNextCommitInfo" | either(bool8, waitForRevCodec, publicKey)) ::
("remotePerCommitmentSecrets" | byteAligned(ShaChain.shaChainCodec)) ::
("originChannels" | originsMapCodec) ::
@ -464,6 +471,7 @@ private[channel] object ChannelCodecs4 {
val waitingForSigsCodec: Codec[InteractiveTxSigningSession.WaitingForSigs] = (
("fundingParams" | fundingParamsCodec) ::
("fundingTxIndex" | uint32) ::
("fundingTx" | partiallySignedSharedTransactionCodec) ::
("localCommit" | either(bool8, unsignedLocalCommitCodec, localCommitCodec(commitmentSpecCodec))) ::
("remoteCommit" | remoteCommitCodec(commitmentSpecCodec))).as[InteractiveTxSigningSession.WaitingForSigs]
@ -475,6 +483,10 @@ private[channel] object ChannelCodecs4 {
.\(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 spliceStatusCodec: Codec[SpliceStatus] = discriminated[SpliceStatus].by(uint8)
.\(0x01) { case status: SpliceStatus if !status.isInstanceOf[SpliceStatus.SpliceWaitingForSigs] => SpliceStatus.NoSplice }(provide(SpliceStatus.NoSplice))
.\(0x02) { case status: SpliceStatus.SpliceWaitingForSigs => status }(interactiveTxWaitingForSigsCodec.as[channel.SpliceStatus.SpliceWaitingForSigs])
val DATA_WAIT_FOR_FUNDING_CONFIRMED_00_Codec: Codec[DATA_WAIT_FOR_FUNDING_CONFIRMED] = (
("commitments" | commitmentsCodec) ::
("waitingSince" | blockHeight) ::
@ -513,7 +525,8 @@ private[channel] object ChannelCodecs4 {
("channelUpdate" | lengthDelimited(channelUpdateCodec)) ::
("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) ::
("closingFeerates" | optional(bool8, closingFeeratesCodec))).as[DATA_NORMAL]
("closingFeerates" | optional(bool8, closingFeeratesCodec)) ::
("spliceStatus" | spliceStatusCodec)).as[DATA_NORMAL]
val DATA_SHUTDOWN_05_Codec: Codec[DATA_SHUTDOWN] = (
("commitments" | commitmentsCodec) ::

View file

@ -33,6 +33,12 @@ sealed trait OpenDualFundedChannelTlv extends Tlv
sealed trait AcceptDualFundedChannelTlv extends Tlv
sealed trait SpliceInitTlv extends Tlv
sealed trait SpliceAckTlv extends Tlv
sealed trait SpliceLockedTlv extends Tlv
object ChannelTlv {
/** Commitment to where the funds will go in case of a mutual close, which remote node will enforce in case we're compromised. */
@ -50,11 +56,11 @@ object ChannelTlv {
tlv => Features(tlv.channelType.features.map(f => f -> FeatureSupport.Mandatory).toMap).toByteVector
))
case class RequireConfirmedInputsTlv() extends OpenDualFundedChannelTlv with AcceptDualFundedChannelTlv
case class RequireConfirmedInputsTlv() extends OpenDualFundedChannelTlv with AcceptDualFundedChannelTlv with SpliceInitTlv with SpliceAckTlv
val requireConfirmedInputsCodec: Codec[RequireConfirmedInputsTlv] = tlvField(provide(RequireConfirmedInputsTlv()))
case class PushAmountTlv(amount: MilliSatoshi) extends OpenDualFundedChannelTlv with AcceptDualFundedChannelTlv
case class PushAmountTlv(amount: MilliSatoshi) extends OpenDualFundedChannelTlv with AcceptDualFundedChannelTlv with SpliceInitTlv with SpliceAckTlv
val pushAmountCodec: Codec[PushAmountTlv] = tlvField(tmillisatoshi)
@ -91,7 +97,30 @@ object OpenDualFundedChannelTlv {
.typecase(UInt64(2), requireConfirmedInputsCodec)
.typecase(UInt64(0x47000007), pushAmountCodec)
)
}
object SpliceInitTlv {
import ChannelTlv._
val spliceInitTlvCodec: Codec[TlvStream[SpliceInitTlv]] = tlvStream(discriminated[SpliceInitTlv].by(varint)
.typecase(UInt64(2), requireConfirmedInputsCodec)
.typecase(UInt64(0x47000007), tlvField(tmillisatoshi.as[PushAmountTlv]))
)
}
object SpliceAckTlv {
import ChannelTlv._
val spliceAckTlvCodec: Codec[TlvStream[SpliceAckTlv]] = tlvStream(discriminated[SpliceAckTlv].by(varint)
.typecase(UInt64(2), requireConfirmedInputsCodec)
.typecase(UInt64(0x47000007), tlvField(tmillisatoshi.as[PushAmountTlv]))
)
}
object SpliceLockedTlv {
val spliceLockedTlvCodec: Codec[TlvStream[SpliceLockedTlv]] = tlvStream(discriminated[SpliceLockedTlv].by(varint))
}
object AcceptDualFundedChannelTlv {

View file

@ -16,11 +16,10 @@
package fr.acinq.eclair.wire.protocol
import fr.acinq.bitcoin.scalacompat.ByteVector32
import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.eclair.UInt64
import fr.acinq.eclair.wire.protocol.CommonCodecs._
import fr.acinq.eclair.wire.protocol.TlvCodecs.{tlvField, tlvStream}
import fr.acinq.eclair.wire.protocol.TlvCodecs.{tlvField, tlvStream, tu16}
import scodec.Codec
import scodec.bits.HexStringSyntax
import scodec.codecs._
@ -62,12 +61,15 @@ sealed trait CommitSigTlv extends Tlv
object CommitSigTlv {
case class FundingTxIdTlv(txId: ByteVector32) extends CommitSigTlv
/** @param size the number of [[CommitSig]] messages in the batch */
case class BatchTlv(size: Int) extends CommitSigTlv
private val fundingTxIdCodec: Codec[FundingTxIdTlv] = tlvField(bytes32)
object BatchTlv {
val codec: Codec[BatchTlv] = tlvField(tu16)
}
val commitSigTlvCodec: Codec[TlvStream[CommitSigTlv]] = tlvStream(discriminated[CommitSigTlv].by(varint)
.typecase(UInt64(0x47010003), fundingTxIdCodec)
.typecase(UInt64(0x47010005), BatchTlv.codec)
)
}

View file

@ -92,7 +92,6 @@ object TxInitRbfTlv {
val txInitRbfTlvCodec: Codec[TlvStream[TxInitRbfTlv]] = tlvStream(discriminated[TxInitRbfTlv].by(varint)
.typecase(UInt64(0), tlvField(satoshiSigned.as[SharedOutputContributionTlv]))
)
}
object TxAckRbfTlv {
@ -102,7 +101,6 @@ object TxAckRbfTlv {
val txAckRbfTlvCodec: Codec[TlvStream[TxAckRbfTlv]] = tlvStream(discriminated[TxAckRbfTlv].by(varint)
.typecase(UInt64(0), tlvField(satoshiSigned.as[SharedOutputContributionTlv]))
)
}
sealed trait TxAbortTlv extends Tlv

View file

@ -403,7 +403,22 @@ object LightningMessageCodecs {
//
//
val spliceInitCodec: Codec[SpliceInit] = (
("channelId" | bytes32) ::
("fundingContribution" | satoshiSigned) ::
("lockTime" | uint32) ::
("feerate" | feeratePerKw) ::
("tlvStream" | SpliceInitTlv.spliceInitTlvCodec)).as[SpliceInit]
val spliceAckCodec: Codec[SpliceAck] = (
("channelId" | bytes32) ::
("fundingContribution" | satoshiSigned) ::
("tlvStream" | SpliceAckTlv.spliceAckTlvCodec)).as[SpliceAck]
val spliceLockedCodec: Codec[SpliceLocked] = (
("channelId" | bytes32) ::
("fundingTxid" | bytes32) ::
("tlvStream" | SpliceLockedTlv.spliceLockedTlvCodec)).as[SpliceLocked]
//
//
@ -455,10 +470,12 @@ object LightningMessageCodecs {
.typecase(264, replyChannelRangeCodec)
.typecase(265, gossipTimestampFilterCodec)
.typecase(513, onionMessageCodec)
// NB: blank lines to minimize merge conflicts
//
// NB: blank lines to minimize merge conflicts
//
.typecase(37000, spliceInitCodec)
.typecase(37002, spliceAckCodec)
.typecase(37004, spliceLockedCodec)
//
//

View file

@ -49,7 +49,8 @@ sealed trait HasTemporaryChannelId extends LightningMessage { def temporaryChann
sealed trait HasChannelId extends LightningMessage { def channelId: ByteVector32 } // <- not in the spec
sealed trait HasChainHash extends LightningMessage { def chainHash: ByteVector32 } // <- not in the spec
sealed trait HasSerialId extends LightningMessage { def serialId: UInt64 } // <- not in the spec
sealed trait UpdateMessage extends HtlcMessage // <- not in the spec
sealed trait ForbiddenMessageDuringSplice extends LightningMessage // <- not in the spec
sealed trait UpdateMessage extends HtlcMessage with ForbiddenMessageDuringSplice // <- not in the spec
sealed trait HtlcSettlementMessage extends UpdateMessage { def id: Long } // <- not in the spec
sealed trait HtlcFailureMessage extends HtlcSettlementMessage // <- not in the spec
// @formatter:on
@ -277,9 +278,51 @@ case class ChannelReady(channelId: ByteVector32,
val alias_opt: Option[Alias] = tlvStream.get[ShortChannelIdTlv].map(_.alias)
}
case class SpliceInit(channelId: ByteVector32,
fundingContribution: Satoshi,
lockTime: Long,
feerate: FeeratePerKw,
tlvStream: TlvStream[SpliceInitTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId {
val requireConfirmedInputs: Boolean = tlvStream.get[ChannelTlv.RequireConfirmedInputsTlv].nonEmpty
val pushAmount: MilliSatoshi = tlvStream.get[ChannelTlv.PushAmountTlv].map(_.amount).getOrElse(0 msat)
}
object SpliceInit {
def apply(channelId: ByteVector32, fundingContribution: Satoshi, lockTime: Long, feerate: FeeratePerKw, pushAmount: MilliSatoshi, requireConfirmedInputs: Boolean): SpliceInit = {
val tlvs: Set[SpliceInitTlv] = Set(
Some(ChannelTlv.PushAmountTlv(pushAmount)),
if (requireConfirmedInputs) Some(ChannelTlv.RequireConfirmedInputsTlv()) else None,
).flatten
SpliceInit(channelId, fundingContribution, lockTime, feerate, TlvStream(tlvs))
}
}
case class SpliceAck(channelId: ByteVector32,
fundingContribution: Satoshi,
tlvStream: TlvStream[SpliceAckTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId {
val requireConfirmedInputs: Boolean = tlvStream.get[ChannelTlv.RequireConfirmedInputsTlv].nonEmpty
val pushAmount: MilliSatoshi = tlvStream.get[ChannelTlv.PushAmountTlv].map(_.amount).getOrElse(0 msat)
}
object SpliceAck {
def apply(channelId: ByteVector32, fundingContribution: Satoshi, pushAmount: MilliSatoshi, requireConfirmedInputs: Boolean): SpliceAck = {
val tlvs: Set[SpliceAckTlv] = Set(
Some(ChannelTlv.PushAmountTlv(pushAmount)),
if (requireConfirmedInputs) Some(ChannelTlv.RequireConfirmedInputsTlv()) else None,
).flatten
SpliceAck(channelId, fundingContribution, TlvStream(tlvs))
}
}
case class SpliceLocked(channelId: ByteVector32,
fundingTxHash: ByteVector32,
tlvStream: TlvStream[SpliceLockedTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId {
val fundingTxid: ByteVector32 = fundingTxHash.reverse
}
case class Shutdown(channelId: ByteVector32,
scriptPubKey: ByteVector,
tlvStream: TlvStream[ShutdownTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId
tlvStream: TlvStream[ShutdownTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId with ForbiddenMessageDuringSplice
case class ClosingSigned(channelId: ByteVector32,
feeSatoshis: Satoshi,
@ -331,7 +374,7 @@ case class CommitSig(channelId: ByteVector32,
signature: ByteVector64,
htlcSignatures: List[ByteVector64],
tlvStream: TlvStream[CommitSigTlv] = TlvStream.empty) extends HtlcMessage with HasChannelId {
val fundingTxId_opt: Option[ByteVector32] = tlvStream.get[CommitSigTlv.FundingTxIdTlv].map(_.txId)
val batchSize: Int = tlvStream.get[CommitSigTlv.BatchTlv].map(_.size).getOrElse(1)
}
case class RevokeAndAck(channelId: ByteVector32,

View file

@ -67,6 +67,7 @@
"remoteNextHtlcId" : 4147
},
"active" : [ {
"fundingTxIndex" : 0,
"fundingTx" : {
"outPoint" : "3dd6450c0bb55d6e4ef6ba6bd62d9061af1690e0c6ebca5b79246ac1228f7307:1",
"amountSatoshis" : 16777215
@ -106,6 +107,7 @@
"remotePerCommitmentPoint" : "03daadaed37bcfed40d15e34979fbf2a0643e748e8960363bb8e930cefe2255c35"
}
} ],
"inactive" : [ ],
"remoteNextCommitInfo" : "034dcc0704325064a1fa68edc13adb5fd173051775df73a298ec291f22ad9d19f6",
"remotePerCommitmentSecrets" : null,
"originChannels" : { }

View file

@ -67,6 +67,7 @@
"remoteNextHtlcId" : 151
},
"active" : [ {
"fundingTxIndex" : 0,
"fundingTx" : {
"outPoint" : "115641011cceeb4a1709a6cbd8f5f1b387460ee5fd2e48be3fbd1ae0e9e1cf6e:0",
"amountSatoshis" : 15000000
@ -106,6 +107,7 @@
"remotePerCommitmentPoint" : "02b82bbd59e0d22665671d9e47d8733058b92f18e906e9403753661aa03dc9e4dd"
}
} ],
"inactive" : [ ],
"remoteNextCommitInfo" : "02a4471183c519e54b8ee66fb41cbe06fed1153fce258db72ce67f9a9e044f0a16",
"remotePerCommitmentSecrets" : null,
"originChannels" : { }

View file

@ -81,6 +81,7 @@
"remoteNextHtlcId" : 0
},
"active" : [ {
"fundingTxIndex" : 0,
"fundingTx" : {
"outPoint" : "1bade1718aaf98ab1f91a97ed5b34ab47bfb78085e384f67c156793544f68659:0",
"amountSatoshis" : 15000000
@ -120,6 +121,7 @@
"remotePerCommitmentPoint" : "02e7e1abac1feb54ee3ac2172c9e2231f77765df57664fb44a6dc2e4aa9e6a9a6a"
}
} ],
"inactive" : [ ],
"remoteNextCommitInfo" : "03fd10fe44564e2d7e1550099785c2c1bad32a5ae0feeef6e27f0c108d18b4931d",
"remotePerCommitmentSecrets" : null,
"originChannels" : { }

View file

@ -74,6 +74,7 @@
"remoteNextHtlcId" : 0
},
"active" : [ {
"fundingTxIndex" : 0,
"fundingTx" : {
"outPoint" : "f4e3ba374da1a85abcd12a86c9a25b1391bda144619c770fe03f3881c6ad17e9:0",
"amountSatoshis" : 1000000
@ -114,6 +115,7 @@
"remotePerCommitmentPoint" : "032a992c123095216f7937a8b0baf442211eeb57942d586854a61a0dc6b01ca6ee"
}
} ],
"inactive" : [ ],
"remoteNextCommitInfo" : "030af74aa1e98668a504d50fe6f664aff3fbdb5c8681f0667c34cdb80024fb950f",
"remotePerCommitmentSecrets" : null,
"originChannels" : { }

View file

@ -76,6 +76,7 @@
"remoteNextHtlcId" : 0
},
"active" : [ {
"fundingTxIndex" : 0,
"fundingTx" : {
"outPoint" : "7d314422179e4e93e201da84b7b86cf9a23470933877f10db675f9ada8dea683:0",
"amountSatoshis" : 1000000
@ -115,6 +116,7 @@
"remotePerCommitmentPoint" : "0324b50221ad635b97f597802fbe5b2d6414fdf41f224ac1869d3772314e9fbfa5"
}
} ],
"inactive" : [ ],
"remoteNextCommitInfo" : "0209317c45de4cff05adbf9d69edbc334a1c89325bade86f4194c6665336b7e9f8",
"remotePerCommitmentSecrets" : null,
"originChannels" : { }

View file

@ -82,6 +82,7 @@
"remoteNextHtlcId" : 0
},
"active" : [ {
"fundingTxIndex" : 0,
"fundingTx" : {
"outPoint" : "7443277377ab5ca44330a332d79e6ff33d21a3b8889559f54894982af47e1cdb:0",
"amountSatoshis" : 1500000
@ -121,6 +122,7 @@
"remotePerCommitmentPoint" : "037d0b91e7bf58eec2eddf033d457b17140a341533808a346c869ada9ecea0cec0"
}
} ],
"inactive" : [ ],
"remoteNextCommitInfo" : "02a7d9d163632731c7211ced4ee21ae181bb0dfa73f5538607c081dd63d89f9820",
"remotePerCommitmentSecrets" : null,
"originChannels" : { }

View file

@ -79,6 +79,10 @@ object TestDatabases {
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))
case d: DATA_NORMAL => d.copy(commitments = freeze2(d.commitments))
.modify(_.spliceStatus).using {
case s: SpliceStatus.SpliceWaitingForSigs => s
case _ => SpliceStatus.NoSplice
}
case d: DATA_CLOSING => d.copy(commitments = freeze2(d.commitments))
case d: DATA_NEGOTIATING => d.copy(commitments = freeze2(d.commitments))
case d: DATA_SHUTDOWN => d.copy(commitments = freeze2(d.commitments))

View file

@ -131,7 +131,7 @@ class SingleKeyOnChainWallet extends OnChainWallet with OnchainPubkeyCache {
override def getP2wpkhPubkey()(implicit ec: ExecutionContext): Future[Crypto.PublicKey] = Future.successful(pubkey)
override def fundTransaction(tx: Transaction, feeRate: FeeratePerKw, replaceable: Boolean, externalInputsWeight: Map[OutPoint, Long])(implicit ec: ExecutionContext): Future[FundTransactionResponse] = synchronized {
val currentAmountIn = tx.txIn.flatMap(txIn => inputs.find(_.txid == txIn.outPoint.txid)).map(_.txOut.head.amount).sum
val currentAmountIn = tx.txIn.flatMap(txIn => inputs.find(_.txid == txIn.outPoint.txid).flatMap(_.txOut.lift(txIn.outPoint.index.toInt))).map(_.amount).sum
val amountOut = tx.txOut.map(_.amount).sum
// We add a single input to reach the desired feerate.
val inputAmount = amountOut + 100_000.sat
@ -139,10 +139,10 @@ class SingleKeyOnChainWallet extends OnChainWallet with OnchainPubkeyCache {
inputs = inputs :+ inputTx
val dummyWitness = Script.witnessPay2wpkh(pubkey, ByteVector.fill(73)(0))
val dummySignedTx = tx.copy(
txIn = tx.txIn.map(_.copy(witness = dummyWitness)) :+ TxIn(OutPoint(inputTx, 0), ByteVector.empty, 0, dummyWitness),
txIn = tx.txIn.filterNot(i => externalInputsWeight.contains(i.outPoint)).map(_.copy(witness = dummyWitness)) :+ TxIn(OutPoint(inputTx, 0), ByteVector.empty, 0, dummyWitness),
txOut = tx.txOut :+ TxOut(inputAmount, Script.pay2wpkh(pubkey)),
)
val fee = Transactions.weight2fee(feeRate, dummySignedTx.weight())
val fee = Transactions.weight2fee(feeRate, dummySignedTx.weight() + externalInputsWeight.values.sum.toInt)
val fundedTx = tx.copy(
txIn = tx.txIn :+ TxIn(OutPoint(inputTx, 0), Nil, 0),
txOut = tx.txOut :+ TxOut(inputAmount + currentAmountIn - amountOut - fee, Script.pay2wpkh(pubkey)),
@ -163,7 +163,10 @@ class SingleKeyOnChainWallet extends OnChainWallet with OnchainPubkeyCache {
Future.successful(SignTransactionResponse(signedTx, complete))
}
override def publishTransaction(tx: Transaction)(implicit ec: ExecutionContext): Future[ByteVector32] = Future.successful(tx.txid)
override def publishTransaction(tx: Transaction)(implicit ec: ExecutionContext): Future[ByteVector32] = {
inputs = inputs :+ tx
Future.successful(tx.txid)
}
override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = {
val tx = Transaction(2, Nil, Seq(TxOut(amount, pubkeyScript)), 0)
@ -178,7 +181,7 @@ class SingleKeyOnChainWallet extends OnChainWallet with OnchainPubkeyCache {
override def getTransaction(txId: ByteVector32)(implicit ec: ExecutionContext): Future[Transaction] = synchronized {
inputs.find(_.txid == txId) match {
case Some(tx) => Future.successful(tx)
case None => Future.failed(new RuntimeException("tx not found"))
case None => Future.failed(new RuntimeException(s"txid=$txId not found"))
}
}

View file

@ -492,7 +492,8 @@ object CommitmentsSpec {
Commitments(
ChannelParams(randomBytes32(), ChannelConfig.standard, ChannelFeatures(), localParams, remoteParams, ChannelFlags(announceChannel = announceChannel)),
CommitmentChanges(LocalChanges(Nil, Nil, Nil), RemoteChanges(Nil, Nil, Nil), localNextHtlcId = 1, remoteNextHtlcId = 1),
List(Commitment(LocalFundingStatus.SingleFundedUnconfirmedFundingTx(None), RemoteFundingStatus.Locked, localCommit, remoteCommit, None)),
List(Commitment(0, LocalFundingStatus.SingleFundedUnconfirmedFundingTx(None), RemoteFundingStatus.Locked, localCommit, remoteCommit, None)),
inactive = Nil,
Right(randomKey().publicKey),
ShaChain.init,
Map.empty,
@ -509,7 +510,8 @@ object CommitmentsSpec {
Commitments(
ChannelParams(randomBytes32(), ChannelConfig.standard, ChannelFeatures(), localParams, remoteParams, ChannelFlags(announceChannel = announceChannel)),
CommitmentChanges(LocalChanges(Nil, Nil, Nil), RemoteChanges(Nil, Nil, Nil), localNextHtlcId = 1, remoteNextHtlcId = 1),
List(Commitment(LocalFundingStatus.SingleFundedUnconfirmedFundingTx(None), RemoteFundingStatus.Locked, localCommit, remoteCommit, None)),
List(Commitment(0, LocalFundingStatus.SingleFundedUnconfirmedFundingTx(None), RemoteFundingStatus.Locked, localCommit, remoteCommit, None)),
inactive = Nil,
Right(randomKey().publicKey),
ShaChain.init,
Map.empty,

View file

@ -19,7 +19,6 @@ package fr.acinq.eclair.channel
import akka.testkit.{TestFSMRef, TestProbe}
import com.softwaremill.quicklens.{ModifyPimp, QuicklensAt}
import fr.acinq.bitcoin.scalacompat._
import fr.acinq.eclair.TestConstants.Alice.nodeParams
import fr.acinq.eclair.TestUtils.NoLoggingDiagnostics
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.WatchFundingSpentTriggered
import fr.acinq.eclair.channel.Helpers.Closing
@ -27,7 +26,7 @@ import fr.acinq.eclair.channel.fsm.Channel
import fr.acinq.eclair.channel.states.{ChannelStateTestsBase, ChannelStateTestsTags}
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.wire.protocol.UpdateAddHtlc
import fr.acinq.eclair.{BlockHeight, FeatureSupport, Features, MilliSatoshiLong, TestKitBaseClass, TimestampSecond, TimestampSecondLong, randomKey}
import fr.acinq.eclair.{BlockHeight, MilliSatoshiLong, TestKitBaseClass, TimestampSecond, TimestampSecondLong, randomKey}
import org.scalatest.Tag
import org.scalatest.funsuite.AnyFunSuiteLike
import scodec.bits.HexStringSyntax
@ -39,15 +38,14 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat
implicit val log: akka.event.LoggingAdapter = akka.event.NoLogging
test("compute the funding tx min depth according to funding amount") {
assert(Helpers.Funding.minDepthFundee(nodeParams.channelConf, Features(), Btc(1)).contains(4))
assert(Helpers.Funding.minDepthFundee(nodeParams.channelConf.copy(minDepthBlocks = 6), Features(), Btc(1)).contains(6)) // 4 conf would be enough but we use min-depth=6
assert(Helpers.Funding.minDepthFundee(nodeParams.channelConf, Features(), Btc(6.25)).contains(16)) // we use scaling_factor=15 and a fixed block reward of 6.25BTC
assert(Helpers.Funding.minDepthFundee(nodeParams.channelConf, Features(), Btc(12.50)).contains(31))
assert(Helpers.Funding.minDepthFundee(nodeParams.channelConf, Features(), Btc(12.60)).contains(32))
assert(Helpers.Funding.minDepthFundee(nodeParams.channelConf, Features(), Btc(30)).contains(73))
assert(Helpers.Funding.minDepthFundee(nodeParams.channelConf, Features(), Btc(50)).contains(121))
assert(Helpers.Funding.minDepthFundee(nodeParams.channelConf, Features(Features.ZeroConf -> FeatureSupport.Optional), Btc(50)).isEmpty)
test("scale funding tx min depth according to funding amount") {
assert(ChannelParams.minDepthScaled(defaultMinDepth = 3, Btc(1)) == 4)
assert(ChannelParams.minDepthScaled(defaultMinDepth = 6, Btc(1)) == 6) // 4 conf would be enough but we use min-depth=6
assert(ChannelParams.minDepthScaled(defaultMinDepth = 3, Btc(6.25)) == 16) // we use scaling_factor=15 and a fixed block reward of 6.25BTC
assert(ChannelParams.minDepthScaled(defaultMinDepth = 3, Btc(12.5)) == 31)
assert(ChannelParams.minDepthScaled(defaultMinDepth = 3, Btc(12.6)) == 32)
assert(ChannelParams.minDepthScaled(defaultMinDepth = 3, Btc(30)) == 73)
assert(ChannelParams.minDepthScaled(defaultMinDepth = 3, Btc(50)) == 121)
}
test("compute refresh delay") {

View file

@ -2073,14 +2073,16 @@ class InteractiveTxBuilderSpec extends TestKitBaseClass with AnyFunSuiteLike wit
}
}
test("allow all output types") {
test("allow standard output types") {
val probe = TestProbe()
val wallet = new SingleKeyOnChainWallet()
val params = createFixtureParams(100_000 sat, 0 sat, FeeratePerKw(5000 sat), 330 sat, 0)
val testCases = Seq(
TxAddOutput(params.channelId, UInt64(1), 25_000 sat, Script.write(Script.pay2pkh(randomKey().publicKey))),
TxAddOutput(params.channelId, UInt64(1), 25_000 sat, Script.write(Script.pay2sh(OP_1 :: Nil))),
TxAddOutput(params.channelId, UInt64(1), 25_000 sat, Script.write(OP_1 :: Nil)),
TxAddOutput(params.channelId, UInt64(1), 25_000 sat, Script.write(Script.pay2wpkh(randomKey().publicKey))),
TxAddOutput(params.channelId, UInt64(1), 25_000 sat, Script.write(Script.pay2wsh(OP_1 :: Nil))),
TxAddOutput(params.channelId, UInt64(1), 25_000 sat, Script.write(Script.pay2tr(randomKey().xOnlyPublicKey()))),
)
testCases.foreach { output =>
val alice = params.spawnTxBuilderAlice(wallet)

View file

@ -52,6 +52,8 @@ object ChannelStateTestsTags {
val DisableWumbo = "disable_wumbo"
/** If set, channels will use option_dual_fund. */
val DualFunding = "dual_funding"
/** If set, peers will support splicing. */
val Splicing = "splicing"
/** If set, channels will use option_static_remotekey. */
val StaticRemoteKey = "static_remotekey"
/** If set, channels will use option_anchor_outputs. */
@ -184,6 +186,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
.modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.ZeroConf))(_.updated(Features.ZeroConf, FeatureSupport.Optional))
.modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.ScidAlias))(_.updated(Features.ScidAlias, FeatureSupport.Optional))
.modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.DualFunding))(_.updated(Features.DualFunding, FeatureSupport.Optional))
.modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.Splicing))(_.updated(Features.SplicePrototype, FeatureSupport.Optional))
.initFeatures()
val bobInitFeatures = Bob.nodeParams.features
.modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.DisableWumbo))(_.removed(Features.Wumbo))
@ -196,6 +199,7 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
.modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.ZeroConf))(_.updated(Features.ZeroConf, FeatureSupport.Optional))
.modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.ScidAlias))(_.updated(Features.ScidAlias, FeatureSupport.Optional))
.modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.DualFunding))(_.updated(Features.DualFunding, FeatureSupport.Optional))
.modify(_.activated).usingIf(tags.contains(ChannelStateTestsTags.Splicing))(_.updated(Features.SplicePrototype, FeatureSupport.Optional))
.initFeatures()
val channelType = ChannelTypes.defaultFromFeatures(aliceInitFeatures, bobInitFeatures, announceChannel = channelFlags.announceChannel)
@ -323,14 +327,16 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
bob2blockchain.expectMsgType[WatchPublished]
alice ! WatchPublishedTriggered(fundingTx)
bob ! WatchPublishedTriggered(fundingTx)
alice2blockchain.expectMsgType[WatchFundingConfirmed]
bob2blockchain.expectMsgType[WatchFundingConfirmed]
} else {
alice2blockchain.expectMsgType[WatchFundingConfirmed]
bob2blockchain.expectMsgType[WatchFundingConfirmed]
alice ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx)
bob ! WatchFundingConfirmedTriggered(BlockHeight(400000), 42, fundingTx)
alice2blockchain.expectMsgType[WatchFundingSpent]
bob2blockchain.expectMsgType[WatchFundingSpent]
}
alice2blockchain.expectMsgType[WatchFundingSpent]
bob2blockchain.expectMsgType[WatchFundingSpent]
alice2bob.expectMsgType[ChannelReady]
alice2bob.forward(bob)
bob2alice.expectMsgType[ChannelReady]
@ -429,12 +435,22 @@ trait ChannelStateTestsBase extends Assertions with Eventually {
val rHasChanges = r.stateData.asInstanceOf[ChannelDataWithCommitments].commitments.changes.localHasChanges
s ! CMD_SIGN(Some(sender.ref))
sender.expectMsgType[RES_SUCCESS[CMD_SIGN]]
s2r.expectMsgType[CommitSig]
s2r.forward(r)
var sigs2r = 0
var batchSize = 0
do {
val sig = s2r.expectMsgType[CommitSig]
s2r.forward(r)
sigs2r += 1
batchSize = sig.batchSize
} while (sigs2r < batchSize)
r2s.expectMsgType[RevokeAndAck]
r2s.forward(s)
r2s.expectMsgType[CommitSig]
r2s.forward(s)
var sigr2s = 0
do {
r2s.expectMsgType[CommitSig]
r2s.forward(s)
sigr2s += 1
} while (sigr2s < batchSize)
s2r.expectMsgType[RevokeAndAck]
s2r.forward(r)
if (rHasChanges) {

View file

@ -81,7 +81,7 @@ class WaitForAcceptDualFundedChannelStateSpec extends TestKitBaseClass with Fixt
val listener = TestProbe()
alice.underlyingActor.context.system.eventStream.subscribe(listener.ref, classOf[ChannelIdAssigned])
bob2alice.forward(alice, accept)
assert(listener.expectMsgType[ChannelIdAssigned].channelId == Helpers.computeChannelId(open, accept))
assert(listener.expectMsgType[ChannelIdAssigned].channelId == Helpers.computeChannelId(open.revocationBasepoint, accept.revocationBasepoint))
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CREATED)
aliceOpenReplyTo.expectNoMessage()

View file

@ -89,7 +89,7 @@ class WaitForOpenDualFundedChannelStateSpec extends TestKitBaseClass with Fixtur
val accept = bob2alice.expectMsgType[AcceptDualFundedChannel]
val channelIdAssigned = bobListener.expectMsgType[ChannelIdAssigned]
assert(channelIdAssigned.temporaryChannelId == ByteVector32.Zeroes)
assert(channelIdAssigned.channelId == Helpers.computeChannelId(open, accept))
assert(channelIdAssigned.channelId == Helpers.computeChannelId(open.revocationBasepoint, accept.revocationBasepoint))
assert(!accept.requireConfirmedInputs)
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CREATED)

View file

@ -260,10 +260,11 @@ class WaitForChannelReadyStateSpec extends TestKitBaseClass with FixtureAnyFunSu
awaitCond(alice.stateName == CLOSING)
}
test("recv WatchFundingSpentTriggered (other commit)") { f =>
test("recv WatchFundingSpentTriggered (unrecognized commit)") { f =>
import f._
alice ! WatchFundingSpentTriggered(Transaction(0, Nil, Nil, 0))
awaitCond(alice.stateName == ERR_INFORMATION_LEAK)
alice2blockchain.expectNoMessage(100 millis)
assert(alice.stateName == WAIT_FOR_CHANNEL_READY)
}
test("recv Error") { f =>

View file

@ -270,14 +270,17 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
assert(alice2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx1.txid)
alice2bob.expectMsgType[ChannelReady]
awaitCond(alice2.stateName == WAIT_FOR_DUAL_FUNDING_READY)
assert(alice2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.active.size == 1)
assert(alice2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.inactive.isEmpty)
assert(alice2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.latest.fundingTxId == fundingTx1.txid)
bob2 ! WatchFundingConfirmedTriggered(BlockHeight(42000), 42, fundingTx1)
assert(bobListener.expectMsgType[TransactionConfirmed].tx == fundingTx1)
assert(bob2blockchain.expectMsgType[WatchFundingSpent].txId == fundingTx1.txid)
bob2alice.expectMsgType[ChannelReady]
awaitCond(bob2.stateName == WAIT_FOR_DUAL_FUNDING_READY)
assert(alice2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.latest.fundingTxId == fundingTx1.txid)
assert(bob2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.active.size == 1)
assert(bob2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.inactive.isEmpty)
assert(bob2.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_READY].commitments.latest.fundingTxId == fundingTx1.txid)
}
@ -614,7 +617,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
// Bob broadcasts his commit tx.
alice ! WatchFundingSpentTriggered(bobCommitTx1)
aliceListener.expectMsgType[TransactionPublished]
assert(aliceListener.expectMsgType[TransactionPublished].tx.txid == bobCommitTx1.txid)
val claimMain = alice2blockchain.expectMsgType[TxPublisher.PublishFinalTx]
assert(claimMain.input.txid == bobCommitTx1.txid)
assert(alice2blockchain.expectMsgType[WatchTxConfirmed].txId == bobCommitTx1.txid)
@ -688,10 +691,11 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
awaitCond(bob2.stateName == CLOSING)
}
test("recv WatchFundingSpentTriggered (other commit)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
test("recv WatchFundingSpentTriggered (unrecognized commit)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
import f._
alice ! WatchFundingSpentTriggered(Transaction(0, Nil, Nil, 0))
awaitCond(alice.stateName == ERR_INFORMATION_LEAK)
alice2blockchain.expectNoMessage(100 millis)
assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
}
test("recv INPUT_DISCONNECTED (unsigned rbf attempt)", Tag(ChannelStateTestsTags.DualFunding)) { f =>

View file

@ -204,11 +204,12 @@ class WaitForDualFundingReadyStateSpec extends TestKitBaseClass with FixtureAnyF
awaitCond(alice.stateName == CLOSING)
}
test("recv WatchFundingSpentTriggered (other commit)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
test("recv WatchFundingSpentTriggered (unrecognized commit)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._
alice2bob.expectMsgType[ChannelReady]
alice ! WatchFundingSpentTriggered(Transaction(0, Nil, Nil, 0))
awaitCond(alice.stateName == ERR_INFORMATION_LEAK)
alice2blockchain.expectNoMessage(100 millis)
assert(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY)
}
test("recv Error", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>

View file

@ -242,10 +242,11 @@ class WaitForFundingConfirmedStateSpec extends TestKitBaseClass with FixtureAnyF
awaitCond(alice.stateName == CLOSING)
}
test("recv WatchFundingSpentTriggered (other commit)") { f =>
test("recv WatchFundingSpentTriggered (unrecognized commit)") { f =>
import f._
alice ! WatchFundingSpentTriggered(Transaction(0, Nil, Nil, 0))
awaitCond(alice.stateName == ERR_INFORMATION_LEAK)
alice2blockchain.expectNoMessage(100 millis)
assert(alice.stateName == WAIT_FOR_FUNDING_CONFIRMED)
}
test("recv Error") { f =>

View file

@ -3266,10 +3266,11 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(addSettled.htlc == htlc3)
}
test("recv WatchFundingSpentTriggered (other commit)") { f =>
test("recv WatchFundingSpentTriggered (unrecognized commit)") { f =>
import f._
alice ! WatchFundingSpentTriggered(Transaction(0, Nil, Nil, 0))
awaitCond(alice.stateName == ERR_INFORMATION_LEAK)
alice2blockchain.expectNoMessage(100 millis)
assert(alice.stateName == NORMAL)
}
test("recv Error") { f =>

View file

@ -409,10 +409,8 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
// alice then finds out bob is lying
bob2alice.send(alice, invalidReestablish)
val error = alice2bob.expectMsgType[Error]
assert(alice2blockchain.expectMsgType[PublishFinalTx].tx.txid == aliceCommitTx.txid)
val claimMainOutput = alice2blockchain.expectMsgType[PublishFinalTx].tx
Transaction.correctlySpends(claimMainOutput, aliceCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
assert(error == Error(channelId(alice), InvalidRevokedCommitProof(channelId(alice), 0, 42, invalidReestablish.yourLastPerCommitmentSecret).getMessage))
assert(error == Error(channelId(alice), PleasePublishYourCommitment(channelId(alice)).getMessage))
awaitCond(alice.stateName == WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT)
}
test("change relay fee while offline", Tag(IgnoreChannelUpdates)) { f =>
@ -799,10 +797,11 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
channelUpdateListener.expectMsgType[LocalChannelUpdate]
}
test("recv WatchFundingSpentTriggered (other commit)") { f =>
test("recv WatchFundingSpentTriggered (unrecognized commit)") { f =>
import f._
alice ! WatchFundingSpentTriggered(Transaction(0, Nil, Nil, 0))
awaitCond(alice.stateName == ERR_INFORMATION_LEAK)
alice2blockchain.expectNoMessage(100 millis)
assert(alice.stateName == NORMAL)
}
def disconnect(alice: TestFSMRef[ChannelState, ChannelData, Channel], bob: TestFSMRef[ChannelState, ChannelData, Channel]): Unit = {

View file

@ -903,10 +903,11 @@ class ShutdownStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike wit
alice2blockchain.expectNoMessage(1 second)
}
test("recv WatchFundingSpentTriggered (other commit)") { f =>
test("recv WatchFundingSpentTriggered (unrecognized commit)") { f =>
import f._
alice ! WatchFundingSpentTriggered(Transaction(0, Nil, Nil, 0))
awaitCond(alice.stateName == ERR_INFORMATION_LEAK)
alice2blockchain.expectNoMessage(100 millis)
assert(alice.stateName == SHUTDOWN)
}
test("recv Error") { f =>

View file

@ -248,7 +248,7 @@ class NegotiatingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
// alice initiates the negotiation with a very low feerate
val aliceCloseSig = alice2bob.expectMsgType[ClosingSigned]
assert(aliceCloseSig.feeSatoshis == 1685.sat)
assert(aliceCloseSig.feeRange_opt == Some(FeeRange(1348 sat, 2022 sat)))
assert(aliceCloseSig.feeRange_opt.contains(FeeRange(1348 sat, 2022 sat)))
alice2bob.forward(bob)
// bob chooses alice's highest fee
val bobCloseSig = bob2alice.expectMsgType[ClosingSigned]
@ -541,10 +541,12 @@ class NegotiatingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
awaitCond(bob.stateName == CLOSING)
}
test("recv WatchFundingSpentTriggered (other commit)") { f =>
test("recv WatchFundingSpentTriggered (unrecognized commit)") { f =>
import f._
bobClose(f)
alice ! WatchFundingSpentTriggered(Transaction(0, Nil, Nil, 0))
awaitCond(alice.stateName == ERR_INFORMATION_LEAK)
alice2blockchain.expectNoMessage(100 millis)
assert(alice.stateName == NEGOTIATING)
}
test("recv Error") { f =>

View file

@ -1638,10 +1638,12 @@ class ClosingStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
assert(new String(error.data.toArray) == FundingTxSpent(channelId(alice), initialState.spendingTxs.head.txid).getMessage)
}
test("recv WatchFundingSpentTriggered (other commit)") { f =>
test("recv WatchFundingSpentTriggered (unrecognized commit)") { f =>
import f._
mutualClose(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain)
alice ! WatchFundingSpentTriggered(Transaction(0, Nil, Nil, 0))
awaitCond(alice.stateName == ERR_INFORMATION_LEAK)
alice2blockchain.expectNoMessage(100 millis)
assert(alice.stateName == CLOSING)
}
test("recv CMD_CLOSE") { f =>

View file

@ -75,7 +75,7 @@ class PendingChannelsRateLimiterSpec extends ScalaTestWithActorTestKit(ConfigFac
val peerBelowLimit2 = randomKey().publicKey
val channelsBelowLimit2 = Seq(
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(peerBelowLimit2, channelIdBelowLimit2), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)),
DATA_NORMAL(commitments(peerBelowLimit2, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None), None, null, None, None, None),
DATA_NORMAL(commitments(peerBelowLimit2, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None), None, null, None, None, None, SpliceStatus.NoSplice),
DATA_SHUTDOWN(commitments(peerBelowLimit2, randomBytes32()), Shutdown(randomBytes32(), ByteVector.empty), Shutdown(randomBytes32(), ByteVector.empty), None),
DATA_CLOSING(commitments(peerBelowLimit2, randomBytes32()), BlockHeight(0), ByteVector.empty, List(), List(closingTx))
)
@ -83,7 +83,7 @@ class PendingChannelsRateLimiterSpec extends ScalaTestWithActorTestKit(ConfigFac
val privatePeer2 = randomKey().publicKey
val privateChannels = Seq(
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(privatePeer1, channelIdPrivate1), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)),
DATA_NORMAL(commitments(privatePeer2, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None), None, null, None, None, None),
DATA_NORMAL(commitments(privatePeer2, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None), None, null, None, None, None, SpliceStatus.NoSplice),
)
val publicChannels = channelsOnWhitelistAtLimit ++ channelsAtLimit1 ++ channelsAtLimit2 ++ channelsBelowLimit1 ++ channelsBelowLimit2
val publicPeers = publicChannels.map(_.commitments.remoteNodeId).toSet
@ -291,7 +291,7 @@ class PendingChannelsRateLimiterSpec extends ScalaTestWithActorTestKit(ConfigFac
val channels = Seq(
DATA_WAIT_FOR_CHANNEL_READY(commitments(randomKey().publicKey, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)),
DATA_WAIT_FOR_DUAL_FUNDING_READY(commitments(randomKey().publicKey, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None)),
DATA_NORMAL(commitments(randomKey().publicKey, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None), None, null, None, None, None),
DATA_NORMAL(commitments(randomKey().publicKey, randomBytes32()), ShortIds(RealScidStatus.Unknown, ShortChannelId.generateLocalAlias(), None), None, null, None, None, None, SpliceStatus.NoSplice),
DATA_SHUTDOWN(commitments(randomKey().publicKey, randomBytes32()), Shutdown(randomBytes32(), ByteVector.empty), Shutdown(randomBytes32(), ByteVector.empty), None),
DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments(randomKey().publicKey, randomBytes32()), BlockHeight(0), None, Left(FundingCreated(randomBytes32(), ByteVector32.Zeroes, 3, randomBytes64()))),
)

View file

@ -711,7 +711,8 @@ object PaymentPacketSpec {
new Commitments(
ChannelParams(channelId, ChannelConfig.standard, channelFeatures, localParams, remoteParams, channelFlags),
CommitmentChanges(localChanges, remoteChanges, 0, 0),
List(Commitment(LocalFundingStatus.SingleFundedUnconfirmedFundingTx(None), RemoteFundingStatus.Locked, localCommit, remoteCommit, None)),
List(Commitment(0, LocalFundingStatus.SingleFundedUnconfirmedFundingTx(None), RemoteFundingStatus.Locked, localCommit, remoteCommit, None)),
inactive = Nil,
Right(randomKey().publicKey),
ShaChain.init,
Map.empty,

View file

@ -0,0 +1,46 @@
package fr.acinq.eclair.testutils
import akka.testkit.TestProbe
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi}
import fr.acinq.eclair.MilliSatoshi
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchFundingSpent, WatchPublished}
import fr.acinq.eclair.channel.AvailableBalanceChanged
import org.scalatest.Assertions
import scala.reflect.ClassTag
case class PimpTestProbe(probe: TestProbe) extends Assertions {
/**
* Generic method to perform validation on an expected message.
*
* @param asserts should contains asserts on the message
*/
def expectMsgTypeHaving[T](asserts: T => Unit)(implicit t: ClassTag[T]): T = {
val msg = probe.expectMsgType[T]
asserts(msg)
msg
}
def expectWatchFundingSpent(txid: ByteVector32): WatchFundingSpent =
expectMsgTypeHaving[WatchFundingSpent](w => assert(w.txId == txid, "txid"))
def expectWatchFundingConfirmed(txid: ByteVector32): WatchFundingConfirmed =
expectMsgTypeHaving[WatchFundingConfirmed](w => assert(w.txId == txid, "txid"))
def expectWatchPublished(txid: ByteVector32): WatchPublished =
expectMsgTypeHaving[WatchPublished](w => assert(w.txId == txid, "txid"))
def expectAvailableBalanceChanged(balance: MilliSatoshi, capacity: Satoshi): AvailableBalanceChanged =
expectMsgTypeHaving[AvailableBalanceChanged] { e =>
// NB: we check raw local balance, not availableBalanceForSend, because the latter is more difficult to compute
assert(e.commitments.active.map(_.localCommit.spec.toLocal).min == balance, "balance")
assert(e.commitments.active.map(_.capacity).min == capacity, "capacity")
}
}
object PimpTestProbe {
implicit def convert(probe: TestProbe): PimpTestProbe = PimpTestProbe(probe)
}

View file

@ -323,11 +323,11 @@ object ChannelCodecsSpec {
val commitments = Commitments(
ChannelParams(channelId, ChannelConfig.standard, ChannelFeatures(), localParams, remoteParams, channelFlags),
CommitmentChanges(LocalChanges(Nil, Nil, Nil), RemoteChanges(Nil, Nil, Nil), localNextHtlcId = 32, remoteNextHtlcId = 4),
Seq(Commitment(LocalFundingStatus.SingleFundedUnconfirmedFundingTx(None), RemoteFundingStatus.NotLocked, localCommit, remoteCommit, None)),
Seq(Commitment(fundingTxIndex = 0, LocalFundingStatus.SingleFundedUnconfirmedFundingTx(None), RemoteFundingStatus.NotLocked, localCommit, remoteCommit, None)),
remoteNextCommitInfo = Right(randomKey().publicKey),
remotePerCommitmentSecrets = ShaChain.init,
originChannels = origins)
DATA_NORMAL(commitments, ShortIds(RealScidStatus.Final(RealShortChannelId(42)), ShortChannelId.generateLocalAlias(), None), None, channelUpdate, None, None, None)
DATA_NORMAL(commitments, ShortIds(RealScidStatus.Final(RealShortChannelId(42)), ShortChannelId.generateLocalAlias(), None), None, channelUpdate, None, None, None, SpliceStatus.NoSplice)
}
}

View file

@ -136,6 +136,7 @@ class ChannelCodecs4Spec extends AnyFunSuite {
)
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)),
fundingTxIndex = 0,
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)

View file

@ -75,6 +75,21 @@ trait Channel {
}
}
val spliceIn: Route = postRequest("splicein") { implicit f =>
formFields(channelIdFormParam, "amountIn".as[Satoshi], "pushMsat".as[MilliSatoshi].?) {
(channelId, amountIn, pushMsat_opt) => complete(eclairApi.spliceIn(channelId, amountIn, pushMsat_opt))
}
}
val spliceOut: Route = postRequest("spliceout") { implicit f =>
formFields(channelIdFormParam, "amountOut".as[Satoshi], "scriptPubKey".as[ByteVector](bytesUnmarshaller)) {
(channelId, amountOut, scriptPubKey) => complete(eclairApi.spliceOut(channelId, amountOut, Left(scriptPubKey)))
} ~
formFields(channelIdFormParam, "amountOut".as[Satoshi], "address".as[String]) {
(channelId, amountOut, address) => complete(eclairApi.spliceOut(channelId, amountOut, Right(address)))
}
}
val close: Route = postRequest("close") { implicit t =>
withChannelsIdentifier { channels =>
formFields("scriptPubKey".as[ByteVector](bytesUnmarshaller).?, "preferredFeerateSatByte".as[FeeratePerByte].?, "minFeerateSatByte".as[FeeratePerByte].?, "maxFeerateSatByte".as[FeeratePerByte].?) {
@ -132,6 +147,6 @@ trait Channel {
complete(eclairApi.channelBalances())
}
val channelRoutes: Route = open ~ rbfOpen ~ close ~ forceClose ~ channel ~ channels ~ allChannels ~ allUpdates ~ channelStats ~ channelBalances
val channelRoutes: Route = open ~ rbfOpen ~ spliceIn ~ spliceOut ~ close ~ forceClose ~ channel ~ channels ~ allChannels ~ allUpdates ~ channelStats ~ channelBalances
}