1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-03-12 10:30:45 +01:00

Remove spurious interactive-tx commit_sig retransmission

We fully implement https://github.com/lightning/bolts/pull/1214 to stop
retransmitting `commit_sig` when our peer has already received it. We
also correctly set `next_commitment_number` to let our peer know whether
we have received their `commit_sig` or not.

We also retransmit `tx_signatures` (and, if requested, `commit_sig`)
after sending `channel_ready` in the 0-conf case. This was missing and
was a bug.
This commit is contained in:
t-bast 2024-12-13 10:16:16 +01:00
parent 21917f55dd
commit 9625a6e8c6
No known key found for this signature in database
GPG key ID: 34F377B0100ED6BB
5 changed files with 300 additions and 142 deletions

View file

@ -2242,7 +2242,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
val nextFundingTlv: Set[ChannelReestablishTlv] = Set(ChannelReestablishTlv.NextFundingTlv(d.signingSession.fundingTx.txId))
val channelReestablish = ChannelReestablish(
channelId = d.channelId,
nextLocalCommitmentNumber = 1,
nextLocalCommitmentNumber = d.signingSession.reconnectNextLocalCommitmentNumber,
nextRemoteRevocationNumber = 0,
yourLastPerCommitmentSecret = PrivateKey(ByteVector32.Zeroes),
myCurrentPerCommitmentPoint = myFirstPerCommitmentPoint,
@ -2257,6 +2257,19 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
val yourLastPerCommitmentSecret = remotePerCommitmentSecrets.lastIndex.flatMap(remotePerCommitmentSecrets.getHash).getOrElse(ByteVector32.Zeroes)
val channelKeyPath = keyManager.keyPath(d.commitments.params.localParams, d.commitments.params.channelConfig)
val myCurrentPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, d.commitments.localCommitIndex)
// If we disconnected while signing a funding transaction, we may need our peer to retransmit their commit_sig.
val nextLocalCommitmentNumber = d match {
case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => d.status match {
case DualFundingStatus.RbfWaitingForSigs(status) => status.reconnectNextLocalCommitmentNumber
case _ => d.commitments.localCommitIndex + 1
}
case d: DATA_NORMAL => d.spliceStatus match {
case SpliceStatus.SpliceWaitingForSigs(status) => status.reconnectNextLocalCommitmentNumber
case _ => d.commitments.localCommitIndex + 1
}
case _ => d.commitments.localCommitIndex + 1
}
// If we disconnected while signing a funding transaction, we may need our peer to (re)transmit their tx_signatures.
val rbfTlv: Set[ChannelReestablishTlv] = d match {
case d: DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED => d.status match {
case DualFundingStatus.RbfWaitingForSigs(status) => Set(ChannelReestablishTlv.NextFundingTlv(status.fundingTx.txId))
@ -2280,7 +2293,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
val channelReestablish = ChannelReestablish(
channelId = d.channelId,
nextLocalCommitmentNumber = d.commitments.localCommitIndex + 1,
nextLocalCommitmentNumber = nextLocalCommitmentNumber,
nextRemoteRevocationNumber = d.commitments.remoteCommitIndex,
yourLastPerCommitmentSecret = PrivateKey(yourLastPerCommitmentSecret),
myCurrentPerCommitmentPoint = myCurrentPerCommitmentPoint,
@ -2321,8 +2334,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Event(channelReestablish: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_SIGNED) =>
channelReestablish.nextFundingTxId_opt match {
case Some(fundingTxId) if fundingTxId == d.signingSession.fundingTx.txId =>
// We retransmit our commit_sig, and will send our tx_signatures once we've received their commit_sig.
case Some(fundingTxId) if fundingTxId == d.signingSession.fundingTx.txId && channelReestablish.nextLocalCommitmentNumber == 0 =>
// They haven't received our commit_sig: we retransmit it, and will send our tx_signatures once we've received
// their commit_sig or their tx_signatures (depending on who must send tx_signatures first).
val commitSig = d.signingSession.remoteCommit.sign(keyManager, d.channelParams, d.signingSession.fundingTxIndex, d.signingSession.fundingParams.remoteFundingPubKey, d.signingSession.commitInput)
goto(WAIT_FOR_DUAL_FUNDING_SIGNED) sending commitSig
case _ => goto(WAIT_FOR_DUAL_FUNDING_SIGNED)
@ -2333,20 +2347,25 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
case Some(fundingTxId) =>
d.status match {
case DualFundingStatus.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.
if (channelReestablish.nextLocalCommitmentNumber == 0) {
// They haven't received our commit_sig: we retransmit it.
// We're also waiting for signatures from them, and will send our tx_signatures once we receive them.
val commitSig = signingSession.remoteCommit.sign(keyManager, d.commitments.params, signingSession.fundingTxIndex, signingSession.fundingParams.remoteFundingPubKey, signingSession.commitInput)
goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) sending commitSig
case _ if d.latestFundingTx.sharedTx.txId == fundingTxId =>
val toSend = d.latestFundingTx.sharedTx match {
case fundingTx: InteractiveTxBuilder.PartiallySignedSharedTransaction =>
// We have not received their tx_signatures: we retransmit our commit_sig because we don't know if they received it.
val commitSig = d.commitments.latest.remoteCommit.sign(keyManager, d.commitments.params, d.commitments.latest.fundingTxIndex, d.commitments.latest.remoteFundingPubKey, d.commitments.latest.commitInput)
Seq(commitSig, fundingTx.localSigs)
case fundingTx: InteractiveTxBuilder.FullySignedSharedTransaction =>
// We've already received their tx_signatures, which means they've received and stored our commit_sig, we only need to retransmit our tx_signatures.
Seq(fundingTx.localSigs)
} else {
// They have already received our commit_sig, but we were waiting for them to send either commit_sig or
// tx_signatures first. We wait for their message before sending our tx_signatures.
goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED)
}
case _ if d.latestFundingTx.sharedTx.txId == fundingTxId =>
// We've already received their commit_sig and sent our tx_signatures. We retransmit our tx_signatures
// and our commit_sig if they haven't received it already.
if (channelReestablish.nextLocalCommitmentNumber == 0) {
val commitSig = d.commitments.latest.remoteCommit.sign(keyManager, d.commitments.params, d.commitments.latest.fundingTxIndex, d.commitments.latest.remoteFundingPubKey, d.commitments.latest.commitInput)
goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) sending Seq(commitSig, d.latestFundingTx.sharedTx.localSigs)
} else {
goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) sending d.latestFundingTx.sharedTx.localSigs
}
goto(WAIT_FOR_DUAL_FUNDING_CONFIRMED) sending toSend
case _ =>
// The fundingTxId must be for an RBF attempt that we didn't store (we got disconnected before receiving
// their tx_complete): we tell them to abort that RBF attempt.
@ -2356,14 +2375,31 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
}
case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_CHANNEL_READY) =>
log.debug("re-sending channelReady")
log.debug("re-sending channel_ready")
val channelReady = createChannelReady(d.aliases, d.commitments.params)
goto(WAIT_FOR_CHANNEL_READY) sending channelReady
case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) =>
log.debug("re-sending channelReady")
case Event(channelReestablish: ChannelReestablish, d: DATA_WAIT_FOR_DUAL_FUNDING_READY) =>
log.debug("re-sending channel_ready")
val channelReady = createChannelReady(d.aliases, d.commitments.params)
// We've already received their commit_sig and sent our tx_signatures. We retransmit our tx_signatures
// and our commit_sig if they haven't received it already.
channelReestablish.nextFundingTxId_opt match {
case Some(fundingTxId) if fundingTxId == d.commitments.latest.fundingTxId =>
d.commitments.latest.localFundingStatus.localSigs_opt match {
case Some(txSigs) if channelReestablish.nextLocalCommitmentNumber == 0 =>
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)
goto(WAIT_FOR_DUAL_FUNDING_READY) sending Seq(commitSig, txSigs, channelReady)
case Some(txSigs) =>
log.info("re-sending tx_signatures for fundingTxIndex={} fundingTxId={}", d.commitments.latest.fundingTxIndex, d.commitments.latest.fundingTxId)
goto(WAIT_FOR_DUAL_FUNDING_READY) sending Seq(txSigs, channelReady)
case None =>
log.warning("cannot retransmit tx_signatures, we don't have them (status={})", d.commitments.latest.localFundingStatus)
goto(WAIT_FOR_DUAL_FUNDING_READY) sending channelReady
}
case _ => goto(WAIT_FOR_DUAL_FUNDING_READY) sending channelReady
}
case Event(channelReestablish: ChannelReestablish, d: DATA_NORMAL) =>
Syncing.checkSync(keyManager, d.commitments, channelReestablish) match {
@ -2389,23 +2425,26 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder with
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.
if (channelReestablish.nextLocalCommitmentNumber == d.commitments.remoteCommitIndex) {
// They haven't received our commit_sig: we retransmit it.
// We're also waiting for signatures from them, and will send our tx_signatures once we receive them.
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)
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
// We've already received their commit_sig and sent our tx_signatures. We retransmit our
// tx_signatures and our commit_sig if they haven't received it already.
if (channelReestablish.nextLocalCommitmentNumber == d.commitments.remoteCommitIndex) {
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)
sendQueue = sendQueue :+ commitSig :+ fundingTx.localSigs
case fundingTx: InteractiveTxBuilder.FullySignedSharedTransaction =>
sendQueue = sendQueue :+ commitSig :+ dfu.sharedTx.localSigs
} else {
log.info("re-sending tx_signatures for fundingTxIndex={} fundingTxId={}", d.commitments.latest.fundingTxIndex, d.commitments.latest.fundingTxId)
sendQueue = sendQueue :+ fundingTx.localSigs
sendQueue = sendQueue :+ dfu.sharedTx.localSigs
}
case fundingStatus =>
// They have not received our tx_signatures, but they must have received our commit_sig, otherwise we would be in the case above.

