Skip to content

Commit

Permalink
Check tx confirmation proofs
Browse files Browse the repository at this point in the history
When we're notified that a tx has been confirmed, we:
- ask bitcoind for a "txout" proof i.e a proof that the tx was included in a block
- verify this proof
- verify the proof of work of the block in which it was published and its descendants by checking that
the block hash matches the block difficulty and (only on mainnet) that the diffculty is above a given target
  • Loading branch information
sstone committed May 16, 2023
1 parent fa985da commit 3ec4af7
Show file tree
Hide file tree
Showing 4 changed files with 194 additions and 12 deletions.
1 change: 1 addition & 0 deletions eclair-core/src/main/resources/reference.conf
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ eclair {
// - ignore: eclair will leave these utxos locked and start
startup-locked-utxos-behavior = "stop"
final-pubkey-refresh-delay = 3 seconds
min-difficulty = 387294044 // difficulty of block 600000
}

node-alias = "eclair"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import fr.acinq.eclair.{BlockHeight, KamonExt, NodeParams, RealShortChannelId, T
import java.util.concurrent.atomic.AtomicLong
import scala.concurrent.duration._
import scala.concurrent.{ExecutionContext, Future}
import scala.util.{Failure, Success}
import scala.util.{Failure, Success, Try}

/**
* Created by PM on 21/02/2016.
Expand Down Expand Up @@ -415,21 +415,34 @@ private class ZmqWatcher(nodeParams: NodeParams, blockHeight: AtomicLong, client

private def checkConfirmed(w: WatchConfirmed[_ <: WatchConfirmedTriggered]): Future[Unit] = {
log.debug("checking confirmations of txid={}", w.txId)

def checkConfirmationProof(): Future[Unit] = {
client.getTxConfirmationProof(w.txId).map(headerInfos => {
if (nodeParams.chainHash == Block.LivenetGenesisBlock.hash) {
// 0x1715a35cL = 387294044 = difficulty of block 600000
val minDiff = Try(context.system.settings.config.getLong("eclair.bitcoind.min-difficulty")).getOrElse(0x1715a35cL)
require(headerInfos.forall(hi => hi.header.bits < minDiff))
}
})
}

// NB: this is very inefficient since internally we call `getrawtransaction` three times, but it doesn't really
// matter because this only happens once, when the watched transaction has reached min_depth
client.getTxConfirmations(w.txId).flatMap {
case Some(confirmations) if confirmations >= w.minDepth =>
client.getTransaction(w.txId).flatMap { tx =>
client.getTransactionShortId(w.txId).map {
case (height, index) => w match {
case w: WatchFundingConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchFundingConfirmedTriggered(height, index, tx))
case w: WatchFundingDeeplyBuried => context.self ! TriggerEvent(w.replyTo, w, WatchFundingDeeplyBuriedTriggered(height, index, tx))
case w: WatchTxConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchTxConfirmedTriggered(height, index, tx))
case w: WatchParentTxConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchParentTxConfirmedTriggered(height, index, tx))
case w: WatchAlternativeCommitTxConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchAlternativeCommitTxConfirmedTriggered(height, index, tx))
checkConfirmationProof().andThen(_ =>
client.getTransaction(w.txId).flatMap { tx =>
client.getTransactionShortId(w.txId).map {
case (height, index) => w match {
case w: WatchFundingConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchFundingConfirmedTriggered(height, index, tx))
case w: WatchFundingDeeplyBuried => context.self ! TriggerEvent(w.replyTo, w, WatchFundingDeeplyBuriedTriggered(height, index, tx))
case w: WatchTxConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchTxConfirmedTriggered(height, index, tx))
case w: WatchParentTxConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchParentTxConfirmedTriggered(height, index, tx))
case w: WatchAlternativeCommitTxConfirmed => context.self ! TriggerEvent(w.replyTo, w, WatchAlternativeCommitTxConfirmedTriggered(height, index, tx))
}
}
}
}
)
case _ => Future.successful((): Unit)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ package fr.acinq.eclair.blockchain.bitcoind.rpc

import fr.acinq.bitcoin.scalacompat.Crypto.PublicKey
import fr.acinq.bitcoin.scalacompat._
import fr.acinq.bitcoin.{Bech32, Block}
import fr.acinq.bitcoin.{Bech32, Block, BlockHeader}
import fr.acinq.eclair.ShortChannelId.coordinates
import fr.acinq.eclair.blockchain.OnChainWallet
import fr.acinq.eclair.blockchain.OnChainWallet.{FundTransactionResponse, MakeFundingTxResponse, OnChainBalance, SignTransactionResponse}
Expand Down Expand Up @@ -74,6 +74,70 @@ class BitcoinCoreClient(val rpcClient: BitcoinJsonRPCClient) extends OnChainWall
case t: JsonRPCError if t.error.code == -5 => None // Invalid or non-wallet transaction id (code: -5)
}

/**
*
* @param txid transaction id
* @return a list of block header information, starting from the block in which the transaction was published, up to the current tip
*/
def getTxConfirmationProof(txid: ByteVector32)(implicit ec: ExecutionContext): Future[List[BlockHeaderInfo]] = {
import KotlinUtils._

/**
* Scala wrapper for Block.verifyTxOutProof
*
* @param proof tx output proof, as provided by bitcoind
* @return a (Header, List(txhash, position)) tuple. Header is the header of the block used to compute the input proof, and
* (txhash, position) is a list of transaction ids that were verified, and their position in the block
*/
def verifyTxOutProof(proof: ByteVector): (BlockHeader, List[(ByteVector32, Int)]) = {
val check = Block.verifyTxOutProof(proof.toArray)
(check.getFirst, check.getSecond.asScala.toList.map(p => (kmp2scala(p.getFirst), p.getSecond.intValue())))
}

for {
confirmations_opt <- getTxConfirmations(txid)
if (confirmations_opt.isDefined && confirmations_opt.get > 0)
// get the merkle proof for our txid
proof <- getTxOutProof(txid)
// verify this merkle proof. if valid, we get the header for the block the tx was published in, and the tx hashes
// that can be used to rebuild the block's merkle root
(header, txHashesAndPos) = verifyTxOutProof(proof)
// inclusionData contains a header and a list of (txid, position) that can be used to re-build the header's merkle root
// check that the block hash included in the proof matches the block in which the tx was published
Some(blockHash) <- getTxBlockHash(txid)
_ = require(header.blockId.contentEquals(blockHash.toArray), "confirmation proof is not valid (block id mismatch)")
// check that our txid is included in the merkle root of the block it was published in
txids = txHashesAndPos.map { case (txhash, _) => txhash.reverse }
_ = require(txids.contains(txid), "confirmation proof is not valid (txid not found)")
// get the block in which our tx was confirmed and all following blocks
headerInfos <- getBlockInfos(blockHash, confirmations_opt.get)
_ = require(headerInfos.head.header.blockId.contentEquals(blockHash.toArray), "block header id mismatch")
} yield headerInfos
}

def getTxOutProof(txid: ByteVector32)(implicit ec: ExecutionContext): Future[ByteVector] =
rpcClient.invoke("gettxoutproof", Array(txid)).collect { case JString(raw) => ByteVector.fromValidHex(raw) }

// returns a chain a blocks of a given size starting at `blockId`
def getBlockInfos(blockId: ByteVector32, count: Int)(implicit ec: ExecutionContext): Future[List[BlockHeaderInfo]] = {
import KotlinUtils._

def loop(blocks: List[BlockHeaderInfo]): Future[List[BlockHeaderInfo]] = if (blocks.size == count) Future.successful(blocks) else {
getBlockHeaderInfo(blocks.last.nextBlockHash.get.reverse).flatMap(info => loop(blocks :+ info))
}

getBlockHeaderInfo(blockId).flatMap(info => loop(List(info))).map(blocks => {
for (i <- 0 until blocks.size - 1) {
require(BlockHeader.checkProofOfWork(blocks(i).header))
require(blocks(i).height == blocks(0).height + i)
require(blocks(i).confirmation == blocks(0).confirmation - i)
require(blocks(i).nextBlockHash.contains(kmp2scala(blocks(i + 1).header.hash)))
require(blocks(i + 1).header.hashPreviousBlock == blocks(i).header.hash)
}
blocks
})
}

/** Get the hash of the block containing a given transaction. */
private def getTxBlockHash(txid: ByteVector32)(implicit ec: ExecutionContext): Future[Option[ByteVector32]] =
rpcClient.invoke("getrawtransaction", txid, 1 /* verbose output is needed to get the block hash */)
Expand Down Expand Up @@ -207,6 +271,32 @@ class BitcoinCoreClient(val rpcClient: BitcoinJsonRPCClient) extends OnChainWall
case _ => Nil
}

