1
0
mirror of https://github.com/ACINQ/eclair.git synced 2024-11-19 01:43:22 +01:00

Unlock non-wallet inputs (#2828)

When funding a transaction that contains inputs that are external to our
bitcoin wallet, bitcoin core will add a lock to those external inputs,
thinking that they may be wallet inputs that it doesn't know about yet.
In our case, those are never wallet inputs, they are instead:

- the current channel output, when creating a splice transaction
- an output of the commitment transaction, when force-closing

We previously explicitly filtered those inputs before calling `unlock`,
which was wrong. We now also unlock those utxos.
This commit is contained in:
Bastien Teinturier 2024-02-22 09:20:27 +01:00 committed by GitHub
parent 0393cfc275
commit cd4d9fd4b0
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
5 changed files with 50 additions and 20 deletions

View File

@ -117,7 +117,7 @@ trait DualFundingHandlers extends CommonFundingHandlers {
* never sent us their signatures, or the transaction wasn't accepted in our mempool), their inputs may still be locked.
*/
def rollbackDualFundingTxs(txs: Seq[SignedSharedTransaction]): Unit = {
val inputs = txs.flatMap(_.tx.localInputs).distinctBy(_.serialId).map(i => TxIn(i.outPoint, Nil, 0))
val inputs = txs.flatMap(sharedTx => sharedTx.tx.localInputs ++ sharedTx.tx.sharedInput_opt.toSeq).distinctBy(_.serialId).map(i => TxIn(i.outPoint, Nil, 0))
if (inputs.nonEmpty) {
wallet.rollback(Transaction(2, inputs, Nil, 0))
}

View File

@ -832,12 +832,15 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon
}
private def unlockAndStop(session: InteractiveTxSession): Behavior[Command] = {
val localInputs = session.localInputs ++ session.toSend.collect { case addInput: Input.Local => addInput }
val localInputs = session.localInputs ++ session.toSend.collect {
case addInput: Input.Local => addInput
case addInput: Input.Shared => addInput
}
unlockAndStop(localInputs.map(_.outPoint).toSet)
}
private def unlockAndStop(tx: SharedTransaction): Behavior[Command] = {
val localInputs = tx.localInputs.map(_.outPoint).toSet
val localInputs = tx.localInputs.map(_.outPoint).toSet ++ tx.sharedInput_opt.map(_.outPoint).toSet
unlockAndStop(localInputs)
}

View File

@ -20,7 +20,7 @@ import akka.actor.typed.eventstream.EventStream
import akka.actor.typed.scaladsl.{ActorContext, Behaviors}
import akka.actor.typed.{ActorRef, Behavior}
import fr.acinq.bitcoin.psbt.Psbt
import fr.acinq.bitcoin.scalacompat.{ByteVector32, OutPoint, Satoshi, Script, Transaction, TxOut}
import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, Script, Transaction, TxOut}
import fr.acinq.eclair.NotificationsLogger.NotifyNodeOperator
import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient
import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient.{FundTransactionOptions, InputWeight}
@ -364,7 +364,7 @@ private class ReplaceableTxFunder(nodeParams: NodeParams,
psbt match {
case Left(failure) =>
log.error(s"cannot sign ${cmd.desc}: $failure")
unlockAndStop(locallySignedTx.txInfo.input.outPoint, locallySignedTx.txInfo.tx, TxPublisher.TxRejectedReason.UnknownTxFailure)
unlockAndStop(locallySignedTx.txInfo.tx, TxPublisher.TxRejectedReason.UnknownTxFailure)
case Right(psbt1) =>
// The transaction that we want to fund/replace has one input, the first one. Additional inputs are provided by our on-chain wallet.
val ourWalletInputs = locallySignedTx.txInfo.tx.txIn.indices.tail
@ -402,13 +402,13 @@ private class ReplaceableTxFunder(nodeParams: NodeParams,
log.error(s"cannot sign ${cmd.desc}: ", reason)
// We reply with the failure only once the utxos are unlocked, otherwise there is a risk that our parent stops
// itself, which will automatically stop us before we had a chance to unlock them.
unlockAndStop(locallySignedTx.txInfo.input.outPoint, locallySignedTx.txInfo.tx, TxPublisher.TxRejectedReason.UnknownTxFailure)
unlockAndStop(locallySignedTx.txInfo.tx, TxPublisher.TxRejectedReason.UnknownTxFailure)
}
}
}
def unlockAndStop(input: OutPoint, tx: Transaction, failure: TxPublisher.TxRejectedReason): Behavior[Command] = {
val toUnlock = tx.txIn.filterNot(_.outPoint == input).map(_.outPoint)
def unlockAndStop(tx: Transaction, failure: TxPublisher.TxRejectedReason): Behavior[Command] = {
val toUnlock = tx.txIn.map(_.outPoint)
log.debug("unlocking utxos={}", toUnlock.mkString(", "))
context.pipeToSelf(bitcoinClient.unlockOutpoints(toUnlock))(_ => UtxosUnlocked)
Behaviors.receiveMessagePartial {

View File

@ -18,7 +18,7 @@ package fr.acinq.eclair.channel.publish
import akka.actor.typed.scaladsl.{ActorContext, Behaviors, TimerScheduler}
import akka.actor.typed.{ActorRef, Behavior}
import fr.acinq.bitcoin.scalacompat.{OutPoint, SatoshiLong, Transaction}
import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Transaction}
import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher
import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient
import fr.acinq.eclair.blockchain.fee.{ConfirmationTarget, FeeratePerKw, FeeratesPerKw}
@ -55,7 +55,6 @@ object ReplaceableTxPublisher {
private case class WrappedFundingResult(result: ReplaceableTxFunder.FundingResult) extends Command
private case class WrappedTxResult(result: MempoolTxMonitor.TxResult) extends Command
private case class BumpFee(targetFeerate: FeeratePerKw) extends Command
private case object UnlockUtxos extends Command
private case object UtxosUnlocked extends Command
// @formatter:on
@ -252,7 +251,7 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams,
case UpdateConfirmationTarget(target) =>
confirmationTarget = target
Behaviors.same
case Stop => unlockAndStop(cmd.input, Seq(tx.signedTx))
case Stop => unlockAndStop(Seq(tx.signedTx))
}
}
@ -321,13 +320,15 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams,
case Stop =>
// We don't know yet which transaction won, so we try abandoning both and unlocking their utxos.
// One of the calls will fail (for the transaction that is in the mempool), but we will simply ignore that failure.
unlockAndStop(cmd.input, Seq(previousTx.signedTx, bumpedTx.signedTx))
unlockAndStop(Seq(previousTx.signedTx, bumpedTx.signedTx))
}
}
// Clean up the failed transaction attempt. Once that's done, go back to the waiting state with the new transaction.
def cleanUpFailedTxAndWait(failedTx: Transaction, mempoolTx: FundedTx): Behavior[Command] = {
val toUnlock = failedTx.txIn.map(_.outPoint).toSet -- mempoolTx.signedTx.txIn.map(_.outPoint).toSet
// Note that we don't need to filter inputs from the previous transaction, they have already been unlocked when the
// previous transaction was published (but the bitcoin wallet ensures that they won't be double-spent).
val toUnlock = failedTx.txIn.map(_.outPoint).toSet
if (toUnlock.isEmpty) {
context.self ! UtxosUnlocked
} else {
@ -358,12 +359,12 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams,
def sendResult(result: TxPublisher.PublishTxResult, toUnlock_opt: Option[Seq[Transaction]]): Behavior[Command] = {
replyTo ! result
toUnlock_opt match {
case Some(txs) => unlockAndStop(cmd.input, txs)
case Some(txs) => unlockAndStop(txs)
case None => stop()
}
}
def unlockAndStop(input: OutPoint, txs: Seq[Transaction]): Behavior[Command] = {
def unlockAndStop(txs: Seq[Transaction]): Behavior[Command] = {
// Note that we unlock utxos but we don't abandon failed transactions:
// - if they were successfully published:
// - the utxos have automatically been unlocked, so the call to unlock is a (safe) no-op
@ -371,7 +372,7 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams,
// - if they failed to be published:
// - we must unlock the utxos, otherwise they would stay locked forever
// - abandoning the transaction would be a no-op, as it was never added to our wallet
val toUnlock = txs.flatMap(_.txIn).filterNot(_.outPoint == input).map(_.outPoint).toSet
val toUnlock = txs.flatMap(_.txIn).map(_.outPoint).toSet
if (toUnlock.isEmpty) {
context.self ! UtxosUnlocked
} else {

View File

@ -719,14 +719,23 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A
}
test("unlock outpoints correctly") {
assume(!useEclairSigner)
val sender = TestProbe()
val pubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(randomKey().publicKey, randomKey().publicKey)))
val bitcoinClient = makeBitcoinCoreClient()
val nonWalletScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(randomKey().publicKey, randomKey().publicKey)))
val nonWalletUtxo = {
bitcoinClient.sendToPubkeyScript(nonWalletScript, 150_000 sat, FeeratePerKw(1000 sat)).pipeTo(sender.ref)
val txId = sender.expectMsgType[TxId]
bitcoinClient.getTransaction(txId).pipeTo(sender.ref)
val tx = sender.expectMsgType[Transaction]
OutPoint(tx, tx.txOut.indexWhere(_.publicKeyScript == nonWalletScript))
}
{
// test #1: unlock outpoints that are actually locked
// test #1: unlock wallet outpoints that are actually locked
// create a huge tx so we make sure it has > 1 inputs
bitcoinClient.makeFundingTx(pubkeyScript, 250 btc, FeeratePerKw(1000 sat)).pipeTo(sender.ref)
bitcoinClient.makeFundingTx(nonWalletScript, 250 btc, FeeratePerKw(1000 sat)).pipeTo(sender.ref)
val MakeFundingTxResponse(fundingTx, _, _) = sender.expectMsgType[MakeFundingTxResponse]
assert(fundingTx.txIn.size > 2)
bitcoinClient.listLockedOutpoints().pipeTo(sender.ref)
@ -736,7 +745,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A
}
{
// test #2: some outpoints are locked, some are unlocked
bitcoinClient.makeFundingTx(pubkeyScript, 250 btc, FeeratePerKw(1000 sat)).pipeTo(sender.ref)
bitcoinClient.makeFundingTx(nonWalletScript, 250 btc, FeeratePerKw(1000 sat)).pipeTo(sender.ref)
val MakeFundingTxResponse(fundingTx, _, _) = sender.expectMsgType[MakeFundingTxResponse]
assert(fundingTx.txIn.size > 2)
bitcoinClient.listLockedOutpoints().pipeTo(sender.ref)
@ -755,6 +764,23 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A
bitcoinClient.listLockedOutpoints().pipeTo(sender.ref)
sender.expectMsg(Set.empty[OutPoint])
}
{
// test #3: lock and unlock non-wallet inputs
val txNotFunded = Transaction(2, Seq(TxIn(nonWalletUtxo, Nil, 0)), Seq(TxOut(250_000 sat, Script.pay2wpkh(randomKey().publicKey))), 0)
bitcoinClient.fundTransaction(txNotFunded, FeeratePerKw(1000 sat), replaceable = true, externalInputsWeight = Map(nonWalletUtxo -> 400L)).pipeTo(sender.ref)
val fundedTx = sender.expectMsgType[FundTransactionResponse].tx
assert(fundedTx.txIn.size > 1)
// the external input is also considered locked
bitcoinClient.listLockedOutpoints().pipeTo(sender.ref)
sender.expectMsg(fundedTx.txIn.map(_.outPoint).toSet)
// unlocking works for both wallet and non-wallet utxos
bitcoinClient.unlockOutpoints(fundedTx.txIn.map(_.outPoint)).pipeTo(sender.ref)
sender.expectMsg(true)
bitcoinClient.listLockedOutpoints().pipeTo(sender.ref)
sender.expectMsg(Set.empty[OutPoint])
}
}
test("sign transactions") {