View file

@ -1082,6 +1082,11 @@ object InteractiveTxSigningSession {
liquidityPurchase_opt: Option[LiquidityAds.PurchaseBasicInfo]) extends InteractiveTxSigningSession {
val commitInput: InputInfo = localCommit.fold(_.commitTx.input, _.commitTxAndRemoteSig.commitTx.input)
val localCommitIndex: Long = localCommit.fold(_.index, _.index)
// This value tells our peer whether we need them to retransmit their commit_sig on reconnection or not.
val reconnectNextLocalCommitmentNumber: Long = localCommit match {
case Left(commit) => commit.index
case Right(commit) => commit.index + 1
}
def receiveCommitSig(nodeParams: NodeParams, channelParams: ChannelParams, remoteCommitSig: CommitSig)(implicit log: LoggingAdapter): Either[ChannelException, InteractiveTxSigningSession] = {
localCommit match {

View file

@ -21,7 +21,7 @@ import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, ByteVector64, SatoshiLong, TxId}
import fr.acinq.eclair.TestUtils.randomTxId
import fr.acinq.eclair.blockchain.SingleKeyOnChainWallet
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchPublished}
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchFundingConfirmed, WatchPublished, WatchPublishedTriggered}
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.fsm.Channel
@ -375,15 +375,16 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny
bob ! INPUT_DISCONNECTED
awaitCond(bob.stateName == OFFLINE)
reconnect(f, fundingTxId)
reconnect(f, fundingTxId, aliceExpectsCommitSig = true, bobExpectsCommitSig = true)
}
test("recv INPUT_DISCONNECTED (commit_sig not received, next_commitment_number = 0)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
test("recv INPUT_DISCONNECTED (commit_sig received by Alice)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
import f._
val fundingTxId = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED].signingSession.fundingTx.txId
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectMsgType[CommitSig] // Bob doesn't receive Alice's commit_sig
bob2alice.expectMsgType[CommitSig] // Alice doesn't receive Bob's commit_sig
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
@ -392,10 +393,10 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny
bob ! INPUT_DISCONNECTED
awaitCond(bob.stateName == OFFLINE)
reconnect(f, fundingTxId, aliceCommitmentNumber = 0, bobCommitmentNumber = 0)
reconnect(f, fundingTxId, aliceExpectsCommitSig = false, bobExpectsCommitSig = true)
}
test("recv INPUT_DISCONNECTED (commit_sig partially received)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
test("recv INPUT_DISCONNECTED (commit_sig received by Bob)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
import f._
val fundingTxId = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED].signingSession.fundingTx.txId
@ -411,13 +412,12 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny
bob ! INPUT_DISCONNECTED
awaitCond(bob.stateName == OFFLINE)
reconnect(f, fundingTxId)
reconnect(f, fundingTxId, aliceExpectsCommitSig = true, bobExpectsCommitSig = false)
}
test("recv INPUT_DISCONNECTED (commit_sig partially received, next_commitment_number = 0)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
test("recv INPUT_DISCONNECTED (commit_sig received by Bob, zero-conf)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.ZeroConf), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._
val fundingTxId = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_SIGNED].signingSession.fundingTx.txId
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[CommitSig] // Alice doesn't receive Bob's commit_sig
@ -425,12 +425,47 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_SIGNED)
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
// Note that this case can only happen when Bob doesn't need Alice's signatures to publish the transaction (when
// Bob was the only one to contribute to the funding transaction).
val fundingTx = bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.tx.buildUnsignedTx()
assert(bob2blockchain.expectMsgType[WatchPublished].txId == fundingTx.txid)
bob ! WatchPublishedTriggered(fundingTx)
assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid)
bob2alice.expectMsgType[ChannelReady]
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_READY)
alice ! INPUT_DISCONNECTED
awaitCond(alice.stateName == OFFLINE)
bob ! INPUT_DISCONNECTED
awaitCond(bob.stateName == OFFLINE)
reconnect(f, fundingTxId, aliceCommitmentNumber = 0)
val listener = TestProbe()
alice.underlyingActor.context.system.eventStream.subscribe(listener.ref, classOf[TransactionPublished])
val aliceInit = Init(alice.underlyingActor.nodeParams.features.initFeatures())
val bobInit = Init(bob.underlyingActor.nodeParams.features.initFeatures())
alice ! INPUT_RECONNECTED(bob, aliceInit, bobInit)
bob ! INPUT_RECONNECTED(alice, bobInit, aliceInit)
val channelReestablishAlice = alice2bob.expectMsgType[ChannelReestablish]
assert(channelReestablishAlice.nextFundingTxId_opt.contains(fundingTx.txid))
assert(channelReestablishAlice.nextLocalCommitmentNumber == 0)
alice2bob.forward(bob, channelReestablishAlice)
val channelReestablishBob = bob2alice.expectMsgType[ChannelReestablish]
assert(channelReestablishBob.nextFundingTxId_opt.isEmpty)
assert(channelReestablishBob.nextLocalCommitmentNumber == 1)
bob2alice.forward(alice, channelReestablishBob)
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
bob2alice.expectMsgType[TxSignatures]
bob2alice.forward(alice)
alice2bob.expectMsgType[TxSignatures]
alice2bob.forward(bob)
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_READY)
assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid)
assert(listener.expectMsgType[TransactionPublished].tx.txid == fundingTx.txid)
}
test("recv INPUT_DISCONNECTED (commit_sig received)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
@ -450,7 +485,7 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny
bob ! INPUT_DISCONNECTED
awaitCond(bob.stateName == OFFLINE)
reconnect(f, fundingTxId)
reconnect(f, fundingTxId, aliceExpectsCommitSig = false, bobExpectsCommitSig = false)
}
test("recv INPUT_DISCONNECTED (tx_signatures received)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
@ -466,7 +501,7 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny
bob2alice.forward(alice)
bob2alice.expectMsgType[TxSignatures]
bob2alice.forward(alice)
alice2bob.expectMsgType[TxSignatures]
alice2bob.expectMsgType[TxSignatures] // Bob doesn't receive Alice's tx_signatures
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
@ -490,7 +525,52 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny
assert(listener.expectMsgType[TransactionPublished].tx.txid == fundingTxId)
}
private def reconnect(f: FixtureParam, fundingTxId: TxId, aliceCommitmentNumber: Long = 1, bobCommitmentNumber: Long = 1): Unit = {
test("recv INPUT_DISCONNECTED (tx_signatures received, zero-conf)", Tag(ChannelStateTestsTags.DualFunding), Tag(ChannelStateTestsTags.ZeroConf), Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs)) { f =>
import f._
val listener = TestProbe()
bob.underlyingActor.context.system.eventStream.subscribe(listener.ref, classOf[TransactionPublished])
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
bob2alice.expectMsgType[TxSignatures]
bob2alice.forward(alice)
alice2bob.expectMsgType[TxSignatures] // Bob doesn't receive Alice's tx_signatures
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
awaitCond(bob.stateName == WAIT_FOR_DUAL_FUNDING_CONFIRMED)
val fundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.signedTx_opt.get
assert(alice2blockchain.expectMsgType[WatchPublished].txId == fundingTx.txid)
alice ! WatchPublishedTriggered(fundingTx)
assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == fundingTx.txid)
alice2bob.expectMsgType[ChannelReady]
awaitCond(alice.stateName == WAIT_FOR_DUAL_FUNDING_READY)
alice ! INPUT_DISCONNECTED
awaitCond(alice.stateName == OFFLINE)
bob ! INPUT_DISCONNECTED
awaitCond(bob.stateName == OFFLINE)
val aliceInit = Init(alice.underlyingActor.nodeParams.features.initFeatures())
val bobInit = Init(bob.underlyingActor.nodeParams.features.initFeatures())
alice ! INPUT_RECONNECTED(bob, aliceInit, bobInit)
bob ! INPUT_RECONNECTED(alice, bobInit, aliceInit)
assert(alice2bob.expectMsgType[ChannelReestablish].nextFundingTxId_opt.isEmpty)
alice2bob.forward(bob)
assert(bob2alice.expectMsgType[ChannelReestablish].nextFundingTxId_opt.contains(fundingTx.txid))
bob2alice.forward(alice)
alice2bob.expectMsgType[TxSignatures]
alice2bob.forward(bob)
alice2bob.expectMsgType[ChannelReady]
alice2bob.forward(bob)
assert(bob2blockchain.expectMsgType[WatchPublished].txId == fundingTx.txid)
assert(listener.expectMsgType[TransactionPublished].tx.txid == fundingTx.txid)
}
private def reconnect(f: FixtureParam, fundingTxId: TxId, aliceExpectsCommitSig: Boolean, bobExpectsCommitSig: Boolean): Unit = {
import f._
val listener = TestProbe()
@ -501,17 +581,24 @@ class WaitForDualFundingSignedStateSpec extends TestKitBaseClass with FixtureAny
alice ! INPUT_RECONNECTED(bob, aliceInit, bobInit)
bob ! INPUT_RECONNECTED(alice, bobInit, aliceInit)
val channelReestablishAlice = alice2bob.expectMsgType[ChannelReestablish]
val nextLocalCommitmentNumberAlice = if (aliceExpectsCommitSig) 0 else 1
assert(channelReestablishAlice.nextFundingTxId_opt.contains(fundingTxId))
assert(channelReestablishAlice.nextLocalCommitmentNumber == 1)
alice2bob.forward(bob, channelReestablishAlice.copy(nextLocalCommitmentNumber = aliceCommitmentNumber))
assert(channelReestablishAlice.nextLocalCommitmentNumber == nextLocalCommitmentNumberAlice)
alice2bob.forward(bob, channelReestablishAlice)
val channelReestablishBob = bob2alice.expectMsgType[ChannelReestablish]
val nextLocalCommitmentNumberBob = if (bobExpectsCommitSig) 0 else 1
assert(channelReestablishBob.nextFundingTxId_opt.contains(fundingTxId))
assert(channelReestablishBob.nextLocalCommitmentNumber == 1)
bob2alice.forward(alice, channelReestablishBob.copy(nextLocalCommitmentNumber = bobCommitmentNumber))
assert(channelReestablishBob.nextLocalCommitmentNumber == nextLocalCommitmentNumberBob)
bob2alice.forward(alice, channelReestablishBob)
if (aliceExpectsCommitSig) {
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
}
if (bobExpectsCommitSig) {
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
}
bob2alice.expectMsgType[TxSignatures]
bob2alice.forward(alice)
alice2bob.expectMsgType[TxSignatures]

View file

@ -933,7 +933,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
val (channelReestablishAlice, channelReestablishBob) = reconnectRbf(f)
assert(channelReestablishAlice.nextFundingTxId_opt.contains(rbfTxId))
assert(channelReestablishAlice.nextLocalCommitmentNumber == 1)
assert(channelReestablishAlice.nextLocalCommitmentNumber == 0)
assert(channelReestablishBob.nextFundingTxId_opt.isEmpty)
assert(channelReestablishBob.nextLocalCommitmentNumber == 1)
@ -948,7 +948,43 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
bob2alice.expectNoMessage(100 millis)
}
test("recv INPUT_DISCONNECTED (rbf commit_sig partially received)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
test("recv INPUT_DISCONNECTED (rbf commit_sig received by Alice)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
import f._
initiateRbf(f)
alice2bob.expectMsgType[TxComplete]
alice2bob.forward(bob)
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectMsgType[CommitSig] // Bob doesn't receive Alice's commit_sig
awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs])
awaitCond(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.isInstanceOf[DualFundingStatus.RbfWaitingForSigs])
val rbfTxId = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status.asInstanceOf[DualFundingStatus.RbfWaitingForSigs].signingSession.fundingTx.txId
val (channelReestablishAlice, channelReestablishBob) = reconnectRbf(f)
assert(channelReestablishAlice.nextFundingTxId_opt.contains(rbfTxId))
assert(channelReestablishAlice.nextLocalCommitmentNumber == 1)
assert(channelReestablishBob.nextFundingTxId_opt.contains(rbfTxId))
assert(channelReestablishBob.nextLocalCommitmentNumber == 0)
// Alice retransmits commit_sig, and they exchange tx_signatures afterwards.
bob2alice.expectNoMessage(100 millis)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[TxSignatures]
bob2alice.forward(alice)
alice2bob.expectMsgType[TxSignatures]
alice2bob.forward(bob)
val nextFundingTx = alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].latestFundingTx.sharedTx.asInstanceOf[FullySignedSharedTransaction]
assert(aliceListener.expectMsgType[TransactionPublished].tx.txid == nextFundingTx.signedTx.txid)
assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == nextFundingTx.signedTx.txid)
assert(bobListener.expectMsgType[TransactionPublished].tx.txid == nextFundingTx.signedTx.txid)
assert(bob2blockchain.expectMsgType[WatchFundingConfirmed].txId == nextFundingTx.signedTx.txid)
awaitCond(alice.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations)
awaitCond(bob.stateData.asInstanceOf[DATA_WAIT_FOR_DUAL_FUNDING_CONFIRMED].status == DualFundingStatus.WaitingForConfirmations)
}
test("recv INPUT_DISCONNECTED (rbf commit_sig received by Bob)", Tag(ChannelStateTestsTags.DualFunding)) { f =>
import f._
initiateRbf(f)
@ -964,15 +1000,14 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
val (channelReestablishAlice, channelReestablishBob) = reconnectRbf(f)
assert(channelReestablishAlice.nextFundingTxId_opt.contains(rbfTxId))
assert(channelReestablishAlice.nextLocalCommitmentNumber == 1)
assert(channelReestablishAlice.nextLocalCommitmentNumber == 0)
assert(channelReestablishBob.nextFundingTxId_opt.contains(rbfTxId))
assert(channelReestablishBob.nextLocalCommitmentNumber == 1)
// Alice and Bob exchange signatures and complete the RBF attempt.
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
// Bob retransmits commit_sig and tx_signatures, then Alice sends her tx_signatures.
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectNoMessage(100 millis)
bob2alice.expectMsgType[TxSignatures]
bob2alice.forward(alice)
alice2bob.expectMsgType[TxSignatures]
@ -1007,11 +1042,7 @@ class WaitForDualFundingConfirmedStateSpec extends TestKitBaseClass with Fixture
assert(channelReestablishBob.nextFundingTxId_opt.contains(rbfTx.txId))
assert(channelReestablishBob.nextLocalCommitmentNumber == 1)
// Alice and Bob exchange signatures and complete the RBF attempt.
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
// Alice and Bob exchange tx_signatures and complete the RBF attempt.
bob2alice.expectMsgType[TxSignatures]
bob2alice.forward(alice)
alice2bob.expectMsgType[TxSignatures]

View file

@ -1767,57 +1767,9 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
disconnect(f)
val (channelReestablishAlice, channelReestablishBob) = reconnect(f)
assert(channelReestablishAlice.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId))
assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex + 1)
assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex)
assert(channelReestablishBob.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId))
assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex + 1)
// Alice and Bob retransmit commit_sig and tx_signatures.
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
bob2alice.expectMsgType[TxSignatures]
bob2alice.forward(alice)
alice2bob.expectMsgType[TxSignatures]
alice2bob.forward(bob)
sender.expectMsgType[RES_SPLICE]
val spliceTx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.latest.localFundingStatus.signedTx_opt.get
alice2blockchain.expectWatchFundingConfirmed(spliceTx.txid)
bob2blockchain.expectWatchFundingConfirmed(spliceTx.txid)
alice ! WatchFundingConfirmedTriggered(BlockHeight(42), 0, spliceTx)
alice2bob.expectMsgType[SpliceLocked]
alice2bob.forward(bob)
bob ! WatchFundingConfirmedTriggered(BlockHeight(42), 0, spliceTx)
bob2alice.expectMsgType[SpliceLocked]
bob2alice.forward(alice)
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 1)
awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.active.size == 1)
resolveHtlcs(f, htlcs)
}
test("disconnect (commit_sig not received, reestablish with previous commitment_number)") { f =>
import f._
val htlcs = setupHtlcs(f)
val aliceCommitIndex = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommitIndex
val bobCommitIndex = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommitIndex
val sender = initiateSpliceWithoutSigs(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey)))
alice2bob.expectMsgType[CommitSig] // Bob doesn't receive Alice's commit_sig
bob2alice.expectMsgType[CommitSig] // Alice doesn't receive Bob's commit_sig
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus.isInstanceOf[SpliceStatus.SpliceWaitingForSigs])
val spliceStatus = alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus.asInstanceOf[SpliceStatus.SpliceWaitingForSigs]
disconnect(f)
val (channelReestablishAlice, channelReestablishBob) = reconnect(f, sendReestablish = false)
assert(channelReestablishAlice.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId))
assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex + 1)
alice2bob.forward(bob, channelReestablishAlice.copy(nextLocalCommitmentNumber = aliceCommitIndex))
assert(channelReestablishBob.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId))
assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex + 1)
bob2alice.forward(alice, channelReestablishBob.copy(nextLocalCommitmentNumber = bobCommitIndex))
assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex)
// Alice and Bob retransmit commit_sig and tx_signatures.
alice2bob.expectMsgType[CommitSig]
@ -1865,13 +1817,13 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
assert(channelReestablishAlice.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId))
assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex + 1)
assert(channelReestablishBob.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId))
assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex + 1)
assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex)
// Alice and Bob retransmit commit_sig and tx_signatures.
// Alice retransmits commit_sig, and they exchange tx_signatures afterwards.
bob2alice.expectNoMessage(100 millis)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectNoMessage(100 millis)
bob2alice.expectMsgType[TxSignatures]
bob2alice.forward(alice)
alice2bob.expectMsgType[TxSignatures]
@ -1893,7 +1845,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
resolveHtlcs(f, htlcs)
}
test("disconnect (commit_sig received by alice, reestablish with previous commitment_number)") { f =>
test("disconnect (commit_sig received by bob)") { f =>
import f._
val htlcs = setupHtlcs(f)
@ -1902,26 +1854,25 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
assert(aliceCommitIndex != bobCommitIndex)
val sender = initiateSpliceWithoutSigs(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey)))
alice2bob.expectMsgType[CommitSig] // Bob doesn't receive Alice's commit_sig
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[CommitSig] // Alice doesn't receive Bob's commit_sig
bob2alice.expectMsgType[TxSignatures] // Alice doesn't receive Bob's tx_signatures
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus.isInstanceOf[SpliceStatus.SpliceWaitingForSigs])
awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].spliceStatus == SpliceStatus.NoSplice)
val spliceStatus = alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus.asInstanceOf[SpliceStatus.SpliceWaitingForSigs]
disconnect(f)
val (channelReestablishAlice, channelReestablishBob) = reconnect(f, sendReestablish = false)
val (channelReestablishAlice, channelReestablishBob) = reconnect(f)
assert(channelReestablishAlice.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId))
assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex + 1)
alice2bob.forward(bob, channelReestablishAlice)
assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex)
assert(channelReestablishBob.nextFundingTxId_opt.contains(spliceStatus.signingSession.fundingTx.txId))
assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex + 1)
bob2alice.forward(alice, channelReestablishBob.copy(nextLocalCommitmentNumber = bobCommitIndex))
// Alice and Bob retransmit commit_sig and tx_signatures.
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
// Bob retransmit commit_sig and tx_signatures, Alice sends tx_signatures afterwards.
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectNoMessage(100 millis)
bob2alice.expectMsgType[TxSignatures]
bob2alice.forward(alice)
alice2bob.expectMsgType[TxSignatures]
@ -1943,7 +1894,7 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
resolveHtlcs(f, htlcs)
}
test("disconnect (tx_signatures sent by bob)") { f =>
test("disconnect (commit_sig received)") { f =>
import f._
val htlcs = setupHtlcs(f)
@ -1966,11 +1917,8 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex + 1)
bob2blockchain.expectWatchFundingConfirmed(spliceTxId)
// Alice and Bob retransmit commit_sig and tx_signatures.
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
// Alice and Bob retransmit tx_signatures.
alice2bob.expectNoMessage(100 millis)
bob2alice.expectMsgType[TxSignatures]
bob2alice.forward(alice)
alice2bob.expectMsgType[TxSignatures]
@ -2184,9 +2132,9 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
disconnect(f)
val (channelReestablishAlice, channelReestablishBob) = reconnect(f)
assert(channelReestablishAlice.nextFundingTxId_opt.contains(rbfTxId))
assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex + 1)
assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex)
assert(channelReestablishBob.nextFundingTxId_opt.contains(rbfTxId))
assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex + 1)
assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex)
bob2blockchain.expectWatchFundingConfirmed(spliceTx.txid)
// Alice and Bob retransmit commit_sig and tx_signatures.
@ -2233,14 +2181,62 @@ class NormalSplicesStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLik
assert(channelReestablishAlice.nextFundingTxId_opt.contains(rbfTxId))
assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex + 1)
assert(channelReestablishBob.nextFundingTxId_opt.contains(rbfTxId))
assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex)
bob2blockchain.expectWatchFundingConfirmed(spliceTx.txid)
// Alice retransmits commit_sig, and they exchange tx_signatures afterwards.
bob2alice.expectNoMessage(100 millis)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
alice2bob.expectNoMessage(100 millis)
bob2alice.expectMsgType[TxSignatures]
bob2alice.forward(alice)
alice2bob.expectMsgType[TxSignatures]
alice2bob.forward(bob)
probe.expectMsgType[RES_SPLICE]
val rbfTx = confirmRbfTx(f)
assert(rbfTx.txid != spliceTx.txid)
resolveHtlcs(f, htlcs)
}
test("disconnect (RBF commit_sig received by bob)") { f =>
import f._
val htlcs = setupHtlcs(f)
val spliceTx = initiateSplice(f, spliceIn_opt = Some(SpliceIn(500_000 sat)), spliceOut_opt = Some(SpliceOut(100_000 sat, defaultSpliceOutScriptPubKey)))
assert(alice2blockchain.expectMsgType[WatchFundingConfirmed].txId == spliceTx.txid)
// Bob uses the channel before Alice tries to RBF.
val (_, add) = addHtlc(40_000_000 msat, bob, alice, bob2alice, alice2bob)
crossSign(bob, alice, bob2alice, alice2bob)
failHtlc(add.id, alice, bob, alice2bob, bob2alice)
crossSign(alice, bob, alice2bob, bob2alice)
val aliceCommitIndex = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommitIndex
val bobCommitIndex = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommitIndex
val probe = initiateRbfWithoutSigs(f, FeeratePerKw(15_000 sat), sInputsCount = 2, sOutputsCount = 2)
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
bob2alice.expectMsgType[CommitSig] // Alice doesn't receive Bob's commit_sig
bob2alice.expectMsgType[TxSignatures] // Alice doesn't receive Bob's tx_signatures
awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus.isInstanceOf[SpliceStatus.SpliceWaitingForSigs])
awaitCond(bob.stateData.asInstanceOf[DATA_NORMAL].spliceStatus == SpliceStatus.NoSplice)
val rbfTxId = alice.stateData.asInstanceOf[DATA_NORMAL].spliceStatus.asInstanceOf[SpliceStatus.SpliceWaitingForSigs].signingSession.fundingTx.txId
disconnect(f)
val (channelReestablishAlice, channelReestablishBob) = reconnect(f)
assert(channelReestablishAlice.nextFundingTxId_opt.contains(rbfTxId))
assert(channelReestablishAlice.nextLocalCommitmentNumber == aliceCommitIndex)
assert(channelReestablishBob.nextFundingTxId_opt.contains(rbfTxId))
assert(channelReestablishBob.nextLocalCommitmentNumber == bobCommitIndex + 1)
bob2blockchain.expectWatchFundingConfirmed(spliceTx.txid)
// Alice and Bob retransmit commit_sig and tx_signatures.
alice2bob.expectMsgType[CommitSig]
alice2bob.forward(bob)
// Bob retransmits commit_sig, and they exchange tx_signatures afterwards.
bob2alice.expectMsgType[CommitSig]
bob2alice.forward(alice)
alice2bob.expectNoMessage(100 millis)
bob2alice.expectMsgType[TxSignatures]
bob2alice.forward(alice)
alice2bob.expectMsgType[TxSignatures]