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:
parent
daf947fb60
commit
de6d3c1709
61 changed files with 2448 additions and 463 deletions
|
@ -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" ;;
|
||||
|
|
|
@ -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]]]] = {
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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")
|
||||
}
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
|
@ -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 {
|
||||
|
|
|
@ -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.
|
||||
*
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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 = {
|
||||
|
|
|
@ -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 _ => ()
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}
|
||||
))
|
||||
|
||||
}
|
||||
|
|
|
@ -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] = (
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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] = (
|
||||
|
|
|
@ -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] = (
|
||||
|
|
|
@ -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) ::
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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) ::
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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)
|
||||
)
|
||||
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
//
|
||||
|
||||
//
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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" : { }
|
||||
|
|
|
@ -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" : { }
|
||||
|
|
|
@ -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" : { }
|
||||
|
|
|
@ -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" : { }
|
||||
|
|
|
@ -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" : { }
|
||||
|
|
|
@ -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" : { }
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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"))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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") {
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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 =>
|
||||
|
|
|
@ -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 =>
|
||||
|
|
|
@ -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 =>
|
||||
|
|
|
@ -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 =>
|
||||
|
|
File diff suppressed because it is too large
Load diff
|
@ -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 =>
|
||||
|
|
|
@ -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 = {
|
||||
|
|
|
@ -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 =>
|
||||
|
|
|
@ -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 =>
|
||||
|
|
|
@ -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 =>
|
||||
|
|
|
@ -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()))),
|
||||
)
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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)
|
||||
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
||||
}
|
||||
|
|
Loading…
Add table
Reference in a new issue