//------------------------- BLOCKS -------------------------//
def getBlockHash(height: Int)(implicit ec: ExecutionContext): Future[ByteVector32] = {
rpcClient.invoke("getblockhash", height).map(json => {
val JString(hash) = json
ByteVector32.fromValidHex(hash)
})
}

def getBlockHeaderInfo(blockId: ByteVector32)(implicit ec: ExecutionContext): Future[BlockHeaderInfo] = {
import fr.acinq.bitcoin.{ByteVector32 => ByteVector32Kt}
rpcClient.invoke("getblockheader", blockId.toString()).map(json => {
val JInt(confirmations) = json \ "confirmations"
val JInt(height) = json \ "height"
val JInt(time) = json \ "time"
val JInt(version) = json \ "version"
val JInt(nonce) = json \ "nonce"
val JString(bits) = json \ "bits"
val merkleRoot = ByteVector32Kt.fromValidHex((json \ "merkleroot").extract[String]).reversed()
val previousblockhash = ByteVector32Kt.fromValidHex((json \ "previousblockhash").extract[String]).reversed()
val nextblockhash = (json \ "nextblockhash").extractOpt[String].map(h => ByteVector32.fromValidHex(h).reverse)
val header = new BlockHeader(version.longValue, previousblockhash, merkleRoot, time.longValue, java.lang.Long.parseLong(bits, 16), nonce.longValue)
require(header.blockId == KotlinUtils.scala2kmp(blockId))
BlockHeaderInfo(header, confirmations.toLong, height.toLong, nextblockhash)
})
}

