mirror of
https://github.com/ACINQ/eclair.git
synced 2025-01-18 05:12:34 +01:00
Fix splicing issues
Upon re-connection, when a splice has not been fully completed, nodes will re-send signatures for the previous remote commit tx. This signature will be ignored by the receiving nodes if it has already received it before it was disconnected, simply by comparing them (signatures are deterministic). With taproot channels, we also need to attach musig2 nonces for splices in progress to channel_reestablish, which are needed to re-generate the signature for the old commit tx.
This commit is contained in:
parent
c0dd17a303
commit
7eff6e53b5
@ -246,7 +246,7 @@ case class LocalCommit(index: Long, spec: CommitmentSpec, commitTxAndRemoteSig:
|
||||
object LocalCommit {
|
||||
def fromCommitSig(keyManager: ChannelKeyManager, params: ChannelParams, fundingTxId: TxId,
|
||||
fundingTxIndex: Long, remoteFundingPubKey: PublicKey, commitInput: InputInfo,
|
||||
commit: CommitSig, localCommitIndex: Long, spec: CommitmentSpec, localPerCommitmentPoint: PublicKey, localNonce_opt: Option[(SecretNonce, IndividualNonce)]): Either[ChannelException, LocalCommit] = {
|
||||
commit: CommitSig, localCommitIndex: Long, spec: CommitmentSpec, localPerCommitmentPoint: PublicKey, localNonce_opt: Option[(SecretNonce, IndividualNonce)])(implicit log: LoggingAdapter): Either[ChannelException, LocalCommit] = {
|
||||
val (localCommitTx, htlcTxs) = Commitment.makeLocalTxs(keyManager, params.channelConfig, params.channelFeatures, localCommitIndex, params.localParams, params.remoteParams, fundingTxIndex, remoteFundingPubKey, commitInput, localPerCommitmentPoint, spec)
|
||||
if (!localCommitTx.checkSig(commit, remoteFundingPubKey, TxOwner.Remote, params.commitmentFormat)) {
|
||||
return Left(InvalidCommitmentSignature(params.channelId, fundingTxId, fundingTxIndex, localCommitTx.tx))
|
||||
@ -260,6 +260,10 @@ object LocalCommit {
|
||||
val fundingPubkey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex).publicKey
|
||||
val Some(localNonce) = localNonce_opt
|
||||
if (!localCommitTx.checkPartialSignature(psig, fundingPubkey, localNonce._2, remoteFundingPubKey)) {
|
||||
log.debug(s"fromCommitSig: invalid partial signature $psig fundingPubkey = $fundingPubkey, fundingTxIndex = $fundingTxIndex localCommitIndex = $localCommitIndex localNonce = $localNonce remoteFundingPubKey = $remoteFundingPubKey")
|
||||
|
||||
val localNonce1 = keyManager.verificationNonce(params.localParams.fundingKeyPath, fundingTxIndex, keyManager.keyPath(params.localParams, params.channelConfig), localCommitIndex)
|
||||
log.debug(s"with $localNonce1 ${localCommitTx.checkPartialSignature(psig, fundingPubkey, localNonce1._2, remoteFundingPubKey)}")
|
||||
return Left(InvalidCommitmentSignature(params.channelId, fundingTxId, fundingTxIndex, localCommitTx.tx))
|
||||
}
|
||||
}
|
||||
@ -288,9 +292,10 @@ case class RemoteCommit(index: Long, spec: CommitmentSpec, txid: TxId, remotePer
|
||||
def sign(keyManager: ChannelKeyManager, params: ChannelParams, fundingTxIndex: Long, remoteFundingPubKey: PublicKey, commitInput: InputInfo, remoteNonce_opt: Option[IndividualNonce])(implicit log: LoggingAdapter): CommitSig = {
|
||||
val (remoteCommitTx, htlcTxs) = Commitment.makeRemoteTxs(keyManager, params.channelConfig, params.channelFeatures, index, params.localParams, params.remoteParams, fundingTxIndex, remoteFundingPubKey, commitInput, remotePerCommitmentPoint, spec)
|
||||
val (sig, tlvStream) = if (params.commitmentFormat.useTaproot) {
|
||||
val localNonce = keyManager.verificationNonce(params.localParams.fundingKeyPath, fundingTxIndex, keyManager.keyPath(params.localParams, params.channelConfig), index)
|
||||
val localNonce = keyManager.signingNonce(params.localParams.fundingKeyPath, fundingTxIndex)
|
||||
val Some(remoteNonce) = remoteNonce_opt
|
||||
val Right(localPartialSigOfRemoteTx) = keyManager.partialSign(remoteCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex), remoteFundingPubKey, TxOwner.Remote, localNonce, remoteNonce)
|
||||
log.debug(s"RemoteCommit.sign localPartialSigOfRemoteTx = $localPartialSigOfRemoteTx fundingTxIndex = $fundingTxIndex remote commit index = $index remote nonce = $remoteNonce")
|
||||
val tlvStream: TlvStream[CommitSigTlv] = TlvStream(CommitSigTlv.PartialSignatureWithNonceTlv(PartialSignatureWithNonce(localPartialSigOfRemoteTx, localNonce._2)))
|
||||
(ByteVector64.Zeroes, tlvStream)
|
||||
} else {
|
||||
@ -682,7 +687,7 @@ case class Commitment(fundingTxIndex: Long,
|
||||
val localNonce = keyManager.signingNonce(params.localParams.fundingKeyPath, fundingTxIndex)
|
||||
val Some(remoteNonce) = nextRemoteNonce_opt
|
||||
val Right(psig) = keyManager.partialSign(remoteCommitTx, keyManager.fundingPublicKey(params.localParams.fundingKeyPath, fundingTxIndex), remoteFundingPubKey, TxOwner.Remote, localNonce, remoteNonce)
|
||||
log.debug(s"sendCommit: creating partial sig $psig for remote commit tx ${remoteCommitTx.tx.txid} with remote nonce $remoteNonce and remoteNextPerCommitmentPoint = $remoteNextPerCommitmentPoint")
|
||||
log.debug(s"sendCommit: creating partial sig $psig for remote commit tx ${remoteCommitTx.tx.txid} with fundingTxIndex = $fundingTxIndex remoteCommit.index (should add +1) = ${remoteCommit.index} remote nonce $remoteNonce and remoteNextPerCommitmentPoint = $remoteNextPerCommitmentPoint")
|
||||
Set(CommitSigTlv.PartialSignatureWithNonceTlv(PartialSignatureWithNonce(psig, localNonce._2)))
|
||||
} else {
|
||||
Set.empty
|
||||
@ -1104,6 +1109,10 @@ case class Commitments(params: ChannelParams,
|
||||
}
|
||||
val channelKeyPath = keyManager.keyPath(params.localParams, params.channelConfig)
|
||||
val localPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, localCommitIndex + 1)
|
||||
|
||||
val fundingIndexes = active.map(_.fundingTxIndex).toSet
|
||||
if (fundingIndexes.size > 1) log.warning(s"more than 1 funding tx index")
|
||||
|
||||
// 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) =>
|
||||
val localNonce_opt = if (params.commitmentFormat.useTaproot) {
|
||||
@ -1250,12 +1259,20 @@ case class Commitments(params: ChannelParams,
|
||||
}
|
||||
|
||||
/** This function should be used to ignore a commit_sig that we've already received. */
|
||||
def ignoreRetransmittedCommitSig(commitSig: CommitSig): Boolean = {
|
||||
val latestRemoteSigOrPartialSig = latest.localCommit.commitTxAndRemoteSig.remoteSig match {
|
||||
case RemoteSignature.FullSignature(sig) => Left(sig)
|
||||
case _: RemoteSignature.PartialSignatureWithNonce => Right(latest.localCommit.commitTxAndRemoteSig.remoteSig)
|
||||
}
|
||||
params.channelFeatures.hasFeature(Features.DualFunding) && commitSig.batchSize == 1 && latestRemoteSigOrPartialSig == commitSig.sigOrPartialSig
|
||||
def ignoreRetransmittedCommitSig(commitSig: CommitSig, keyManager: ChannelKeyManager): Boolean = commitSig.sigOrPartialSig match {
|
||||
case _ if !params.channelFeatures.hasFeature(Features.DualFunding) => false
|
||||
case _ if commitSig.batchSize != 1 => false
|
||||
case Left(sig) =>
|
||||
latest.localCommit.commitTxAndRemoteSig.remoteSig match {
|
||||
case f: RemoteSignature.FullSignature => f.sig == sig
|
||||
case _: RemoteSignature.PartialSignatureWithNonce => false
|
||||
}
|
||||
case Right(psig) =>
|
||||
// we cannot compare partial signatures directly as they are not deterministic (a new signing nonce is used every time a signature is computed)
|
||||
// => instead we simply check that the provided partial signature is valid for our latest commit tx
|
||||
val localFundingKey = keyManager.fundingPublicKey(params.localParams.fundingKeyPath, latest.fundingTxIndex).publicKey
|
||||
val Some(localNonce) = generateLocalNonce(keyManager, latest.fundingTxIndex, latest.localCommit.index)
|
||||
latest.localCommit.commitTxAndRemoteSig.commitTx.checkPartialSignature(psig, localFundingKey, localNonce, latest.remoteFundingPubKey)
|
||||
}
|
||||
|
||||
def localFundingSigs(fundingTxId: TxId): Option[TxSignatures] = {
|
||||
@ -1375,32 +1392,33 @@ case class Commitments(params: ChannelParams,
|
||||
}
|
||||
|
||||
/**
|
||||
* Create local verification nonces for the next funding tx
|
||||
* Generate local verification nonces for a specific funding tx index and commit tx index
|
||||
*
|
||||
* @param keyManager key manager that will generate actual nonces
|
||||
* @return a list of 2 verification nonces for the next funding tx: one for the current commitment index, one for the next commitment index
|
||||
* @param keyManager key manager that will generate actual nonces
|
||||
* @param fundingIndex funding tx index
|
||||
* @param commitIndex commit tx index
|
||||
* @return a public nonce for thr provided fundint tx index and commit tx index if taproot is used, None otherwise
|
||||
*/
|
||||
def generateLocalNonces(keyManager: ChannelKeyManager, fundingIndex: Long): List[IndividualNonce] = {
|
||||
def generateLocalNonce(keyManager: ChannelKeyManager, fundingIndex: Long, commitIndex: Long): Option[IndividualNonce] = {
|
||||
if (latest.params.commitmentFormat.useTaproot) {
|
||||
|
||||
def localNonce(commitIndex: Long) = {
|
||||
val (_, nonce) = keyManager.verificationNonce(params.localParams.fundingKeyPath, fundingIndex, keyManager.keyPath(params.localParams, params.channelConfig), commitIndex)
|
||||
nonce
|
||||
}
|
||||
|
||||
List(localNonce(localCommitIndex), localNonce(localCommitIndex + 1))
|
||||
Some(keyManager.verificationNonce(params.localParams.fundingKeyPath, fundingIndex, keyManager.keyPath(params.localParams, params.channelConfig), commitIndex)._2)
|
||||
} else {
|
||||
List.empty
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create local verification nonces for the next funding tx
|
||||
* Create local verification nonces a specific funding tx index and a range of commit tx indexes
|
||||
*
|
||||
* @param keyManager key manager that will generate actual nonces
|
||||
* @return a list of 2 verification nonces for the next funding tx: one for the current commitment index, one for the next commitment index
|
||||
* @param fundingIndex funding tx index
|
||||
* @param commitIndexes range of commit tx indexes
|
||||
* @return a list of nonces if raproot is used, or an empty list
|
||||
*/
|
||||
def generateLocalNonces(keyManager: ChannelKeyManager): List[IndividualNonce] = generateLocalNonces(keyManager, latest.commitment.fundingTxIndex + 1)
|
||||
def generateLocalNonces(keyManager: ChannelKeyManager, fundingIndex: Long, commitIndexes: Long*): List[IndividualNonce] = {
|
||||
commitIndexes.toList.flatMap(commitIndex => generateLocalNonce(keyManager, fundingIndex, commitIndex))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
object Commitments {
|
||||
|
@ -204,8 +204,15 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
import Channel._
|
||||
|
||||
val keyManager: ChannelKeyManager = nodeParams.channelKeyManager
|
||||
|
||||
// remote nonces, one for each active commitment, with the same ordering
|
||||
var remoteNextLocalNonces: List[IndividualNonce] = List.empty
|
||||
var pendingRemoteNextLocalNonce: Option[IndividualNonce] = None // will be added to remoteNextLocalNonces once a splice has been completed
|
||||
|
||||
// // will be added to remoteNextLocalNonces once a splice has been completed
|
||||
var pendingRemoteNextLocalNonce: Option[IndividualNonce] = None
|
||||
|
||||
// remote nonce for the current splice in progress
|
||||
var spliceRemoteNonce: Option[IndividualNonce] = None
|
||||
|
||||
def setRemoteNextLocalNonces(info: String, n: List[IndividualNonce]): Unit = {
|
||||
this.remoteNextLocalNonces = n
|
||||
@ -600,7 +607,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
stay() using d1 storing() sending signingSession1.localSigs calling endQuiescence(d1)
|
||||
}
|
||||
}
|
||||
case _ if d.commitments.ignoreRetransmittedCommitSig(commit) =>
|
||||
case _ if d.commitments.ignoreRetransmittedCommitSig(commit, keyManager) =>
|
||||
// We haven't received our peer's tx_signatures for the latest funding transaction and asked them to resend it on reconnection.
|
||||
// They also resend their corresponding commit_sig, but we have already received it so we should ignore it.
|
||||
// Note that the funding transaction may have confirmed while we were reconnecting.
|
||||
@ -610,6 +617,10 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
// 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
|
||||
if (commit.batchSize == 1 && sigs.size < d.commitments.active.size) {
|
||||
log.warning(s"invalid sig count for $sigs\n old commit sig is ${d.commitments.latest.localCommit.commitTxAndRemoteSig.remoteSig} ")
|
||||
}
|
||||
|
||||
d.commitments.receiveCommit(sigs, keyManager) match {
|
||||
case Right((commitments1, revocation)) =>
|
||||
log.debug("received a new sig, spec:\n{}", commitments1.latest.specs2String)
|
||||
@ -1033,11 +1044,14 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
} else if (d.commitments.latest.localFundingStatus.isInstanceOf[LocalFundingStatus.DualFundedUnconfirmedFundingTx]) {
|
||||
log.info("rejecting splice request: the previous funding transaction is unconfirmed (txId={})", d.commitments.latest.fundingTxId)
|
||||
stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidSpliceWithUnconfirmedTx(d.channelId, d.commitments.latest.fundingTxId).getMessage)
|
||||
} else if (d.commitments.latest.params.commitmentFormat.useTaproot && msg.nexLocalNonces.isEmpty) {
|
||||
log.info("rejecting splice request: missing musig2 nonces)")
|
||||
stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, MissingNextLocalNonce(d.channelId).getMessage)
|
||||
} else {
|
||||
val parentCommitment = d.commitments.latest.commitment
|
||||
val localFundingPubKey = nodeParams.channelKeyManager.fundingPublicKey(d.commitments.params.localParams.fundingKeyPath, parentCommitment.fundingTxIndex + 1).publicKey
|
||||
val fundingScript = Funding.makeFundingPubKeyScript(localFundingPubKey, msg.fundingPubKey, d.commitments.latest.params.commitmentFormat)
|
||||
val nextLocalNonces = d.commitments.generateLocalNonces(keyManager)
|
||||
val nextLocalNonces = d.commitments.generateLocalNonces(keyManager, parentCommitment.fundingTxIndex + 1, parentCommitment.localCommit.index, parentCommitment.localCommit.index + 1)
|
||||
val sharedInput = if (d.commitments.latest.params.commitmentFormat.useTaproot) {
|
||||
Musig2Input(parentCommitment)
|
||||
} else {
|
||||
@ -1083,6 +1097,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
wallet,
|
||||
msg.firstRemoteNonce
|
||||
))
|
||||
log.info(s"received SpliceInit with msg.firstRemoteNonce = ${msg.firstRemoteNonce} and msg.secondRemoteNonce = ${msg.secondRemoteNonce}")
|
||||
txBuilder ! InteractiveTxBuilder.Start(self)
|
||||
// README: the splice_init message contains the remote musig2 nonce for the next commit tx that will be built in the interactive tx session
|
||||
log.debug(s"updating pendingRemoteNextLocalNonce $pendingRemoteNextLocalNonce with ${msg.secondRemoteNonce}")
|
||||
@ -1142,6 +1157,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
wallet,
|
||||
msg.firstRemoteNonce
|
||||
))
|
||||
log.info(s"received SpliceInit with msg.firstRemoteNonce = ${msg.firstRemoteNonce} and msg.secondRemoteNonce = ${msg.secondRemoteNonce}")
|
||||
txBuilder ! InteractiveTxBuilder.Start(self)
|
||||
// README: the splice_ack message contains the remote musig2 nonce for the next commit tx that will be built in the interactive tx session
|
||||
log.debug(s"updating pendingRemoteNextLocalNonce $pendingRemoteNextLocalNonce with ${msg.secondRemoteNonce}")
|
||||
@ -1178,6 +1194,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
case Right(rbf) if nodeParams.currentBlockHeight < rbf.latestFundingTx.createdAt + nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks =>
|
||||
log.info("rejecting rbf attempt: last attempt was less than {} blocks ago", nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks)
|
||||
stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, InvalidRbfAttemptTooSoon(d.channelId, rbf.latestFundingTx.createdAt, rbf.latestFundingTx.createdAt + nodeParams.channelConf.remoteRbfLimits.attemptDeltaBlocks).getMessage)
|
||||
case Right(_) if d.commitments.latest.params.commitmentFormat.useTaproot && msg.nexLocalNonces.isEmpty =>
|
||||
log.info("rejecting rbf attempt: missing musig2 nonces")
|
||||
stay() using d.copy(spliceStatus = SpliceStatus.SpliceAborted) sending TxAbort(d.channelId, MissingNextLocalNonce(d.channelId).getMessage)
|
||||
case Right(rbf) =>
|
||||
val fundingScript = d.commitments.latest.commitInput.txOut.publicKeyScript
|
||||
LiquidityAds.validateRequest(nodeParams.privateKey, d.channelId, fundingScript, msg.feerate, isChannelCreation = false, msg.requestFunding_opt, nodeParams.willFundRates_opt, feeCreditUsed_opt = None) match {
|
||||
@ -1189,7 +1208,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
// Otherwise we keep the same contribution we made to the previous funding transaction.
|
||||
val fundingContribution = willFund_opt.map(_.purchase.amount).getOrElse(rbf.latestFundingTx.fundingParams.localContribution)
|
||||
log.info("accepting rbf with remote.in.amount={} local.in.amount={}", msg.fundingContribution, fundingContribution)
|
||||
val localNonces = d.commitments.generateLocalNonces(keyManager, rbf.parentCommitment.fundingTxIndex + 1)
|
||||
val localNonces = d.commitments.generateLocalNonces(keyManager, rbf.parentCommitment.fundingTxIndex + 1, rbf.localCommitIndex, rbf.localCommitIndex + 1)
|
||||
val txAckRbf = TxAckRbf(d.channelId, fundingContribution, rbf.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote, willFund_opt.map(_.willFund), localNonces)
|
||||
val sharedInput = if (d.commitments.latest.params.commitmentFormat.useTaproot) {
|
||||
Musig2Input(rbf.parentCommitment)
|
||||
@ -2191,19 +2210,41 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
}
|
||||
case _ => Set.empty
|
||||
}
|
||||
val myNextLocalNonce = if (d.commitments.params.commitmentFormat.useTaproot) {
|
||||
val nonces = d.commitments.active.map(c => keyManager.verificationNonce(d.commitments.params.localParams.fundingKeyPath, c.fundingTxIndex, channelKeyPath, d.commitments.localCommitIndex + 1))
|
||||
Set(NextLocalNoncesTlv(nonces.map(_._2).toList))
|
||||
// for taproot channels, send a "next remote nonce" for each active commitment
|
||||
val myNextLocalNonces = if (d.commitments.params.commitmentFormat.useTaproot) {
|
||||
val nonces = d.commitments.active.map(c => keyManager.verificationNonce(d.commitments.params.localParams.fundingKeyPath, c.fundingTxIndex, channelKeyPath, d.commitments.localCommitIndex + 1)._2)
|
||||
Set(NextLocalNoncesTlv(nonces.toList))
|
||||
} else {
|
||||
Set.empty
|
||||
}
|
||||
|
||||
// if a splice is in progress, we also need to re-send nonces for it: we include a nonce for the current commit tx, and one for the next commit tx
|
||||
val mySpliceNonces = d match {
|
||||
case _ if !d.commitments.params.commitmentFormat.useTaproot => Set.empty
|
||||
case d: DATA_NORMAL => d.spliceStatus match {
|
||||
case w: SpliceStatus.SpliceWaitingForSigs if d.commitments.params.commitmentFormat.useTaproot =>
|
||||
val spliceNonces = List(
|
||||
keyManager.verificationNonce(d.commitments.params.localParams.fundingKeyPath, w.signingSession.fundingTxIndex, channelKeyPath, w.signingSession.localCommitIndex)._2,
|
||||
keyManager.verificationNonce(d.commitments.params.localParams.fundingKeyPath, w.signingSession.fundingTxIndex, channelKeyPath, w.signingSession.localCommitIndex + 1)._2,
|
||||
)
|
||||
Set(ChannelReestablishTlv.SpliceNoncesTlv(spliceNonces))
|
||||
case _ =>
|
||||
val spliceNonces = List(
|
||||
keyManager.verificationNonce(d.commitments.params.localParams.fundingKeyPath, d.commitments.latest.fundingTxIndex, channelKeyPath, d.commitments.localCommitIndex)._2,
|
||||
keyManager.verificationNonce(d.commitments.params.localParams.fundingKeyPath, d.commitments.latest.fundingTxIndex, channelKeyPath, d.commitments.localCommitIndex + 1)._2,
|
||||
)
|
||||
Set(ChannelReestablishTlv.SpliceNoncesTlv(spliceNonces))
|
||||
}
|
||||
case _ => Set.empty
|
||||
}
|
||||
|
||||
val channelReestablish = ChannelReestablish(
|
||||
channelId = d.channelId,
|
||||
nextLocalCommitmentNumber = d.commitments.localCommitIndex + 1,
|
||||
nextRemoteRevocationNumber = d.commitments.remoteCommitIndex,
|
||||
yourLastPerCommitmentSecret = PrivateKey(yourLastPerCommitmentSecret),
|
||||
myCurrentPerCommitmentPoint = myCurrentPerCommitmentPoint,
|
||||
tlvStream = TlvStream(rbfTlv ++ myNextLocalNonce)
|
||||
tlvStream = TlvStream(rbfTlv ++ myNextLocalNonces ++ mySpliceNonces)
|
||||
)
|
||||
// we update local/remote connection-local global/local features, we don't persist it right now
|
||||
val d1 = Helpers.updateFeatures(d, localInit, remoteInit)
|
||||
@ -2300,10 +2341,26 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
goto(WAIT_FOR_DUAL_FUNDING_READY) sending channelReady
|
||||
|
||||
case Event(channelReestablish: ChannelReestablish, d: DATA_NORMAL) =>
|
||||
log.debug(s"received $channelReestablish")
|
||||
if (d.commitments.params.commitmentFormat.useTaproot) {
|
||||
require(channelReestablish.nextLocalNonces.size == d.commitments.active.size, "missing next local nonce")
|
||||
d.spliceStatus match {
|
||||
case _: SpliceStatus.SpliceWaitingForSigs if channelReestablish.nextLocalNonces.size == d.commitments.active.size =>
|
||||
require(channelReestablish.spliceNonces.size == 2, "missing splice nonces (splicing in progress")
|
||||
this.pendingRemoteNextLocalNonce = channelReestablish.secondSpliceNonce
|
||||
setRemoteNextLocalNonces(s"received ChannelReestablish (waiting for sigs)", channelReestablish.nextLocalNonces)
|
||||
case _: SpliceStatus.SpliceWaitingForSigs if channelReestablish.nextLocalNonces.size == d.commitments.active.size + 1 =>
|
||||
this.pendingRemoteNextLocalNonce = channelReestablish.nextLocalNonces.headOption
|
||||
setRemoteNextLocalNonces(s"received ChannelReestablish (waiting for sigs)", channelReestablish.nextLocalNonces.tail)
|
||||
case _ if channelReestablish.nextLocalNonces.size == d.commitments.active.size - 1 =>
|
||||
require(channelReestablish.spliceNonces.size == 2, "missing splice nonces")
|
||||
this.pendingRemoteNextLocalNonce = None
|
||||
setRemoteNextLocalNonces(s"received ChannelReestablish (with splice nonces)", channelReestablish.secondSpliceNonce.get :: channelReestablish.nextLocalNonces)
|
||||
case _ =>
|
||||
require(channelReestablish.nextLocalNonces.size >= d.commitments.active.size, "missing next local nonce")
|
||||
setRemoteNextLocalNonces("received ChannelReestablish", channelReestablish.nextLocalNonces)
|
||||
this.pendingRemoteNextLocalNonce = None
|
||||
}
|
||||
}
|
||||
setRemoteNextLocalNonces("received ChannelReestablish", channelReestablish.nextLocalNonces)
|
||||
|
||||
Syncing.checkSync(keyManager, d.commitments, channelReestablish) match {
|
||||
case syncFailure: SyncResult.Failure =>
|
||||
@ -2335,7 +2392,16 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
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("re-sending commit_sig for splice attempt with fundingTxIndex={} fundingTxId={}", signingSession.fundingTxIndex, signingSession.fundingTx.txId)
|
||||
val commitSig = signingSession.remoteCommit.sign(keyManager, d.commitments.params, signingSession.fundingTxIndex, signingSession.fundingParams.remoteFundingPubKey, signingSession.commitInput, remoteNextLocalNonces.headOption)
|
||||
// channel_reestablishes includes splice nonces for nextLocalCommitmentNumber - 1 and nextLocalCommitmentNumber
|
||||
val spliceNonce = if (signingSession.remoteCommit.index == channelReestablish.nextLocalCommitmentNumber) {
|
||||
channelReestablish.firstSpliceNonce
|
||||
} else if (signingSession.remoteCommit.index == channelReestablish.nextLocalCommitmentNumber - 1) {
|
||||
channelReestablish.firstSpliceNonce
|
||||
} else {
|
||||
// we should never end up here, it would have been handled in checkSync()
|
||||
throw new RuntimeException("invalid nextLocalCommitmentNumber in ChannelReestablish")
|
||||
}
|
||||
val commitSig = signingSession.remoteCommit.sign(keyManager, d.commitments.params, signingSession.fundingTxIndex, signingSession.fundingParams.remoteFundingPubKey, signingSession.commitInput, spliceNonce)
|
||||
sendQueue = sendQueue :+ commitSig
|
||||
d.spliceStatus
|
||||
case _ if d.commitments.latest.fundingTxId == fundingTxId =>
|
||||
@ -2345,7 +2411,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
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("re-sending commit_sig and tx_signatures for fundingTxIndex={} fundingTxId={}", d.commitments.latest.fundingTxIndex, d.commitments.latest.fundingTxId)
|
||||
val commitSig = d.commitments.latest.remoteCommit.sign(keyManager, d.commitments.params, d.commitments.latest.fundingTxIndex, d.commitments.latest.remoteFundingPubKey, d.commitments.latest.commitInput, remoteNextLocalNonces.headOption)
|
||||
val commitSig = d.commitments.latest.remoteCommit.sign(keyManager, d.commitments.params, d.commitments.latest.fundingTxIndex, d.commitments.latest.remoteFundingPubKey, d.commitments.latest.commitInput, channelReestablish.firstSpliceNonce)
|
||||
sendQueue = sendQueue :+ commitSig :+ fundingTx.localSigs
|
||||
case fundingTx: InteractiveTxBuilder.FullySignedSharedTransaction =>
|
||||
log.info("re-sending tx_signatures for fundingTxIndex={} fundingTxId={}", d.commitments.latest.fundingTxIndex, d.commitments.latest.fundingTxId)
|
||||
@ -3116,7 +3182,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
Left(InvalidSpliceRequest(d.channelId))
|
||||
} 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 nextLocalNonces = d.commitments.generateLocalNonces(keyManager)
|
||||
val nextLocalNonces = d.commitments.generateLocalNonces(keyManager, parentCommitment.fundingTxIndex + 1, parentCommitment.localCommit.index, parentCommitment.localCommit.index + 1)
|
||||
val spliceInit = SpliceInit(d.channelId,
|
||||
fundingContribution = fundingContribution,
|
||||
lockTime = nodeParams.currentBlockHeight.toLong,
|
||||
@ -3144,7 +3210,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
|
||||
log.warning(s"cannot do rbf: insufficient funds (commitTxFees=$commitTxFees reserve=${rbf.parentCommitment.localChannelReserve(d.commitments.params)})")
|
||||
Left(InvalidSpliceRequest(d.channelId))
|
||||
} else {
|
||||
val nextLocalNonces = d.commitments.generateLocalNonces(keyManager, rbf.parentCommitment.fundingTxIndex + 1)
|
||||
val nextLocalNonces = d.commitments.generateLocalNonces(keyManager, rbf.parentCommitment.fundingTxIndex + 1, rbf.localCommitIndex, rbf.localCommitIndex + 1)
|
||||
val txInitRbf = TxInitRbf(d.channelId, cmd.lockTime, cmd.targetFeerate, fundingContribution, rbf.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote, cmd.requestFunding_opt, nextLocalNonces)
|
||||
Right(txInitRbf)
|
||||
}
|
||||
|
@ -116,6 +116,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
||||
if (input.requireConfirmedInputs) Some(ChannelTlv.RequireConfirmedInputsTlv()) else None,
|
||||
input.requestFunding_opt.map(ChannelTlv.RequestFundingTlv),
|
||||
input.pushAmount_opt.map(amount => ChannelTlv.PushAmountTlv(amount)),
|
||||
// include our verification nonces at funding_index = 0, commit_index = 0 and funding_index = 0, commit_index = 1
|
||||
if (input.channelType.commitmentFormat.useTaproot) Some(ChannelTlv.NextLocalNoncesTlv(
|
||||
List(
|
||||
keyManager.verificationNonce(input.localParams.fundingKeyPath, fundingTxIndex = 0, channelKeyPath, 0)._2,
|
||||
@ -190,6 +191,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
||||
willFund_opt.map(l => ChannelTlv.ProvideFundingTlv(l.willFund)),
|
||||
open.useFeeCredit_opt.map(c => ChannelTlv.FeeCreditUsedTlv(c)),
|
||||
d.init.pushAmount_opt.map(amount => ChannelTlv.PushAmountTlv(amount)),
|
||||
// include our verification nonces at funding_index = 0, commit_index = 0 and funding_index = 0, commit_index = 1
|
||||
if (channelParams.commitmentFormat.useTaproot) Some(ChannelTlv.NextLocalNoncesTlv(
|
||||
List(
|
||||
keyManager.verificationNonce(localParams.fundingKeyPath, fundingTxIndex = 0, channelKeyPath, 0)._2,
|
||||
@ -531,7 +533,8 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
||||
cmd.replyTo ! RES_FAILURE(cmd, InvalidRbfFeerate(d.channelId, cmd.targetFeerate, minNextFeerate))
|
||||
stay()
|
||||
} else {
|
||||
val txInitRbf = TxInitRbf(d.channelId, cmd.lockTime, cmd.targetFeerate, d.latestFundingTx.fundingParams.localContribution, d.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote, cmd.requestFunding_opt, d.commitments.generateLocalNonces(keyManager))
|
||||
val localNonces = d.commitments.generateLocalNonces(keyManager, d.commitments.latest.fundingTxIndex, d.commitments.localCommitIndex, d.commitments.localCommitIndex + 1)
|
||||
val txInitRbf = TxInitRbf(d.channelId, cmd.lockTime, cmd.targetFeerate, d.latestFundingTx.fundingParams.localContribution, d.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote, cmd.requestFunding_opt, localNonces)
|
||||
stay() using d.copy(status = DualFundingStatus.RbfRequested(cmd)) sending txInitRbf
|
||||
}
|
||||
case _ =>
|
||||
@ -601,7 +604,7 @@ trait ChannelOpenDualFunded extends DualFundingHandlers with ErrorHandlers {
|
||||
wallet,
|
||||
msg.firstRemoteNonce))
|
||||
txBuilder ! InteractiveTxBuilder.Start(self)
|
||||
val nextLocalNonces = d.commitments.generateLocalNonces(keyManager)
|
||||
val nextLocalNonces = d.commitments.generateLocalNonces(keyManager, d.commitments.latest.fundingTxIndex, d.commitments.localCommitIndex, d.commitments.localCommitIndex + 1)
|
||||
setRemoteNextLocalNonces("received TxInitRbf", msg.secondRemoteNonce.toList)
|
||||
val toSend = Seq(
|
||||
Some(TxAckRbf(d.channelId, fundingParams.localContribution, d.latestFundingTx.fundingParams.requireConfirmedInputs.forRemote, willFund_opt.map(_.willFund), nextLocalNonces)),
|
||||
|
@ -81,11 +81,12 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
|
||||
// See https://github.com/lightningnetwork/lightning-rfc/pull/714.
|
||||
val localShutdownScript = input.localParams.upfrontShutdownScript_opt.getOrElse(ByteVector.empty)
|
||||
val tlvStream: TlvStream[OpenChannelTlv] = if (input.channelType.commitmentFormat.useTaproot) {
|
||||
val localNonce = keyManager.verificationNonce(input.localParams.fundingKeyPath, fundingTxIndex = 0, channelKeyPath, 0)
|
||||
// include our verification nonce at funding_index = 0, commit_index = 0
|
||||
val (_, localNonce) = keyManager.verificationNonce(input.localParams.fundingKeyPath, fundingTxIndex = 0, channelKeyPath, 0)
|
||||
TlvStream(
|
||||
ChannelTlv.UpfrontShutdownScriptTlv(localShutdownScript),
|
||||
ChannelTlv.ChannelTypeTlv(input.channelType),
|
||||
ChannelTlv.NextLocalNonceTlv(localNonce._2)
|
||||
ChannelTlv.NextLocalNonceTlv(localNonce)
|
||||
)
|
||||
} else {
|
||||
TlvStream(
|
||||
@ -146,6 +147,7 @@ trait ChannelOpenSingleFunded extends SingleFundingHandlers with ErrorHandlers {
|
||||
// See https://github.com/lightningnetwork/lightning-rfc/pull/714.
|
||||
val localShutdownScript = d.initFundee.localParams.upfrontShutdownScript_opt.getOrElse(ByteVector.empty)
|
||||
val tlvStream: TlvStream[AcceptChannelTlv] = if (d.initFundee.channelType.commitmentFormat.useTaproot) {
|
||||
// include our verification nonce at funding_index = 0, commit_index = 0
|
||||
val localNonce = keyManager.verificationNonce(d.initFundee.localParams.fundingKeyPath, fundingTxIndex = 0, channelKeyPath, 0)
|
||||
TlvStream(
|
||||
ChannelTlv.UpfrontShutdownScriptTlv(localShutdownScript),
|
||||
|
@ -476,6 +476,9 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
|
||||
|
||||
import InteractiveTxBuilder._
|
||||
|
||||
if (channelParams.commitmentFormat.useTaproot) {
|
||||
require(nextRemoteNonce_opt.isDefined, "next remote nonce is missing")
|
||||
}
|
||||
private val log = context.log
|
||||
private val keyManager = nodeParams.channelKeyManager
|
||||
private val localFundingPubKey: PublicKey = keyManager.fundingPublicKey(channelParams.localParams.fundingKeyPath, purpose.fundingTxIndex).publicKey
|
||||
|
@ -267,9 +267,16 @@ object ChannelReestablishTlv {
|
||||
val codec: Codec[NextFundingTlv] = tlvField(txIdAsHash)
|
||||
}
|
||||
|
||||
case class SpliceNoncesTlv(nonces: List[IndividualNonce]) extends ChannelReestablishTlv
|
||||
|
||||
object SpliceNoncesTlv {
|
||||
val codec: Codec[SpliceNoncesTlv] = tlvField(list(publicNonce))
|
||||
}
|
||||
|
||||
val channelReestablishTlvCodec: Codec[TlvStream[ChannelReestablishTlv]] = tlvStream(discriminated[ChannelReestablishTlv].by(varint)
|
||||
.typecase(UInt64(0), NextFundingTlv.codec)
|
||||
.typecase(UInt64(4), nexLocalNoncesTlvCodec)
|
||||
.typecase(UInt64(6), SpliceNoncesTlv.codec)
|
||||
)
|
||||
}
|
||||
|
||||
|
@ -213,6 +213,9 @@ case class ChannelReestablish(channelId: ByteVector32,
|
||||
tlvStream: TlvStream[ChannelReestablishTlv] = TlvStream.empty) extends ChannelMessage with HasChannelId {
|
||||
val nextFundingTxId_opt: Option[TxId] = tlvStream.get[ChannelReestablishTlv.NextFundingTlv].map(_.txId)
|
||||
val nextLocalNonces: List[IndividualNonce] = tlvStream.get[ChannelTlv.NextLocalNoncesTlv].map(_.nonces).getOrElse(List.empty)
|
||||
val spliceNonces: List[IndividualNonce] = tlvStream.get[ChannelReestablishTlv.SpliceNoncesTlv].map(_.nonces).getOrElse(List.empty)
|
||||
val firstSpliceNonce: Option[IndividualNonce] = if (spliceNonces.isEmpty) None else Some(spliceNonces(0))
|
||||
val secondSpliceNonce: Option[IndividualNonce] = if (spliceNonces.isEmpty) None else Some(spliceNonces(1))
|
||||
}
|
||||
|
||||
case class OpenChannel(chainHash: BlockHash,
|
||||
|
@ -42,7 +42,7 @@ import fr.acinq.eclair.payment.relay.Relayer
|
||||
import fr.acinq.eclair.testutils.PimpTestProbe.convert
|
||||
import fr.acinq.eclair.transactions.DirectedHtlc.{incoming, outgoing}
|
||||
import fr.acinq.eclair.transactions.Transactions
|
||||
import fr.acinq.eclair.transactions.Transactions.ClaimLocalAnchorOutputTx
|
||||
import fr.acinq.eclair.transactions.Transactions.{AnchorOutputsCommitmentFormat, ClaimLocalAnchorOutputTx}
|
||||
import fr.acinq.eclair.wire.protocol._
|
||||
import org.scalatest.Inside.inside
|
||||
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
|
||||
@ -367,21 +367,6 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.spec.toRemote == 700_000_000.msat)
|
||||
}
|
||||
|
||||
test("recv CMD_SPLICE (splice-in, simple taproot channels)", Tag(OptionSimpleTaprootStaging), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f =>
|
||||
import f._
|
||||
|
||||
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
|
||||
assert(initialState.commitments.latest.capacity == 1_500_000.sat)
|
||||
assert(initialState.commitments.latest.localCommit.spec.toLocal == 800_000_000.msat)
|
||||
assert(initialState.commitments.latest.localCommit.spec.toRemote == 700_000_000.msat)
|
||||
|
||||
initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)))
|
||||
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.capacity == 2_000_000.sat)
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.spec.toLocal == 1_300_000_000.msat)
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localCommit.spec.toRemote == 700_000_000.msat)
|
||||
}
|
||||
|
||||
test("recv CMD_SPLICE (splice-in, non dual-funded channel)") { () =>
|
||||
val f = init(tags = Set(ChannelStateTestsTags.DualFunding))
|
||||
import f._
|
||||
@ -633,7 +618,11 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
assert(commitment.localCommit.spec.toLocal == 650_000_000.msat)
|
||||
assert(commitment.localChannelReserve == 15_000.sat)
|
||||
val commitFees = Transactions.commitTxTotalCost(commitment.remoteParams.dustLimit, commitment.remoteCommit.spec, commitment.params.commitmentFormat)
|
||||
assert(commitFees > 20_000.sat)
|
||||
if (commitment.params.commitmentFormat.useTaproot) {
|
||||
assert(commitFees > 7_000.sat)
|
||||
} else {
|
||||
assert(commitFees > 20_000.sat)
|
||||
}
|
||||
|
||||
val sender = TestProbe()
|
||||
val cmd = CMD_SPLICE(sender.ref, spliceIn_opt = None, Some(SpliceOut(630_000 sat, defaultSpliceOutScriptPubKey)), requestFunding_opt = None)
|
||||
@ -756,12 +745,6 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
resolveHtlcs(f, htlcs)
|
||||
}
|
||||
|
||||
test("recv CMD_SPLICE (splice-in + splice-out, simple taproot channels)", Tag(OptionSimpleTaprootStaging), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f =>
|
||||
val htlcs = setupHtlcs(f)
|
||||
initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey)))
|
||||
resolveHtlcs(f, htlcs, spliceOutFee = 0.sat)
|
||||
}
|
||||
|
||||
test("recv CMD_BUMP_FUNDING_FEE (splice-in + splice-out)") { f =>
|
||||
import f._
|
||||
|
||||
@ -813,49 +796,6 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
initiateSplice(bob, alice, bob2alice, alice2bob, Some(SpliceIn(100_000 sat)), None)
|
||||
}
|
||||
|
||||
test("recv CMD_BUMP_FUNDING_FEE (splice-in + splice-out, simple taproot channels)", Tag(OptionSimpleTaprootStaging), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f =>
|
||||
import f._
|
||||
|
||||
val spliceTx = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(300_000 sat, defaultSpliceOutScriptPubKey)))
|
||||
val spliceCommitment = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.find(_.fundingTxId == spliceTx.txid).get
|
||||
|
||||
// Alice RBFs the splice transaction.
|
||||
// Our dummy bitcoin wallet adds an additional input at every funding attempt.
|
||||
val rbfTx1 = initiateRbf(f, FeeratePerKw(15_000 sat), sInputsCount = 2, sOutputsCount = 2)
|
||||
assert(rbfTx1.txIn.size == spliceTx.txIn.size + 1)
|
||||
spliceTx.txIn.foreach(txIn => assert(rbfTx1.txIn.map(_.outPoint).contains(txIn.outPoint)))
|
||||
assert(rbfTx1.txOut.size == spliceTx.txOut.size)
|
||||
|
||||
// Bob RBFs the splice transaction: he needs to add an input to pay the fees.
|
||||
// Our dummy bitcoin wallet adds an additional input for Alice: a real bitcoin wallet would simply lower the previous change output.
|
||||
val sender2 = initiateRbfWithoutSigs(bob, alice, bob2alice, alice2bob, FeeratePerKw(20_000 sat), sInputsCount = 1, sOutputsCount = 1, rInputsCount = 3, rOutputsCount = 2)
|
||||
val rbfTx2 = exchangeSpliceSigs(alice, bob, alice2bob, bob2alice, sender2)
|
||||
assert(rbfTx2.txIn.size > rbfTx1.txIn.size)
|
||||
rbfTx1.txIn.foreach(txIn => assert(rbfTx2.txIn.map(_.outPoint).contains(txIn.outPoint)))
|
||||
assert(rbfTx2.txOut.size == rbfTx1.txOut.size + 1)
|
||||
|
||||
// There are three pending splice transactions that double-spend each other.
|
||||
inside(alice.stateData.asInstanceOf[DATA_NORMAL]) { data =>
|
||||
val commitments = data.commitments.active.filter(_.fundingTxIndex == spliceCommitment.fundingTxIndex)
|
||||
assert(commitments.size == 3)
|
||||
assert(commitments.map(_.fundingTxId) == Seq(rbfTx2, rbfTx1, spliceTx).map(_.txid))
|
||||
// The contributions are the same across RBF attempts.
|
||||
commitments.foreach(c => assert(c.localCommit.spec.toLocal == spliceCommitment.localCommit.spec.toLocal))
|
||||
commitments.foreach(c => assert(c.localCommit.spec.toRemote == spliceCommitment.localCommit.spec.toRemote))
|
||||
}
|
||||
|
||||
// The last RBF attempt confirms.
|
||||
confirmSpliceTx(f, rbfTx2)
|
||||
inside(alice.stateData.asInstanceOf[DATA_NORMAL]) { data =>
|
||||
assert(data.commitments.active.map(_.fundingTxId) == Seq(rbfTx2.txid))
|
||||
data.commitments.active.foreach(c => assert(c.localCommit.spec.toLocal == spliceCommitment.localCommit.spec.toLocal))
|
||||
data.commitments.active.foreach(c => assert(c.localCommit.spec.toRemote == spliceCommitment.localCommit.spec.toRemote))
|
||||
}
|
||||
|
||||
// We can keep doing more splice transactions now that one of the previous transactions confirmed.
|
||||
initiateSplice(bob, alice, bob2alice, alice2bob, Some(SpliceIn(100_000 sat)), None)
|
||||
}
|
||||
|
||||
test("recv CMD_BUMP_FUNDING_FEE (splice-in + splice-out from non-initiator)") { f =>
|
||||
import f._
|
||||
|
||||
@ -1443,35 +1383,6 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.forall(_.localCommit.spec.htlcs.size == 1))
|
||||
}
|
||||
|
||||
test("recv CMD_ADD_HTLC with multiple commitments (simple taproot channels)", Tag(OptionSimpleTaprootStaging), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f =>
|
||||
import f._
|
||||
initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)))
|
||||
val sender = TestProbe()
|
||||
alice ! CMD_ADD_HTLC(sender.ref, 500_000 msat, randomBytes32(), CltvExpiryDelta(144).toCltvExpiry(currentBlockHeight), TestConstants.emptyOnionPacket, None, 1.0, None, localOrigin(sender.ref))
|
||||
sender.expectMsgType[RES_SUCCESS[CMD_ADD_HTLC]]
|
||||
alice2bob.expectMsgType[UpdateAddHtlc]
|
||||
alice2bob.forward(bob)
|
||||
alice ! CMD_SIGN()
|
||||
val sigA1 = alice2bob.expectMsgType[CommitSig]
|
||||
assert(sigA1.batchSize == 2)
|
||||
alice2bob.forward(bob)
|
||||
val sigA2 = alice2bob.expectMsgType[CommitSig]
|
||||
assert(sigA2.batchSize == 2)
|
||||
alice2bob.forward(bob)
|
||||
bob2alice.expectMsgType[RevokeAndAck]
|
||||
bob2alice.forward(alice)
|
||||
val sigB1 = bob2alice.expectMsgType[CommitSig]
|
||||
assert(sigB1.batchSize == 2)
|
||||
bob2alice.forward(alice)
|
||||
val sigB2 = bob2alice.expectMsgType[CommitSig]
|
||||
assert(sigB2.batchSize == 2)
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[RevokeAndAck]
|
||||
alice2bob.forward(bob)
|
||||
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.forall(_.localCommit.spec.htlcs.size == 1))
|
||||
awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.forall(_.localCommit.spec.htlcs.size == 1))
|
||||
}
|
||||
|
||||
test("recv CMD_ADD_HTLC with multiple commitments and reconnect") { f =>
|
||||
import f._
|
||||
initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)))
|
||||
@ -1609,30 +1520,6 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.head.localCommit.spec.htlcs.size == 1)
|
||||
}
|
||||
|
||||
test("recv UpdateAddHtlc before splice confirms (zero-conf, simple taproot channels)", Tag(OptionSimpleTaprootStaging), Tag(ZeroConf), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f =>
|
||||
import f._
|
||||
|
||||
val spliceTx = initiateSplice(f, spliceOut_opt = Some(SpliceOut(50_000 sat, defaultSpliceOutScriptPubKey)))
|
||||
alice ! WatchPublishedTriggered(spliceTx)
|
||||
val spliceLockedAlice = alice2bob.expectMsgType[SpliceLocked]
|
||||
bob ! WatchPublishedTriggered(spliceTx)
|
||||
val spliceLockedBob = bob2alice.expectMsgType[SpliceLocked]
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 2)
|
||||
val (preimage, htlc) = addHtlc(25_000_000 msat, alice, bob, alice2bob, bob2alice)
|
||||
crossSign(alice, bob, alice2bob, bob2alice)
|
||||
|
||||
alice2bob.forward(bob, spliceLockedAlice)
|
||||
bob2alice.forward(alice, spliceLockedBob)
|
||||
|
||||
fulfillHtlc(htlc.id, preimage, bob, alice, bob2alice, alice2bob)
|
||||
crossSign(bob, alice, bob2alice, alice2bob)
|
||||
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 1)
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.head.localCommit.spec.htlcs.isEmpty)
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.size == 1)
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.head.localCommit.spec.htlcs.size == 1)
|
||||
}
|
||||
|
||||
test("recv UpdateAddHtlc while splice is being locked", Tag(ChannelStateTestsTags.ZeroConf), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
|
||||
import f._
|
||||
|
||||
@ -1701,74 +1588,6 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
bobCommitments.inactive.foreach(c => assert(c.localCommit.index < bobCommitments.localCommitIndex))
|
||||
}
|
||||
|
||||
test("recv UpdateAddHtlc while splice is being locked (simple taproot channels)", Tag(OptionSimpleTaprootStaging), Tag(ZeroConf), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f =>
|
||||
import f._
|
||||
|
||||
val spliceTx1 = initiateSplice(f, spliceOut_opt = Some(SpliceOut(50_000 sat, defaultSpliceOutScriptPubKey)))
|
||||
bob ! WatchPublishedTriggered(spliceTx1)
|
||||
bob2alice.expectMsgType[SpliceLocked] // we ignore Bob's splice_locked for the first splice
|
||||
|
||||
val spliceTx2 = initiateSplice(f, spliceOut_opt = Some(SpliceOut(50_000 sat, defaultSpliceOutScriptPubKey)))
|
||||
alice ! WatchPublishedTriggered(spliceTx2)
|
||||
val spliceLockedAlice = alice2bob.expectMsgType[SpliceLocked]
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 3)
|
||||
|
||||
// Alice adds a new HTLC, and sends commit_sigs before receiving Bob's splice_locked.
|
||||
//
|
||||
// Alice Bob
|
||||
// | splice_locked |
|
||||
// |----------------------------->|
|
||||
// | update_add_htlc |
|
||||
// |----------------------------->|
|
||||
// | commit_sig | batch_size = 3
|
||||
// |----------------------------->|
|
||||
// | splice_locked |
|
||||
// |<-----------------------------|
|
||||
// | commit_sig | batch_size = 3
|
||||
// |----------------------------->|
|
||||
// | commit_sig | batch_size = 3
|
||||
// |----------------------------->|
|
||||
// | revoke_and_ack |
|
||||
// |<-----------------------------|
|
||||
// | commit_sig | batch_size = 1
|
||||
// |<-----------------------------|
|
||||
// | revoke_and_ack |
|
||||
// |----------------------------->|
|
||||
|
||||
alice2bob.forward(bob, spliceLockedAlice)
|
||||
val (preimage, htlc) = addHtlc(20_000_000 msat, alice, bob, alice2bob, bob2alice)
|
||||
alice ! CMD_SIGN()
|
||||
val commitSigsAlice = (1 to 3).map(_ => alice2bob.expectMsgType[CommitSig])
|
||||
alice2bob.forward(bob, commitSigsAlice(0))
|
||||
bob ! WatchPublishedTriggered(spliceTx2)
|
||||
val spliceLockedBob = bob2alice.expectMsgType[SpliceLocked]
|
||||
assert(spliceLockedBob.fundingTxId == spliceTx2.txid)
|
||||
bob2alice.forward(alice, spliceLockedBob)
|
||||
alice2bob.forward(bob, commitSigsAlice(1))
|
||||
alice2bob.forward(bob, commitSigsAlice(2))
|
||||
bob2alice.expectMsgType[RevokeAndAck]
|
||||
bob2alice.forward(alice)
|
||||
assert(bob2alice.expectMsgType[CommitSig].batchSize == 1)
|
||||
bob2alice.forward(alice)
|
||||
alice2bob.expectMsgType[RevokeAndAck]
|
||||
alice2bob.forward(bob)
|
||||
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 1)
|
||||
assert(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.size == 2)
|
||||
assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 1)
|
||||
assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.inactive.size == 2)
|
||||
|
||||
// Bob fulfills the HTLC.
|
||||
fulfillHtlc(htlc.id, preimage, bob, alice, bob2alice, alice2bob)
|
||||
crossSign(bob, alice, bob2alice, alice2bob)
|
||||
val aliceCommitments = alice.stateData.asInstanceOf[DATA_NORMAL].commitments
|
||||
assert(aliceCommitments.active.head.localCommit.spec.htlcs.isEmpty)
|
||||
aliceCommitments.inactive.foreach(c => assert(c.localCommit.index < aliceCommitments.localCommitIndex))
|
||||
val bobCommitments = bob.stateData.asInstanceOf[DATA_NORMAL].commitments
|
||||
assert(bobCommitments.active.head.localCommit.spec.htlcs.isEmpty)
|
||||
bobCommitments.inactive.foreach(c => assert(c.localCommit.index < bobCommitments.localCommitIndex))
|
||||
}
|
||||
|
||||
private def disconnect(f: FixtureParam): Unit = {
|
||||
import f._
|
||||
|
||||
@ -2482,7 +2301,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
}
|
||||
}
|
||||
|
||||
def testForceCloseWithMultipleSplicesSimple(f: FixtureParam, useAnchorOutputs: Boolean = false): Unit = {
|
||||
test("force-close with multiple splices (simple)") { f =>
|
||||
import f._
|
||||
|
||||
val htlcs = setupHtlcs(f)
|
||||
@ -2505,7 +2324,11 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
val commitTx2 = assertPublished(alice2blockchain, "commit-tx")
|
||||
Transaction.correctlySpends(commitTx2, Seq(fundingTx2), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
|
||||
|
||||
if (useAnchorOutputs) {
|
||||
val isAnchorOutputs = alice.stateData.asInstanceOf[DATA_CLOSING].commitments.params.commitmentFormat match {
|
||||
case _: AnchorOutputsCommitmentFormat => true
|
||||
case _ => false
|
||||
}
|
||||
if (isAnchorOutputs) {
|
||||
val claimAnchor = assertPublished(alice2blockchain, "local-anchor")
|
||||
}
|
||||
val claimMainDelayed2 = assertPublished(alice2blockchain, "local-main-delayed")
|
||||
@ -2516,7 +2339,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
val watchConfirmedCommit2 = alice2blockchain.expectWatchTxConfirmed(commitTx2.txid)
|
||||
val watchConfirmedClaimMainDelayed2 = alice2blockchain.expectWatchTxConfirmed(claimMainDelayed2.txid)
|
||||
// watch for all htlc outputs from local commit-tx to be spent
|
||||
if (useAnchorOutputs) {
|
||||
if (isAnchorOutputs) {
|
||||
alice2blockchain.expectMsgType[WatchOutputSpent]
|
||||
}
|
||||
val watchHtlcsOut = htlcs.aliceToBob.map(_ => alice2blockchain.expectMsgType[WatchOutputSpent])
|
||||
@ -2557,14 +2380,6 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
assert(Helpers.Closing.isClosed(alice.stateData.asInstanceOf[DATA_CLOSING], None).exists(_.isInstanceOf[LocalClose]))
|
||||
}
|
||||
|
||||
test("force-close with multiple splices (simple)") { f =>
|
||||
testForceCloseWithMultipleSplicesSimple(f)
|
||||
}
|
||||
|
||||
test("force-close with multiple splices (simple, simple taproot channels)", Tag(OptionSimpleTaprootStaging), Tag(AnchorOutputsZeroFeeHtlcTxs)) { f =>
|
||||
testForceCloseWithMultipleSplicesSimple(f, useAnchorOutputs = true)
|
||||
}
|
||||
|
||||
test("force-close with multiple splices (previous active remote)", Tag(ChannelStateTestsTags.StaticRemoteKey), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
|
||||
import f._
|
||||
|
||||
@ -3265,5 +3080,18 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
|
||||
assert(finalState.commitments.latest.localCommit.spec.toLocal == 805_000_000.msat)
|
||||
assert(finalState.commitments.latest.localCommit.spec.toRemote == 695_000_000.msat)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class NormalSplicesStateWithTaprootChannelsSpec extends NormalSplicesStateSpec {
|
||||
override def withFixture(test: OneArgTest): Outcome = {
|
||||
val tags = test.tags + ChannelStateTestsTags.DualFunding + ChannelStateTestsTags.FundingDeeplyBuried + ChannelStateTestsTags.OptionSimpleTaprootStaging + ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs
|
||||
val setup = init(tags = tags)
|
||||
import setup._
|
||||
reachNormal(setup, tags)
|
||||
alice2bob.ignoreMsg { case _: ChannelUpdate => true }
|
||||
bob2alice.ignoreMsg { case _: ChannelUpdate => true }
|
||||
awaitCond(alice.stateName == NORMAL)
|
||||
awaitCond(bob.stateName == NORMAL)
|
||||
withFixture(test.toNoArgTest(setup))
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue
Block a user