state: track UTXO provenance

This commit changes the state system and database format to track the
provenance of UTXOs, in addition to the outputs themselves.
Specifically, it tracks the following additional metadata:

- the height at which the UTXO was created;
- whether or not the UTXO was created from a coinbase transaction or
  not.

This metadata will allow us to:

- check the coinbase maturity consensus rule;
- check the coinbase inputs => no transparent outputs rule;
- implement lookup of transactions by utxo (using the height to find the
  block and then scanning the block) for a future RPC mechanism.

Closes #1342
This commit is contained in:
Henry de Valence 2020-11-23 12:02:57 -08:00
parent d2173cae7e
commit 342eb166ff
19 changed files with 210 additions and 140 deletions

1
Cargo.lock generated
View File

@ -3263,6 +3263,7 @@ dependencies = [
"once_cell", "once_cell",
"primitive-types", "primitive-types",
"proptest", "proptest",
"proptest-derive",
"rocksdb", "rocksdb",
"serde", "serde",
"spandoc", "spandoc",

View File

@ -167,7 +167,7 @@ where
let mut async_checks = FuturesUnordered::new(); let mut async_checks = FuturesUnordered::new();
let known_utxos = known_utxos(&block); let known_utxos = new_outputs(&block);
for transaction in &block.transactions { for transaction in &block.transactions {
let rsp = transaction_verifier let rsp = transaction_verifier
.ready_and() .ready_and()
@ -221,15 +221,24 @@ where
} }
} }
fn known_utxos(block: &Block) -> Arc<HashMap<transparent::OutPoint, transparent::Output>> { fn new_outputs(block: &Block) -> Arc<HashMap<transparent::OutPoint, zs::Utxo>> {
let mut map = HashMap::default(); let mut new_outputs = HashMap::default();
let height = block.coinbase_height().expect("block has coinbase height");
for transaction in &block.transactions { for transaction in &block.transactions {
let hash = transaction.hash(); let hash = transaction.hash();
let from_coinbase = transaction.is_coinbase();
for (index, output) in transaction.outputs().iter().cloned().enumerate() { for (index, output) in transaction.outputs().iter().cloned().enumerate() {
let index = index as u32; let index = index as u32;
map.insert(transparent::OutPoint { hash, index }, output); new_outputs.insert(
transparent::OutPoint { hash, index },
zs::Utxo {
output,
height,
from_coinbase,
},
);
} }
} }
Arc::new(map) Arc::new(new_outputs)
} }

View File