//------------------------- FUNDING -------------------------//

def fundTransaction(tx: Transaction, options: FundTransactionOptions)(implicit ec: ExecutionContext): Future[FundTransactionResponse] = {
Expand Down Expand Up @@ -623,6 +713,10 @@ object BitcoinCoreClient {

case class Utxo(txid: ByteVector32, amount: MilliBtc, confirmations: Long, safe: Boolean, label_opt: Option[String])

case class TransactionInfo(tx: Transaction, confirmations: Int, blockId: Option[ByteVector32])

case class BlockHeaderInfo(header: BlockHeader, confirmation: Long, height: Long, nextBlockHash: Option[ByteVector32])

def toSatoshi(btcAmount: BigDecimal): Satoshi = Satoshi(btcAmount.bigDecimal.scaleByPowerOfTen(8).longValue)

}
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import fr.acinq.eclair.blockchain.WatcherSpec.{createSpendManyP2WPKH, createSpen
import fr.acinq.eclair.blockchain.bitcoind.BitcoindService.BitcoinReq
import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinCoreClient._
import fr.acinq.eclair.blockchain.bitcoind.rpc.BitcoinJsonRPCAuthMethod.UserPassword
import fr.acinq.eclair.blockchain.bitcoind.rpc.{BasicBitcoinJsonRPCClient, BitcoinCoreClient, JsonRPCError}
import fr.acinq.eclair.blockchain.bitcoind.rpc.{BasicBitcoinJsonRPCClient, BitcoinCoreClient, BitcoinJsonRPCClient, JsonRPCError}
import fr.acinq.eclair.blockchain.fee.{FeeratePerByte, FeeratePerKw}
import fr.acinq.eclair.transactions.{Scripts, Transactions}
import fr.acinq.eclair.{BlockHeight, TestConstants, TestKitBaseClass, addressToPublicKeyScript, randomBytes32, randomKey}
Expand Down Expand Up @@ -1321,4 +1321,78 @@ class BitcoinCoreClientSpec extends TestKitBaseClass with BitcoindService with A
assert(sender.expectMsgType[Transaction].txid == tx.txid)
}

test("get block header info") {
import fr.acinq.bitcoin.scalacompat.KotlinUtils._
val sender = TestProbe()
val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient)
bitcoinClient.getBlockHeight().pipeTo(sender.ref)
val height = sender.expectMsgType[BlockHeight]
bitcoinClient.getBlockHash(height.toInt).pipeTo(sender.ref)
val lastBlockId = sender.expectMsgType[ByteVector32]
bitcoinClient.getBlockHeaderInfo(lastBlockId).pipeTo(sender.ref)
val lastBlockInfo = sender.expectMsgType[BlockHeaderInfo]
assert(lastBlockInfo.nextBlockHash.isEmpty)

bitcoinClient.getBlockHash(height.toInt - 1).pipeTo(sender.ref)
val blockId = sender.expectMsgType[ByteVector32]
bitcoinClient.getBlockHeaderInfo(blockId).pipeTo(sender.ref)
val blockInfo = sender.expectMsgType[BlockHeaderInfo]
assert(lastBlockInfo.header.hashPreviousBlock == blockInfo.header.hash)
assert(blockInfo.nextBlockHash.contains(kmp2scala(lastBlockInfo.header.hash)))
}

