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:
parent
21917f55dd
commit
9625a6e8c6
5 changed files with 300 additions and 142 deletions
|
@ -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.
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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]
|
||||
|
|
|
@ -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]
|
||||
|
|
|
@ -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]
|
||||
|
|
Loading…
Add table
Reference in a new issue