@ -3,6 +3,7 @@ use std::{collections::HashMap, future::Future, pin::Pin, sync::Arc};
use tracing::Instrument; use tracing::Instrument;
use zebra_chain::{parameters::ConsensusBranchId, transaction::Transaction, transparent}; use zebra_chain::{parameters::ConsensusBranchId, transaction::Transaction, transparent};
use zebra_state::Utxo;
use crate::BoxError; use crate::BoxError;
@ -40,7 +41,7 @@ impl<ZS> Verifier<ZS> {
pub struct Request { pub struct Request {
pub transaction: Arc<Transaction>, pub transaction: Arc<Transaction>,
pub input_index: usize, pub input_index: usize,
pub known_utxos: Arc<HashMap<transparent::OutPoint, transparent::Output>>, pub known_utxos: Arc<HashMap<transparent::OutPoint, Utxo>>,
} }
impl<ZS> tower::Service<Request> for Verifier<ZS> impl<ZS> tower::Service<Request> for Verifier<ZS>
@ -81,17 +82,21 @@ where
async move { async move {
tracing::trace!("awaiting outpoint lookup"); tracing::trace!("awaiting outpoint lookup");
let output = if let Some(output) = known_utxos.get(&outpoint) { let utxo = if let Some(output) = known_utxos.get(&outpoint) {
tracing::trace!("UXTO in known_utxos, discarding query"); tracing::trace!("UXTO in known_utxos, discarding query");
output.clone() output.clone()
} else if let zebra_state::Response::Utxo(output) = query.await? { } else if let zebra_state::Response::Utxo(utxo) = query.await? {
output utxo
} else { } else {
unreachable!("AwaitUtxo always responds with Utxo") unreachable!("AwaitUtxo always responds with Utxo")
}; };
tracing::trace!(?output, "got UTXO"); tracing::trace!(?utxo, "got UTXO");
zebra_script::is_valid(transaction, branch_id, (input_index as u32, output))?; zebra_script::is_valid(
transaction,
branch_id,
(input_index as u32, utxo.output),
)?;
tracing::trace!("script verification succeeded"); tracing::trace!("script verification succeeded");
Ok(()) Ok(())

View File

@ -56,13 +56,13 @@ pub enum Request {
Block { Block {
transaction: Arc<Transaction>, transaction: Arc<Transaction>,
/// Additional UTXOs which are known at the time of verification. /// Additional UTXOs which are known at the time of verification.
known_utxos: Arc<HashMap<transparent::OutPoint, transparent::Output>>, known_utxos: Arc<HashMap<transparent::OutPoint, zs::Utxo>>,
}, },
/// Verify the supplied transaction as part of the mempool. /// Verify the supplied transaction as part of the mempool.
Mempool { Mempool {
transaction: Arc<Transaction>, transaction: Arc<Transaction>,
/// Additional UTXOs which are known at the time of verification. /// Additional UTXOs which are known at the time of verification.
known_utxos: Arc<HashMap<transparent::OutPoint, transparent::Output>>, known_utxos: Arc<HashMap<transparent::OutPoint, zs::Utxo>>,
}, },
} }

View File

@ -36,4 +36,5 @@ spandoc = "0.2"
tempdir = "0.3.7" tempdir = "0.3.7"
tokio = { version = "0.3", features = ["full"] } tokio = { version = "0.3", features = ["full"] }
proptest = "0.10.1" proptest = "0.10.1"
proptest-derive = "0.2"
primitive-types = "0.7.3" primitive-types = "0.7.3"

View File

@ -11,4 +11,5 @@ pub const MIN_TRASPARENT_COINBASE_MATURITY: u32 = 100;
/// coinbase transactions. /// coinbase transactions.
pub const MAX_BLOCK_REORG_HEIGHT: u32 = MIN_TRASPARENT_COINBASE_MATURITY - 1; pub const MAX_BLOCK_REORG_HEIGHT: u32 = MIN_TRASPARENT_COINBASE_MATURITY - 1;
pub const DATABASE_FORMAT_VERSION: u32 = 2; /// The database format version, incremented each time the database format changes.
pub const DATABASE_FORMAT_VERSION: u32 = 3;

View File

@ -15,6 +15,7 @@ mod request;
mod response; mod response;
mod service; mod service;
mod util; mod util;
mod utxo;
// TODO: move these to integration tests. // TODO: move these to integration tests.
#[cfg(test)] #[cfg(test)]
@ -26,3 +27,4 @@ pub use error::{BoxError, CloneError, CommitBlockError, ValidateContextError};
pub use request::{FinalizedBlock, HashOrHeight, PreparedBlock, Request}; pub use request::{FinalizedBlock, HashOrHeight, PreparedBlock, Request};
pub use response::Response; pub use response::Response;
pub use service::init; pub use service::init;
pub use utxo::Utxo;

View File

@ -5,6 +5,8 @@ use zebra_chain::{
transaction, transparent, transaction, transparent,
}; };
use crate::Utxo;
// Allow *only* this unused import, so that rustdoc link resolution // Allow *only* this unused import, so that rustdoc link resolution
// will work with inline links. // will work with inline links.
#[allow(unused_imports)] #[allow(unused_imports)]
@ -71,7 +73,7 @@ pub struct PreparedBlock {
/// Note: although these transparent outputs are newly created, they may not /// Note: although these transparent outputs are newly created, they may not
/// be unspent, since a later transaction in a block can spend outputs of an /// be unspent, since a later transaction in a block can spend outputs of an
/// earlier transaction. /// earlier transaction.
pub new_outputs: HashMap<transparent::OutPoint, transparent::Output>, pub new_outputs: HashMap<transparent::OutPoint, Utxo>,
// TODO: add these parameters when we can compute anchors. // TODO: add these parameters when we can compute anchors.
// sprout_anchor: sprout::tree::Root, // sprout_anchor: sprout::tree::Root,
// sapling_anchor: sapling::tree::Root, // sapling_anchor: sapling::tree::Root,
@ -88,6 +90,13 @@ pub struct FinalizedBlock {
pub(crate) block: Arc<Block>, pub(crate) block: Arc<Block>,
pub(crate) hash: block::Hash, pub(crate) hash: block::Hash,
pub(crate) height: block::Height, pub(crate) height: block::Height,
/// New transparent outputs created in this block, indexed by
/// [`Outpoint`](transparent::Outpoint).
///
/// Note: although these transparent outputs are newly created, they may not
/// be unspent, since a later transaction in a block can spend outputs of an
/// earlier transaction.
pub(crate) new_outputs: HashMap<transparent::OutPoint, Utxo>,
} }
// Doing precomputation in this From impl means that it will be done in // Doing precomputation in this From impl means that it will be done in
@ -100,10 +109,45 @@ impl From<Arc<Block>> for FinalizedBlock {
.expect("finalized blocks must have a valid coinbase height"); .expect("finalized blocks must have a valid coinbase height");
let hash = block.hash(); let hash = block.hash();
let mut new_outputs = HashMap::default();
for transaction in &block.transactions {
let hash = transaction.hash();
let from_coinbase = transaction.is_coinbase();
for (index, output) in transaction.outputs().iter().cloned().enumerate() {
let index = index as u32;
new_outputs.insert(
transparent::OutPoint { hash, index },
Utxo {
output,
height,
from_coinbase,
},
);
}
}
Self { Self {
block, block,
height, height,
hash, hash,
new_outputs,
}
}
}
impl From<PreparedBlock> for FinalizedBlock {
fn from(prepared: PreparedBlock) -> Self {
let PreparedBlock {
block,
height,
hash,
new_outputs,
} = prepared;
Self {
block,
height,
hash,
new_outputs,
} }
} }
} }