test("get chains of block headers") {
import fr.acinq.bitcoin.scalacompat.KotlinUtils._
val sender = TestProbe()
val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient)

bitcoinClient.getBlockHash(140).pipeTo(sender.ref)
val blockId = sender.expectMsgType[ByteVector32]
bitcoinClient.getBlockInfos(blockId, 5).pipeTo(sender.ref)
val blockInfos = sender.expectMsgType[List[BlockHeaderInfo]]
for (i <- 0 until blockInfos.size - 1) {
require(blockInfos(i).nextBlockHash.contains(kmp2scala(blockInfos(i + 1).header.hash)))
require(blockInfos(i + 1).header.hashPreviousBlock == blockInfos(i).header.hash)
}
}

test("verify tx publication proofs") {
val sender = TestProbe()
val bitcoinClient = new BitcoinCoreClient(bitcoinrpcclient)
val address = getNewAddress(sender)

// we create a dummy confirmed tx, we'll use its txout proof later
val dummyTx = sendToAddress(address, 5 btc, sender)

val tx = sendToAddress(address, 5 btc, sender)
// transaction is not confirmed yet
bitcoinClient.getTxConfirmations(tx.txid).pipeTo(sender.ref)
sender.expectMsg(Some(0))

// let's confirm our transaction.
generateBlocks(6)
bitcoinClient.getTxConfirmations(tx.txid).pipeTo(sender.ref)
sender.expectMsg(Some(6))

bitcoinClient.getTxOutProof(tx.txid).pipeTo(sender.ref)
val proof = sender.expectMsgType[ByteVector]
val check = fr.acinq.bitcoin.Block.verifyTxOutProof(proof.toArray)
val header = check.getFirst
bitcoinClient.getTxConfirmationProof(tx.txid).pipeTo(sender.ref)
val headerInfos = sender.expectMsgType[List[BlockHeaderInfo]]
assert(header == headerInfos.head.header)

// try again with a bitcoin client that returns a proof that is not valid for our tx but from the same block where it was confirmed
bitcoinClient.getTxOutProof(dummyTx.txid).pipeTo(sender.ref)
val dumyProof = sender.expectMsgType[ByteVector]
val evilBitcoinClient = new BitcoinCoreClient(new BitcoinJsonRPCClient {
override def invoke(method: String, params: Any*)(implicit ec: ExecutionContext): Future[JValue] = method match {
case "gettxoutproof" => Future.successful(JString(dumyProof.toHex))
case _ => bitcoinrpcclient.invoke(method, params: _*)(ec)
}
})
evilBitcoinClient.getTxConfirmationProof(tx.txid).pipeTo(sender.ref)
val error = sender.expectMsgType[Failure]
assert(error.cause.getMessage.contains("txid not found"))
}
}

0 comments on commit 3ec4af7

Please sign in to comment.