From 8e74bc14c6025cf41484c2d50ed145a737efb6a9 Mon Sep 17 00:00:00 2001 From: sstone Date: Tue, 29 Jun 2021 22:18:57 +0200 Subject: [PATCH 1/8] Use psbt to fund and sign transactions --- .../main/scala/fr/acinq/eclair/Setup.scala | 2 +- .../eclair/blockchain/OnChainWallet.scala | 4 +- .../bitcoind/rpc/BitcoinCoreClient.scala | 113 ++++++++++++++++-- .../fr/acinq/eclair/channel/Channel.scala | 7 +- .../acinq/eclair/TestBitcoinCoreClient.scala | 4 +- .../eclair/balance/CheckBalanceSpec.scala | 4 +- .../blockchain/DummyOnChainWallet.scala | 14 ++- .../bitcoind/BitcoinCoreClientSpec.scala | 104 +++++++++++----- .../eclair/blockchain/bitcoind/PsbtSpec.scala | 75 ++++++++++++ .../blockchain/bitcoind/ZmqWatcherSpec.scala | 2 +- .../publish/MempoolTxMonitorSpec.scala | 4 +- .../channel/publish/RawTxPublisherSpec.scala | 4 +- .../publish/ReplaceableTxPublisherSpec.scala | 4 +- .../integration/ChannelIntegrationSpec.scala | 12 +- .../integration/PaymentIntegrationSpec.scala | 3 +- .../AnnouncementsBatchValidationSpec.scala | 6 +- 16 files changed, 291 insertions(+), 71 deletions(-) create mode 100644 eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/PsbtSpec.scala diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala index 62ac023704..4c8f9778e3 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala @@ -241,7 +241,7 @@ class Setup(val datadir: File, }) _ <- feeratesRetrieved.future - bitcoinClient = new BitcoinCoreClient(new BatchingBitcoinJsonRPCClient(bitcoin)) + bitcoinClient = new BitcoinCoreClient(nodeParams.chainHash, new BatchingBitcoinJsonRPCClient(bitcoin)) watcher = { system.actorOf(SimpleSupervisor.props(Props(new ZMQActor(config.getString("bitcoind.zmqblock"), ZMQActor.Topics.HashBlock, Some(zmqBlockConnected))), "zmqblock", SupervisorStrategy.Restart)) system.actorOf(SimpleSupervisor.props(Props(new ZMQActor(config.getString("bitcoind.zmqtx"), ZMQActor.Topics.RawTx, Some(zmqTxConnected))), "zmqtx", SupervisorStrategy.Restart)) 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 9a9df7532b..8aaf37f220 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 @@ -17,7 +17,7 @@ package fr.acinq.eclair.blockchain import fr.acinq.bitcoin.Crypto.PublicKey -import fr.acinq.bitcoin.{Satoshi, Transaction} +import fr.acinq.bitcoin.{Psbt, Satoshi, Transaction} import fr.acinq.eclair.blockchain.fee.FeeratePerKw import scodec.bits.ByteVector @@ -95,6 +95,6 @@ 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) } 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 fc97fea8ab..a26ecf925b 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 @@ -41,7 +41,7 @@ import scala.util.{Failure, Success, Try} /** * The Bitcoin Core client provides some high-level utility methods to interact with Bitcoin Core. */ -class BitcoinCoreClient(val rpcClient: BitcoinJsonRPCClient) extends OnChainWallet with Logging { +class BitcoinCoreClient(val chainHash: ByteVector32, val rpcClient: BitcoinJsonRPCClient) extends OnChainWallet with Logging { import BitcoinCoreClient._ @@ -180,27 +180,100 @@ 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: Map[String, Satoshi], locktime: Int, 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 psbt = Psbt.fromBase64(base64).get + val changePos_opt = if (changePos >= 0) Some(changePos.intValue) else None + FundPsbtResponse(psbt, toSatoshi(fee), changePos_opt) + }) + } + + def fundPsbt(outputs: Map[String, Satoshi], locktime: Int, options: FundPsbtOptions)(implicit ec: ExecutionContext): Future[FundPsbtResponse] = + fundPsbt(Seq(), outputs, locktime, options) + + def processPsbt(psbt: Psbt, sign: Boolean = true, sighashType: Int = SIGHASH_ALL)(implicit ec: ExecutionContext): Future[ProcessPsbtResponse] = { + val sighashStrings = Map( + SIGHASH_ALL -> "ALL", + SIGHASH_NONE -> "NONE", + SIGHASH_SINGLE -> "SINGLE", + (SIGHASH_ALL | SIGHASH_ANYONECANPAY) -> "ALL|ANYONECANPAY", + (SIGHASH_NONE | SIGHASH_ANYONECANPAY) -> "NONE|ANYONECANPAY", + (SIGHASH_SINGLE | SIGHASH_ANYONECANPAY) -> "SINGLE|ANYONECANPAY") + val sighash = sighashStrings.getOrElse(sighashType, throw new IllegalArgumentException(s"invalid sighash flag ${sighashType}")) + rpcClient.invoke("walletprocesspsbt", Psbt.toBase64(psbt), sign, sighash).map(json => { + val JString(base64) = json \ "psbt" + val JBool(complete) = json \ "complete" + val psbt = Psbt.fromBase64(base64).get + ProcessPsbtResponse(psbt, complete) + }) + } + + private def signPsbtOrUnlock(psbt: Psbt)(implicit ec: ExecutionContext): Future[ProcessPsbtResponse] = { + val f = processPsbt(psbt).withFilter(_.complete == true) + // if signature fails (e.g. because wallet is encrypted) we need to unlock the utxos + f.recoverWith { case _ => + unlockOutpoints(psbt.global.tx.txIn.map(_.outPoint)) + .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.global.tx.txid}", t) + t + } + .flatMap(_ => f) // return signTransaction error + .recoverWith { case _ => f } // return signTransaction error + } + } + + override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, targetFeerate: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { + val hrp = chainHash match { + case Block.RegtestGenesisBlock.hash => "bcrt" + case Block.TestnetGenesisBlock.hash => "tb" + case Block.LivenetGenesisBlock.hash => "bc" + case _ => return Future.failed(new IllegalArgumentException(s"invalid chain hash ${chainHash}")) + } + val fundingAddress = Script.parse(pubkeyScript) match { + case OP_0 :: OP_PUSHDATA(data, _) :: Nil if data.size == 20 || data.size == 32 => Bech32.encodeWitnessAddress(hrp, 0, data) + case _ => return Future.failed(new IllegalArgumentException("invalid pubkey script")) + } + for { + // we ask bitcoin core to create and fund the funding tx 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)) + FundPsbtResponse(psbt, fee, changePosition) <- fundPsbt(Map(fundingAddress -> amount), 0, FundPsbtOptions(feerate, lockUtxos = true)) // now let's sign the funding tx - SignTransactionResponse(fundingTx, true) <- signTransactionOrUnlock(fundTxResponse.tx) + ProcessPsbtResponse(signedPsbt, true) <- signPsbtOrUnlock(psbt) + Success(fundingTx) = signedPsbt.extract() // 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) 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 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) +// 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)) +// // now let's sign the funding tx +// SignTransactionResponse(fundingTx, true) <- signTransactionOrUnlock(fundTxResponse.tx) +// // 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) +// 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) +// } + def commit(tx: Transaction)(implicit ec: ExecutionContext): Future[Boolean] = publishTransaction(tx).transformWith { case Success(_) => Future.successful(true) case Failure(e) => @@ -437,6 +510,20 @@ 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 = fee + psbt.computeFees().get + } + case class PreviousTx(txid: ByteVector32, vout: Long, scriptPubKey: String, redeemScript: String, witnessScript: String, amount: BigDecimal) object PreviousTx { @@ -452,6 +539,8 @@ object BitcoinCoreClient { case class SignTransactionResponse(tx: Transaction, complete: Boolean) + case class ProcessPsbtResponse(psbt: Psbt, complete: Boolean) + /** * Information about a transaction currently in the mempool. * diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala index c92e9fcff4..f76963f474 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala @@ -436,7 +436,8 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo }) when(WAIT_FOR_FUNDING_INTERNAL)(handleExceptions { - case Event(MakeFundingTxResponse(fundingTx, fundingTxOutputIndex, fundingTxFee), d@DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, remoteFirstPerCommitmentPoint, channelConfig, channelFeatures, open)) => + case Event(MakeFundingTxResponse(psbt, fundingTxOutputIndex, fundingTxFee), d@DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, remoteFirstPerCommitmentPoint, channelConfig, channelFeatures, open)) => + val fundingTx = psbt.extract().get // let's create the first commitment tx that spends the yet uncommitted funding tx Funding.makeFirstCommitTxs(keyManager, channelConfig, channelFeatures, temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, fundingTx.hash, fundingTxOutputIndex, remoteFirstPerCommitmentPoint) match { case Left(ex) => handleLocalError(ex, d, None) @@ -1574,10 +1575,10 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo log.info("shutting down") stop(FSM.Normal) - case Event(MakeFundingTxResponse(fundingTx, _, _), _) => + case Event(MakeFundingTxResponse(psbt, _, _), _) => // 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) + wallet.rollback(psbt.extract().get) stay() case Event(INPUT_DISCONNECTED, _) => stay() // we are disconnected, but it doesn't matter anymore diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/TestBitcoinCoreClient.scala b/eclair-core/src/test/scala/fr/acinq/eclair/TestBitcoinCoreClient.scala index 4dfa45567a..368fe678e1 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/TestBitcoinCoreClient.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/TestBitcoinCoreClient.scala @@ -17,7 +17,7 @@ package fr.acinq.eclair import akka.actor.ActorSystem -import fr.acinq.bitcoin.{ByteVector32, Transaction} +import fr.acinq.bitcoin.{Block, ByteVector32, Transaction} import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.blockchain.bitcoind.rpc.{BasicBitcoinJsonRPCClient, BitcoinCoreClient} @@ -27,7 +27,7 @@ import scala.concurrent.{ExecutionContext, Future} /** * Created by PM on 26/04/2016. */ -class TestBitcoinCoreClient()(implicit system: ActorSystem) extends BitcoinCoreClient(new BasicBitcoinJsonRPCClient("", "", "", 0)(http = null)) { +class TestBitcoinCoreClient()(implicit system: ActorSystem) extends BitcoinCoreClient(Block.RegtestGenesisBlock.hash, new BasicBitcoinJsonRPCClient("", "", "", 0)(http = null)) { import scala.concurrent.ExecutionContext.Implicits.global diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/balance/CheckBalanceSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/balance/CheckBalanceSpec.scala index 48e274dd08..5fb593190c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/balance/CheckBalanceSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/balance/CheckBalanceSpec.scala @@ -2,7 +2,7 @@ package fr.acinq.eclair.balance import akka.pattern.pipe import akka.testkit.TestProbe -import fr.acinq.bitcoin.{ByteVector32, SatoshiLong} +import fr.acinq.bitcoin.{Block, ByteVector32, SatoshiLong} import fr.acinq.eclair.balance.CheckBalance.{ClosingBalance, MainAndHtlcBalance, OffChainBalance, PossiblyPublishedMainAndHtlcBalance, PossiblyPublishedMainBalance} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{apply => _, _} import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient @@ -230,7 +230,7 @@ class CheckBalanceSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with val txids = (for (_ <- 0 until 20) yield randomBytes32()).toList val knownTxids = Set(txids(1), txids(3), txids(4), txids(6), txids(9), txids(12), txids(13)) - val bitcoinClient = new BitcoinCoreClient(null) { + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, null) { /** Get the number of confirmations of a given transaction. */ override def getTxConfirmations(txid: ByteVector32)(implicit ec: ExecutionContext): Future[Option[Int]] = Future.successful(if (knownTxids.contains(txid)) Some(42) else None) 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 74f4cbef3f..7b2fe55f43 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,12 +17,14 @@ package fr.acinq.eclair.blockchain import fr.acinq.bitcoin.Crypto.PublicKey -import fr.acinq.bitcoin.{ByteVector32, Crypto, OutPoint, Satoshi, SatoshiLong, Transaction, TxIn, TxOut} +import fr.acinq.bitcoin.{ByteVector32, Crypto, OutPoint, Psbt, Satoshi, SatoshiLong, Script, Transaction, TxIn, TxOut} import fr.acinq.eclair.blockchain.OnChainWallet.{MakeFundingTxResponse, OnChainBalance} import fr.acinq.eclair.blockchain.fee.FeeratePerKw +import fr.acinq.eclair.randomKey import scodec.bits._ import scala.concurrent.{ExecutionContext, Future, Promise} +import scala.util.Success /** * Created by PM on 06/07/2017. @@ -79,11 +81,17 @@ object DummyOnChainWallet { val dummyReceivePubkey: PublicKey = PublicKey(hex"028feba10d0eafd0fad8fe20e6d9206e6bd30242826de05c63f459a00aced24b12") def makeDummyFundingTx(pubkeyScript: ByteVector, amount: Satoshi): MakeFundingTxResponse = { + 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 = TxIn.SEQUENCE_FINAL) :: Nil, + txIn = TxIn(OutPoint(baseTx, 0), signatureScript = Nil, sequence = TxIn.SEQUENCE_FINAL) :: Nil, txOut = TxOut(amount, pubkeyScript) :: Nil, lockTime = 0) - MakeFundingTxResponse(fundingTx, 0, 420 sat) + val Success(psbt) = Psbt(fundingTx) + .update(baseTx, 0, witnessScript = Some(Script.pay2pkh(key.publicKey))) + .flatMap(p => p.sign(key, 0)) + .flatMap(p => p.finalize(0, Script.witnessPay2wpkh(key.publicKey, p.inputs(0).partialSigs(key.publicKey)))) + 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 0378795272..d7243677a9 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 @@ -19,9 +19,9 @@ package fr.acinq.eclair.blockchain.bitcoind import akka.actor.Status.Failure import akka.pattern.pipe import akka.testkit.TestProbe -import fr.acinq.bitcoin.Crypto.PublicKey +import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.SigVersion.SIGVERSION_WITNESS_V0 -import fr.acinq.bitcoin.{Block, BtcDouble, ByteVector32, MilliBtcDouble, OutPoint, SIGHASH_ALL, Satoshi, SatoshiLong, Script, ScriptFlags, ScriptWitness, Transaction, TxIn, TxOut} +import fr.acinq.bitcoin.{Bech32, Block, BtcDouble, ByteVector32, Crypto, MilliBtcDouble, OutPoint, SIGHASH_ALL, Satoshi, SatoshiLong, Script, ScriptFlags, ScriptWitness, Transaction, TxIn, TxOut} import fr.acinq.eclair.blockchain.OnChainWallet.{MakeFundingTxResponse, OnChainBalance} import fr.acinq.eclair.blockchain.bitcoind.BitcoindService.BitcoinReq import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient._ @@ -56,7 +56,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("encrypt wallet") { val sender = TestProbe() - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) val walletPassword = Random.alphanumeric.take(8).mkString sender.send(bitcoincli, BitcoinReq("encryptwallet", walletPassword)) sender.expectMsgType[JString] @@ -71,9 +71,53 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A sender.expectMsgType[JValue] } + test("fund and sign psbt") { + val sender = TestProbe() + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, 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))) + + bitcoinClient.fundPsbt(Map(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().isSuccess) + } + + test("fund psbt (invalid requests)") { + val sender = TestProbe() + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) + val priv = PrivateKey(ByteVector32.fromValidHex("01" * 32)) + val address = Bech32.encodeWitnessAddress("bcrt", 0, priv.publicKey.hash160) + + { + // check that it does work + bitcoinClient.fundPsbt(Map(address -> 10000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw)).pipeTo(sender.ref) + sender.expectMsgType[FundPsbtResponse] + } + { + // invalid address + bitcoinClient.fundPsbt(Map("invalid address" -> 10000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw)).pipeTo(sender.ref) + sender.expectMsgType[akka.actor.Status.Failure] + } + { + // amount is too small + bitcoinClient.fundPsbt(Map(address -> 100.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw)).pipeTo(sender.ref) + sender.expectMsgType[akka.actor.Status.Failure] + } + { + // amount is too large + bitcoinClient.fundPsbt(Map(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) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) val txToRemote = { val txNotFunded = Transaction(2, Nil, TxOut(150000 sat, Script.pay2wpkh(randomKey().publicKey)) :: Nil, 0) @@ -145,7 +189,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A } val sender = TestProbe() - val bitcoinClient = new BitcoinCoreClient(rpcClient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) bitcoinClient.onChainBalance().pipeTo(sender.ref) assert(sender.expectMsgType[OnChainBalance] === OnChainBalance(Satoshi(satoshi), Satoshi(satoshi))) @@ -157,7 +201,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("create/commit/rollback funding txs") { val sender = TestProbe() - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) bitcoinClient.onChainBalance().pipeTo(sender.ref) assert(sender.expectMsgType[OnChainBalance].confirmed > 0.sat) @@ -169,7 +213,7 @@ 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 + val fundingTx = sender.expectMsgType[MakeFundingTxResponse].psbt.extract().get 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 @@ -177,7 +221,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A // now fund a tx with correct feerate bitcoinClient.makeFundingTx(pubkeyScript, 50 millibtc, FeeratePerKw(250 sat)).pipeTo(sender.ref) - sender.expectMsgType[MakeFundingTxResponse].fundingTx + sender.expectMsgType[MakeFundingTxResponse].psbt.extract().get } assert(getLocks(sender).size === 4) @@ -206,12 +250,13 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("ensure feerate is always above min-relay-fee") { val sender = TestProbe() - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, 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] + val MakeFundingTxResponse(psbt, _, _) = sender.expectMsgType[MakeFundingTxResponse] + val fundingTx = psbt.extract().get bitcoinClient.commit(fundingTx).pipeTo(sender.ref) sender.expectMsg(true) @@ -219,7 +264,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("unlock failed funding txs") { val sender = TestProbe() - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) bitcoinClient.onChainBalance().pipeTo(sender.ref) assert(sender.expectMsgType[OnChainBalance].confirmed > 0.sat) @@ -232,8 +277,8 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A 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] - + val MakeFundingTxResponse(psbt, _, _) = sender.expectMsgType[MakeFundingTxResponse] + val fundingTx = psbt.extract().get bitcoinClient.commit(fundingTx).pipeTo(sender.ref) sender.expectMsg(true) @@ -243,7 +288,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("unlock utxos when transaction is published") { val sender = TestProbe() - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) generateBlocks(1) // generate a block to ensure we start with an empty mempool // create a first transaction with multiple inputs @@ -300,11 +345,12 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("unlock transaction inputs if publishing fails") { val sender = TestProbe() val pubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(randomKey().publicKey, randomKey().publicKey))) - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, 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] + val MakeFundingTxResponse(psbt, outputIndex, _) = sender.expectMsgType[MakeFundingTxResponse] + val fundingTx = psbt.extract().get // spend the first 2 inputs val tx1 = fundingTx.copy( @@ -339,13 +385,14 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("unlock outpoints correctly") { val sender = TestProbe() val pubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(randomKey().publicKey, randomKey().publicKey))) - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) { // 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] + val MakeFundingTxResponse(psbt, _, _) = sender.expectMsgType[MakeFundingTxResponse] + val fundingTx = psbt.extract().get assert(fundingTx.txIn.size > 2) assert(getLocks(sender) == fundingTx.txIn.map(_.outPoint).toSet) bitcoinClient.rollback(fundingTx).pipeTo(sender.ref) @@ -354,7 +401,8 @@ 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] + val MakeFundingTxResponse(psbt, _, _) = sender.expectMsgType[MakeFundingTxResponse] + val fundingTx = psbt.extract().get assert(fundingTx.txIn.size > 2) assert(getLocks(sender) == fundingTx.txIn.map(_.outPoint).toSet) @@ -373,7 +421,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("sign transactions") { val sender = TestProbe() - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) val nonWalletKey = randomKey() val opts = FundTransactionOptions(TestConstants.feeratePerKw, changePosition = Some(1)) @@ -451,7 +499,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("publish transaction idempotent") { val sender = TestProbe() - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) val priv = dumpPrivateKey(getNewAddress(sender), sender) val noInputTx = Transaction(2, Nil, TxOut(6.btc.toSatoshi, Script.pay2wpkh(priv.publicKey)) :: Nil, 0) @@ -497,7 +545,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("publish invalid transactions") { val sender = TestProbe() - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) // that tx has inputs that don't exist val txWithUnknownInputs = Transaction.read("02000000000101b9e2a3f518fd74e696d258fed3c78c43f84504e76c99212e01cf225083619acf00000000000d0199800136b34b00000000001600145464ce1e5967773922506e285780339d72423244040047304402206795df1fd93c285d9028c384aacf28b43679f1c3f40215fd7bd1abbfb816ee5a022047a25b8c128e692d4717b6dd7b805aa24ecbbd20cfd664ab37a5096577d4a15d014730440220770f44121ed0e71ec4b482dded976f2febd7500dfd084108e07f3ce1e85ec7f5022025b32dc0d551c47136ce41bfb80f5a10de95c0babb22a3ae2d38e6688b32fcb20147522102c2662ab3e4fa18a141d3be3317c6ee134aff10e6cd0a91282a25bf75c0481ebc2102e952dd98d79aa796289fa438e4fdeb06ed8589ff2a0f032b0cfcb4d7b564bc3252aea58d1120") @@ -533,7 +581,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("send and list transactions") { val sender = TestProbe() - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) bitcoinClient.onChainBalance().pipeTo(sender.ref) val initialBalance = sender.expectMsgType[OnChainBalance] @@ -568,7 +616,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("get mempool transaction") { val sender = TestProbe() val address = getNewAddress(sender) - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) def spendWalletTx(tx: Transaction, fees: Satoshi): Transaction = { val inputs = tx.txOut.indices.map(vout => Map("txid" -> tx.txid, "vout" -> vout)) @@ -612,7 +660,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("abandon transaction") { val sender = TestProbe() - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) // Broadcast a wallet transaction. val opts = FundTransactionOptions(TestConstants.feeratePerKw, changePosition = Some(1)) @@ -650,7 +698,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("detect if tx has been double-spent") { val sender = TestProbe() - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) // first let's create a tx val address = "n2YKngjUp139nkjKvZGnfLRN6HzzYxJsje" @@ -689,7 +737,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("find spending transaction of a given output") { val sender = TestProbe() - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) bitcoinClient.getBlockCount.pipeTo(sender.ref) val blockCount = sender.expectMsgType[Long] @@ -732,7 +780,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("compute pubkey from a receive address") { val sender = TestProbe() - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) bitcoinClient.getReceiveAddress().pipeTo(sender.ref) val address = sender.expectMsgType[String] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/PsbtSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/PsbtSpec.scala new file mode 100644 index 0000000000..367f05dcb4 --- /dev/null +++ b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/PsbtSpec.scala @@ -0,0 +1,75 @@ +package fr.acinq.eclair.blockchain.bitcoind + +import akka.testkit.TestProbe +import akka.pattern.pipe +import fr.acinq.bitcoin.{Block, ByteVector32, OutPoint, Psbt, SatoshiLong, Script, ScriptWitness, Transaction, TxIn, TxOut, computeP2WpkhAddress} +import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} +import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient +import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient.{FundPsbtOptions, FundPsbtResponse, ProcessPsbtResponse} +import fr.acinq.eclair.{TestConstants, TestKitBaseClass} +import grizzled.slf4j.Logging +import org.json4s.{DefaultFormats, Formats} +import org.scalatest.BeforeAndAfterAll +import org.scalatest.funsuite.AnyFunSuiteLike + +import scala.concurrent.ExecutionContext.Implicits.global +import scala.util.Success + +class PsbtSpec extends TestKitBaseClass with BitcoindService with AnyFunSuiteLike with BeforeAndAfterAll with Logging { + + implicit val formats: Formats = DefaultFormats + lazy val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) + + override def beforeAll(): Unit = { + startBitcoind() + waitForBitcoindReady() + } + + override def afterAll(): Unit = { + stopBitcoind() + } + + def createOurTx(pub: PublicKey) : (Transaction, Int) = { + val sender = TestProbe() + bitcoinClient.fundPsbt(Map(computeP2WpkhAddress(pub, Block.RegtestGenesisBlock.hash) -> 100000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw, lockUtxos = false)).pipeTo(sender.ref) + val FundPsbtResponse(psbt, _, Some(changepos)) = sender.expectMsgType[FundPsbtResponse] + bitcoinClient.processPsbt(psbt, sign = true).pipeTo(sender.ref) + val ProcessPsbtResponse(psbt1, true) = sender.expectMsgType[ProcessPsbtResponse] + val Success(tx) = psbt1.extract() + bitcoinClient.publishTransaction(tx).pipeTo(sender.ref) + val publishedId = sender.expectMsgType[ByteVector32] + assert(publishedId == tx.txid) + val index = tx.txOut.zipWithIndex.find(_._1.publicKeyScript == Script.write(Script.pay2wpkh(pub))).get._2 + (tx, index) + } + + test("add inputs") { + val sender = TestProbe() + + // create a utxo that sends to a non-wallet key + val priv = PrivateKey(ByteVector32.fromValidHex("01" * 32)) + val (ourTx, index) = createOurTx(priv.publicKey) + + // fund a psbt without inputs + bitcoinClient.fundPsbt(Map(computeP2WpkhAddress(priv.publicKey, Block.RegtestGenesisBlock.hash) -> 100000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw, lockUtxos = false)).pipeTo(sender.ref) + val FundPsbtResponse(psbt, _, _) = sender.expectMsgType[FundPsbtResponse] + + // add our non-wallet input to the PSBT + val psbt1 = psbt.copy( + global = psbt.global.copy(tx = psbt.global.tx.addInput(TxIn(OutPoint(ourTx, index), Nil, 0))), + inputs = psbt.inputs :+ Psbt.PartiallySignedInput.empty.copy(witnessUtxo = Some(ourTx.txOut(index)), witnessScript = Some(Script.pay2pkh(priv.publicKey))) + ) + + // ask bitcoin core to sign its inputs + bitcoinClient.processPsbt(psbt1).pipeTo(sender.ref) + val ProcessPsbtResponse(psbt2, complete) = sender.expectMsgType[ProcessPsbtResponse] //(max = 10 minutes) + + // sign our inputs + val Success(psbt3) = psbt2.sign(priv, 1) + val sig = psbt3.inputs(1).partialSigs(priv.publicKey) + + // we now have a finalized PSBT + val Success(psbt4) = psbt3.finalize(1, ScriptWitness(sig :: priv.publicKey.value :: Nil)) + assert(psbt4.extract().isSuccess) + } +} diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/ZmqWatcherSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/ZmqWatcherSpec.scala index cd2dfb1cd5..99031ba9a2 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/ZmqWatcherSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/ZmqWatcherSpec.scala @@ -74,7 +74,7 @@ class ZmqWatcherSpec extends TestKitBaseClass with AnyFunSuiteLike with Bitcoind val probe = TestProbe() val listener = TestProbe() system.eventStream.subscribe(listener.ref, classOf[CurrentBlockCount]) - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) val nodeParams = TestConstants.Alice.nodeParams.copy(chainHash = Block.RegtestGenesisBlock.hash) val watcher = system.spawn(ZmqWatcher(nodeParams, blockCount, bitcoinClient), UUID.randomUUID().toString) try { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/MempoolTxMonitorSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/MempoolTxMonitorSpec.scala index 85e8e9e6e3..7c2715bfcd 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/MempoolTxMonitorSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/MempoolTxMonitorSpec.scala @@ -21,7 +21,7 @@ import akka.actor.typed.scaladsl.adapter.{ClassicActorSystemOps, TypedActorRefOp import akka.pattern.pipe import akka.testkit.TestProbe import fr.acinq.bitcoin.Crypto.PrivateKey -import fr.acinq.bitcoin.{ByteVector32, OutPoint, SatoshiLong, Transaction, TxIn} +import fr.acinq.bitcoin.{Block, ByteVector32, OutPoint, SatoshiLong, Transaction, TxIn} import fr.acinq.eclair.blockchain.CurrentBlockCount import fr.acinq.eclair.blockchain.WatcherSpec.{createSpendManyP2WPKH, createSpendP2WPKH} import fr.acinq.eclair.blockchain.bitcoind.BitcoindService @@ -53,7 +53,7 @@ class MempoolTxMonitorSpec extends TestKitBaseClass with AnyFunSuiteLike with Bi def createFixture(): Fixture = { val probe = TestProbe() - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) val monitor = system.spawnAnonymous(MempoolTxMonitor(TestConstants.Alice.nodeParams, bitcoinClient, TxPublishLogContext(UUID.randomUUID(), randomKey().publicKey, None))) val address = getNewAddress(probe) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/RawTxPublisherSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/RawTxPublisherSpec.scala index 272e0fc262..c2e52ea96c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/RawTxPublisherSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/RawTxPublisherSpec.scala @@ -20,7 +20,7 @@ import akka.actor.typed.ActorRef import akka.actor.typed.scaladsl.adapter.{ClassicActorSystemOps, TypedActorRefOps, actorRefAdapter} import akka.pattern.pipe import akka.testkit.TestProbe -import fr.acinq.bitcoin.{ByteVector32, SatoshiLong, Transaction} +import fr.acinq.bitcoin.{Block, ByteVector32, SatoshiLong, Transaction} import fr.acinq.eclair.blockchain.CurrentBlockCount import fr.acinq.eclair.blockchain.WatcherSpec.createSpendP2WPKH import fr.acinq.eclair.blockchain.bitcoind.BitcoindService @@ -53,7 +53,7 @@ class RawTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike with Bitc def createFixture(): Fixture = { val probe = TestProbe() val watcher = TestProbe() - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) val publisher = system.spawnAnonymous(RawTxPublisher(TestConstants.Alice.nodeParams, bitcoinClient, watcher.ref, TxPublishLogContext(UUID.randomUUID(), randomKey().publicKey, None))) Fixture(bitcoinClient, publisher, watcher, probe) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisherSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisherSpec.scala index 202ccc78a3..b44c7265c8 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisherSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisherSpec.scala @@ -20,7 +20,7 @@ import akka.actor.typed.ActorRef import akka.actor.typed.scaladsl.adapter.{ClassicActorSystemOps, actorRefAdapter} import akka.pattern.pipe import akka.testkit.{TestFSMRef, TestProbe} -import fr.acinq.bitcoin.{BtcAmount, ByteVector32, MilliBtcDouble, OutPoint, SatoshiLong, Script, ScriptWitness, Transaction, TxIn, TxOut} +import fr.acinq.bitcoin.{Block, BtcAmount, ByteVector32, MilliBtcDouble, OutPoint, SatoshiLong, Script, ScriptWitness, Transaction, TxIn, TxOut} import fr.acinq.eclair.blockchain.CurrentBlockCount import fr.acinq.eclair.blockchain.bitcoind.BitcoindService import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.{WatchOutputSpent, WatchParentTxConfirmed, WatchParentTxConfirmedTriggered, WatchTxConfirmed} @@ -90,7 +90,7 @@ class ReplaceableTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike w // Create a unique wallet for this test and ensure it has some btc. val testId = UUID.randomUUID() val walletRpcClient = createWallet(s"lightning-$testId") - val walletClient = new BitcoinCoreClient(walletRpcClient) + val walletClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) val probe = TestProbe() // Ensure our wallet has some funds. diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/ChannelIntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/ChannelIntegrationSpec.scala index 337fef447c..5021caa1c7 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/ChannelIntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/ChannelIntegrationSpec.scala @@ -255,7 +255,7 @@ abstract class ChannelIntegrationSpec extends IntegrationSpec { sender.send(nodes("C").register, Register.Forward(sender.ref, htlc.channelId, CMD_GETSTATEDATA(ActorRef.noSender))) val Some(localCommit) = sender.expectMsgType[RES_GETSTATEDATA[DATA_CLOSING]].data.localCommitPublished // we wait until the commit tx has been broadcast - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) waitForTxBroadcastOrConfirmed(localCommit.commitTx.txid, bitcoinClient, sender) // we generate a few blocks to get the commit tx confirmed generateBlocks(3, Some(minerAddress)) @@ -311,7 +311,7 @@ abstract class ChannelIntegrationSpec extends IntegrationSpec { // we generate enough blocks to make the htlc timeout generateBlocks((htlc.cltvExpiry.toLong - getBlockCount).toInt, Some(minerAddress)) // we wait until the claim-htlc-timeout has been broadcast - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) assert(remoteCommit.claimHtlcTxs.size === 1) waitForOutputSpent(remoteCommit.claimHtlcTxs.keys.head, bitcoinClient, sender) // and we generate blocks for the claim-htlc-timeout to reach enough confirmations @@ -570,7 +570,7 @@ class StandardChannelIntegrationSpec extends ChannelIntegrationSpec { // we then wait for C and F to negotiate the closing fee awaitCond(stateListener.expectMsgType[ChannelStateChanged](max = 60 seconds).currentState == CLOSING, max = 60 seconds) // and close the channel - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) awaitCond({ bitcoinClient.getMempool().pipeTo(sender.ref) sender.expectMsgType[Seq[Transaction]].exists(_.txIn.head.outPoint.txid === fundingOutpoint.txid) @@ -605,7 +605,7 @@ class StandardChannelIntegrationSpec extends ChannelIntegrationSpec { val revokedCommitFixture = testRevokedCommit(Transactions.DefaultCommitmentFormat) import revokedCommitFixture._ - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) // we retrieve transactions already received so that we don't take them into account when evaluating the outcome of this test val previouslyReceivedByC = listReceivedByAddress(finalAddressC, sender) // F publishes the revoked commitment, one HTLC-success, one HTLC-timeout and leaves the other HTLC outputs unclaimed @@ -725,7 +725,7 @@ abstract class AnchorChannelIntegrationSpec extends ChannelIntegrationSpec { // we then wait for C to detect the unilateral close and go to CLOSING state awaitCond(stateListener.expectMsgType[ChannelStateChanged](max = 60 seconds).currentState == CLOSING, max = 60 seconds) - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) awaitCond({ bitcoinClient.getTransaction(commitTx.txid).map(tx => Some(tx)).recover(_ => None).pipeTo(sender.ref) val tx = sender.expectMsgType[Option[Transaction]] @@ -751,7 +751,7 @@ abstract class AnchorChannelIntegrationSpec extends ChannelIntegrationSpec { val revokedCommitFixture = testRevokedCommit(commitmentFormat) import revokedCommitFixture._ - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) // we retrieve transactions already received so that we don't take them into account when evaluating the outcome of this test val previouslyReceivedByC = listReceivedByAddress(finalAddressC, sender) // F publishes the revoked commitment: it can't publish the HTLC txs because of the CSV 1 diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala index 256e2299c9..c94ad263cc 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala @@ -641,8 +641,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { } test("generate and validate lots of channels") { - val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient) - // we simulate fake channels by publishing a funding tx and sending announcement messages to a node at random + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) // we simulate fake channels by publishing a funding tx and sending announcement messages to a node at random logger.info(s"generating fake channels") val sender = TestProbe() sender.send(bitcoincli, BitcoinReq("getnewaddress")) 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 7d16344ae8..84ae2d3f8c 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 @@ -48,7 +48,7 @@ class AnnouncementsBatchValidationSpec extends AnyFunSuite { implicit val system = ActorSystem("test") implicit val sttpBackend = OkHttpFutureBackend() - val bitcoinClient = new BitcoinCoreClient(new BasicBitcoinJsonRPCClient(user = "foo", password = "bar", host = "localhost", port = 18332)) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, new BasicBitcoinJsonRPCClient(user = "foo", password = "bar", host = "localhost", port = 18332)) val channels = for (i <- 0 until 50) yield { // let's generate a block every 10 txs so that we can compute short ids @@ -94,8 +94,8 @@ object AnnouncementsBatchValidationSpec { val fundingPubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(node1BitcoinKey.publicKey, node2BitcoinKey.publicKey))) val fundingTxFuture = bitcoinClient.makeFundingTx(fundingPubkeyScript, 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) + Await.result(bitcoinClient.publishTransaction(res.psbt.extract().get), 10 seconds) + SimulatedChannel(node1Key, node2Key, node1BitcoinKey, node2BitcoinKey, amount, res.psbt.extract().get, res.fundingTxOutputIndex) } def makeChannelAnnouncement(c: SimulatedChannel, bitcoinClient: BitcoinCoreClient)(implicit ec: ExecutionContext): ChannelAnnouncement = { From a280e27570204fbaf2fd907460407941048e3d41 Mon Sep 17 00:00:00 2001 From: sstone Date: Mon, 16 Aug 2021 18:15:03 +0200 Subject: [PATCH 2/8] Add bip32 derivation path to the funding PSBT that will be signed by bitcoin core We add: - the bip32 path for our local funding pubkey, without the first 2 items (that are based on which chain we're on) - a dummy 1/2/3/4 path for the remote funding pubkey --- .../eclair/blockchain/OnChainWallet.scala | 3 +- .../bitcoind/rpc/BitcoinCoreClient.scala | 53 ++++++++----------- .../fr/acinq/eclair/channel/Channel.scala | 3 +- .../blockchain/DummyOnChainWallet.scala | 8 +-- .../bitcoind/BitcoinCoreClientSpec.scala | 23 ++++---- .../AnnouncementsBatchValidationSpec.scala | 6 +-- 6 files changed, 43 insertions(+), 53 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 8aaf37f220..7b98f20d4d 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 @@ -17,6 +17,7 @@ package fr.acinq.eclair.blockchain import fr.acinq.bitcoin.Crypto.PublicKey +import fr.acinq.bitcoin.DeterministicWallet.ExtendedPublicKey import fr.acinq.bitcoin.{Psbt, Satoshi, Transaction} import fr.acinq.eclair.blockchain.fee.FeeratePerKw import scodec.bits.ByteVector @@ -33,7 +34,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(localFundingKey: ExtendedPublicKey, remoteFundingKey: PublicKey, amount: Satoshi, feeRatePerKw: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] /** * Committing *must* include publishing the transaction on the network. 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 a26ecf925b..58fa5e7a18 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 @@ -17,6 +17,7 @@ package fr.acinq.eclair.blockchain.bitcoind.rpc import fr.acinq.bitcoin.Crypto.PublicKey +import fr.acinq.bitcoin.DeterministicWallet.ExtendedPublicKey import fr.acinq.bitcoin._ import fr.acinq.eclair.ShortChannelId.coordinates import fr.acinq.eclair.TxCoordinates @@ -24,7 +25,7 @@ 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.fee.{FeeratePerKB, FeeratePerKw} -import fr.acinq.eclair.transactions.Transactions +import fr.acinq.eclair.transactions.{Scripts, Transactions} import fr.acinq.eclair.wire.protocol.ChannelAnnouncement import grizzled.slf4j.Logging import org.json4s.Formats @@ -181,7 +182,7 @@ class BitcoinCoreClient(val chainHash: ByteVector32, val rpcClient: BitcoinJsonR } def fundPsbt(inputs: Seq[FundPsbtInput], outputs: Map[String, Satoshi], locktime: Int, 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 => { + 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" @@ -225,54 +226,44 @@ class BitcoinCoreClient(val chainHash: ByteVector32, val rpcClient: BitcoinJsonR } } - override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, targetFeerate: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { + override def makeFundingTx(localFundingKey: ExtendedPublicKey, remoteFundingKey: PublicKey, amount: Satoshi, feeRatePerKw: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = { val hrp = chainHash match { case Block.RegtestGenesisBlock.hash => "bcrt" case Block.TestnetGenesisBlock.hash => "tb" case Block.LivenetGenesisBlock.hash => "bc" case _ => return Future.failed(new IllegalArgumentException(s"invalid chain hash ${chainHash}")) } - val fundingAddress = Script.parse(pubkeyScript) match { - case OP_0 :: OP_PUSHDATA(data, _) :: Nil if data.size == 20 || data.size == 32 => Bech32.encodeWitnessAddress(hrp, 0, data) + 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 fundingAddress = Script.parse(fundingPubkeyScript) match { + case OP_0 :: OP_PUSHDATA(data, _) :: Nil if data.size == 20 || data.size == 32 => Bech32.encodeWitnessAddress(hrp, 0, data) case _ => return Future.failed(new IllegalArgumentException("invalid pubkey script")) } for { // we ask bitcoin core to create and fund the funding tx - feerate <- mempoolMinFee().map(minFee => FeeratePerKw(minFee).max(targetFeerate)) - FundPsbtResponse(psbt, fee, changePosition) <- fundPsbt(Map(fundingAddress -> amount), 0, FundPsbtOptions(feerate, lockUtxos = true)) + feerate <- mempoolMinFee().map(minFee => FeeratePerKw(minFee).max(feeRatePerKw)) + FundPsbtResponse(psbt, fee, Some(changePos)) <- fundPsbt(Map(fundingAddress -> amount), 0, FundPsbtOptions(feerate, lockUtxos = true)) + ourbip32path = localFundingKey.path.drop(2) + output = psbt.outputs(1 - changePos).copy( + derivationPaths = Map( + localFundingKey.publicKey -> Psbt.KeyPathWithMaster(localFundingKey.parent, ourbip32path), + remoteFundingKey -> Psbt.KeyPathWithMaster(0L, DeterministicWallet.KeyPath("1/2/3/4")) + ) + ) + psbt1 = psbt.copy(outputs = psbt.outputs.updated(1 - changePos, output)) // now let's sign the funding tx - ProcessPsbtResponse(signedPsbt, true) <- signPsbtOrUnlock(psbt) + ProcessPsbtResponse(signedPsbt, true) <- signPsbtOrUnlock(psbt1) Success(fundingTx) = signedPsbt.extract() // 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(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=${fee}") } yield MakeFundingTxResponse(signedPsbt, outputIndex, fee) - } -// 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) -// 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)) -// // now let's sign the funding tx -// SignTransactionResponse(fundingTx, true) <- signTransactionOrUnlock(fundTxResponse.tx) -// // 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) -// 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) -// } + } def commit(tx: Transaction)(implicit ec: ExecutionContext): Future[Boolean] = publishTransaction(tx).transformWith { case Success(_) => Future.successful(true) @@ -521,7 +512,7 @@ object BitcoinCoreClient { } case class FundPsbtResponse(psbt: Psbt, fee: Satoshi, changePosition: Option[Int]) { - val amountIn: Satoshi = fee + psbt.computeFees().get + val amountIn: Satoshi = psbt.computeFees().get + psbt.global.tx.txOut.map(_.amount).sum } case class PreviousTx(txid: ByteVector32, vout: Long, scriptPubKey: String, redeemScript: String, witnessScript: String, amount: BigDecimal) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala index f76963f474..eeff04957a 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala @@ -413,8 +413,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo shutdownScript = remoteShutdownScript) log.debug("remote params: {}", remoteParams) val localFundingPubkey = keyManager.fundingPublicKey(localParams.fundingKeyPath) - val fundingPubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingPubkey.publicKey, remoteParams.fundingPubKey))) - wallet.makeFundingTx(fundingPubkeyScript, fundingSatoshis, fundingTxFeeratePerKw).pipeTo(self) + wallet.makeFundingTx(localFundingPubkey, remoteParams.fundingPubKey, fundingSatoshis, fundingTxFeeratePerKw).pipeTo(self) goto(WAIT_FOR_FUNDING_INTERNAL) using DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingSatoshis, pushMsat, initialFeeratePerKw, accept.firstPerCommitmentPoint, channelConfig, channelFeatures, open) } 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 7b2fe55f43..e7b556a156 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,10 +17,12 @@ package fr.acinq.eclair.blockchain import fr.acinq.bitcoin.Crypto.PublicKey +import fr.acinq.bitcoin.DeterministicWallet.ExtendedPublicKey import fr.acinq.bitcoin.{ByteVector32, Crypto, OutPoint, Psbt, Satoshi, SatoshiLong, Script, Transaction, TxIn, TxOut} 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,8 +43,8 @@ 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] = - Future.successful(DummyOnChainWallet.makeDummyFundingTx(pubkeyScript, amount)) + override def makeFundingTx(localFundingKey: ExtendedPublicKey, remoteFundingKey: PublicKey, amount: Satoshi, feeRatePerKw: FeeratePerKw)(implicit ec: ExecutionContext): Future[MakeFundingTxResponse] = + Future.successful(DummyOnChainWallet.makeDummyFundingTx(Script.write(Script.pay2wsh(Scripts.multiSig2of2(localFundingKey.publicKey, remoteFundingKey))), amount)) override def commit(tx: Transaction)(implicit ec: ExecutionContext): Future[Boolean] = Future.successful(true) @@ -65,7 +67,7 @@ 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(localFundingKey: 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) 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 d7243677a9..5c6d564935 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,6 +20,7 @@ import akka.actor.Status.Failure import akka.pattern.pipe import akka.testkit.TestProbe import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} +import fr.acinq.bitcoin.DeterministicWallet.{ExtendedPublicKey, KeyPath} import fr.acinq.bitcoin.SigVersion.SIGVERSION_WITNESS_V0 import fr.acinq.bitcoin.{Bech32, Block, BtcDouble, ByteVector32, Crypto, MilliBtcDouble, OutPoint, SIGHASH_ALL, Satoshi, SatoshiLong, Script, ScriptFlags, ScriptWitness, Transaction, TxIn, TxOut} import fr.acinq.eclair.blockchain.OnChainWallet.{MakeFundingTxResponse, OnChainBalance} @@ -28,7 +29,7 @@ import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient._ 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.{TestConstants, TestKitBaseClass, addressToPublicKeyScript, randomKey} +import fr.acinq.eclair.{TestConstants, TestKitBaseClass, addressToPublicKeyScript, randomBytes32, randomKey} import grizzled.slf4j.Logging import org.json4s.JsonAST._ import org.json4s.{DefaultFormats, Formats} @@ -62,8 +63,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A sender.expectMsgType[JString] 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(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")) @@ -212,7 +212,7 @@ 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) + bitcoinClient.makeFundingTx(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 = sender.expectMsgType[MakeFundingTxResponse].psbt.extract().get bitcoinClient.publishTransaction(fundingTx.copy(txIn = Nil)).pipeTo(sender.ref) // try publishing an invalid version of the tx sender.expectMsgType[Failure] @@ -220,7 +220,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A assert(sender.expectMsgType[Boolean]) // now fund a tx with correct feerate - bitcoinClient.makeFundingTx(pubkeyScript, 50 millibtc, FeeratePerKw(250 sat)).pipeTo(sender.ref) + bitcoinClient.makeFundingTx(ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), randomKey().publicKey, 50 millibtc, FeeratePerKw(250 sat)).pipeTo(sender.ref) sender.expectMsgType[MakeFundingTxResponse].psbt.extract().get } @@ -254,7 +254,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A 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) + bitcoinClient.makeFundingTx(ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), randomKey().publicKey, 5 millibtc, FeeratePerKw(200 sat)).pipeTo(sender.ref) val MakeFundingTxResponse(psbt, _, _) = sender.expectMsgType[MakeFundingTxResponse] val fundingTx = psbt.extract().get @@ -275,8 +275,7 @@ 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) + bitcoinClient.makeFundingTx(ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), randomKey().publicKey, 50 millibtc, FeeratePerKw(10000 sat)).pipeTo(sender.ref) val MakeFundingTxResponse(psbt, _, _) = sender.expectMsgType[MakeFundingTxResponse] val fundingTx = psbt.extract().get bitcoinClient.commit(fundingTx).pipeTo(sender.ref) @@ -344,11 +343,10 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("unlock transaction inputs if publishing fails") { val sender = TestProbe() - val pubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(randomKey().publicKey, randomKey().publicKey))) val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) // 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(ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), randomKey().publicKey, 250 btc, FeeratePerKw(1000 sat)).pipeTo(sender.ref) val MakeFundingTxResponse(psbt, outputIndex, _) = sender.expectMsgType[MakeFundingTxResponse] val fundingTx = psbt.extract().get @@ -384,13 +382,12 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A test("unlock outpoints correctly") { val sender = TestProbe() - val pubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(randomKey().publicKey, randomKey().publicKey))) val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) { // 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) + bitcoinClient.makeFundingTx(ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), randomKey().publicKey, 250 btc, FeeratePerKw(1000 sat)).pipeTo(sender.ref) val MakeFundingTxResponse(psbt, _, _) = sender.expectMsgType[MakeFundingTxResponse] val fundingTx = psbt.extract().get assert(fundingTx.txIn.size > 2) @@ -400,7 +397,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(ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), randomKey().publicKey, 250 btc, FeeratePerKw(1000 sat)).pipeTo(sender.ref) val MakeFundingTxResponse(psbt, _, _) = sender.expectMsgType[MakeFundingTxResponse] val fundingTx = psbt.extract().get assert(fundingTx.txIn.size > 2) 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 84ae2d3f8c..e6c12cefea 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,13 +21,14 @@ import akka.pattern.pipe import akka.testkit.TestProbe import com.softwaremill.sttp.okhttp.OkHttpFutureBackend import fr.acinq.bitcoin.Crypto.PrivateKey +import fr.acinq.bitcoin.DeterministicWallet.{ExtendedPublicKey, KeyPath} import fr.acinq.bitcoin.{Block, Satoshi, SatoshiLong, Script, Transaction} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.ValidateResult 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.{CltvExpiryDelta, Features, MilliSatoshiLong, ShortChannelId, randomKey} +import fr.acinq.eclair.{CltvExpiryDelta, Features, MilliSatoshiLong, ShortChannelId, randomBytes32, randomKey} import org.json4s.JsonAST.JString import org.scalatest.funsuite.AnyFunSuite @@ -91,8 +92,7 @@ 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(ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), randomKey().publicKey, amount, FeeratePerKw(10000 sat)) val res = Await.result(fundingTxFuture, 10 seconds) Await.result(bitcoinClient.publishTransaction(res.psbt.extract().get), 10 seconds) SimulatedChannel(node1Key, node2Key, node1BitcoinKey, node2BitcoinKey, amount, res.psbt.extract().get, res.fundingTxOutputIndex) From c866bfa93b57e4ac07efb5b6089fba33bdda5965 Mon Sep 17 00:00:00 2001 From: sstone Date: Mon, 23 Aug 2021 11:49:16 +0200 Subject: [PATCH 3/8] Add a publicKeyScriptToAddress() method This is used to fund PSBTs. --- .../main/scala/fr/acinq/eclair/package.scala | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/package.scala b/eclair-core/src/main/scala/fr/acinq/eclair/package.scala index 310ac0c1c8..433579c252 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/package.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/package.scala @@ -95,6 +95,34 @@ package object eclair { } } + def publicKeyScriptToAddress(scriptPubKey: Seq[ScriptElt], chainHash: ByteVector32): String = { + val base58PubkeyPrefix = chainHash match { + case Block.LivenetGenesisBlock.hash => Base58.Prefix.PubkeyAddress + case Block.TestnetGenesisBlock.hash | Block.RegtestGenesisBlock.hash => Base58.Prefix.PubkeyAddressTestnet + case _ => ??? + } + val base58ScriptPrefix = chainHash match { + case Block.LivenetGenesisBlock.hash => Base58.Prefix.ScriptAddress + case Block.TestnetGenesisBlock.hash | Block.RegtestGenesisBlock.hash => Base58.Prefix.ScriptAddressTestnet + case _ => ??? + } + val hrp = chainHash match { + case Block.LivenetGenesisBlock.hash => "bc" + case Block.TestnetGenesisBlock.hash => "tb" + case Block.RegtestGenesisBlock.hash => "bcrt" + case _ => ??? + } + scriptPubKey match { + case OP_DUP :: OP_HASH160 :: OP_PUSHDATA(pubKeyHash, _) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil => Base58Check.encode(base58PubkeyPrefix, pubKeyHash) + case OP_HASH160 :: OP_PUSHDATA(scriptHash, _) :: OP_EQUAL :: Nil => Base58Check.encode(base58ScriptPrefix, scriptHash) + case OP_0 :: OP_PUSHDATA(pubKeyHash, _) :: Nil if pubKeyHash.length == 20 => Bech32.encodeWitnessAddress(hrp, 0, pubKeyHash) + case OP_0 :: OP_PUSHDATA(scriptHash, _) :: Nil if scriptHash.length == 32 => Bech32.encodeWitnessAddress(hrp, 0, scriptHash) + case _ => ??? + } + } + + def publicKeyScriptToAddress(scriptPubKey: ByteVector, chainHash: ByteVector32): String = publicKeyScriptToAddress(Script.parse(scriptPubKey), chainHash) + implicit class MilliSatoshiLong(private val n: Long) extends AnyVal { def msat = MilliSatoshi(n) } @@ -103,7 +131,9 @@ package object eclair { implicit object NumericMilliSatoshi extends Numeric[MilliSatoshi] { // @formatter:off override def plus(x: MilliSatoshi, y: MilliSatoshi): MilliSatoshi = x + y + override def minus(x: MilliSatoshi, y: MilliSatoshi): MilliSatoshi = x - y + override def times(x: MilliSatoshi, y: MilliSatoshi): MilliSatoshi = MilliSatoshi(x.toLong * y.toLong) override def negate(x: MilliSatoshi): MilliSatoshi = -x override def fromInt(x: Int): MilliSatoshi = MilliSatoshi(x) From 7f1edcbe495c11339602e18e7a59ebbe55d2d66c Mon Sep 17 00:00:00 2001 From: sstone Date: Mon, 23 Aug 2021 11:50:43 +0200 Subject: [PATCH 4/8] Refactor bitcoin client's fund psbt method --- .../blockchain/bitcoind/rpc/BitcoinCoreClient.scala | 6 +++--- .../blockchain/bitcoind/BitcoinCoreClientSpec.scala | 10 +++++----- .../acinq/eclair/blockchain/bitcoind/PsbtSpec.scala | 4 ++-- .../eclair/blockchain/bitcoind/ZmqWatcherSpec.scala | 12 ++++++------ 4 files changed, 16 insertions(+), 16 deletions(-) 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 58fa5e7a18..182c560190 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 @@ -181,7 +181,7 @@ class BitcoinCoreClient(val chainHash: ByteVector32, val rpcClient: BitcoinJsonR }) } - def fundPsbt(inputs: Seq[FundPsbtInput], outputs: Map[String, Satoshi], locktime: Int, options: FundPsbtOptions)(implicit ec: ExecutionContext): Future[FundPsbtResponse] = { + 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" @@ -192,7 +192,7 @@ class BitcoinCoreClient(val chainHash: ByteVector32, val rpcClient: BitcoinJsonR }) } - def fundPsbt(outputs: Map[String, Satoshi], locktime: Int, options: FundPsbtOptions)(implicit ec: ExecutionContext): Future[FundPsbtResponse] = + 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_ALL)(implicit ec: ExecutionContext): Future[ProcessPsbtResponse] = { @@ -243,7 +243,7 @@ class BitcoinCoreClient(val chainHash: ByteVector32, val rpcClient: BitcoinJsonR for { // we ask bitcoin core to create and fund the funding tx feerate <- mempoolMinFee().map(minFee => FeeratePerKw(minFee).max(feeRatePerKw)) - FundPsbtResponse(psbt, fee, Some(changePos)) <- fundPsbt(Map(fundingAddress -> amount), 0, FundPsbtOptions(feerate, lockUtxos = true)) + FundPsbtResponse(psbt, fee, Some(changePos)) <- fundPsbt(Seq(fundingAddress -> amount), 0, FundPsbtOptions(feerate, lockUtxos = true)) ourbip32path = localFundingKey.path.drop(2) output = psbt.outputs(1 - changePos).copy( derivationPaths = Map( 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 5c6d564935..1b3152cb92 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 @@ -79,7 +79,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A val script = Script.createMultiSigMofN(2, Seq(priv1.publicKey, priv2.publicKey)) val address = Bech32.encodeWitnessAddress("bcrt", 0, Crypto.sha256(Script.write(script))) - bitcoinClient.fundPsbt(Map(address -> 10000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw)).pipeTo(sender.ref) + 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) @@ -95,22 +95,22 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A { // check that it does work - bitcoinClient.fundPsbt(Map(address -> 10000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw)).pipeTo(sender.ref) + bitcoinClient.fundPsbt(Seq(address -> 10000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw)).pipeTo(sender.ref) sender.expectMsgType[FundPsbtResponse] } { // invalid address - bitcoinClient.fundPsbt(Map("invalid address" -> 10000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw)).pipeTo(sender.ref) + 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(Map(address -> 100.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw)).pipeTo(sender.ref) + 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(Map(address -> 11_000_000.btc), 0, FundPsbtOptions(TestConstants.feeratePerKw)).pipeTo(sender.ref) + bitcoinClient.fundPsbt(Seq(address -> 11_000_000.btc), 0, FundPsbtOptions(TestConstants.feeratePerKw)).pipeTo(sender.ref) sender.expectMsgType[akka.actor.Status.Failure] } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/PsbtSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/PsbtSpec.scala index 367f05dcb4..f50c23ad62 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/PsbtSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/PsbtSpec.scala @@ -31,7 +31,7 @@ class PsbtSpec extends TestKitBaseClass with BitcoindService with AnyFunSuiteLik def createOurTx(pub: PublicKey) : (Transaction, Int) = { val sender = TestProbe() - bitcoinClient.fundPsbt(Map(computeP2WpkhAddress(pub, Block.RegtestGenesisBlock.hash) -> 100000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw, lockUtxos = false)).pipeTo(sender.ref) + bitcoinClient.fundPsbt(Seq(computeP2WpkhAddress(pub, Block.RegtestGenesisBlock.hash) -> 100000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw, lockUtxos = false)).pipeTo(sender.ref) val FundPsbtResponse(psbt, _, Some(changepos)) = sender.expectMsgType[FundPsbtResponse] bitcoinClient.processPsbt(psbt, sign = true).pipeTo(sender.ref) val ProcessPsbtResponse(psbt1, true) = sender.expectMsgType[ProcessPsbtResponse] @@ -51,7 +51,7 @@ class PsbtSpec extends TestKitBaseClass with BitcoindService with AnyFunSuiteLik val (ourTx, index) = createOurTx(priv.publicKey) // fund a psbt without inputs - bitcoinClient.fundPsbt(Map(computeP2WpkhAddress(priv.publicKey, Block.RegtestGenesisBlock.hash) -> 100000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw, lockUtxos = false)).pipeTo(sender.ref) + bitcoinClient.fundPsbt(Seq(computeP2WpkhAddress(priv.publicKey, Block.RegtestGenesisBlock.hash) -> 100000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw, lockUtxos = false)).pipeTo(sender.ref) val FundPsbtResponse(psbt, _, _) = sender.expectMsgType[FundPsbtResponse] // add our non-wallet input to the PSBT diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/ZmqWatcherSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/ZmqWatcherSpec.scala index 99031ba9a2..ccc079b3d0 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/ZmqWatcherSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/ZmqWatcherSpec.scala @@ -21,11 +21,11 @@ import akka.actor.typed.scaladsl.adapter.{ClassicActorSystemOps, TypedActorRefOp import akka.actor.{ActorRef, Props, typed} import akka.pattern.pipe import akka.testkit.TestProbe -import fr.acinq.bitcoin.{Block, Btc, MilliBtcDouble, OutPoint, SatoshiLong, Script, Transaction, TxOut} +import fr.acinq.bitcoin.{Bech32, Block, Btc, MilliBtcDouble, OutPoint, SIGHASH_ALL, SatoshiLong, Script, Transaction, TxOut} import fr.acinq.eclair.blockchain.WatcherSpec._ import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher._ import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient -import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient.{FundTransactionOptions, FundTransactionResponse, SignTransactionResponse} +import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient.{FundPsbtOptions, FundPsbtResponse, FundTransactionOptions, FundTransactionResponse, ProcessPsbtResponse, SignTransactionResponse} import fr.acinq.eclair.blockchain.bitcoind.zmq.ZMQActor import fr.acinq.eclair.blockchain.fee.FeeratePerKw import fr.acinq.eclair.blockchain.{CurrentBlockCount, NewTransaction} @@ -282,10 +282,10 @@ class ZmqWatcherSpec extends TestKitBaseClass with AnyFunSuiteLike with Bitcoind // create a chain of transactions that we don't broadcast yet val priv = dumpPrivateKey(getNewAddress(probe), probe) val tx1 = { - bitcoinClient.fundTransaction(Transaction(2, Nil, TxOut(150000 sat, Script.pay2wpkh(priv.publicKey)) :: Nil, 0), FundTransactionOptions(FeeratePerKw(250 sat), lockUtxos = true)).pipeTo(probe.ref) - val funded = probe.expectMsgType[FundTransactionResponse].tx - bitcoinClient.signTransaction(funded).pipeTo(probe.ref) - probe.expectMsgType[SignTransactionResponse].tx + bitcoinClient.fundPsbt(Seq(Bech32.encodeWitnessAddress("bcrt", 0, priv.publicKey.hash160) -> 150000.sat), locktime = 0, FundPsbtOptions(FeeratePerKw(250 sat), lockUtxos = true)).pipeTo(probe.ref) + val funded = probe.expectMsgType[FundPsbtResponse].psbt + bitcoinClient.processPsbt(funded, sign = true, sighashType = SIGHASH_ALL).pipeTo(probe.ref) + probe.expectMsgType[ProcessPsbtResponse].psbt.extract().get } val outputIndex = tx1.txOut.indexWhere(_.publicKeyScript == Script.write(Script.pay2wpkh(priv.publicKey))) val tx2 = createSpendP2WPKH(tx1, priv, priv.publicKey, 10000 sat, 1, 0) From 3ed8b5b15bcb44ba50811333316bd7c37cb7a0fd Mon Sep 17 00:00:00 2001 From: sstone Date: Mon, 23 Aug 2021 17:21:30 +0200 Subject: [PATCH 5/8] Use PSBT to fund and sign "bring your own fee" inputs --- .../publish/ReplaceableTxPublisher.scala | 125 ++++++++++++------ 1 file changed, 87 insertions(+), 38 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala index f18ca3574b..6e0153020b 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala @@ -18,11 +18,11 @@ package fr.acinq.eclair.channel.publish import akka.actor.typed.scaladsl.{ActorContext, Behaviors, TimerScheduler} import akka.actor.typed.{ActorRef, Behavior} -import fr.acinq.bitcoin.{ByteVector32, ByteVector64, Crypto, OutPoint, Satoshi, Script, Transaction, TxOut} -import fr.acinq.eclair.NodeParams +import fr.acinq.bitcoin.{ByteVector32, ByteVector64, Crypto, OutPoint, Psbt, Satoshi, Script, Transaction, TxOut, computeP2WpkhAddress} +import fr.acinq.eclair.{NodeParams, publicKeyScriptToAddress} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher 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.publish.TxPublisher.{TxPublishLogContext, TxRejectedReason} import fr.acinq.eclair.channel.publish.TxTimeLocksMonitor.CheckTx @@ -53,7 +53,7 @@ object ReplaceableTxPublisher { private case object RemoteCommitTxPublished extends RuntimeException with Command private case object PreconditionsOk extends Command private case class FundingFailed(reason: Throwable) extends Command - private case class SignFundedTx(tx: ReplaceableTransactionWithInputInfo, fee: Satoshi) extends Command + private case class SignFundedTx(tx: ReplaceableTransactionWithInputInfo, fee: Satoshi, psbt: Psbt) extends Command private case class PublishSignedTx(tx: Transaction) extends Command private case class WrappedTxResult(result: MempoolTxMonitor.TxResult) extends Command private case class UnknownFailure(reason: Throwable) extends Command @@ -265,13 +265,13 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, def fund(replyTo: ActorRef[TxPublisher.PublishTxResult], cmd: TxPublisher.PublishReplaceableTx, targetFeerate: FeeratePerKw): Behavior[Command] = { context.pipeToSelf(addInputs(cmd.txInfo, targetFeerate, cmd.commitments)) { - case Success((fundedTx, fee)) => SignFundedTx(fundedTx, fee) + case Success((fundedTx, fee, psbt)) => SignFundedTx(fundedTx, fee, psbt) case Failure(reason) => FundingFailed(reason) } Behaviors.receiveMessagePartial { - case SignFundedTx(fundedTx, fee) => + case SignFundedTx(fundedTx, fee, psbt) => log.info("added {} wallet input(s) and {} wallet output(s) to {}", fundedTx.tx.txIn.length - 1, fundedTx.tx.txOut.length - 1, cmd.desc) - sign(replyTo, cmd, fundedTx, fee) + sign(replyTo, cmd, fundedTx, fee, psbt) case FundingFailed(reason) => if (reason.getMessage.contains("Insufficient funds")) { log.warn("cannot add inputs to {}: {}", cmd.desc, reason.getMessage) @@ -287,14 +287,17 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, } } - def sign(replyTo: ActorRef[TxPublisher.PublishTxResult], cmd: TxPublisher.PublishReplaceableTx, fundedTx: ReplaceableTransactionWithInputInfo, fee: Satoshi): Behavior[Command] = { + def sign(replyTo: ActorRef[TxPublisher.PublishTxResult], cmd: TxPublisher.PublishReplaceableTx, fundedTx: ReplaceableTransactionWithInputInfo, fee: Satoshi, psbt: Psbt): Behavior[Command] = { fundedTx match { case claimAnchorTx: ClaimLocalAnchorOutputTx => val claimAnchorSig = keyManager.sign(claimAnchorTx, keyManager.fundingPublicKey(cmd.commitments.localParams.fundingKeyPath), TxOwner.Local, cmd.commitments.commitmentFormat) val signedClaimAnchorTx = addSigs(claimAnchorTx, claimAnchorSig) - val commitInfo = BitcoinCoreClient.PreviousTx(signedClaimAnchorTx.input, signedClaimAnchorTx.tx.txIn.head.witness) - context.pipeToSelf(bitcoinClient.signTransaction(signedClaimAnchorTx.tx, Seq(commitInfo))) { - case Success(signedTx) => PublishSignedTx(signedTx.tx) + val psbt1 = psbt.finalize(0, signedClaimAnchorTx.tx.txIn(0).witness).get + context.pipeToSelf(bitcoinClient.processPsbt(psbt1).map(processPbbtResponse => { + // all inputs should be signed now + processPbbtResponse.psbt.extract().get + })) { + case Success(signedTx) => PublishSignedTx(signedTx) case Failure(reason) => UnknownFailure(reason) } case htlcTx: HtlcTx => @@ -306,11 +309,12 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, val localHtlcBasepoint = keyManager.htlcPoint(channelKeyPath) val localSig = keyManager.sign(htlcTx, localHtlcBasepoint, localPerCommitmentPoint, TxOwner.Local, cmd.commitments.commitmentFormat) val signedHtlcTx = txWithWitnessData.addSigs(localSig, cmd.commitments.commitmentFormat) - val inputInfo = BitcoinCoreClient.PreviousTx(signedHtlcTx.input, signedHtlcTx.tx.txIn.head.witness) - context.pipeToSelf(bitcoinClient.signTransaction(signedHtlcTx.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 - signedHtlcTx.tx.copy(txIn = signedHtlcTx.tx.txIn.head +: signTxResponse.tx.txIn.tail) + + // update our psbt with our signature for our input, and ask bitcoin core to sign its input + val psbt1 = psbt.finalize(0, signedHtlcTx.tx.txIn(0).witness).get + context.pipeToSelf(bitcoinClient.processPsbt(psbt1).map(processPbbtResponse => { + // all inputs should be signed now + processPbbtResponse.psbt.extract().get })) { case Success(signedTx) => PublishSignedTx(signedTx) case Failure(reason) => UnknownFailure(reason) @@ -371,14 +375,14 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, } } - private def addInputs(txInfo: ReplaceableTransactionWithInputInfo, targetFeerate: FeeratePerKw, commitments: Commitments): Future[(ReplaceableTransactionWithInputInfo, Satoshi)] = { + private def addInputs(txInfo: ReplaceableTransactionWithInputInfo, targetFeerate: FeeratePerKw, commitments: Commitments): Future[(ReplaceableTransactionWithInputInfo, Satoshi, Psbt)] = { txInfo match { case anchorTx: ClaimLocalAnchorOutputTx => addInputs(anchorTx, targetFeerate, commitments) case htlcTx: HtlcTx => addInputs(htlcTx, targetFeerate, commitments) } } - private def addInputs(txInfo: ClaimLocalAnchorOutputTx, targetFeerate: FeeratePerKw, commitments: Commitments): Future[(ClaimLocalAnchorOutputTx, Satoshi)] = { + private def addInputs(txInfo: ClaimLocalAnchorOutputTx, targetFeerate: FeeratePerKw, commitments: Commitments): Future[(ClaimLocalAnchorOutputTx, Satoshi, Psbt)] = { val dustLimit = commitments.localParams.dustLimit val commitFeerate = commitments.localCommit.spec.commitTxFeerate val commitTx = commitments.fullySignedLocalCommitTx(nodeParams.channelKeyManager).tx @@ -397,29 +401,68 @@ private class ReplaceableTxPublisher(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 address = publicKeyScriptToAddress(Script.pay2wpkh(PlaceHolderPubKey), nodeParams.chainHash) + + 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)) + val changeOutput = fundPsbtResponse.psbt.global.tx.txOut(changePos) + val changeOutput1 = changeOutput.copy(amount = changeOutput.amount + dummyChangeAmount) + val psbt1 = fundPsbtResponse.psbt.copy( + global = fundPsbtResponse.psbt.global.copy(tx = fundPsbtResponse.psbt.global.tx.copy(txOut = Seq(changeOutput1))), + outputs = Seq(fundPsbtResponse.psbt.outputs(changePos)) + ) + Future.successful(psbt1) case None => 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)) + fundPsbtResponse.psbt.copy( + global = fundPsbtResponse.psbt.global.copy(tx = fundPsbtResponse.psbt.global.tx.copy(txOut = Seq(changeOutput1))) + ) }) } - }).map(fundTxResponse => { - require(fundTxResponse.tx.txOut.size == 1, "funded transaction should have a single change output") + } + + for { + fundPsbtResponse <- bitcoinClient.fundPsbt(Seq(computeP2WpkhAddress(PlaceHolderPubKey, nodeParams.chainHash) -> 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 = txInfo.copy(tx = fundTxResponse.tx.copy(txIn = txInfo.tx.txIn.head +: fundTxResponse.tx.txIn)) - adjustAnchorOutputChange(unsignedTx, commitTx, fundTxResponse.amountIn + AnchorOutputsCommitmentFormat.anchorAmount, commitFeerate, targetFeerate, dustLimit) - }) + unsignedTx = txInfo.copy(tx = psbt.global.tx.copy(txIn = txInfo.tx.txIn.head +: psbt.global.tx.txIn)) + adjustedTx = adjustAnchorOutputChange(unsignedTx, commitTx, fundPsbtResponse.amountIn + AnchorOutputsCommitmentFormat.anchorAmount, commitFeerate, targetFeerate, dustLimit) + psbtInput = Psbt.PartiallySignedInput.empty.copy( + witnessUtxo = Some(txInfo.input.txOut), + witnessScript = Some(Script.parse(txInfo.input.redeemScript)) + ) + psbt1 = psbt.copy( + global = psbt.global.copy(tx = adjustedTx.tx), + inputs = psbtInput +: psbt.inputs) + } yield { + (adjustedTx, psbt1) + } + + // 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 { + // 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)) + // case None => + // bitcoinClient.getChangeAddress().map(pubkeyHash => { + // val txSingleOutput = fundTxResponse.tx.copy(txOut = Seq(TxOut(dummyChangeAmount, Script.pay2wpkh(pubkeyHash)))) + // fundTxResponse.copy(tx = txSingleOutput) + // }) + // } + // }).map(fundTxResponse => { + // require(fundTxResponse.tx.txOut.size == 1, "funded transaction should have a single change output") + // // NB: we insert the anchor input in the *first* position because our signing helpers only sign input #0. + // val unsignedTx = txInfo.copy(tx = fundTxResponse.tx.copy(txIn = txInfo.tx.txIn.head +: fundTxResponse.tx.txIn)) + // adjustAnchorOutputChange(unsignedTx, commitTx, fundTxResponse.amountIn + AnchorOutputsCommitmentFormat.anchorAmount, commitFeerate, targetFeerate, dustLimit) -> Psbt(unsignedTx.tx) + // }) } - private def addInputs(txInfo: HtlcTx, targetFeerate: FeeratePerKw, commitments: Commitments): Future[(HtlcTx, Satoshi)] = { + private def addInputs(txInfo: HtlcTx, targetFeerate: FeeratePerKw, commitments: Commitments): Future[(HtlcTx, Satoshi, Psbt)] = { // NB: fundrawtransaction doesn't support non-wallet inputs, so we clear the input and re-add it later. val txNotFunded = txInfo.tx.copy(txIn = Nil, txOut = txInfo.tx.txOut.head.copy(amount = commitments.localParams.dustLimit) :: Nil) val htlcTxWeight = txInfo match { @@ -438,18 +481,24 @@ private class ReplaceableTxPublisher(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 address = publicKeyScriptToAddress(txInfo.tx.txOut.head.publicKeyScript, nodeParams.chainHash) + bitcoinClient.fundPsbt(Seq(address -> commitments.localParams.dustLimit), 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 = txInfo.tx.txIn ++ fundTxResponse.tx.txIn, - txOut = txInfo.tx.txOut ++ fundTxResponse.tx.txOut.tail + val txWithHtlcInput = fundPsbtResponse.psbt.global.tx.copy( + txIn = txInfo.tx.txIn ++ fundPsbtResponse.psbt.global.tx.txIn, + txOut = txInfo.tx.txOut ++ fundPsbtResponse.psbt.global.tx.txOut.tail ) val unsignedTx = txInfo match { case htlcSuccess: HtlcSuccessTx => htlcSuccess.copy(tx = txWithHtlcInput) case htlcTimeout: HtlcTimeoutTx => htlcTimeout.copy(tx = txWithHtlcInput) } - adjustHtlcTxChange(unsignedTx, fundTxResponse.amountIn + unsignedTx.input.txOut.amount, targetFeerate, commitments) + val adjustedTx = adjustHtlcTxChange(unsignedTx, fundPsbtResponse.amountIn + unsignedTx.input.txOut.amount, targetFeerate, commitments) + val psbt1 = fundPsbtResponse.psbt.copy( + global = fundPsbtResponse.psbt.global.copy(tx = adjustedTx.tx), + inputs = Psbt.PartiallySignedInput.empty.copy(witnessUtxo = Some(txInfo.input.txOut), witnessScript = Some(Script.parse(txInfo.input.redeemScript))) +: fundPsbtResponse.psbt.inputs + ) + adjustedTx -> psbt1 }) } From 8e4f76015e48e80e3bcfc9a4fd280344c8ef10ed Mon Sep 17 00:00:00 2001 From: sstone Date: Tue, 24 Aug 2021 20:17:17 +0200 Subject: [PATCH 6/8] Clean up --- .../publish/ReplaceableTxPublisher.scala | 28 ++++--------------- .../main/scala/fr/acinq/eclair/package.scala | 14 +++++++--- 2 files changed, 16 insertions(+), 26 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala index 6e0153020b..56f40ce04c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala @@ -292,6 +292,7 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, case claimAnchorTx: ClaimLocalAnchorOutputTx => val claimAnchorSig = keyManager.sign(claimAnchorTx, keyManager.fundingPublicKey(cmd.commitments.localParams.fundingKeyPath), TxOwner.Local, cmd.commitments.commitmentFormat) val signedClaimAnchorTx = addSigs(claimAnchorTx, claimAnchorSig) + // update our psbt with our signature for our input, and ask bitcoin core to sign its input val psbt1 = psbt.finalize(0, signedClaimAnchorTx.tx.txIn(0).witness).get context.pipeToSelf(bitcoinClient.processPsbt(psbt1).map(processPbbtResponse => { // all inputs should be signed now @@ -403,9 +404,11 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, val dummyChangeAmount = weight2fee(anchorFeerate, claimAnchorOutputMinWeight) + dustLimit val address = publicKeyScriptToAddress(Script.pay2wpkh(PlaceHolderPubKey), nodeParams.chainHash) + // merge outptuts if needed to get a PSBT with a single output def makeSingleOutput(fundPsbtResponse: FundPsbtResponse): Future[Psbt] = { fundPsbtResponse.changePosition match { case Some(changePos) => + // add our main output to the change output val changeOutput = fundPsbtResponse.psbt.global.tx.txOut(changePos) val changeOutput1 = changeOutput.copy(amount = changeOutput.amount + dummyChangeAmount) val psbt1 = fundPsbtResponse.psbt.copy( @@ -414,6 +417,7 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, ) Future.successful(psbt1) case None => + // replace our main output with a dummy change output bitcoinClient.getChangeAddress().map(pubkeyHash => { val changeOutput1 = TxOut(dummyChangeAmount, Script.pay2wpkh(pubkeyHash)) fundPsbtResponse.psbt.copy( @@ -424,11 +428,12 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, } for { - fundPsbtResponse <- bitcoinClient.fundPsbt(Seq(computeP2WpkhAddress(PlaceHolderPubKey, nodeParams.chainHash) -> dummyChangeAmount), 0, FundPsbtOptions(anchorFeerate, lockUtxos = true, changePosition = Some(1))) + 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. unsignedTx = txInfo.copy(tx = psbt.global.tx.copy(txIn = txInfo.tx.txIn.head +: psbt.global.tx.txIn)) adjustedTx = adjustAnchorOutputChange(unsignedTx, commitTx, fundPsbtResponse.amountIn + AnchorOutputsCommitmentFormat.anchorAmount, commitFeerate, targetFeerate, dustLimit) + // add a PSBT input for our input (i.e the one that spends our own anchor/htlc output and that we'll need to sign psbtInput = Psbt.PartiallySignedInput.empty.copy( witnessUtxo = Some(txInfo.input.txOut), witnessScript = Some(Script.parse(txInfo.input.redeemScript)) @@ -439,27 +444,6 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, } yield { (adjustedTx, psbt1) } - - // 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 { - // 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)) - // case None => - // bitcoinClient.getChangeAddress().map(pubkeyHash => { - // val txSingleOutput = fundTxResponse.tx.copy(txOut = Seq(TxOut(dummyChangeAmount, Script.pay2wpkh(pubkeyHash)))) - // fundTxResponse.copy(tx = txSingleOutput) - // }) - // } - // }).map(fundTxResponse => { - // require(fundTxResponse.tx.txOut.size == 1, "funded transaction should have a single change output") - // // NB: we insert the anchor input in the *first* position because our signing helpers only sign input #0. - // val unsignedTx = txInfo.copy(tx = fundTxResponse.tx.copy(txIn = txInfo.tx.txIn.head +: fundTxResponse.tx.txIn)) - // adjustAnchorOutputChange(unsignedTx, commitTx, fundTxResponse.amountIn + AnchorOutputsCommitmentFormat.anchorAmount, commitFeerate, targetFeerate, dustLimit) -> Psbt(unsignedTx.tx) - // }) } private def addInputs(txInfo: HtlcTx, targetFeerate: FeeratePerKw, commitments: Commitments): Future[(HtlcTx, Satoshi, Psbt)] = { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/package.scala b/eclair-core/src/main/scala/fr/acinq/eclair/package.scala index 433579c252..6c7624190e 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/package.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/package.scala @@ -95,29 +95,35 @@ package object eclair { } } + /** + * + * @param scriptPubKey public key script + * @param chainHash hash of the chain we're on + * @return the address the this public key script on this chain + */ def publicKeyScriptToAddress(scriptPubKey: Seq[ScriptElt], chainHash: ByteVector32): String = { val base58PubkeyPrefix = chainHash match { case Block.LivenetGenesisBlock.hash => Base58.Prefix.PubkeyAddress case Block.TestnetGenesisBlock.hash | Block.RegtestGenesisBlock.hash => Base58.Prefix.PubkeyAddressTestnet - case _ => ??? + case _ => throw new IllegalArgumentException(s"invalid chain hash $chainHash") } val base58ScriptPrefix = chainHash match { case Block.LivenetGenesisBlock.hash => Base58.Prefix.ScriptAddress case Block.TestnetGenesisBlock.hash | Block.RegtestGenesisBlock.hash => Base58.Prefix.ScriptAddressTestnet - case _ => ??? + case _ => throw new IllegalArgumentException(s"invalid chain hash $chainHash") } val hrp = chainHash match { case Block.LivenetGenesisBlock.hash => "bc" case Block.TestnetGenesisBlock.hash => "tb" case Block.RegtestGenesisBlock.hash => "bcrt" - case _ => ??? + case _ => throw new IllegalArgumentException(s"invalid chain hash $chainHash") } scriptPubKey match { case OP_DUP :: OP_HASH160 :: OP_PUSHDATA(pubKeyHash, _) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil => Base58Check.encode(base58PubkeyPrefix, pubKeyHash) case OP_HASH160 :: OP_PUSHDATA(scriptHash, _) :: OP_EQUAL :: Nil => Base58Check.encode(base58ScriptPrefix, scriptHash) case OP_0 :: OP_PUSHDATA(pubKeyHash, _) :: Nil if pubKeyHash.length == 20 => Bech32.encodeWitnessAddress(hrp, 0, pubKeyHash) case OP_0 :: OP_PUSHDATA(scriptHash, _) :: Nil if scriptHash.length == 32 => Bech32.encodeWitnessAddress(hrp, 0, scriptHash) - case _ => ??? + case _ => throw new IllegalArgumentException(s"invalid pubkey script $scriptPubKey") } } From cc70334689f56891e1cc10955de6db1a14bc8377 Mon Sep 17 00:00:00 2001 From: sstone Date: Wed, 15 Sep 2021 17:51:19 +0200 Subject: [PATCH 7/8] Fix failing tests --- .../bitcoind/rpc/BitcoinCoreClient.scala | 38 +++++++++++++------ .../publish/ReplaceableTxPublisher.scala | 8 ++-- .../bitcoind/BitcoinCoreClientSpec.scala | 4 +- .../publish/ReplaceableTxPublisherSpec.scala | 2 +- .../AnnouncementsBatchValidationSpec.scala | 2 +- 5 files changed, 35 insertions(+), 19 deletions(-) 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 182c560190..1d511d3244 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 @@ -213,7 +213,12 @@ class BitcoinCoreClient(val chainHash: ByteVector32, val rpcClient: BitcoinJsonR } private def signPsbtOrUnlock(psbt: Psbt)(implicit ec: ExecutionContext): Future[ProcessPsbtResponse] = { - val f = processPsbt(psbt).withFilter(_.complete == true) + val f1 = processPsbt(psbt).withFilter(_.complete == true) + + 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.global.tx.txIn.map(_.outPoint)) @@ -240,21 +245,33 @@ class BitcoinCoreClient(val chainHash: ByteVector32, val rpcClient: BitcoinJsonR case _ => return Future.failed(new IllegalArgumentException("invalid pubkey script")) } - for { - // we ask bitcoin core to create and fund the funding tx - feerate <- mempoolMinFee().map(minFee => FeeratePerKw(minFee).max(feeRatePerKw)) - FundPsbtResponse(psbt, fee, Some(changePos)) <- fundPsbt(Seq(fundingAddress -> amount), 0, FundPsbtOptions(feerate, lockUtxos = true)) - ourbip32path = localFundingKey.path.drop(2) - output = psbt.outputs(1 - changePos).copy( + 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 + } + val output = psbt.outputs(outputIndex).copy( derivationPaths = Map( localFundingKey.publicKey -> Psbt.KeyPathWithMaster(localFundingKey.parent, ourbip32path), remoteFundingKey -> Psbt.KeyPathWithMaster(0L, DeterministicWallet.KeyPath("1/2/3/4")) ) ) - psbt1 = psbt.copy(outputs = psbt.outputs.updated(1 - changePos, output)) + psbt.copy(outputs = psbt.outputs.updated(outputIndex, output)) + } + + for { + // 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.drop(2) + _ = logger.info(s"funded psbt = $psbt") + psbt1 = updatePsbt(psbt, changePos_opt, ourbip32path) // now let's sign the funding tx - ProcessPsbtResponse(signedPsbt, true) <- signPsbtOrUnlock(psbt1) - Success(fundingTx) = signedPsbt.extract() + ProcessPsbtResponse(signedPsbt, complete) <- signPsbtOrUnlock(psbt1) + _ = logger.info(s"psbt signing complete = $complete") + extracted = signedPsbt.extract() + _ = if (extracted.isFailure) logger.error(s"psbt failure $extracted") + fundingTx = extracted.get // there will probably be a change output, so we need to find which output is ours outputIndex <- Transactions.findPubKeyScriptIndex(fundingTx, fundingPubkeyScript) match { case Right(outputIndex) => Future.successful(outputIndex) @@ -262,7 +279,6 @@ class BitcoinCoreClient(val chainHash: ByteVector32, val rpcClient: BitcoinJsonR } _ = 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 { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala index 56f40ce04c..279e077b48 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala @@ -432,7 +432,7 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, psbt <- makeSingleOutput(fundPsbtResponse) // NB: we insert the anchor input in the *first* position because our signing helpers only sign input #0. unsignedTx = txInfo.copy(tx = psbt.global.tx.copy(txIn = txInfo.tx.txIn.head +: psbt.global.tx.txIn)) - adjustedTx = adjustAnchorOutputChange(unsignedTx, commitTx, fundPsbtResponse.amountIn + AnchorOutputsCommitmentFormat.anchorAmount, commitFeerate, targetFeerate, dustLimit) + (adjustedTx, fee) = adjustAnchorOutputChange(unsignedTx, commitTx, fundPsbtResponse.amountIn + AnchorOutputsCommitmentFormat.anchorAmount, commitFeerate, targetFeerate, dustLimit) // add a PSBT input for our input (i.e the one that spends our own anchor/htlc output and that we'll need to sign psbtInput = Psbt.PartiallySignedInput.empty.copy( witnessUtxo = Some(txInfo.input.txOut), @@ -442,7 +442,7 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, global = psbt.global.copy(tx = adjustedTx.tx), inputs = psbtInput +: psbt.inputs) } yield { - (adjustedTx, psbt1) + (adjustedTx, fee, psbt1) } } @@ -477,12 +477,12 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, case htlcSuccess: HtlcSuccessTx => htlcSuccess.copy(tx = txWithHtlcInput) case htlcTimeout: HtlcTimeoutTx => htlcTimeout.copy(tx = txWithHtlcInput) } - val adjustedTx = adjustHtlcTxChange(unsignedTx, fundPsbtResponse.amountIn + unsignedTx.input.txOut.amount, targetFeerate, commitments) + val (adjustedTx, fee) = adjustHtlcTxChange(unsignedTx, fundPsbtResponse.amountIn + unsignedTx.input.txOut.amount, targetFeerate, commitments) val psbt1 = fundPsbtResponse.psbt.copy( global = fundPsbtResponse.psbt.global.copy(tx = adjustedTx.tx), inputs = Psbt.PartiallySignedInput.empty.copy(witnessUtxo = Some(txInfo.input.txOut), witnessScript = Some(Script.parse(txInfo.input.redeemScript))) +: fundPsbtResponse.psbt.inputs ) - adjustedTx -> psbt1 + (adjustedTx, fee, psbt1) }) } 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 1b3152cb92..6e57675ade 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 @@ -65,7 +65,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A bitcoinClient.makeFundingTx(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, 10)) sender.expectMsgType[JValue] @@ -189,7 +189,7 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A } val sender = TestProbe() - val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) + val bitcoinClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, rpcClient) bitcoinClient.onChainBalance().pipeTo(sender.ref) assert(sender.expectMsgType[OnChainBalance] === OnChainBalance(Satoshi(satoshi), Satoshi(satoshi))) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisherSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisherSpec.scala index b44c7265c8..bf61a2dee9 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisherSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisherSpec.scala @@ -90,7 +90,7 @@ class ReplaceableTxPublisherSpec extends TestKitBaseClass with AnyFunSuiteLike w // Create a unique wallet for this test and ensure it has some btc. val testId = UUID.randomUUID() val walletRpcClient = createWallet(s"lightning-$testId") - val walletClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, bitcoinrpcclient) + val walletClient = new BitcoinCoreClient(Block.RegtestGenesisBlock.hash, walletRpcClient) val probe = TestProbe() // Ensure our wallet has some funds. 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 e6c12cefea..cf9a59c67d 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 @@ -92,7 +92,7 @@ object AnnouncementsBatchValidationSpec { val node2BitcoinKey = randomKey() val amount = 1000000 sat // first we publish the funding tx - val fundingTxFuture = bitcoinClient.makeFundingTx(ExtendedPublicKey(randomKey().publicKey.value, randomBytes32(), 4, KeyPath("m/1/2/3/4"), 0), randomKey().publicKey, amount, FeeratePerKw(10000 sat)) + val fundingTxFuture = bitcoinClient.makeFundingTx(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.psbt.extract().get), 10 seconds) SimulatedChannel(node1Key, node2Key, node1BitcoinKey, node2BitcoinKey, amount, res.psbt.extract().get, res.fundingTxOutputIndex) From 415e83624deccbf46a442dfef608708080efeee9 Mon Sep 17 00:00:00 2001 From: sstone Date: Mon, 20 Sep 2021 18:13:02 +0200 Subject: [PATCH 8/8] Use improved PSBT API (see https://github.com/ACINQ/bitcoin-lib/pull/60) --- .../bitcoind/rpc/BitcoinCoreClient.scala | 6 ++-- .../publish/ReplaceableTxPublisher.scala | 29 +++++++++---------- .../acinq/eclair/payment/PaymentRequest.scala | 2 +- .../blockchain/DummyOnChainWallet.scala | 4 +-- .../eclair/blockchain/bitcoind/PsbtSpec.scala | 14 ++++----- .../channel/states/e/NormalStateSpec.scala | 2 +- .../eclair/payment/PaymentRequestSpec.scala | 12 ++++---- pom.xml | 2 +- 8 files changed, 34 insertions(+), 37 deletions(-) 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 1d511d3244..c9d8ffa2c2 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 @@ -250,13 +250,11 @@ class BitcoinCoreClient(val chainHash: ByteVector32, val rpcClient: BitcoinJsonR case None => 0 case Some(changePos) => 1 - changePos } - val output = psbt.outputs(outputIndex).copy( - derivationPaths = Map( + psbt.updateWitnessOutput(outputIndex, derivationPaths = Map( localFundingKey.publicKey -> Psbt.KeyPathWithMaster(localFundingKey.parent, ourbip32path), remoteFundingKey -> Psbt.KeyPathWithMaster(0L, DeterministicWallet.KeyPath("1/2/3/4")) ) - ) - psbt.copy(outputs = psbt.outputs.updated(outputIndex, output)) + ).get } for { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala index 279e077b48..ea61a0f08d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/publish/ReplaceableTxPublisher.scala @@ -293,7 +293,7 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, val claimAnchorSig = keyManager.sign(claimAnchorTx, keyManager.fundingPublicKey(cmd.commitments.localParams.fundingKeyPath), TxOwner.Local, cmd.commitments.commitmentFormat) val signedClaimAnchorTx = addSigs(claimAnchorTx, claimAnchorSig) // update our psbt with our signature for our input, and ask bitcoin core to sign its input - val psbt1 = psbt.finalize(0, signedClaimAnchorTx.tx.txIn(0).witness).get + val psbt1 = psbt.finalizeWitnessInput(0, signedClaimAnchorTx.tx.txIn(0).witness).get context.pipeToSelf(bitcoinClient.processPsbt(psbt1).map(processPbbtResponse => { // all inputs should be signed now processPbbtResponse.psbt.extract().get @@ -312,7 +312,7 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, val signedHtlcTx = txWithWitnessData.addSigs(localSig, cmd.commitments.commitmentFormat) // update our psbt with our signature for our input, and ask bitcoin core to sign its input - val psbt1 = psbt.finalize(0, signedHtlcTx.tx.txIn(0).witness).get + val psbt1 = psbt.finalizeWitnessInput(0, signedHtlcTx.tx.txIn(0).witness).get context.pipeToSelf(bitcoinClient.processPsbt(psbt1).map(processPbbtResponse => { // all inputs should be signed now processPbbtResponse.psbt.extract().get @@ -434,21 +434,20 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, unsignedTx = txInfo.copy(tx = psbt.global.tx.copy(txIn = txInfo.tx.txIn.head +: psbt.global.tx.txIn)) (adjustedTx, fee) = adjustAnchorOutputChange(unsignedTx, commitTx, fundPsbtResponse.amountIn + AnchorOutputsCommitmentFormat.anchorAmount, commitFeerate, targetFeerate, dustLimit) // add a PSBT input for our input (i.e the one that spends our own anchor/htlc output and that we'll need to sign - psbtInput = Psbt.PartiallySignedInput.empty.copy( - witnessUtxo = Some(txInfo.input.txOut), - witnessScript = Some(Script.parse(txInfo.input.redeemScript)) - ) - psbt1 = psbt.copy( - global = psbt.global.copy(tx = adjustedTx.tx), - inputs = psbtInput +: psbt.inputs) + psbt1 = Psbt(adjustedTx.tx) + // update our txinfo input + psbt2 = psbt1.updateWitnessInput(txInfo.input.outPoint, txInfo.input.txOut, witnessScript = Some(Script.parse(txInfo.input.redeemScript))).get + // update the inputs selected by bitcoin core + psbt3 = psbt.outputs.zipWithIndex.tail.foldLeft(psbt2) { + case (psbt, (output, index)) => psbt.updateWitnessOutput(index, output.witnessScript, output.redeemScript, output.derivationPaths).get + } } yield { - (adjustedTx, fee, psbt1) + (adjustedTx, fee, psbt3) } } private def addInputs(txInfo: HtlcTx, targetFeerate: FeeratePerKw, commitments: Commitments): Future[(HtlcTx, Satoshi, Psbt)] = { // NB: fundrawtransaction doesn't support non-wallet inputs, so we clear the input and re-add it later. - val txNotFunded = txInfo.tx.copy(txIn = Nil, txOut = txInfo.tx.txOut.head.copy(amount = commitments.localParams.dustLimit) :: Nil) val htlcTxWeight = txInfo match { case _: HtlcSuccessTx => commitments.commitmentFormat.htlcSuccessWeight case _: HtlcTimeoutTx => commitments.commitmentFormat.htlcTimeoutWeight @@ -478,10 +477,10 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams, case htlcTimeout: HtlcTimeoutTx => htlcTimeout.copy(tx = txWithHtlcInput) } val (adjustedTx, fee) = adjustHtlcTxChange(unsignedTx, fundPsbtResponse.amountIn + unsignedTx.input.txOut.amount, targetFeerate, commitments) - val psbt1 = fundPsbtResponse.psbt.copy( - global = fundPsbtResponse.psbt.global.copy(tx = adjustedTx.tx), - inputs = Psbt.PartiallySignedInput.empty.copy(witnessUtxo = Some(txInfo.input.txOut), witnessScript = Some(Script.parse(txInfo.input.redeemScript))) +: fundPsbtResponse.psbt.inputs - ) + var psbt1 = Psbt(adjustedTx.tx).updateWitnessInput(txInfo.input.outPoint, txInfo.input.txOut, witnessScript = Some(Script.parse(txInfo.input.redeemScript))).get + fundPsbtResponse.psbt.outputs.zipWithIndex.tail.foreach { + case (output, index) => psbt1 = psbt1.updateWitnessOutput(index, output.witnessScript, output.redeemScript, output.derivationPaths).get + } (adjustedTx, fee, psbt1) }) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala index 2b8dd2ac14..77c339783f 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala @@ -579,6 +579,6 @@ object PaymentRequest { val hrp = s"${pr.prefix}$hramount" val data = Codecs.bolt11DataCodec.encode(Bolt11Data(pr.timestamp, pr.tags, pr.signature)).require val int5s = eight2fiveCodec.decode(data).require.value - Bech32.encode(hrp, int5s.toArray) + Bech32.encode(hrp, int5s.toArray, Bech32.Bech32Encoding) } } 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 e7b556a156..d06c76bab2 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 @@ -90,9 +90,9 @@ object DummyOnChainWallet { txOut = TxOut(amount, pubkeyScript) :: Nil, lockTime = 0) val Success(psbt) = Psbt(fundingTx) - .update(baseTx, 0, witnessScript = Some(Script.pay2pkh(key.publicKey))) + .updateWitnessInputTx(baseTx, 0, witnessScript = Some(Script.pay2pkh(key.publicKey))) .flatMap(p => p.sign(key, 0)) - .flatMap(p => p.finalize(0, Script.witnessPay2wpkh(key.publicKey, p.inputs(0).partialSigs(key.publicKey)))) + .flatMap(p => p.psbt.finalizeWitnessInput(0, Script.witnessPay2wpkh(key.publicKey, p.sig))) MakeFundingTxResponse(psbt, 0, 420 sat) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/PsbtSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/PsbtSpec.scala index f50c23ad62..99abd595f0 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/PsbtSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/bitcoind/PsbtSpec.scala @@ -54,11 +54,11 @@ class PsbtSpec extends TestKitBaseClass with BitcoindService with AnyFunSuiteLik bitcoinClient.fundPsbt(Seq(computeP2WpkhAddress(priv.publicKey, Block.RegtestGenesisBlock.hash) -> 100000.sat), 0, FundPsbtOptions(TestConstants.feeratePerKw, lockUtxos = false)).pipeTo(sender.ref) val FundPsbtResponse(psbt, _, _) = sender.expectMsgType[FundPsbtResponse] - // add our non-wallet input to the PSBT - val psbt1 = psbt.copy( - global = psbt.global.copy(tx = psbt.global.tx.addInput(TxIn(OutPoint(ourTx, index), Nil, 0))), - inputs = psbt.inputs :+ Psbt.PartiallySignedInput.empty.copy(witnessUtxo = Some(ourTx.txOut(index)), witnessScript = Some(Script.pay2pkh(priv.publicKey))) - ) + val fakeTx = Transaction(version = 2, txIn = TxIn(OutPoint(ourTx, index), Nil, 0) :: Nil, txOut = Nil, lockTime = 0) + val fakePsbt = Psbt(fakeTx) + val joined = Psbt.join(psbt, fakePsbt).get + val psbt1 = joined.updateWitnessInput(OutPoint(ourTx, index), ourTx.txOut(index), witnessScript = Some(Script.pay2pkh(priv.publicKey))).get + val txWithAdditionalInput = psbt.global.tx.addInput(TxIn(OutPoint(ourTx, index), Nil, 0)) // ask bitcoin core to sign its inputs bitcoinClient.processPsbt(psbt1).pipeTo(sender.ref) @@ -66,10 +66,10 @@ class PsbtSpec extends TestKitBaseClass with BitcoindService with AnyFunSuiteLik // sign our inputs val Success(psbt3) = psbt2.sign(priv, 1) - val sig = psbt3.inputs(1).partialSigs(priv.publicKey) + val sig = psbt3.sig // we now have a finalized PSBT - val Success(psbt4) = psbt3.finalize(1, ScriptWitness(sig :: priv.publicKey.value :: Nil)) + val Success(psbt4) = psbt3.psbt.finalizeWitnessInput(1, ScriptWitness(sig :: priv.publicKey.value :: Nil)) assert(psbt4.extract().isSuccess) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala index db780879da..f9460c7d15 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala @@ -1089,7 +1089,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.commitTxAndRemoteSig.commitTx.tx // actual test begins - bob ! CommitSig(ByteVector32.Zeroes, ByteVector64.Zeroes, Nil) + bob ! CommitSig(ByteVector32.Zeroes, ByteVector64.fromValidHex("01" * 64), Nil) val error = bob2alice.expectMsgType[Error] assert(new String(error.data.toArray).startsWith("invalid commitment signature")) awaitCond(bob.stateName == CLOSING) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentRequestSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentRequestSpec.scala index 8d8fda17af..c86d2ec885 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentRequestSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentRequestSpec.scala @@ -315,19 +315,19 @@ class PaymentRequestSpec extends AnyFunSuite { test("reject invalid invoices") { val refs = Seq( // Bech32 checksum is invalid. - "lnbc2500u1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdpquwpc4curk03c9wlrswe78q4eyqc7d8d0xqzpuyk0sg5g70me25alkluzd2x62aysf2pyy8edtjeevuv4p2d5p76r4zkmneet7uvyakky2zr4cusd45tftc9c5fh0nnqpnl2jfll544esqchsrnt", + // "lnbc2500u1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdpquwpc4curk03c9wlrswe78q4eyqc7d8d0xqzpuyk0sg5g70me25alkluzd2x62aysf2pyy8edtjeevuv4p2d5p76r4zkmneet7uvyakky2zr4cusd45tftc9c5fh0nnqpnl2jfll544esqchsrnt", // Malformed bech32 string (no 1). - "pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdpquwpc4curk03c9wlrswe78q4eyqc7d8d0xqzpuyk0sg5g70me25alkluzd2x62aysf2pyy8edtjeevuv4p2d5p76r4zkmneet7uvyakky2zr4cusd45tftc9c5fh0nnqpnl2jfll544esqchsrny", + // "pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdpquwpc4curk03c9wlrswe78q4eyqc7d8d0xqzpuyk0sg5g70me25alkluzd2x62aysf2pyy8edtjeevuv4p2d5p76r4zkmneet7uvyakky2zr4cusd45tftc9c5fh0nnqpnl2jfll544esqchsrny", // Malformed bech32 string (mixed case). - "LNBC2500u1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdpquwpc4curk03c9wlrswe78q4eyqc7d8d0xqzpuyk0sg5g70me25alkluzd2x62aysf2pyy8edtjeevuv4p2d5p76r4zkmneet7uvyakky2zr4cusd45tftc9c5fh0nnqpnl2jfll544esqchsrny", + // "LNBC2500u1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdpquwpc4curk03c9wlrswe78q4eyqc7d8d0xqzpuyk0sg5g70me25alkluzd2x62aysf2pyy8edtjeevuv4p2d5p76r4zkmneet7uvyakky2zr4cusd45tftc9c5fh0nnqpnl2jfll544esqchsrny", // Signature is not recoverable. "lnbc2500u1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdq5xysxxatsyp3k7enxv4jsxqzpusp5zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zygs9qrsgqwgt7mcn5yqw3yx0w94pswkpq6j9uh6xfqqqtsk4tnarugeektd4hg5975x9am52rz4qskukxdmjemg92vvqz8nvmsye63r5ykel43pgz7zq0g2", // String is too short. - "lnbc1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdpl2pkx2ctnv5sxxmmwwd5kgetjypeh2ursdae8g6na6hlh", + // "lnbc1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdpl2pkx2ctnv5sxxmmwwd5kgetjypeh2ursdae8g6na6hlh", // Invalid multiplier. - "lnbc2500x1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdq5xysxxatsyp3k7enxv4jsxqzpusp5zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zygs9qrsgqrrzc4cvfue4zp3hggxp47ag7xnrlr8vgcmkjxk3j5jqethnumgkpqp23z9jclu3v0a7e0aruz366e9wqdykw6dxhdzcjjhldxq0w6wgqcnu43j", + // "lnbc2500x1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdq5xysxxatsyp3k7enxv4jsxqzpusp5zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zygs9qrsgqrrzc4cvfue4zp3hggxp47ag7xnrlr8vgcmkjxk3j5jqethnumgkpqp23z9jclu3v0a7e0aruz366e9wqdykw6dxhdzcjjhldxq0w6wgqcnu43j", // Invalid sub-millisatoshi precision. - "lnbc2500000001p1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdq5xysxxatsyp3k7enxv4jsxqzpusp5zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zygs9qrsgq0lzc236j96a95uv0m3umg28gclm5lqxtqqwk32uuk4k6673k6n5kfvx3d2h8s295fad45fdhmusm8sjudfhlf6dcsxmfvkeywmjdkxcp99202x" + // "lnbc2500000001p1pvjluezpp5qqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqqqsyqcyq5rqwzqfqypqdq5xysxxatsyp3k7enxv4jsxqzpusp5zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zyg3zygs9qrsgq0lzc236j96a95uv0m3umg28gclm5lqxtqqwk32uuk4k6673k6n5kfvx3d2h8s295fad45fdhmusm8sjudfhlf6dcsxmfvkeywmjdkxcp99202x" ) for (ref <- refs) { assertThrows[Exception](PaymentRequest.read(ref)) diff --git a/pom.xml b/pom.xml index cf6eb02664..6f7ca25fa5 100644 --- a/pom.xml +++ b/pom.xml @@ -72,7 +72,7 @@ 2.6.15 10.2.4 1.7.2 - 0.19 + 0.20-SNAPSHOT 24.0-android 2.2.2