View File

@ -2,9 +2,10 @@ use std::sync::Arc;
use zebra_chain::{ use zebra_chain::{
block::{self, Block}, block::{self, Block},
transaction::Transaction, transaction::Transaction,
transparent,
}; };
use crate::Utxo;
// Allow *only* this unused import, so that rustdoc link resolution // Allow *only* this unused import, so that rustdoc link resolution
// will work with inline links. // will work with inline links.
#[allow(unused_imports)] #[allow(unused_imports)]
@ -33,5 +34,5 @@ pub enum Response {
Block(Option<Arc<Block>>), Block(Option<Arc<Block>>),
/// The response to a `AwaitUtxo` request /// The response to a `AwaitUtxo` request
Utxo(transparent::Output), Utxo(Utxo),
} }

View File

@ -21,7 +21,7 @@ use zebra_chain::{
use crate::{ use crate::{
request::HashOrHeight, BoxError, CommitBlockError, Config, FinalizedBlock, PreparedBlock, request::HashOrHeight, BoxError, CommitBlockError, Config, FinalizedBlock, PreparedBlock,
Request, Response, ValidateContextError, Request, Response, Utxo, ValidateContextError,
}; };
use self::finalized_state::FinalizedState; use self::finalized_state::FinalizedState;
@ -251,12 +251,12 @@ impl StateService {
.or_else(|| self.disk.height(hash)) .or_else(|| self.disk.height(hash))
} }
/// Return the utxo pointed to by `outpoint` if it exists in any chain. /// Return the [`Utxo`] pointed to by `outpoint` if it exists in any chain.
pub fn utxo(&self, outpoint: &transparent::OutPoint) -> Option<transparent::Output> { pub fn utxo(&self, outpoint: &transparent::OutPoint) -> Option<Utxo> {
self.mem self.mem
.utxo(outpoint) .utxo(outpoint)
.or_else(|| self.disk.utxo(outpoint))
.or_else(|| self.queued_blocks.utxo(outpoint)) .or_else(|| self.queued_blocks.utxo(outpoint))
.or_else(|| self.disk.utxo(outpoint))
} }
/// Return an iterator over the relevant chain of the block identified by /// Return an iterator over the relevant chain of the block identified by
@ -404,7 +404,7 @@ impl Service<Request> for StateService {
let (rsp_tx, rsp_rx) = oneshot::channel(); let (rsp_tx, rsp_rx) = oneshot::channel();
self.pending_utxos.scan_block(&finalized.block); self.pending_utxos.check_against(&finalized.new_outputs);
self.disk.queue_and_commit_finalized((finalized, rsp_tx)); self.disk.queue_and_commit_finalized((finalized, rsp_tx));
async move { async move {

View File

@ -11,7 +11,7 @@ use zebra_chain::{
transaction::{self, Transaction}, transaction::{self, Transaction},
}; };
use crate::{BoxError, Config, FinalizedBlock, HashOrHeight}; use crate::{BoxError, Config, FinalizedBlock, HashOrHeight, Utxo};
use self::disk_format::{DiskDeserialize, DiskSerialize, FromDisk, IntoDisk, TransactionLocation}; use self::disk_format::{DiskDeserialize, DiskSerialize, FromDisk, IntoDisk, TransactionLocation};
@ -149,6 +149,7 @@ impl FinalizedState {
block, block,
hash, hash,
height, height,
new_outputs,
} = finalized; } = finalized;
let hash_by_height = self.db.cf_handle("hash_by_height").unwrap(); let hash_by_height = self.db.cf_handle("hash_by_height").unwrap();
@ -205,7 +206,13 @@ impl FinalizedState {
return batch; return batch;
} }
// Index each transaction // Index all new transparent outputs
for (outpoint, utxo) in new_outputs.into_iter() {
batch.zs_insert(utxo_by_outpoint, outpoint, utxo);
}
// Index each transaction, spent inputs, nullifiers
// TODO: move computation into FinalizedBlock as with transparent outputs
for (transaction_index, transaction) in block.transactions.iter().enumerate() { for (transaction_index, transaction) in block.transactions.iter().enumerate() {
let transaction_hash = transaction.hash(); let transaction_hash = transaction.hash();
let transaction_location = TransactionLocation { let transaction_location = TransactionLocation {
@ -228,15 +235,6 @@ impl FinalizedState {
} }
} }
// Index all new transparent outputs
for (index, output) in transaction.outputs().iter().enumerate() {
let outpoint = transparent::OutPoint {
hash: transaction_hash,
index: index as _,
};
batch.zs_insert(utxo_by_outpoint, outpoint, output);
}
// Mark sprout and sapling nullifiers as spent // Mark sprout and sapling nullifiers as spent
for sprout_nullifier in transaction.sprout_nullifiers() { for sprout_nullifier in transaction.sprout_nullifiers() {
batch.zs_insert(sprout_nullifiers, sprout_nullifier, ()); batch.zs_insert(sprout_nullifiers, sprout_nullifier, ());
@ -305,7 +303,7 @@ impl FinalizedState {
/// Returns the `transparent::Output` pointed to by the given /// Returns the `transparent::Output` pointed to by the given
/// `transparent::OutPoint` if it is present. /// `transparent::OutPoint` if it is present.
pub fn utxo(&self, outpoint: &transparent::OutPoint) -> Option<transparent::Output> { pub fn utxo(&self, outpoint: &transparent::OutPoint) -> Option<Utxo> {
let utxo_by_outpoint = self.db.cf_handle("utxo_by_outpoint").unwrap(); let utxo_by_outpoint = self.db.cf_handle("utxo_by_outpoint").unwrap();
self.db.zs_get(utxo_by_outpoint, outpoint) self.db.zs_get(utxo_by_outpoint, outpoint)
} }

View File

@ -5,10 +5,12 @@ use zebra_chain::{
block, block,
block::Block, block::Block,
sapling, sapling,
serialization::{ZcashDeserialize, ZcashSerialize}, serialization::{ZcashDeserialize, ZcashDeserializeInto, ZcashSerialize},
sprout, transaction, transparent, sprout, transaction, transparent,
}; };
use crate::Utxo;
#[derive(Debug, Clone, Copy, PartialEq)] #[derive(Debug, Clone, Copy, PartialEq)]
pub struct TransactionLocation { pub struct TransactionLocation {
pub height: block::Height, pub height: block::Height,
@ -184,19 +186,33 @@ impl FromDisk for block::Height {
} }
} }
impl IntoDisk for transparent::Output { impl IntoDisk for Utxo {
type Bytes = Vec<u8>; type Bytes = Vec<u8>;
fn as_bytes(&self) -> Self::Bytes { fn as_bytes(&self) -> Self::Bytes {
self.zcash_serialize_to_vec() let mut bytes = vec![0; 5];
.expect("serialization to vec doesn't fail") bytes[0..4].copy_from_slice(&self.height.0.to_be_bytes());
bytes[4] = self.from_coinbase as u8;
self.output
.zcash_serialize(&mut bytes)
.expect("serialization to vec doesn't fail");
bytes
} }
} }
impl FromDisk for transparent::Output { impl FromDisk for Utxo {
fn from_bytes(bytes: impl AsRef<[u8]>) -> Self { fn from_bytes(bytes: impl AsRef<[u8]>) -> Self {
Self::zcash_deserialize(bytes.as_ref()) let (meta_bytes, output_bytes) = bytes.as_ref().split_at(5);
.expect("deserialization format should match the serialization format used by IntoDisk") let height = block::Height(u32::from_be_bytes(meta_bytes[0..4].try_into().unwrap()));
let from_coinbase = meta_bytes[4] == 1u8;
let output = output_bytes
.zcash_deserialize_into()
.expect("db has serialized data");
Self {
output,
height,
from_coinbase,
}
} }
} }
@ -370,6 +386,6 @@ mod tests {
fn roundtrip_transparent_output() { fn roundtrip_transparent_output() {
zebra_test::init(); zebra_test::init();
proptest!(|(val in any::<transparent::Output>())| assert_value_properties(val)); proptest!(|(val in any::<Utxo>())| assert_value_properties(val));
} }
} }

View File

@ -15,7 +15,7 @@ use zebra_chain::{
transparent, transparent,
}; };
use crate::{FinalizedBlock, HashOrHeight, PreparedBlock}; use crate::{FinalizedBlock, HashOrHeight, PreparedBlock, Utxo};
use self::chain::Chain; use self::chain::Chain;
@ -64,12 +64,7 @@ impl NonFinalizedState {
self.update_metrics_for_chains(); self.update_metrics_for_chains();
// Construct a finalized block. finalizing.into()
FinalizedBlock {
block: finalizing.block,
hash: finalizing.hash,
height: finalizing.height,
}
} }
/// Commit block to the non-finalized state. /// Commit block to the non-finalized state.
@ -148,7 +143,7 @@ impl NonFinalizedState {
/// Returns the `transparent::Output` pointed to by the given /// Returns the `transparent::Output` pointed to by the given
/// `transparent::OutPoint` if it is present. /// `transparent::OutPoint` if it is present.
pub fn utxo(&self, outpoint: &transparent::OutPoint) -> Option<transparent::Output> { pub fn utxo(&self, outpoint: &transparent::OutPoint) -> Option<Utxo> {
for chain in self.chain_set.iter().rev() { for chain in self.chain_set.iter().rev() {
if let Some(output) = chain.created_utxos.get(outpoint) { if let Some(output) = chain.created_utxos.get(outpoint) {
return Some(output.clone()); return Some(output.clone());

View File

@ -10,7 +10,7 @@ use zebra_chain::{
work::difficulty::PartialCumulativeWork, work::difficulty::PartialCumulativeWork,
}; };
use crate::PreparedBlock; use crate::{PreparedBlock, Utxo};
#[derive(Default, Clone)] #[derive(Default, Clone)]
pub struct Chain { pub struct Chain {
@ -18,7 +18,7 @@ pub struct Chain {
pub height_by_hash: HashMap<block::Hash, block::Height>, pub height_by_hash: HashMap<block::Hash, block::Height>,
pub tx_by_hash: HashMap<transaction::Hash, (block::Height, usize)>, pub tx_by_hash: HashMap<transaction::Hash, (block::Height, usize)>,
pub created_utxos: HashMap<transparent::OutPoint, transparent::Output>, pub created_utxos: HashMap<transparent::OutPoint, Utxo>,
spent_utxos: HashSet<transparent::OutPoint>, spent_utxos: HashSet<transparent::OutPoint>,
sprout_anchors: HashSet<sprout::tree::Root>, sprout_anchors: HashSet<sprout::tree::Root>,
sapling_anchors: HashSet<sapling::tree::Root>, sapling_anchors: HashSet<sapling::tree::Root>,
@ -155,14 +155,13 @@ impl UpdateWith<PreparedBlock> for Chain {
// for each transaction in block // for each transaction in block
for (transaction_index, transaction) in block.transactions.iter().enumerate() { for (transaction_index, transaction) in block.transactions.iter().enumerate() {
let (inputs, outputs, shielded_data, joinsplit_data) = match transaction.deref() { let (inputs, shielded_data, joinsplit_data) = match transaction.deref() {
transaction::Transaction::V4 { transaction::Transaction::V4 {
inputs, inputs,
outputs,
shielded_data, shielded_data,
joinsplit_data, joinsplit_data,
.. ..
} => (inputs, outputs, shielded_data, joinsplit_data), } => (inputs, shielded_data, joinsplit_data),
_ => unreachable!( _ => unreachable!(
"older transaction versions only exist in finalized blocks pre sapling", "older transaction versions only exist in finalized blocks pre sapling",
), ),
@ -179,7 +178,7 @@ impl UpdateWith<PreparedBlock> for Chain {
); );
// add the utxos this produced // add the utxos this produced
self.update_chain_state_with(&(transaction_hash, outputs)); self.update_chain_state_with(&prepared.new_outputs);
// add the utxos this consumed // add the utxos this consumed
self.update_chain_state_with(inputs); self.update_chain_state_with(inputs);
// add sprout anchor and nullifiers // add sprout anchor and nullifiers
@ -209,14 +208,13 @@ impl UpdateWith<PreparedBlock> for Chain {
// for each transaction in block // for each transaction in block
for transaction in &block.transactions { for transaction in &block.transactions {
let (inputs, outputs, shielded_data, joinsplit_data) = match transaction.deref() { let (inputs, shielded_data, joinsplit_data) = match transaction.deref() {
transaction::Transaction::V4 { transaction::Transaction::V4 {
inputs, inputs,
outputs,
shielded_data, shielded_data,
joinsplit_data, joinsplit_data,
.. ..
} => (inputs, outputs, shielded_data, joinsplit_data), } => (inputs, shielded_data, joinsplit_data),
_ => unreachable!( _ => unreachable!(
"older transaction versions only exist in finalized blocks pre sapling", "older transaction versions only exist in finalized blocks pre sapling",
), ),
@ -230,7 +228,7 @@ impl UpdateWith<PreparedBlock> for Chain {
); );
// remove the utxos this produced // remove the utxos this produced
self.revert_chain_state_with(&(transaction_hash, outputs)); self.revert_chain_state_with(&prepared.new_outputs);
// remove the utxos this consumed // remove the utxos this consumed
self.revert_chain_state_with(inputs); self.revert_chain_state_with(inputs);
// remove sprout anchor and nullifiers // remove sprout anchor and nullifiers
@ -241,37 +239,15 @@ impl UpdateWith<PreparedBlock> for Chain {
} }
} }
impl UpdateWith<(transaction::Hash, &Vec<transparent::Output>)> for Chain { impl UpdateWith<HashMap<transparent::OutPoint, Utxo>> for Chain {
fn update_chain_state_with( fn update_chain_state_with(&mut self, utxos: &HashMap<transparent::OutPoint, Utxo>) {
&mut self, self.created_utxos
(transaction_hash, outputs): &(transaction::Hash, &Vec<transparent::Output>), .extend(utxos.iter().map(|(k, v)| (*k, v.clone())));
) {
for (utxo_index, output) in outputs.iter().enumerate() {
self.created_utxos.insert(
transparent::OutPoint {
hash: *transaction_hash,
index: utxo_index as u32,
},
output.clone(),
);
}
} }
fn revert_chain_state_with( fn revert_chain_state_with(&mut self, utxos: &HashMap<transparent::OutPoint, Utxo>) {
&mut self, self.created_utxos
(transaction_hash, outputs): &(transaction::Hash, &Vec<transparent::Output>), .retain(|outpoint, _| !utxos.contains_key(outpoint));
) {
for (utxo_index, _) in outputs.iter().enumerate() {
assert!(
self.created_utxos
.remove(&transparent::OutPoint {
hash: *transaction_hash,
index: utxo_index as u32,
})
.is_some(),
"created_utxos must be present if block was"
);
}
} }
} }

View File

@ -6,7 +6,7 @@ use std::{
use tracing::instrument; use tracing::instrument;
use zebra_chain::{block, transparent}; use zebra_chain::{block, transparent};
use crate::service::QueuedBlock; use crate::{service::QueuedBlock, Utxo};
/// A queue of blocks, awaiting the arrival of parent blocks. /// A queue of blocks, awaiting the arrival of parent blocks.
#[derive(Default)] #[derive(Default)]
@ -18,7 +18,7 @@ pub struct QueuedBlocks {
/// Hashes from `queued_blocks`, indexed by block height. /// Hashes from `queued_blocks`, indexed by block height.
by_height: BTreeMap<block::Height, HashSet<block::Hash>>, by_height: BTreeMap<block::Height, HashSet<block::Hash>>,
/// Known UTXOs. /// Known UTXOs.
known_utxos: HashMap<transparent::OutPoint, transparent::Output>, known_utxos: HashMap<transparent::OutPoint, Utxo>,
} }
impl QueuedBlocks { impl QueuedBlocks {
@ -150,7 +150,7 @@ impl QueuedBlocks {
} }
/// Try to look up this UTXO in any queued block. /// Try to look up this UTXO in any queued block.
pub fn utxo(&self, outpoint: &transparent::OutPoint) -> Option<transparent::Output> { pub fn utxo(&self, outpoint: &transparent::OutPoint) -> Option<Utxo> {
self.known_utxos.get(outpoint).cloned() self.known_utxos.get(outpoint).cloned()
} }
} }

View File

@ -8,7 +8,7 @@ use zebra_chain::{
}; };
use zebra_test::{prelude::*, transcript::Transcript}; use zebra_test::{prelude::*, transcript::Transcript};
use crate::{init, BoxError, Config, Request, Response}; use crate::{init, BoxError, Config, Request, Response, Utxo};
const LAST_BLOCK_HEIGHT: u32 = 10; const LAST_BLOCK_HEIGHT: u32 = 10;
@ -80,16 +80,19 @@ async fn test_populated_state_responds_correctly(
Ok(Response::Transaction(Some(transaction.clone()))), Ok(Response::Transaction(Some(transaction.clone()))),
)); ));
for (index, output) in transaction.outputs().iter().enumerate() { let from_coinbase = transaction.is_coinbase();
for (index, output) in transaction.outputs().iter().cloned().enumerate() {
let outpoint = transparent::OutPoint { let outpoint = transparent::OutPoint {
hash: transaction_hash, hash: transaction_hash,
index: index as _, index: index as _,
}; };
let utxo = Utxo {
output,
height,
from_coinbase,
};
transcript.push(( transcript.push((Request::AwaitUtxo(outpoint), Ok(Response::Utxo(utxo))));
Request::AwaitUtxo(outpoint),
Ok(Response::Utxo(output.clone())),
));
} }
} }
} }

View File

@ -1,12 +1,14 @@
#![allow(dead_code)]
use crate::{BoxError, Response};
use std::collections::HashMap; use std::collections::HashMap;
use std::future::Future; use std::future::Future;
use tokio::sync::broadcast; use tokio::sync::broadcast;
use zebra_chain::{block::Block, transparent};
use zebra_chain::transparent;
use crate::{BoxError, Response, Utxo};
#[derive(Debug, Default)] #[derive(Debug, Default)]
pub struct PendingUtxos(HashMap<transparent::OutPoint, broadcast::Sender<transparent::Output>>); pub struct PendingUtxos(HashMap<transparent::OutPoint, broadcast::Sender<Utxo>>);
impl PendingUtxos { impl PendingUtxos {
/// Returns a future that will resolve to the `transparent::Output` pointed /// Returns a future that will resolve to the `transparent::Output` pointed
@ -33,41 +35,23 @@ impl PendingUtxos {
} }
} }
/// Notify all utxo requests waiting for the `transparent::Output` pointed to /// Notify all requests waiting for the [`Utxo`] pointed to by the given
/// by the given `transparent::OutPoint` that the `Output` has arrived. /// [`transparent::OutPoint`] that the [`Utxo`] has arrived.
pub fn respond(&mut self, outpoint: &transparent::OutPoint, output: transparent::Output) { pub fn respond(&mut self, outpoint: &transparent::OutPoint, utxo: Utxo) {
if let Some(sender) = self.0.remove(&outpoint) { if let Some(sender) = self.0.remove(outpoint) {
// Adding the outpoint as a field lets us crossreference // Adding the outpoint as a field lets us crossreference
// with the trace of the verification that made the request. // with the trace of the verification that made the request.
tracing::trace!(?outpoint, "found pending UTXO"); tracing::trace!(?outpoint, "found pending UTXO");
let _ = sender.send(output); let _ = sender.send(utxo);
} }
} }
/// Check the list of pending UTXO requests against the supplied UTXO index. /// Check the list of pending UTXO requests against the supplied UTXO index.
pub fn check_against(&mut self, utxos: &HashMap<transparent::OutPoint, transparent::Output>) { pub fn check_against(&mut self, utxos: &HashMap<transparent::OutPoint, Utxo>) {
for (outpoint, output) in utxos.iter() { for (outpoint, utxo) in utxos.iter() {
self.respond(outpoint, output.clone()); if let Some(sender) = self.0.remove(outpoint) {
} tracing::trace!(?outpoint, "found pending UTXO");
} let _ = sender.send(utxo.clone());
/// Scan through unindexed transactions in the given `block`
/// to determine whether it contains any requested UTXOs.
pub fn scan_block(&mut self, block: &Block) {
if self.0.is_empty() {
return;
}
tracing::trace!("scanning new block for pending UTXOs");
for transaction in block.transactions.iter() {
let transaction_hash = transaction.hash();
for (index, output) in transaction.outputs().iter().enumerate() {
let outpoint = transparent::OutPoint {
hash: transaction_hash,
index: index as _,
};
self.respond(&outpoint, output.clone());
} }
} }
} }

View File

@ -1,4 +1,4 @@
use std::{collections::HashMap, convert::TryFrom, mem, sync::Arc}; use std::{convert::TryFrom, mem, sync::Arc};
use primitive_types::U256; use primitive_types::U256;
use zebra_chain::{ use zebra_chain::{
@ -21,15 +21,7 @@ impl Prepare for Arc<Block> {
let block = self; let block = self;
let hash = block.hash(); let hash = block.hash();
let height = block.coinbase_height().unwrap(); let height = block.coinbase_height().unwrap();
let new_outputs = crate::utxo::new_outputs(&block);
let mut new_outputs = HashMap::new();
for transaction in &block.transactions {
let hash = transaction.hash();
for (index, output) in transaction.outputs().iter().cloned().enumerate() {
let index = index as u32;
new_outputs.insert(transparent::OutPoint { hash, index }, output);
}
}
PreparedBlock { PreparedBlock {
block, block,

42
zebra-state/src/utxo.rs Normal file
View File

@ -0,0 +1,42 @@
// needed to make clippy happy with derive(Arbitrary)
#![allow(clippy::unit_arg)]
use zebra_chain::{block, transparent};
/// An unspent `transparent::Output`, with accompanying metadata.
#[derive(Clone, Debug, PartialEq, Eq)]
#[cfg_attr(test, derive(proptest_derive::Arbitrary))]
pub struct Utxo {
/// The output itself.
pub output: transparent::Output,
/// The height at which the output was created.
pub height: block::Height,
/// Whether the output originated in a coinbase transaction.
pub from_coinbase: bool,
}
#[cfg(test)]
pub fn new_outputs(block: &block::Block) -> std::collections::HashMap<transparent::OutPoint, Utxo> {
use std::collections::HashMap;
let height = block.coinbase_height().expect("block has coinbase height");
let mut new_outputs = HashMap::default();
for transaction in &block.transactions {
let hash = transaction.hash();
let from_coinbase = transaction.is_coinbase();
for (index, output) in transaction.outputs().iter().cloned().enumerate() {
let index = index as u32;
new_outputs.insert(
transparent::OutPoint { hash, index },
Utxo {
output,
height,
from_coinbase,
},
);
}
}
new_outputs
}