Skip to content

Commit

Permalink
Use psbt to fund and sign transactions
Browse files Browse the repository at this point in the history
  • Loading branch information
sstone committed Jul 22, 2021
1 parent cafaeed commit f9478c1
Show file tree
Hide file tree
Showing 12 changed files with 304 additions and 55 deletions.
2 changes: 1 addition & 1 deletion eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala
Original file line number Diff line number Diff line change
Expand Up @@ -251,7 +251,7 @@ class Setup(val datadir: File,
routerTimeout = after(FiniteDuration(config.getDuration("router.init-timeout").getSeconds, TimeUnit.SECONDS), using = system.scheduler)(Future.failed(new RuntimeException("Router initialization timed out")))
_ <- Future.firstCompletedOf(routerInitialized.future :: routerTimeout :: Nil)

wallet = new BitcoinCoreWallet(bitcoin)
wallet = new BitcoinCoreWallet(Block.RegtestGenesisBlock.hash, bitcoin)
_ = wallet.getReceiveAddress().map(address => logger.info(s"initial wallet address=$address"))

channelsListener = system.spawn(ChannelsListener(channelsListenerReady), name = "channels-listener")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand All @@ -36,7 +36,7 @@ trait EclairWallet {
def getReceiveAddress(label: String = ""): Future[String]

def getReceivePubkey(receiveAddress: Option[String] = None): Future[PublicKey]

def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: FeeratePerKw): Future[MakeFundingTxResponse]

/**
Expand Down Expand Up @@ -67,4 +67,4 @@ trait EclairWallet {

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)
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,9 @@ package fr.acinq.eclair.blockchain.bitcoind

import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin._
import fr.acinq.eclair.addressToPublicKeyScript
import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient.{FundTransactionOptions, FundTransactionResponse, SignTransactionResponse, toSatoshi}
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient.{FundPsbtOptions, FundPsbtResponse, FundTransactionOptions, FundTransactionResponse, ProcessPsbtResponse, SignTransactionResponse, toSatoshi}
import fr.acinq.eclair.blockchain.bitcoind.rpc.{BitcoinJsonRPCClient, ExtendedBitcoinClient}
import fr.acinq.eclair.blockchain.fee.{FeeratePerKB, FeeratePerKw}
import fr.acinq.eclair.transactions.Transactions
Expand All @@ -34,12 +35,53 @@ import scala.util.{Failure, Success}
/**
* Created by PM on 06/07/2017.
*/
class BitcoinCoreWallet(rpcClient: BitcoinJsonRPCClient)(implicit ec: ExecutionContext) extends EclairWallet with Logging {
class BitcoinCoreWallet(chainHash: ByteVector32, rpcClient: BitcoinJsonRPCClient)(implicit ec: ExecutionContext) extends EclairWallet with Logging {

import BitcoinCoreWallet._

val bitcoinClient = new ExtendedBitcoinClient(rpcClient)

def getMinFeerate(feeRatePerKw: FeeratePerKw): Future[FeeratePerKB] = {
val requestedFeeRatePerKB = FeeratePerKB(feeRatePerKw)
rpcClient.invoke("getmempoolinfo").map(json => json \ "mempoolminfee" match {
case JDecimal(feerate) => FeeratePerKB(Btc(feerate).toSatoshi).max(requestedFeeRatePerKB)
case JInt(feerate) => FeeratePerKB(Btc(feerate.toLong).toSatoshi).max(requestedFeeRatePerKB)
case other =>
logger.warn(s"cannot retrieve mempool minimum fee: $other")
requestedFeeRatePerKB
})
}

def fundTransaction(outputs: Map[String, Satoshi], lockUtxos: Boolean, feeRatePerKw: FeeratePerKw): Future[FundTransactionResponse] = {
val requestedFeeRatePerKB = FeeratePerKB(feeRatePerKw)
rpcClient.invoke("getmempoolinfo").map(json => json \ "mempoolminfee" match {
case JDecimal(feerate) => FeeratePerKB(Btc(feerate).toSatoshi).max(requestedFeeRatePerKB)
case JInt(feerate) => FeeratePerKB(Btc(feerate.toLong).toSatoshi).max(requestedFeeRatePerKB)
case other =>
logger.warn(s"cannot retrieve mempool minimum fee: $other")
requestedFeeRatePerKB
}).flatMap(feeRatePerKB => {
bitcoinClient.fundPsbt(outputs, 0, FundPsbtOptions(FeeratePerKw(feeRatePerKB), lockUtxos = lockUtxos)).map {
response => {
FundTransactionResponse(tx = response.psbt.extract().get, changePosition = response.changePosition, fee = response.fee)
}
}
})
}

def fundPsbt(outputs: Map[String, Satoshi], lockUtxos: Boolean, feeRatePerKw: FeeratePerKw): Future[FundPsbtResponse] = {
val requestedFeeRatePerKB = FeeratePerKB(feeRatePerKw)
rpcClient.invoke("getmempoolinfo").map(json => json \ "mempoolminfee" match {
case JDecimal(feerate) => FeeratePerKB(Btc(feerate).toSatoshi).max(requestedFeeRatePerKB)
case JInt(feerate) => FeeratePerKB(Btc(feerate.toLong).toSatoshi).max(requestedFeeRatePerKB)
case other =>
logger.warn(s"cannot retrieve mempool minimum fee: $other")
requestedFeeRatePerKB
}).flatMap(feeRatePerKB => {
bitcoinClient.fundPsbt(outputs, 0, FundPsbtOptions(FeeratePerKw(feeRatePerKB), lockUtxos = lockUtxos))
})
}

def fundTransaction(tx: Transaction, lockUtxos: Boolean, feerate: FeeratePerKw): Future[FundTransactionResponse] = {
val requestedFeeRatePerKB = FeeratePerKB(feerate)
rpcClient.invoke("getmempoolinfo").map(json => json \ "mempoolminfee" match {
Expand All @@ -54,7 +96,23 @@ class BitcoinCoreWallet(rpcClient: BitcoinJsonRPCClient)(implicit ec: ExecutionC
}

def signTransaction(tx: Transaction): Future[SignTransactionResponse] = {
bitcoinClient.signTransaction(tx, Nil)
bitcoinClient.processPsbt(Psbt(tx)).map {
response => SignTransactionResponse(response.psbt.extract().get, response.complete)
}
}

private def signPsbtOrUnlock(psbt: Psbt): Future[ProcessPsbtResponse] = {
val f = bitcoinClient.processPsbt(psbt).withFilter(_.complete == true)
// if signature fails (e.g. because wallet is encrypted) we need to unlock the utxos
f.recoverWith { case _ =>
bitcoinClient.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
}
}

private def signTransactionOrUnlock(tx: Transaction): Future[SignTransactionResponse] = {
Expand Down Expand Up @@ -122,24 +180,33 @@ class BitcoinCoreWallet(rpcClient: BitcoinJsonRPCClient)(implicit ec: ExecutionC
JString(rawKey) <- rpcClient.invoke("getaddressinfo", address).map(_ \ "pubkey")
} yield PublicKey(ByteVector.fromValidHex(rawKey))


override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feerate: FeeratePerKw): Future[MakeFundingTxResponse] = {
val partialFundingTx = Transaction(
version = 2,
txIn = Seq.empty[TxIn],
txOut = TxOut(amount, pubkeyScript) :: Nil,
lockTime = 0)
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 add inputs to the funding tx, and use the specified change address
fundTxResponse <- fundTransaction(partialFundingTx, lockUtxos = true, feerate)
// we ask bitcoin core to create and fund the funding tx
actualFeeRate <- getMinFeerate(feerate)
FundPsbtResponse(psbt, fee, changePosition) <- bitcoinClient.fundPsbt(Map(fundingAddress -> amount), 0, FundPsbtOptions(FeeratePerKw(actualFeeRate), 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)
}

override def commit(tx: Transaction): Future[Boolean] = bitcoinClient.publishTransaction(tx).transformWith {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@

package fr.acinq.eclair.blockchain.bitcoind.rpc

import fr.acinq.bitcoin.Psbt.toBase64
import fr.acinq.bitcoin._
import fr.acinq.eclair.ShortChannelId.coordinates
import fr.acinq.eclair.TxCoordinates
Expand Down Expand Up @@ -101,6 +102,37 @@ class ExtendedBitcoinClient(val rpcClient: BitcoinJsonRPCClient) extends Logging
})
}

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", 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)
})
}

/**
* @return the public key hash of a bech32 raw change address.
*/
Expand Down Expand Up @@ -306,6 +338,20 @@ object ExtendedBitcoinClient {
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 {
Expand All @@ -321,6 +367,8 @@ object ExtendedBitcoinClient {

case class SignTransactionResponse(tx: Transaction, complete: Boolean)

case class ProcessPsbtResponse(psbt: Psbt, complete: Boolean)

/**
* Information about a transaction currently in the mempool.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -437,7 +437,8 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
})

when(WAIT_FOR_FUNDING_INTERNAL)(handleExceptions {
case Event(MakeFundingTxResponse(fundingTx, fundingTxOutputIndex, fundingTxFee), d@DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, initialRelayFees_opt, remoteFirstPerCommitmentPoint, channelConfig, channelFeatures, open)) =>
case Event(MakeFundingTxResponse(psbt, fundingTxOutputIndex, fundingTxFee), d@DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId, localParams, remoteParams, fundingAmount, pushMsat, initialFeeratePerKw, initialRelayFees_opt, 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)
Expand Down Expand Up @@ -1491,10 +1492,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,13 @@
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.fee.FeeratePerKw
import fr.acinq.eclair.randomKey
import scodec.bits._

import scala.concurrent.Future
import scala.util.Success

/**
* Created by PM on 06/07/2017.
Expand Down Expand Up @@ -52,11 +54,17 @@ class TestWallet extends EclairWallet {
object TestWallet {

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)
}

}
Loading

0 comments on commit f9478c1

Please sign in to comment.