From 8b00eceb1d0e6c277bc6b8700c5a2bc63abf7749 Mon Sep 17 00:00:00 2001 From: sstone Date: Wed, 19 Jan 2022 10:59:58 +0100 Subject: [PATCH] Use psbt to fund transactions --- .../eclair/blockchain/OnChainWallet.scala | 11 +- .../bitcoind/rpc/BitcoinCoreClient.scala | 119 ++++++++++++++--- .../fr/acinq/eclair/channel/fsm/Channel.scala | 5 +- .../channel/fsm/ChannelOpenSingleFunder.scala | 8 +- .../channel/publish/ReplaceableTxFunder.scala | 125 ++++++++++++------ .../blockchain/DummyOnChainWallet.scala | 29 +++- .../bitcoind/BitcoinCoreClientSpec.scala | 118 +++++++++++++---- .../AnnouncementsBatchValidationSpec.scala | 17 ++- 8 files changed, 332 insertions(+), 100 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 3f62f45009..02b056bd08 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.{Satoshi, Transaction} +import fr.acinq.bitcoin.scalacompat.DeterministicWallet.ExtendedPublicKey +import fr.acinq.bitcoin.scalacompat.{ByteVector32, KotlinUtils, Satoshi, Transaction} import fr.acinq.eclair.blockchain.fee.FeeratePerKw import scodec.bits.ByteVector @@ -33,7 +35,7 @@ trait OnChainChannelFunder { import OnChainWallet.MakeFundingTxResponse /** Create a channel funding transaction with the provided pubkeyScript. */ - def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] + def makeFundingTx(chainHash: ByteVector32, localFundingKey: ExtendedPublicKey, remoteFundingKey: PublicKey, amount: Satoshi, feeRatePerKw: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] /** * Committing *must* include publishing the transaction on the network. @@ -95,6 +97,7 @@ object OnChainWallet { final case class OnChainBalance(confirmed: Satoshi, unconfirmed: Satoshi) - final case class MakeFundingTxResponse(fundingTx: Transaction, fundingTxOutputIndex: Int, fee: Satoshi) - + final case class MakeFundingTxResponse(psbt: Psbt, fundingTxOutputIndex: Int, fee: Satoshi) { + def fundingTx(): Transaction = KotlinUtils.kmp2scala(psbt.extract().getRight) + } } 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 896b1005fe..d68f1c4f2a 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,18 @@ package fr.acinq.eclair.blockchain.bitcoind.rpc +import fr.acinq.bitcoin.psbt.{KeyPathWithMaster, Psbt} import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey import fr.acinq.bitcoin.scalacompat._ -import fr.acinq.bitcoin.{Bech32, Block} +import fr.acinq.bitcoin.{Bech32, Block, SigHash} +import fr.acinq.bitcoin.scalacompat.DeterministicWallet.ExtendedPublicKey import fr.acinq.eclair.ShortChannelId.coordinates import fr.acinq.eclair.blockchain.OnChainWallet import fr.acinq.eclair.blockchain.OnChainWallet.{MakeFundingTxResponse, OnChainBalance} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{GetTxWithMetaResponse, UtxoStatus, ValidateResult} +import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient.{FundPsbtInput, FundPsbtOptions, FundPsbtResponse, ProcessPsbtResponse} 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 +35,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} /** @@ -220,25 +224,92 @@ class BitcoinCoreClient(val rpcClient: BitcoinJsonRPCClient) extends OnChainWall }) } - def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, targetFeerate: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { - val partialFundingTx = Transaction( - version = 2, - txIn = Seq.empty[TxIn], - txOut = TxOut(amount, pubkeyScript) :: Nil, - lockTime = 0) + 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 psbt = Psbt.read(bin).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}")) + rpcClient.invoke("walletprocesspsbt", Base64.getEncoder.encodeToString(Psbt.write(psbt).toByteArray), sign, sighash).map(json => { + val JString(base64) = json \ "psbt" + val JBool(complete) = json \ "complete" + val psbt = Psbt.read(Base64.getDecoder.decode(base64)).getRight + ProcessPsbtResponse(psbt, complete) + }) + } + + 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 + } + } + + override def makeFundingTx(chainHash: ByteVector32, localFundingKey: ExtendedPublicKey, remoteFundingKey: PublicKey, amount: Satoshi, feeRatePerKw: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { + logger.info(s"funding psbt with local_funding_key=$localFundingKey and remote_funding_key=$remoteFundingKey") + val fundingPubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingKey.publicKey, remoteFundingKey))) + val Some(fundingAddress) = computeScriptAddress(chainHash, fundingPubkeyScript) + + def updatePsbt(psbt: Psbt, changepos_opt: Option[Int], ourbip32path: Seq[Long]): Psbt = { + val outputIndex = changepos_opt match { + case None => 0 + case Some(changePos) => 1 - changePos + } + psbt.updateWitnessOutput(outputIndex, null, null, Map( + KotlinUtils.scala2kmp(localFundingKey.publicKey) -> new KeyPathWithMaster(localFundingKey.parent, DeterministicWallet.KeyPath(ourbip32path).keyPath), + KotlinUtils.scala2kmp(remoteFundingKey) -> new KeyPathWithMaster(0L, DeterministicWallet.KeyPath("1/2/3/4").keyPath) + ).asJava + ).getRight + } + for { - feerate <- mempoolMinFee().map(minFee => FeeratePerKw(minFee).max(targetFeerate)) - // we ask bitcoin core to add inputs to the funding tx, and use the specified change address - fundTxResponse <- fundTransaction(partialFundingTx, FundTransactionOptions(feerate, lockUtxos = true)) + // we ask bitcoin core to create and fund the funding tx + feerate <- mempoolMinFee().map(minFee => FeeratePerKw(minFee).max(feeRatePerKw)) + FundPsbtResponse(psbt, fee, changePos_opt) <- fundPsbt(Seq(fundingAddress -> amount), 0, FundPsbtOptions(feerate, lockUtxos = true, changePosition = Some(1))) + ourbip32path = localFundingKey.path.path.drop(2) + _ = logger.info(s"funded psbt = $psbt") + psbt1 = updatePsbt(psbt, changePos_opt, ourbip32path) // now let's sign the funding tx - SignTransactionResponse(fundingTx, true) <- signTransactionOrUnlock(fundTxResponse.tx) + ProcessPsbtResponse(signedPsbt, complete) <- signPsbtOrUnlock(psbt1) + _ = logger.info(s"psbt signing complete = $complete") + extracted = signedPsbt.extract() + _ = if (extracted.isLeft) logger.error(s"psbt failure $extracted") + fundingTx = extracted.getRight // there will probably be a change output, so we need to find which output is ours - outputIndex <- Transactions.findPubKeyScriptIndex(fundingTx, pubkeyScript) match { + outputIndex <- Transactions.findPubKeyScriptIndex(KotlinUtils.kmp2scala(fundingTx), fundingPubkeyScript) match { case Right(outputIndex) => Future.successful(outputIndex) case Left(skipped) => Future.failed(new RuntimeException(skipped.toString)) } - _ = logger.debug(s"created funding txid=${fundingTx.txid} outputIndex=$outputIndex fee=${fundTxResponse.fee}") - } yield MakeFundingTxResponse(fundingTx, outputIndex, fundTxResponse.fee) + _ = logger.debug(s"created funding txid=${fundingTx.txid} outputIndex=$outputIndex fee=${fee}") + } yield MakeFundingTxResponse(signedPsbt, outputIndex, fee) } def commit(tx: Transaction)(implicit ec: ExecutionContext): Future[Boolean] = publishTransaction(tx).transformWith { @@ -477,6 +548,22 @@ object BitcoinCoreClient { val amountIn: Satoshi = fee + tx.txOut.map(_.amount).sum } + case class FundPsbtInput(txid: ByteVector32, vout: Int, sequence_opt: Option[Long] = None) + + 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 amountIn: Satoshi = Option(psbt.computeFees()).map(KotlinUtils.kmp2scala).get + psbt.getGlobal.getTx.txOut.asScala.map(_.amount).map(KotlinUtils.kmp2scala).sum + } + + case class ProcessPsbtResponse(psbt: Psbt, complete: Boolean) + 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/fsm/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala index 9decc58b0d..41f6e0168d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/Channel.scala @@ -21,7 +21,7 @@ import akka.actor.typed.scaladsl.adapter.{ClassicActorContextOps, actorRefAdapte import akka.actor.{Actor, ActorContext, ActorRef, FSM, OneForOneStrategy, PossiblyHarmful, Props, SupervisorStrategy, typed} import akka.event.Logging.MDC import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} -import fr.acinq.bitcoin.scalacompat.{ByteVector32, Satoshi, SatoshiLong, Transaction} +import fr.acinq.bitcoin.scalacompat.{ByteVector32, KotlinUtils, Satoshi, SatoshiLong, Transaction} import fr.acinq.eclair.Logs.LogCategory import fr.acinq.eclair._ import fr.acinq.eclair.blockchain.OnChainWallet.MakeFundingTxResponse @@ -1243,7 +1243,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, val log.info("shutting down") stop(FSM.Normal) - case Event(MakeFundingTxResponse(fundingTx, _, _), _) => + case Event(MakeFundingTxResponse(psbt, _, _), _) => + val fundingTx = KotlinUtils.kmp2scala(psbt.extract().getRight) // this may happen if connection is lost, or remote sends an error while we were waiting for the funding tx to be created by our wallet // in that case we rollback the tx wallet.rollback(fundingTx) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunder.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunder.scala index 14722ddfe6..df6fed1473 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunder.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/fsm/ChannelOpenSingleFunder.scala @@ -20,7 +20,7 @@ import akka.actor.Status import akka.actor.typed.scaladsl.adapter.actorRefAdapter import akka.pattern.pipe import fr.acinq.bitcoin.ScriptFlags -import fr.acinq.bitcoin.scalacompat.{SatoshiLong, Script, Transaction} +import fr.acinq.bitcoin.scalacompat.{KotlinUtils, SatoshiLong, Script, Transaction} import fr.acinq.eclair.blockchain.OnChainWallet.MakeFundingTxResponse import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ import fr.acinq.eclair.channel.Helpers.{Funding, getRelayFees} @@ -154,8 +154,7 @@ trait ChannelOpenSingleFunder extends FundingHandlers with ErrorHandlers { log.debug("remote params: {}", remoteParams) log.info("remote will use fundingMinDepth={}", accept.minimumDepth) val localFundingPubkey = keyManager.fundingPublicKey(localParams.fundingKeyPath) - val fundingPubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingPubkey.publicKey, remoteParams.fundingPubKey))) - wallet.makeFundingTx(fundingPubkeyScript, fundingSatoshis, fundingTxFeerate).pipeTo(self) + wallet.makeFundingTx(nodeParams.chainHash, localFundingPubkey, remoteParams.fundingPubKey, fundingSatoshis, fundingTxFeerate).pipeTo(self) goto(WAIT_FOR_FUNDING_INTERNAL) using DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingSatoshis, pushMsat, commitTxFeerate, accept.firstPerCommitmentPoint, channelConfig, channelFeatures, open) } @@ -177,7 +176,8 @@ trait ChannelOpenSingleFunder extends FundingHandlers with ErrorHandlers { }) when(WAIT_FOR_FUNDING_INTERNAL)(handleExceptions { - case Event(MakeFundingTxResponse(fundingTx, fundingTxOutputIndex, fundingTxFee), d@DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, commitTxFeerate, remoteFirstPerCommitmentPoint, channelConfig, channelFeatures, open)) => + case Event(MakeFundingTxResponse(psbt, fundingTxOutputIndex, fundingTxFee), d@DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, commitTxFeerate, remoteFirstPerCommitmentPoint, channelConfig, channelFeatures, open)) => + val fundingTx = KotlinUtils.kmp2scala(psbt.extract().getRight) // let's create the first commitment tx that spends the yet uncommitted funding tx Funding.makeFirstCommitTxs(keyManager, channelConfig, channelFeatures, temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, commitTxFeerate, fundingTx.hash, fundingTxOutputIndex, remoteFirstPerCommitmentPoint) match { case Left(ex) => handleLocalError(ex, d, None) 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 7edffd673b..7d3f64a153 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,11 @@ 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.eclair.NotificationsLogger.NotifyNodeOperator +import fr.acinq.bitcoin.psbt.Psbt +import fr.acinq.bitcoin.scalacompat.{ByteVector32, KotlinUtils, OutPoint, Satoshi, Script, ScriptWitness, Transaction, TxIn, TxOut, computeScriptAddress} +import fr.acinq.eclair.NotificationsLogger.{NotifyNodeOperator, logFatalError} import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient -import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient.FundTransactionOptions +import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient.{FundPsbtOptions, FundPsbtResponse, FundTransactionOptions} import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.channel.Commitments import fr.acinq.eclair.channel.publish.ReplaceableTxPrePublisher._ @@ -30,7 +31,9 @@ import fr.acinq.eclair.channel.publish.TxPublisher.TxPublishContext import fr.acinq.eclair.transactions.Transactions._ import fr.acinq.eclair.{NodeParams, NotificationsLogger} -import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration.DurationInt +import scala.concurrent.{Await, ExecutionContext, Future} +import scala.jdk.CollectionConverters.CollectionHasAsScala import scala.util.{Failure, Success} /** @@ -276,6 +279,12 @@ private class ReplaceableTxFunder(nodeParams: NodeParams, } def bump(previousTx: FundedTx, targetFeerate: FeeratePerKw): Behavior[Command] = { + def removeWitness1(txIn: TxIn): TxIn = txIn.copy(witness = ScriptWitness.empty) + + def removeWitness2(inputs: Seq[TxIn]): Seq[TxIn] = inputs.map(removeWitness1) + + def removeWitness(tx: Transaction): Transaction = tx.copy(txIn = removeWitness2(tx.txIn)) + adjustPreviousTxOutput(previousTx, targetFeerate, cmd.commitments) match { case AdjustPreviousTxOutputResult.Skip(reason) => log.warn("skipping {} fee bumping: {} (feerate={})", cmd.desc, reason, targetFeerate) @@ -283,7 +292,8 @@ private class ReplaceableTxFunder(nodeParams: NodeParams, Behaviors.stopped case AdjustPreviousTxOutputResult.TxOutputAdjusted(updatedTx) => log.debug("bumping {} fees without adding new inputs: txid={}", cmd.desc, updatedTx.txInfo.tx.txid) - sign(updatedTx, targetFeerate, previousTx.totalAmountIn) + val updatedTx1 = updatedTx.updateTx(removeWitness(updatedTx.txInfo.tx)) + sign(updatedTx1, targetFeerate, previousTx.totalAmountIn) case AdjustPreviousTxOutputResult.AddWalletInputs(tx) => log.debug("bumping {} fees requires adding new inputs (feerate={})", cmd.desc, targetFeerate) // We restore the original transaction (remove previous attempt's wallet inputs). @@ -353,21 +363,35 @@ private class ReplaceableTxFunder(nodeParams: NodeParams, } def signWalletInputs(locallySignedTx: ReplaceableTxWithWalletInputs, txFeerate: FeeratePerKw, amountIn: Satoshi): Behavior[Command] = { + import KotlinUtils._ + locallySignedTx match { case ClaimLocalAnchorWithWitnessData(anchorTx) => - val commitInfo = BitcoinCoreClient.PreviousTx(anchorTx.input, anchorTx.tx.txIn.head.witness) - context.pipeToSelf(bitcoinClient.signTransaction(anchorTx.tx, Seq(commitInfo))) { - case Success(signedTx) => SignWalletInputsOk(signedTx.tx) + val psbt1 = new Psbt(anchorTx.tx) + val psbt2 = psbt1.updateWitnessInput(anchorTx.input.outPoint, anchorTx.input.txOut, null, fr.acinq.bitcoin.Script.parse(anchorTx.input.redeemScript), null, java.util.Map.of()).getRight + val psbt3 = psbt2.finalizeWitnessInput(0, anchorTx.tx.txIn.head.witness).getRight + context.pipeToSelf(bitcoinClient.processPsbt(psbt3)) { + case Success(processPsbtResponse) => + val extracted = processPsbtResponse.psbt.extract() + if (extracted.isLeft) { + SignWalletInputsFailed(new RuntimeException(extracted.getLeft.toString)) + } else { + SignWalletInputsOk(extracted.getRight) + } case Failure(reason) => SignWalletInputsFailed(reason) } case htlcTx: HtlcWithWitnessData => - val inputInfo = BitcoinCoreClient.PreviousTx(htlcTx.txInfo.input, htlcTx.txInfo.tx.txIn.head.witness) - context.pipeToSelf(bitcoinClient.signTransaction(htlcTx.txInfo.tx, Seq(inputInfo), allowIncomplete = true).map(signTxResponse => { - // NB: bitcoind versions older than 0.21.1 messes up the witness stack for our htlc input, so we need to restore it. - // See https://github.com/bitcoin/bitcoin/issues/21151 - htlcTx.txInfo.tx.copy(txIn = htlcTx.txInfo.tx.txIn.head +: signTxResponse.tx.txIn.tail) - })) { - case Success(signedTx) => SignWalletInputsOk(signedTx) + val psbt1 = new Psbt(htlcTx.txInfo.tx) + val psbt2 = psbt1.updateWitnessInput(htlcTx.txInfo.input.outPoint, htlcTx.txInfo.input.txOut, null, fr.acinq.bitcoin.Script.parse(htlcTx.txInfo.input.redeemScript), null, java.util.Map.of()).getRight + val psbt3 = psbt2.finalizeWitnessInput(0, htlcTx.txInfo.tx.txIn.head.witness).getRight + context.pipeToSelf(bitcoinClient.processPsbt(psbt3)) { + case Success(processPsbtResponse) => + val extracted = processPsbtResponse.psbt.extract() + if (extracted.isLeft) { + SignWalletInputsFailed(new RuntimeException(extracted.getLeft.toString)) + } else { + SignWalletInputsOk(extracted.getRight) + } case Failure(reason) => SignWalletInputsFailed(reason) } } @@ -404,6 +428,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 commitFeerate = commitments.localCommit.spec.commitTxFeerate val commitTx = dummySignedCommitTx(commitments).tx @@ -422,32 +448,50 @@ private class ReplaceableTxFunder(nodeParams: NodeParams, // That's ok, we can increase the fee later by decreasing the output amount. But we need to ensure we'll have enough // to cover the weight of our anchor input, which is why we set it to the following value. val dummyChangeAmount = weight2fee(anchorFeerate, claimAnchorOutputMinWeight) + dustLimit - val txNotFunded = Transaction(2, Nil, TxOut(dummyChangeAmount, Script.pay2wpkh(PlaceHolderPubKey)) :: Nil, 0) - bitcoinClient.fundTransaction(txNotFunded, FundTransactionOptions(anchorFeerate, lockUtxos = true)).flatMap(fundTxResponse => { - // We merge the outputs if there's more than one. - fundTxResponse.changePosition match { + val Some(address) = computeScriptAddress(nodeParams.chainHash, Script.pay2wpkh(PlaceHolderPubKey)) + + // merge outptuts if needed to get a PSBT with a single output + def makeSingleOutput(fundPsbtResponse: FundPsbtResponse): Future[Psbt] = { + fundPsbtResponse.changePosition match { case Some(changePos) => - val changeOutput = fundTxResponse.tx.txOut(changePos) - val txSingleOutput = fundTxResponse.tx.copy(txOut = Seq(changeOutput.copy(amount = changeOutput.amount + dummyChangeAmount))) - Future.successful(fundTxResponse.copy(tx = txSingleOutput)) + // add our main output to the change output + val changeOutput = fundPsbtResponse.psbt.getGlobal.getTx.txOut.get(changePos) + val changeOutput1 = changeOutput.updateAmount(changeOutput.amount + dummyChangeAmount) + val psbt = fundPsbtResponse.psbt + val global = psbt.getGlobal + val tx = global.getTx + val tx1 = tx.updateOutputs(java.util.List.of(changeOutput1)) + val global1 = global.copy(global.getVersion, tx1, global.getExtendedPublicKeys, global.getUnknown) + val psbt1 = psbt.copy(global1, psbt.getInputs, java.util.List.of(psbt.getOutputs.get(changePos))) + Future.successful(psbt1) case None => + // replace our main output with a dummy change output bitcoinClient.getChangeAddress().map(pubkeyHash => { - val txSingleOutput = fundTxResponse.tx.copy(txOut = Seq(TxOut(dummyChangeAmount, Script.pay2wpkh(pubkeyHash)))) - fundTxResponse.copy(tx = txSingleOutput) + val changeOutput1 = TxOut(dummyChangeAmount, Script.pay2wpkh(pubkeyHash)) + val global = fundPsbtResponse.psbt.getGlobal + val tx = global.getTx + val tx1 = tx.updateOutputs(java.util.List.of(changeOutput1)) + val global1 = global.copy(global.getVersion, tx1, global.getExtendedPublicKeys, global.getUnknown) + fundPsbtResponse.psbt.copy(global1, fundPsbtResponse.psbt.getInputs, fundPsbtResponse.psbt.getOutputs) }) } - }).map(fundTxResponse => { - require(fundTxResponse.tx.txOut.size == 1, "funded transaction should have a single change output") + } + + for { + fundPsbtResponse <- bitcoinClient.fundPsbt(Seq(address -> dummyChangeAmount), 0, FundPsbtOptions(anchorFeerate, lockUtxos = true, changePosition = Some(1))) + psbt <- makeSingleOutput(fundPsbtResponse) // NB: we insert the anchor input in the *first* position because our signing helpers only sign input #0. - val unsignedTx = anchorTx.updateTx(fundTxResponse.tx.copy(txIn = anchorTx.txInfo.tx.txIn.head +: fundTxResponse.tx.txIn)) - val totalAmountIn = fundTxResponse.amountIn + AnchorOutputsCommitmentFormat.anchorAmount - (adjustAnchorOutputChange(unsignedTx, commitTx, totalAmountIn, commitFeerate, targetFeerate, dustLimit), totalAmountIn) - }) + tx = KotlinUtils.kmp2scala(psbt.getGlobal.getTx) + unsignedTx = anchorTx.updateTx(tx.copy(txIn = anchorTx.txInfo.tx.txIn.head +: tx.txIn)) + totalAmountIn = fundPsbtResponse.amountIn + AnchorOutputsCommitmentFormat.anchorAmount + adjustedTx = adjustAnchorOutputChange(unsignedTx, commitTx, totalAmountIn, commitFeerate, targetFeerate, dustLimit) + } yield { + (adjustedTx, totalAmountIn) + } } private def addInputs(htlcTx: HtlcWithWitnessData, targetFeerate: FeeratePerKw, commitments: Commitments): Future[(HtlcWithWitnessData, Satoshi)] = { - // NB: fundrawtransaction doesn't support non-wallet inputs, so we clear the input and re-add it later. - val txNotFunded = htlcTx.txInfo.tx.copy(txIn = Nil, txOut = htlcTx.txInfo.tx.txOut.head.copy(amount = commitments.localParams.dustLimit) :: Nil) + import fr.acinq.bitcoin.scalacompat.KotlinUtils._ val htlcTxWeight = htlcTx.txInfo match { case _: HtlcSuccessTx => commitments.commitmentFormat.htlcSuccessWeight case _: HtlcTimeoutTx => commitments.commitmentFormat.htlcTimeoutWeight @@ -464,16 +508,21 @@ private class ReplaceableTxFunder(nodeParams: NodeParams, // NB: we don't take into account the fee paid by our HTLC input: we will take it into account when we adjust the // change output amount (unless bitcoind didn't add any change output, in that case we will overpay the fee slightly). val weightRatio = 1.0 + (htlcInputMaxWeight.toDouble / (htlcTxWeight + claimP2WPKHOutputWeight)) - bitcoinClient.fundTransaction(txNotFunded, FundTransactionOptions(targetFeerate * weightRatio, lockUtxos = true, changePosition = Some(1))).map(fundTxResponse => { + val Some(address) = computeScriptAddress(nodeParams.chainHash, Script.pay2wpkh(PlaceHolderPubKey)) + + // NB: fundpsbt doesn't support non-wallet inputs, so we clear the input and re-add it later. + bitcoinClient.fundPsbt(Seq(address -> commitments.localParams.dustLimit), htlcTx.txInfo.tx.lockTime, FundPsbtOptions(targetFeerate * weightRatio, lockUtxos = true, changePosition = Some(1))).map(fundPsbtResponse => { // We add the HTLC input (from the commit tx) and restore the HTLC output. // NB: we can't modify them because they are signed by our peer (with SIGHASH_SINGLE | SIGHASH_ANYONECANPAY). - val txWithHtlcInput = fundTxResponse.tx.copy( - txIn = htlcTx.txInfo.tx.txIn ++ fundTxResponse.tx.txIn, - txOut = htlcTx.txInfo.tx.txOut ++ fundTxResponse.tx.txOut.tail + val tx: Transaction = fundPsbtResponse.psbt.getGlobal.getTx + val txWithHtlcInput = tx.copy( + txIn = htlcTx.txInfo.tx.txIn ++ tx.txIn, + txOut = htlcTx.txInfo.tx.txOut ++ tx.txOut.tail ) val unsignedTx = htlcTx.updateTx(txWithHtlcInput) - val totalAmountIn = fundTxResponse.amountIn + unsignedTx.txInfo.amountIn - (adjustHtlcTxChange(unsignedTx, totalAmountIn, targetFeerate, commitments.localParams.dustLimit, commitments.commitmentFormat), totalAmountIn) + val totalAmountIn = fundPsbtResponse.amountIn + unsignedTx.txInfo.amountIn + val adjustedTx = adjustHtlcTxChange(unsignedTx, totalAmountIn, targetFeerate, commitments.localParams.dustLimit, commitments.commitmentFormat) + (adjustedTx, totalAmountIn) }) } 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 5476611fd6..0dbbf578fd 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 @@ -18,9 +18,12 @@ package fr.acinq.eclair.blockchain import fr.acinq.bitcoin.TxIn.SEQUENCE_FINAL import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey -import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto, OutPoint, Satoshi, SatoshiLong, Transaction, TxIn, TxOut} +import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto, DeterministicWallet, KotlinUtils, OutPoint, Satoshi, SatoshiLong, Script, Transaction, TxIn, TxOut} +import fr.acinq.bitcoin.psbt.Psbt import fr.acinq.eclair.blockchain.OnChainWallet.{MakeFundingTxResponse, OnChainBalance} import fr.acinq.eclair.blockchain.fee.FeeratePerKw +import fr.acinq.eclair.randomKey +import fr.acinq.eclair.transactions.Scripts import scodec.bits._ import scala.concurrent.{ExecutionContext, Future, Promise} @@ -41,9 +44,9 @@ class DummyOnChainWallet extends OnChainWallet { override def getReceivePubkey(receiveAddress: Option[String] = None)(implicit ec: ExecutionContext): Future[Crypto.PublicKey] = Future.successful(dummyReceivePubkey) - override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { - val tx = DummyOnChainWallet.makeDummyFundingTx(pubkeyScript, amount) - funded += (tx.fundingTx.txid -> tx.fundingTx) + override def makeFundingTx(chainHash: ByteVector32, localFundingKey: DeterministicWallet.ExtendedPublicKey, remoteFundingKey: PublicKey, amount: Satoshi, feeRatePerKw: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { + val tx = DummyOnChainWallet.makeDummyFundingTx(Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingKey.publicKey, remoteFundingKey))), amount) + funded += (tx.fundingTx().txid -> tx.fundingTx()) Future.successful(tx) } @@ -68,7 +71,8 @@ class NoOpOnChainWallet extends OnChainWallet { override def getReceivePubkey(receiveAddress: Option[String] = None)(implicit ec: ExecutionContext): Future[Crypto.PublicKey] = Future.successful(dummyReceivePubkey) - override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = Promise[MakeFundingTxResponse]().future // will never be completed + override def makeFundingTx(chainHash: ByteVector32, localFundingKey: DeterministicWallet.ExtendedPublicKey, remoteFundingKey: PublicKey, amount: Satoshi, feeRatePerKw: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = + Promise[MakeFundingTxResponse]().future // will never be completed override def commit(tx: Transaction)(implicit ec: ExecutionContext): Future[Boolean] = Future.successful(true) @@ -84,11 +88,22 @@ object DummyOnChainWallet { val dummyReceivePubkey: PublicKey = PublicKey(hex"028feba10d0eafd0fad8fe20e6d9206e6bd30242826de05c63f459a00aced24b12") def makeDummyFundingTx(pubkeyScript: ByteVector, amount: Satoshi): MakeFundingTxResponse = { + import KotlinUtils._ + val key = randomKey() + val baseTx = Transaction(version = 2, txIn = Nil, txOut = TxOut(amount, Script.pay2wpkh(key.publicKey)) :: Nil, lockTime = 0) val fundingTx = Transaction(version = 2, - txIn = TxIn(OutPoint(ByteVector32(ByteVector.fill(32)(1)), 42), signatureScript = Nil, sequence = SEQUENCE_FINAL) :: Nil, + txIn = TxIn(OutPoint(baseTx, 0), signatureScript = Nil, sequence = SEQUENCE_FINAL) :: Nil, txOut = TxOut(amount, pubkeyScript) :: Nil, lockTime = 0) - MakeFundingTxResponse(fundingTx, 0, 420 sat) + + // TODO: this is really ugly :( + val psbt = new Psbt(fundingTx) + .updateWitnessInputTx(baseTx, 0, null, fr.acinq.bitcoin.Script.pay2pkh(key.publicKey), null, java.util.Map.of()) + .map(p => p.sign(key, 0).getRight) + .map(p => p.getPsbt.finalizeWitnessInput(0, Script.witnessPay2wpkh(key.publicKey, p.getSig))).getRight + .getRight + + MakeFundingTxResponse(psbt, 0, 420 sat) } } \ No newline at end of file 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 e9a8e7c740..516af237cf 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,8 +20,10 @@ import akka.actor.Status.Failure import akka.pattern.pipe import akka.testkit.TestProbe import fr.acinq.bitcoin -import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey -import fr.acinq.bitcoin.scalacompat.{Block, BtcDouble, ByteVector32, MilliBtcDouble, OutPoint, Satoshi, SatoshiLong, Script, ScriptWitness, Transaction, TxIn, TxOut} +import fr.acinq.bitcoin.Bech32 +import fr.acinq.bitcoin.scalacompat.Crypto.{PrivateKey, PublicKey} +import fr.acinq.bitcoin.scalacompat.DeterministicWallet.{ExtendedPublicKey, KeyPath} +import fr.acinq.bitcoin.scalacompat.{Block, BtcDouble, ByteVector32, Crypto, KotlinUtils, MilliBtcDouble, OutPoint, Satoshi, SatoshiLong, Script, ScriptWitness, Transaction, TxIn, TxOut} import fr.acinq.eclair.blockchain.OnChainWallet.{MakeFundingTxResponse, OnChainBalance} import fr.acinq.eclair.blockchain.WatcherSpec.{createSpendManyP2WPKH, createSpendP2WPKH} import fr.acinq.eclair.blockchain.bitcoind.BitcoindService.BitcoinReq @@ -30,7 +32,7 @@ import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinJsonRPCAuthMethod.UserPass import fr.acinq.eclair.blockchain.bitcoind.rpc.{BasicBitcoinJsonRPCClient, BitcoinCoreClient, JsonRPCError} import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.transactions.{Scripts, Transactions} -import fr.acinq.eclair.{BlockHeight, TestConstants, TestKitBaseClass, addressToPublicKeyScript, randomKey} +import fr.acinq.eclair.{BlockHeight, TestConstants, TestKitBaseClass, addressToPublicKeyScript, randomBytes32, randomKey} import grizzled.slf4j.Logging import org.json4s.JsonAST._ import org.json4s.{DefaultFormats, Formats} @@ -64,15 +66,58 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A sender.expectMsgType[JString](60 seconds) restartBitcoind(sender) - val pubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(randomKey().publicKey, randomKey().publicKey))) - bitcoinClient.makeFundingTx(pubkeyScript, 50 millibtc, FeeratePerKw(10000 sat)).pipeTo(sender.ref) + bitcoinClient.makeFundingTx(Block.RegtestGenesisBlock.hash, ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), randomKey().publicKey, 50 millibtc, FeeratePerKw(10000 sat)).pipeTo(sender.ref) val error = sender.expectMsgType[Failure].cause.asInstanceOf[JsonRPCError].error - assert(error.message.contains("Please enter the wallet passphrase with walletpassphrase first")) + assert(error.message.contains("cannot sign psbt")) sender.send(bitcoincli, BitcoinReq("walletpassphrase", walletPassword, 3600)) // wallet stay unlocked for 3600s sender.expectMsgType[JValue] } + test("fund and sign psbt") { + val sender = TestProbe() + val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val priv1 = PrivateKey(ByteVector32.fromValidHex("01" * 32)) + val priv2 = PrivateKey(ByteVector32.fromValidHex("02" * 32)) + val script = Script.createMultiSigMofN(2, Seq(priv1.publicKey, priv2.publicKey)) + val address = Bech32.encodeWitnessAddress("bcrt", 0, Crypto.sha256(Script.write(script)).toArray) + + bitcoinClient.fundPsbt(Seq(address -> 10000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw)).pipeTo(sender.ref) + val FundPsbtResponse(psbt, _, _) = sender.expectMsgType[FundPsbtResponse] + + bitcoinClient.processPsbt(psbt).pipeTo(sender.ref) + val ProcessPsbtResponse(psbt1, true) = sender.expectMsgType[ProcessPsbtResponse] + assert(psbt1.extract().isRight) + } + + test("fund psbt (invalid requests)") { + val sender = TestProbe() + val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val priv = PrivateKey(ByteVector32.fromValidHex("01" * 32)) + val address = Bech32.encodeWitnessAddress("bcrt", 0, priv.publicKey.hash160.toArray) + + { + // check that it does work + bitcoinClient.fundPsbt(Seq(address -> 10000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw)).pipeTo(sender.ref) + sender.expectMsgType[FundPsbtResponse] + } + { + // invalid address + bitcoinClient.fundPsbt(Seq("invalid address" -> 10000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw)).pipeTo(sender.ref) + sender.expectMsgType[akka.actor.Status.Failure] + } + { + // amount is too small + bitcoinClient.fundPsbt(Seq(address -> 100.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw)).pipeTo(sender.ref) + sender.expectMsgType[akka.actor.Status.Failure] + } + { + // amount is too large + bitcoinClient.fundPsbt(Seq(address -> 11_000_000.btc), 0, FundPsbtOptions(TestConstants.feeratePerKw)).pipeTo(sender.ref) + sender.expectMsgType[akka.actor.Status.Failure] + } + } + test("fund transactions") { val sender = TestProbe() val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) @@ -170,16 +215,24 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A val fundingTxs = for (_ <- 0 to 3) yield { val pubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(randomKey().publicKey, randomKey().publicKey))) - bitcoinClient.makeFundingTx(pubkeyScript, Satoshi(500), FeeratePerKw(250 sat)).pipeTo(sender.ref) - val fundingTx = sender.expectMsgType[MakeFundingTxResponse].fundingTx + bitcoinClient.makeFundingTx( + Block.RegtestGenesisBlock.hash, + ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), + randomKey().publicKey, + Satoshi(500), FeeratePerKw(250 sat)).pipeTo(sender.ref) + val fundingTx = KotlinUtils.kmp2scala(sender.expectMsgType[MakeFundingTxResponse].psbt.extract().getRight) bitcoinClient.publishTransaction(fundingTx.copy(txIn = Nil)).pipeTo(sender.ref) // try publishing an invalid version of the tx sender.expectMsgType[Failure] bitcoinClient.rollback(fundingTx).pipeTo(sender.ref) // rollback the locked outputs assert(sender.expectMsgType[Boolean]) // now fund a tx with correct feerate - bitcoinClient.makeFundingTx(pubkeyScript, 50 millibtc, FeeratePerKw(250 sat)).pipeTo(sender.ref) - sender.expectMsgType[MakeFundingTxResponse].fundingTx + bitcoinClient.makeFundingTx( + Block.RegtestGenesisBlock.hash, + ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), + randomKey().publicKey, + 50 millibtc, FeeratePerKw(250 sat)).pipeTo(sender.ref) + KotlinUtils.kmp2scala(sender.expectMsgType[MakeFundingTxResponse].psbt.extract().getRight) } assert(getLocks(sender).size == 4) @@ -210,10 +263,12 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A val sender = TestProbe() val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) - val pubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(randomKey().publicKey, randomKey().publicKey))) - // 200 sat/kw is below the min-relay-fee - bitcoinClient.makeFundingTx(pubkeyScript, 5 millibtc, FeeratePerKw(200 sat)).pipeTo(sender.ref) - val MakeFundingTxResponse(fundingTx, _, _) = sender.expectMsgType[MakeFundingTxResponse] + bitcoinClient.makeFundingTx( + Block.RegtestGenesisBlock.hash, + ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), + randomKey().publicKey, + 5 millibtc, FeeratePerKw(200 sat)).pipeTo(sender.ref) + val fundingTx = sender.expectMsgType[MakeFundingTxResponse].fundingTx() bitcoinClient.commit(fundingTx).pipeTo(sender.ref) sender.expectMsg(true) @@ -232,9 +287,12 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A assert(getLocks(sender).isEmpty) - val pubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(randomKey().publicKey, randomKey().publicKey))) - bitcoinClient.makeFundingTx(pubkeyScript, 50 millibtc, FeeratePerKw(10000 sat)).pipeTo(sender.ref) - val MakeFundingTxResponse(fundingTx, _, _) = sender.expectMsgType[MakeFundingTxResponse] + bitcoinClient.makeFundingTx( + Block.RegtestGenesisBlock.hash, + ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), + randomKey().publicKey, + 50 millibtc, FeeratePerKw(10000 sat)).pipeTo(sender.ref) + val fundingTx = sender.expectMsgType[MakeFundingTxResponse].fundingTx() bitcoinClient.commit(fundingTx).pipeTo(sender.ref) sender.expectMsg(true) @@ -305,8 +363,14 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) // create a huge tx so we make sure it has > 1 inputs - bitcoinClient.makeFundingTx(pubkeyScript, 250 btc, FeeratePerKw(1000 sat)).pipeTo(sender.ref) - val MakeFundingTxResponse(fundingTx, outputIndex, _) = sender.expectMsgType[MakeFundingTxResponse] + bitcoinClient.makeFundingTx( + Block.RegtestGenesisBlock.hash, + ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), + randomKey().publicKey, + 250 btc, FeeratePerKw(1000 sat)).pipeTo(sender.ref) + val fundingResponse = sender.expectMsgType[MakeFundingTxResponse] + val fundingTx = fundingResponse.fundingTx() + val outputIndex = fundingResponse.fundingTxOutputIndex // spend the first 2 inputs val tx1 = fundingTx.copy( @@ -346,8 +410,12 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A { // test #1: unlock 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) - val MakeFundingTxResponse(fundingTx, _, _) = sender.expectMsgType[MakeFundingTxResponse] + bitcoinClient.makeFundingTx( + Block.RegtestGenesisBlock.hash, + ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), + randomKey().publicKey, + 250 btc, FeeratePerKw(1000 sat)).pipeTo(sender.ref) + val fundingTx: Transaction = sender.expectMsgType[MakeFundingTxResponse].fundingTx() assert(fundingTx.txIn.size > 2) assert(getLocks(sender) == fundingTx.txIn.map(_.outPoint).toSet) bitcoinClient.rollback(fundingTx).pipeTo(sender.ref) @@ -355,8 +423,12 @@ 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) - val MakeFundingTxResponse(fundingTx, _, _) = sender.expectMsgType[MakeFundingTxResponse] + bitcoinClient.makeFundingTx( + Block.RegtestGenesisBlock.hash, + ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), + randomKey().publicKey, + 250 btc, FeeratePerKw(1000 sat)).pipeTo(sender.ref) + val fundingTx: Transaction = sender.expectMsgType[MakeFundingTxResponse].fundingTx() assert(fundingTx.txIn.size > 2) assert(getLocks(sender) == fundingTx.txIn.map(_.outPoint).toSet) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/AnnouncementsBatchValidationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/AnnouncementsBatchValidationSpec.scala index b5d7775a4f..b6ea401bb4 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/AnnouncementsBatchValidationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/AnnouncementsBatchValidationSpec.scala @@ -21,14 +21,15 @@ import akka.pattern.pipe import akka.testkit.TestProbe import sttp.client3.okhttp.OkHttpFutureBackend import fr.acinq.bitcoin.scalacompat.Crypto.PrivateKey -import fr.acinq.bitcoin.scalacompat.{Block, Satoshi, SatoshiLong, Script, Transaction} +import fr.acinq.bitcoin.scalacompat.DeterministicWallet.{ExtendedPublicKey, KeyPath} +import fr.acinq.bitcoin.scalacompat.{Block, KotlinUtils, Satoshi, SatoshiLong, Script, Transaction} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.ValidateResult import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinJsonRPCAuthMethod.UserPassword import fr.acinq.eclair.blockchain.bitcoind.rpc.{BasicBitcoinJsonRPCClient, BitcoinCoreClient} import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.transactions.Scripts import fr.acinq.eclair.wire.protocol.{ChannelAnnouncement, ChannelUpdate} -import fr.acinq.eclair.{BlockHeight, CltvExpiryDelta, Features, MilliSatoshiLong, RealShortChannelId, ShortChannelId, randomKey} +import fr.acinq.eclair.{BlockHeight, CltvExpiryDelta, Features, MilliSatoshiLong, RealShortChannelId, ShortChannelId, randomBytes32, randomKey} import org.json4s.JsonAST.JString import org.scalatest.funsuite.AnyFunSuite @@ -92,11 +93,15 @@ object AnnouncementsBatchValidationSpec { val node2BitcoinKey = randomKey() val amount = 1000000 sat // first we publish the funding tx - val fundingPubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(node1BitcoinKey.publicKey, node2BitcoinKey.publicKey))) - val fundingTxFuture = bitcoinClient.makeFundingTx(fundingPubkeyScript, amount, FeeratePerKw(10000 sat)) + val fundingTxFuture = bitcoinClient.makeFundingTx( + Block.RegtestGenesisBlock.hash, + ExtendedPublicKey(node1BitcoinKey.publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), + node2BitcoinKey.publicKey, + amount, FeeratePerKw(10000 sat)) val res = Await.result(fundingTxFuture, 10 seconds) - Await.result(bitcoinClient.publishTransaction(res.fundingTx), 10 seconds) - SimulatedChannel(node1Key, node2Key, node1BitcoinKey, node2BitcoinKey, amount, res.fundingTx, res.fundingTxOutputIndex) + val fundingTx = KotlinUtils.kmp2scala(res.psbt.extract().getRight) + Await.result(bitcoinClient.publishTransaction(fundingTx), 10 seconds) + SimulatedChannel(node1Key, node2Key, node1BitcoinKey, node2BitcoinKey, amount, fundingTx, res.fundingTxOutputIndex) } def makeChannelAnnouncement(c: SimulatedChannel, bitcoinClient: BitcoinCoreClient)(implicit ec: ExecutionContext): ChannelAnnouncement = {