From 7b262ba17b45db1d35cda999f4dcfb951cc543bf Mon Sep 17 00:00:00 2001 From: sstone Date: Tue, 8 Nov 2022 19:27:53 +0100 Subject: [PATCH] Use PSBT to fund and sign transactions --- .../eclair/blockchain/OnChainWallet.scala | 6 +- .../bitcoind/rpc/BitcoinCoreClient.scala | 129 +++++++++++++++++- .../eclair/channel/InteractiveTxBuilder.scala | 14 +- .../channel/publish/ReplaceableTxFunder.scala | 33 +++-- .../blockchain/DummyOnChainWallet.scala | 26 ++++ .../bitcoind/BitcoinCoreClientSpec.scala | 65 +++++---- 6 files changed, 231 insertions(+), 42 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/OnChainWallet.scala b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/OnChainWallet.scala index 2650da519d..595ed69abf 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/OnChainWallet.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/OnChainWallet.scala @@ -16,8 +16,10 @@ package fr.acinq.eclair.blockchain +import fr.acinq.bitcoin.psbt.Psbt import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, Transaction} +import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient.ProcessPsbtResponse import fr.acinq.eclair.blockchain.fee.FeeratePerKw import scodec.bits.ByteVector @@ -38,7 +40,9 @@ trait OnChainChannelFunder { /** Sign the wallet inputs of the provided transaction. */ def signTransaction(tx: Transaction, allowIncomplete: Boolean)(implicit ec: ExecutionContext): Future[SignTransactionResponse] - /** + def signPsbt(psbt: Psbt)(implicit ec: ExecutionContext): Future[ProcessPsbtResponse] + + /** * Publish a transaction on the bitcoin network. * This method must be idempotent: if the tx was already published, it must return a success. */ diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/bitcoind/rpc/BitcoinCoreClient.scala b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/bitcoind/rpc/BitcoinCoreClient.scala index 2cd4b28db5..1d83678d80 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/bitcoind/rpc/BitcoinCoreClient.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/bitcoind/rpc/BitcoinCoreClient.scala @@ -16,15 +16,17 @@ package fr.acinq.eclair.blockchain.bitcoind.rpc +import fr.acinq.bitcoin.psbt.{KeyPathWithMaster, Psbt, UpdateFailure} import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey +import fr.acinq.bitcoin.scalacompat.DeterministicWallet.ExtendedPublicKey import fr.acinq.bitcoin.scalacompat._ -import fr.acinq.bitcoin.{Bech32, Block} +import fr.acinq.bitcoin.{Bech32, Block, SigHash} import fr.acinq.eclair.ShortChannelId.coordinates import fr.acinq.eclair.blockchain.OnChainWallet import fr.acinq.eclair.blockchain.OnChainWallet.{FundTransactionResponse, MakeFundingTxResponse, OnChainBalance, SignTransactionResponse} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{GetTxWithMetaResponse, UtxoStatus, ValidateResult} import fr.acinq.eclair.blockchain.fee.{FeeratePerKB, FeeratePerKw} -import fr.acinq.eclair.transactions.Transactions +import fr.acinq.eclair.transactions.{Scripts, Transactions} import fr.acinq.eclair.wire.protocol.ChannelAnnouncement import fr.acinq.eclair.{BlockHeight, TimestampSecond, TxCoordinates} import grizzled.slf4j.Logging @@ -32,8 +34,9 @@ import org.json4s.Formats import org.json4s.JsonAST._ import scodec.bits.ByteVector +import java.util.Base64 import scala.concurrent.{ExecutionContext, Future} -import scala.jdk.CollectionConverters.ListHasAsScala +import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava} import scala.util.{Failure, Success, Try} /** @@ -224,7 +227,74 @@ class BitcoinCoreClient(val rpcClient: BitcoinJsonRPCClient) extends OnChainWall fundTransaction(tx, FundTransactionOptions(feeRate, replaceable, lockUtxos)) } + def fundPsbt(inputs: Seq[FundPsbtInput], outputs: Seq[(String, Satoshi)], locktime: Long, options: FundPsbtOptions)(implicit ec: ExecutionContext): Future[FundPsbtResponse] = { + rpcClient.invoke("walletcreatefundedpsbt", inputs.toArray, outputs.map { case (a, b) => a -> b.toBtc.toBigDecimal }, locktime, options).map(json => { + val JString(base64) = json \ "psbt" + val JInt(changePos) = json \ "changepos" + val JDecimal(fee) = json \ "fee" + val bin = Base64.getDecoder.decode(base64) + val decoded = Psbt.read(bin) + require(decoded.isRight, s"cannot decode psbt from $base64") + val psbt = decoded.getRight + val changePos_opt = if (changePos >= 0) Some(changePos.intValue) else None + FundPsbtResponse(psbt, toSatoshi(fee), changePos_opt) + }) + } + + def fundPsbt(outputs: Seq[(String, Satoshi)], locktime: Long, options: FundPsbtOptions)(implicit ec: ExecutionContext): Future[FundPsbtResponse] = + fundPsbt(Seq(), outputs, locktime, options) + + def processPsbt(psbt: Psbt, sign: Boolean = true, sighashType: Int = SigHash.SIGHASH_ALL)(implicit ec: ExecutionContext): Future[ProcessPsbtResponse] = { + val sighashStrings = Map( + SigHash.SIGHASH_ALL -> "ALL", + SigHash.SIGHASH_NONE -> "NONE", + SigHash.SIGHASH_SINGLE -> "SINGLE", + (SigHash.SIGHASH_ALL | SigHash.SIGHASH_ANYONECANPAY) -> "ALL|ANYONECANPAY", + (SigHash.SIGHASH_NONE | SigHash.SIGHASH_ANYONECANPAY) -> "NONE|ANYONECANPAY", + (SigHash.SIGHASH_SINGLE | SigHash.SIGHASH_ANYONECANPAY) -> "SINGLE|ANYONECANPAY") + val sighash = sighashStrings.getOrElse(sighashType, throw new IllegalArgumentException(s"invalid sighash flag ${sighashType}")) + val encoded = Base64.getEncoder.encodeToString(Psbt.write(psbt).toByteArray) + rpcClient.invoke("walletprocesspsbt", encoded, sign, sighash).map(json => { + val JString(base64) = json \ "psbt" + val JBool(complete) = json \ "complete" + val decoded = Psbt.read(Base64.getDecoder.decode(base64)) + require(decoded.isRight, s"cannot decode psbt from $base64") + ProcessPsbtResponse(decoded.getRight, complete) + }) + } + + def utxoUpdatePsbt(psbt: Psbt)(implicit ec: ExecutionContext): Future[Psbt] = { + val encoded = Base64.getEncoder.encodeToString(Psbt.write(psbt).toByteArray) + + rpcClient.invoke("utxoupdatepsbt", encoded).map(json => { + val JString(base64) = json + val bin = Base64.getDecoder.decode(base64) + val decoded = Psbt.read(bin) + require(decoded.isRight, s"cannot decode psbt from $base64") + val psbt = decoded.getRight + psbt + }) + } + + private def signPsbtOrUnlock(psbt: Psbt)(implicit ec: ExecutionContext): Future[ProcessPsbtResponse] = { + val f = for { + ProcessPsbtResponse(psbt1, complete) <- processPsbt(psbt) + _ = if (!complete) throw JsonRPCError(Error(0, "cannot sign psbt")) + } yield ProcessPsbtResponse(psbt1, complete) + // if signature fails (e.g. because wallet is encrypted) we need to unlock the utxos + f.recoverWith { case _ => + unlockOutpoints(psbt.getGlobal.getTx.txIn.asScala.toSeq.map(_.outPoint).map(KotlinUtils.kmp2scala)) + .recover { case t: Throwable => // no-op, just add a log in case of failure + logger.warn(s"Cannot unlock failed transaction's UTXOs txid=${psbt.getGlobal.getTx.txid}", t) + t + } + .flatMap(_ => f) // return signTransaction error + .recoverWith { case _ => f } // return signTransaction error + } + } + def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, targetFeerate: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { + import KotlinUtils._ val partialFundingTx = Transaction( version = 2, txIn = Seq.empty[TxIn], @@ -235,7 +305,10 @@ class BitcoinCoreClient(val rpcClient: BitcoinJsonRPCClient) extends OnChainWall // we ask bitcoin core to add inputs to the funding tx, and use the specified change address fundTxResponse <- fundTransaction(partialFundingTx, FundTransactionOptions(feerate, lockUtxos = true)) // now let's sign the funding tx - SignTransactionResponse(fundingTx, true) <- signTransactionOrUnlock(fundTxResponse.tx) + psbt = new Psbt(fundTxResponse.tx) + updatedPsbt <- utxoUpdatePsbt(psbt) + ProcessPsbtResponse(signedPsbt, true) <- signPsbtOrUnlock(updatedPsbt) + fundingTx = signedPsbt.extract().getRight // there will probably be a change output, so we need to find which output is ours outputIndex <- Transactions.findPubKeyScriptIndex(fundingTx, pubkeyScript) match { case Right(outputIndex) => Future.successful(outputIndex) @@ -257,6 +330,10 @@ class BitcoinCoreClient(val rpcClient: BitcoinJsonRPCClient) extends OnChainWall //------------------------- SIGNING -------------------------// + def signPsbt(psbt: Psbt)(implicit ec: ExecutionContext): Future[ProcessPsbtResponse] = { + utxoUpdatePsbt(psbt).flatMap(p => processPsbt(p)) + } + def signTransaction(tx: Transaction)(implicit ec: ExecutionContext): Future[SignTransactionResponse] = signTransaction(tx, Nil) def signTransaction(tx: Transaction, allowIncomplete: Boolean)(implicit ec: ExecutionContext): Future[SignTransactionResponse] = signTransaction(tx, Nil, allowIncomplete) @@ -493,6 +570,50 @@ object BitcoinCoreClient { } } + case class FundPsbtInput(txid: ByteVector32, vout: Int, sequence: Option[Long] = None, weight: Option[Int] = None) + + object FundPsbtInput { + def apply(outPoint: OutPoint, sequence: Option[Long], weight: Option[Int]): FundPsbtInput = FundPsbtInput(outPoint.txid, outPoint.index.toInt, sequence, weight) + } + + case class FundPsbtOptions(feeRate: BigDecimal, replaceable: Boolean, lockUnspents: Boolean, changePosition: Option[Int], add_inputs: Boolean) + + object FundPsbtOptions { + def apply(feerate: FeeratePerKw, replaceable: Boolean = true, lockUtxos: Boolean = false, changePosition: Option[Int] = None, add_inputs: Boolean = true): FundPsbtOptions = { + FundPsbtOptions(BigDecimal(FeeratePerKB(feerate).toLong).bigDecimal.scaleByPowerOfTen(-8), replaceable, lockUtxos, changePosition, add_inputs) + } + } + + case class FundPsbtResponse(psbt: Psbt, fee: Satoshi, changePosition: Option[Int]) { + val tx: Transaction = KotlinUtils.kmp2scala(psbt.getGlobal.getTx) + val amountIn: Satoshi = fee + tx.txOut.map(_.amount).sum + } + + case class ProcessPsbtResponse(psbt: Psbt, complete: Boolean) { + + // this can only work if the psbt if fully signed + def extractFinalTx: Either[UpdateFailure, Transaction] = { + val extracted = psbt.extract() + if (extracted.isLeft) Left(extracted.getLeft) else Right(KotlinUtils.kmp2scala(extracted.getRight)) + } + + // use if you're not expecting the psbt to be fully signed + def extractPartiallySignedTx: Transaction = { + import KotlinUtils._ + + var partiallySignedTx: Transaction = psbt.getGlobal.getTx + for (i <- 0 until psbt.getInputs.size()) { + val scriptWitness = psbt.getInputs.get(i).getScriptWitness + if (scriptWitness != null) { + partiallySignedTx = partiallySignedTx.updateWitness(i, scriptWitness) + } + } + partiallySignedTx + } + + def finalTx = extractFinalTx.getOrElse(throw new RuntimeException("cannot extract transaction from psbt")) + } + case class PreviousTx(txid: ByteVector32, vout: Long, scriptPubKey: String, redeemScript: String, witnessScript: String, amount: BigDecimal) object PreviousTx { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/InteractiveTxBuilder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/InteractiveTxBuilder.scala index 108e7f807b..026bca5f31 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/InteractiveTxBuilder.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/InteractiveTxBuilder.scala @@ -19,10 +19,11 @@ package fr.acinq.eclair.channel import akka.actor.typed.scaladsl.{ActorContext, Behaviors, StashBuffer} import akka.actor.typed.{ActorRef, Behavior} import fr.acinq.bitcoin.ScriptFlags +import fr.acinq.bitcoin.psbt.{Global, Psbt} import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.bitcoin.scalacompat.{ByteVector32, LexicographicalOrdering, OutPoint, Satoshi, SatoshiLong, Script, Transaction, TxIn, TxOut} import fr.acinq.eclair.blockchain.OnChainChannelFunder -import fr.acinq.eclair.blockchain.OnChainWallet.SignTransactionResponse +import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient.ProcessPsbtResponse import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.channel.Helpers.Funding import fr.acinq.eclair.crypto.ShaChain @@ -812,15 +813,18 @@ private class InteractiveTxBuilder(replyTo: ActorRef[InteractiveTxBuilder.Respon } private def signTx(unsignedTx: SharedTransaction, remoteSigs_opt: Option[TxSignatures]): Unit = { + import fr.acinq.bitcoin.scalacompat.KotlinUtils._ + val tx = unsignedTx.buildUnsignedTx() if (unsignedTx.localInputs.isEmpty) { context.self ! SignTransactionResult(PartiallySignedSharedTransaction(unsignedTx, TxSignatures(fundingParams.channelId, tx, Nil)), remoteSigs_opt) } else { - context.pipeToSelf(wallet.signTransaction(tx, allowIncomplete = true).map { - case SignTransactionResponse(signedTx, _) => + context.pipeToSelf(wallet.signPsbt(new Psbt(tx)).map { + case ppr@ProcessPsbtResponse(signedTx, _) => val localOutpoints = unsignedTx.localInputs.map(toOutPoint).toSet - val sigs = signedTx.txIn.filter(txIn => localOutpoints.contains(txIn.outPoint)).map(_.witness) - PartiallySignedSharedTransaction(unsignedTx, TxSignatures(fundingParams.channelId, tx, sigs)) + val partiallySignedTx = ppr.extractPartiallySignedTx + val sigs = partiallySignedTx.txIn.filter(txIn => localOutpoints.contains(txIn.outPoint)).map(_.witness) + PartiallySignedSharedTransaction(unsignedTx, TxSignatures(fundingParams.channelId, partiallySignedTx, sigs)) }) { case Failure(t) => WalletFailure(t) case Success(signedTx) => SignTransactionResult(signedTx, remoteSigs_opt) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxFunder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxFunder.scala index 4fe4e7a577..09ff849534 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxFunder.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxFunder.scala @@ -19,10 +19,12 @@ package fr.acinq.eclair.channel.publish import akka.actor.typed.eventstream.EventStream import akka.actor.typed.scaladsl.{ActorContext, Behaviors} import akka.actor.typed.{ActorRef, Behavior} -import fr.acinq.bitcoin.scalacompat.{ByteVector32, OutPoint, Satoshi, Script, Transaction, TxOut} +import fr.acinq.bitcoin.psbt.Psbt +import fr.acinq.bitcoin.scalacompat.{ByteVector32, OutPoint, Satoshi, Script, Transaction, TxOut, computeBIP44Address, computeBIP84Address} +import fr.acinq.bitcoin.utils.EitherKt import fr.acinq.eclair.NotificationsLogger.NotifyNodeOperator import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient -import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient.{FundTransactionOptions, InputWeight} +import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient.{FundPsbtInput, FundPsbtOptions, FundTransactionOptions, InputWeight} import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.channel.Commitments import fr.acinq.eclair.channel.publish.ReplaceableTxPrePublisher._ @@ -314,10 +316,19 @@ private class ReplaceableTxFunder(nodeParams: NodeParams, } def signWalletInputs(locallySignedTx: ReplaceableTxWithWalletInputs, txFeerate: FeeratePerKw, amountIn: Satoshi): Behavior[Command] = { - val inputInfo = BitcoinCoreClient.PreviousTx(locallySignedTx.txInfo.input, locallySignedTx.txInfo.tx.txIn.head.witness) - context.pipeToSelf(bitcoinClient.signTransaction(locallySignedTx.txInfo.tx, Seq(inputInfo))) { - case Success(signedTx) => SignWalletInputsOk(signedTx.tx) - case Failure(reason) => SignWalletInputsFailed(reason) + import fr.acinq.bitcoin.scalacompat.KotlinUtils._ + + val psbt = new Psbt(locallySignedTx.txInfo.tx) + val updated = psbt.updateWitnessInput(locallySignedTx.txInfo.input.outPoint, locallySignedTx.txInfo.input.txOut, null, fr.acinq.bitcoin.Script.parse(locallySignedTx.txInfo.input.redeemScript), null, java.util.Map.of()) + val signed = EitherKt.flatMap(updated, (psbt: Psbt) => psbt.finalizeWitnessInput(0, locallySignedTx.txInfo.tx.txIn.head.witness)) + val psbt1 = signed.getRight + val f = bitcoinClient.utxoUpdatePsbt(psbt1).flatMap(p => bitcoinClient.processPsbt(p)) + context.pipeToSelf(f) { + case Success(processPsbtResponse) => + val signedTx = processPsbtResponse.finalTx + SignWalletInputsOk(signedTx) + case Failure(reason) => + SignWalletInputsFailed(reason) } Behaviors.receiveMessagePartial { case SignWalletInputsOk(signedTx) => @@ -352,6 +363,8 @@ private class ReplaceableTxFunder(nodeParams: NodeParams, } private def addInputs(anchorTx: ClaimLocalAnchorWithWitnessData, targetFeerate: FeeratePerKw, commitments: Commitments): Future[(ClaimLocalAnchorWithWitnessData, Satoshi)] = { + import fr.acinq.bitcoin.scalacompat.KotlinUtils._ + val dustLimit = commitments.localParams.dustLimit val commitTx = dummySignedCommitTx(commitments).tx // NB: fundrawtransaction requires at least one output, and may add at most one additional change output. @@ -368,8 +381,11 @@ private class ReplaceableTxFunder(nodeParams: NodeParams, val changeOutput = fundTxResponse.tx.txOut(changePos) val txSingleOutput = fundTxResponse.tx.copy(txOut = Seq(changeOutput)) // We ask bitcoind to sign the wallet inputs to learn their final weight and adjust the change amount. - bitcoinClient.signTransaction(txSingleOutput, allowIncomplete = true).map(signTxResponse => { - val dummySignedTx = addSigs(anchorTx.updateTx(signTxResponse.tx).txInfo, PlaceHolderSig) + val psbt = new Psbt(txSingleOutput) + bitcoinClient.utxoUpdatePsbt(psbt).flatMap(p => bitcoinClient.processPsbt(p)).map(processPsbtResponse => { + // we cannot extract the final tx from the psbt because it is not fully signed yet + val partiallySignedTx = processPsbtResponse.extractPartiallySignedTx + val dummySignedTx = addSigs(anchorTx.updateTx(partiallySignedTx).txInfo, PlaceHolderSig) val packageWeight = commitTx.weight() + dummySignedTx.tx.weight() val anchorTxFee = weight2fee(targetFeerate, packageWeight) - weight2fee(commitments.localCommit.spec.commitTxFeerate, commitTx.weight()) val changeAmount = dustLimit.max(fundTxResponse.amountIn - anchorTxFee) @@ -395,5 +411,4 @@ private class ReplaceableTxFunder(nodeParams: NodeParams, (unsignedTx, fundTxResponse.amountIn) }) } - } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala index e57726d3ee..f94243c2fa 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/DummyOnChainWallet.scala @@ -17,16 +17,20 @@ package fr.acinq.eclair.blockchain import fr.acinq.bitcoin.TxIn.SEQUENCE_FINAL +import fr.acinq.bitcoin.psbt.Psbt import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto, OutPoint, Satoshi, SatoshiLong, Script, Transaction, TxIn, TxOut} import fr.acinq.bitcoin.{Bech32, SigHash, SigVersion} import fr.acinq.eclair.blockchain.OnChainWallet.{FundTransactionResponse, MakeFundingTxResponse, OnChainBalance, SignTransactionResponse} +import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient +import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient.ProcessPsbtResponse import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.transactions.Transactions import fr.acinq.eclair.{randomBytes32, randomKey} import scodec.bits._ import scala.concurrent.{ExecutionContext, Future, Promise} +import scala.jdk.CollectionConverters.SeqHasAsJava /** * Created by PM on 06/07/2017. @@ -48,6 +52,8 @@ class DummyOnChainWallet extends OnChainWallet { override def signTransaction(tx: Transaction, allowIncomplete: Boolean)(implicit ec: ExecutionContext): Future[SignTransactionResponse] = Future.successful(SignTransactionResponse(tx, complete = true)) + override def signPsbt(psbt: Psbt)(implicit ec: ExecutionContext): Future[BitcoinCoreClient.ProcessPsbtResponse] = Future.successful(ProcessPsbtResponse(psbt, complete = true)) + override def publishTransaction(tx: Transaction)(implicit ec: ExecutionContext): Future[ByteVector32] = Future.successful(tx.txid) override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { @@ -88,6 +94,8 @@ class NoOpOnChainWallet extends OnChainWallet { override def signTransaction(tx: Transaction, allowIncomplete: Boolean)(implicit ec: ExecutionContext): Future[SignTransactionResponse] = Promise().future // will never be completed + override def signPsbt(psbt: Psbt)(implicit ec: ExecutionContext): Future[ProcessPsbtResponse] = Promise().future // will never be completed + override def publishTransaction(tx: Transaction)(implicit ec: ExecutionContext): Future[ByteVector32] = Future.successful(tx.txid) override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = Promise().future // will never be completed @@ -154,6 +162,24 @@ class SingleKeyOnChainWallet extends OnChainWallet { Future.successful(SignTransactionResponse(signedTx, complete)) } + override def signPsbt(psbt: Psbt)(implicit ec: ExecutionContext): Future[ProcessPsbtResponse] = { + import fr.acinq.bitcoin.scalacompat.KotlinUtils._ + + val tx: Transaction = psbt.getGlobal.getTx + val signedPsbt = tx.txIn.zipWithIndex.foldLeft(new Psbt(tx)) { + case (currentPsbt, (txIn, index)) => inputs.find(_.txid == txIn.outPoint.txid) match { + case Some(inputTx) => + val sig = Transaction.signInput(tx, index, Script.pay2pkh(pubkey), SigHash.SIGHASH_ALL, inputTx.txOut.head.amount, SigVersion.SIGVERSION_WITNESS_V0, privkey) + val updated = currentPsbt.updateWitnessInput(OutPoint(inputTx, 0), inputTx.txOut.head, null, Script.pay2pkh(pubkey).map(scala2kmp).asJava, null, java.util.Map.of()).getRight + val finalized = updated.finalizeWitnessInput(OutPoint(inputTx, 0), Script.witnessPay2wpkh(pubkey, sig)) + finalized.getRight + case None => currentPsbt + } + } + val complete = signedPsbt.extract().isRight + Future.successful(ProcessPsbtResponse(signedPsbt, complete)) + } + override def publishTransaction(tx: Transaction)(implicit ec: ExecutionContext): Future[ByteVector32] = Future.successful(tx.txid) override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/BitcoinCoreClientSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/BitcoinCoreClientSpec.scala index fd03955064..eee0503db3 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/BitcoinCoreClientSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/BitcoinCoreClientSpec.scala @@ -20,9 +20,11 @@ import akka.actor.Status.Failure import akka.pattern.pipe import akka.testkit.TestProbe import fr.acinq.bitcoin +import fr.acinq.bitcoin.psbt.Psbt import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey -import fr.acinq.bitcoin.scalacompat.{Block, Btc, BtcDouble, ByteVector32, MilliBtcDouble, OutPoint, Satoshi, SatoshiLong, Script, ScriptWitness, Transaction, TxIn, TxOut, computeP2WpkhAddress} -import fr.acinq.bitcoin.{Bech32, SigHash, SigVersion} +import fr.acinq.bitcoin.scalacompat.{Block, Btc, BtcDouble, ByteVector32, KotlinUtils, MilliBtcDouble, OutPoint, Satoshi, SatoshiLong, Script, ScriptWitness, Transaction, TxIn, TxOut, computeBIP84Address, computeP2WpkhAddress, computeScriptAddress} +import fr.acinq.bitcoin.utils.EitherKt +import fr.acinq.bitcoin.{Bech32, Bitcoin, SigHash, SigVersion} import fr.acinq.eclair.blockchain.OnChainWallet.{FundTransactionResponse, MakeFundingTxResponse, OnChainBalance, SignTransactionResponse} import fr.acinq.eclair.blockchain.WatcherSpec.{createSpendManyP2WPKH, createSpendP2WPKH} import fr.acinq.eclair.blockchain.bitcoind.BitcoindService.BitcoinReq @@ -75,6 +77,8 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A } test("fund transactions") { + import KotlinUtils._ + val sender = TestProbe() val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) @@ -92,15 +96,15 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A fundTxResponse.tx.txIn.foreach(txIn => assert(txIn.signatureScript.isEmpty && txIn.witness.isNull)) fundTxResponse.tx.txIn.foreach(txIn => assert(txIn.sequence == bitcoin.TxIn.SEQUENCE_FINAL - 2)) - bitcoinClient.signTransaction(fundTxResponse.tx, Nil).pipeTo(sender.ref) - val signTxResponse = sender.expectMsgType[SignTransactionResponse] + bitcoinClient.utxoUpdatePsbt(new Psbt(fundTxResponse.tx)).flatMap(p => bitcoinClient.processPsbt(p)).pipeTo(sender.ref) + val signTxResponse = sender.expectMsgType[ProcessPsbtResponse] assert(signTxResponse.complete) - assert(signTxResponse.tx.txOut.size == 2) + assert(signTxResponse.finalTx.txOut.size == 2) - bitcoinClient.publishTransaction(signTxResponse.tx).pipeTo(sender.ref) - sender.expectMsg(signTxResponse.tx.txid) + bitcoinClient.publishTransaction(signTxResponse.finalTx).pipeTo(sender.ref) + sender.expectMsg(signTxResponse.finalTx.txid) generateBlocks(1) - signTxResponse.tx + signTxResponse.finalTx } { // txs with no outputs are not supported. @@ -137,6 +141,8 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A } test("fund transactions with external inputs") { + import KotlinUtils._ + val sender = TestProbe() val defaultWallet = new BitcoinCoreClient(bitcoinrpcclient) val walletExternalFunds = new BitcoinCoreClient(createWallet("external_inputs", sender)) @@ -152,16 +158,17 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A // We receive more funds on an address that does not belong to our wallet. val externalInputWeight = 310 val (alicePriv, bobPriv, carolPriv) = (randomKey(), randomKey(), randomKey()) - val (outpoint1, inputScript1) = { + val (outpoint1, inputScript1, txOut1) = { val script = Script.createMultiSigMofN(1, Seq(alicePriv.publicKey, bobPriv.publicKey)) val txNotFunded = Transaction(2, Nil, Seq(TxOut(250_000 sat, Script.pay2wsh(script))), 0) defaultWallet.fundTransaction(txNotFunded, FundTransactionOptions(FeeratePerKw(2500 sat), changePosition = Some(1))).pipeTo(sender.ref) val fundedTx = sender.expectMsgType[FundTransactionResponse].tx - defaultWallet.signTransaction(fundedTx, Nil).pipeTo(sender.ref) - val signedTx = sender.expectMsgType[SignTransactionResponse].tx + //defaultWallet.signTransaction(fundedTx, Nil).pipeTo(sender.ref) + defaultWallet.utxoUpdatePsbt(new Psbt(fundedTx)).flatMap(p => defaultWallet.processPsbt(p)).pipeTo(sender.ref) + val signedTx = sender.expectMsgType[ProcessPsbtResponse].finalTx defaultWallet.publishTransaction(signedTx).pipeTo(sender.ref) sender.expectMsg(signedTx.txid) - (OutPoint(signedTx, 0), script) + (OutPoint(signedTx, 0), script, signedTx.txOut(0)) } // We make sure these utxos are confirmed. @@ -194,10 +201,14 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A assert(amountIn2 == fundedTx2.amountIn) // We sign our external input. val externalSig = Transaction.signInput(fundedTx2.tx, 0, inputScript1, SigHash.SIGHASH_ALL, 250_000 sat, SigVersion.SIGVERSION_WITNESS_V0, alicePriv) - val partiallySignedTx = fundedTx2.tx.updateWitness(0, Script.witnessMultiSigMofN(Seq(alicePriv, bobPriv).map(_.publicKey), Seq(externalSig))) // And let bitcoind sign the wallet input. - walletExternalFunds.signTransaction(partiallySignedTx, Nil).pipeTo(sender.ref) - val signedTx = sender.expectMsgType[SignTransactionResponse].tx + walletExternalFunds.utxoUpdatePsbt(new Psbt(fundedTx2.tx)).flatMap(p => walletExternalFunds.processPsbt(p)).pipeTo(sender.ref) + val psbt = sender.expectMsgType[ProcessPsbtResponse].psbt + val updated = psbt.updateWitnessInput(outpoint1, txOut1, null, null, null, java.util.Map.of()) + val finalized = EitherKt.flatMap(updated, (psbt: Psbt) => psbt.finalizeWitnessInput(0, Script.witnessMultiSigMofN(Seq(alicePriv, bobPriv).map(_.publicKey), Seq(externalSig)))) + val psbt1: Psbt = finalized.getRight + val signedTx: Transaction = psbt1.extract().getRight + walletExternalFunds.publishTransaction(signedTx).pipeTo(sender.ref) sender.expectMsg(signedTx.txid) // The weight of our external input matches our estimation and the resulting feerate is correct. @@ -219,12 +230,16 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A walletExternalFunds.fundTransaction(txNotFunded, FundTransactionOptions(targetFeerate, inputWeights = Seq(InputWeight(externalOutpoint, externalInputWeight)), changePosition = Some(1))).pipeTo(sender.ref) val fundedTx = sender.expectMsgType[FundTransactionResponse] assert(fundedTx.tx.txIn.length >= 2) + // bitcoind signs the wallet input. + walletExternalFunds.utxoUpdatePsbt(new Psbt(fundedTx.tx)).flatMap(p => walletExternalFunds.processPsbt(p)).pipeTo(sender.ref) + val psbt = sender.expectMsgType[ProcessPsbtResponse].psbt + // We sign our external input. val externalSig = Transaction.signInput(fundedTx.tx, 0, inputScript2, SigHash.SIGHASH_ALL, 300_000 sat, SigVersion.SIGVERSION_WITNESS_V0, alicePriv) - val partiallySignedTx = fundedTx.tx.updateWitness(0, Script.witnessMultiSigMofN(Seq(alicePriv, carolPriv).map(_.publicKey), Seq(externalSig))) - // And let bitcoind sign the wallet input. - walletExternalFunds.signTransaction(partiallySignedTx, Nil).pipeTo(sender.ref) - val signedTx = sender.expectMsgType[SignTransactionResponse].tx + val updated = psbt.updateWitnessInput(externalOutpoint, tx2.txOut(0), null, null, null, java.util.Map.of()) + val finalized = EitherKt.flatMap(updated, (psbt: Psbt) => psbt.finalizeWitnessInput(0, Script.witnessMultiSigMofN(Seq(alicePriv, carolPriv).map(_.publicKey), Seq(externalSig)))) + val signedTx: Transaction = finalized.getRight.extract().getRight + walletExternalFunds.publishTransaction(signedTx).pipeTo(sender.ref) sender.expectMsg(signedTx.txid) // The resulting feerate takes into account our unconfirmed parent as well. @@ -252,12 +267,16 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A val fundedTx = sender.expectMsgType[FundTransactionResponse] assert(fundedTx.tx.txIn.length >= 2) assert(fundedTx.tx.txOut.length == 2) + + // bitcoind signs the wallet input. + walletExternalFunds.utxoUpdatePsbt(new Psbt(fundedTx.tx)).flatMap(p => walletExternalFunds.processPsbt(p)).pipeTo(sender.ref) + val psbt = sender.expectMsgType[ProcessPsbtResponse].psbt + // We sign our external input. val externalSig = Transaction.signInput(fundedTx.tx, 0, inputScript2, SigHash.SIGHASH_ALL, 300_000 sat, SigVersion.SIGVERSION_WITNESS_V0, alicePriv) - val partiallySignedTx = fundedTx.tx.updateWitness(0, Script.witnessMultiSigMofN(Seq(alicePriv, carolPriv).map(_.publicKey), Seq(externalSig))) - // And let bitcoind sign the wallet input. - walletExternalFunds.signTransaction(partiallySignedTx, Nil).pipeTo(sender.ref) - val signedTx = sender.expectMsgType[SignTransactionResponse].tx + val updated = psbt.updateWitnessInput(OutPoint(tx2, 0), tx2.txOut(0), null, null, null, java.util.Map.of()) + val finalized = EitherKt.flatMap(updated, (psbt: Psbt) => psbt.finalizeWitnessInput(0, Script.witnessMultiSigMofN(Seq(alicePriv, carolPriv).map(_.publicKey), Seq(externalSig)))) + val signedTx: Transaction = finalized.getRight.extract().getRight walletExternalFunds.publishTransaction(signedTx).pipeTo(sender.ref) sender.expectMsg(signedTx.txid) // We have replaced the previous transaction.