state: introduce PreparedBlock, FinalizedBlock

This change introduces two new types:

- `PreparedBlock`, representing a block which has undergone semantic
  validation and has been prepared for contextual validation;
- `FinalizedBlock`, representing a block which is ready to be finalized
  immediately;

and changes the `Request::CommitBlock`,`Request::CommitFinalizedBlock`
variants to use these types instead of their previous fields.

This change solves the problem of passing data between semantic
validation and contextual validation, and cleans up the state code by
allowing it to pass around a bundle of data.  Previously, the state code
just passed around an `Arc<Block>`, which forced it to needlessly
recompute block hashes and other data, and was incompatible with the
already-known but not-yet-implemented data transfer requirements, namely
passing in the Sprout and Sapling anchors computed during contextual
validation.

This commit propagates the `PreparedBlock` and `FinalizedBlock` types
through the state code but only uses their data opportunistically, e.g.,
changing .hash() computations to use the precomputed hash.  In the
future, these structures can be extended to pass data through the
verification pipeline for reuse as appropriate.  For instance, these
changes allow the sprout and sapling anchors to be propagated through
the state.
This commit is contained in:
Henry de Valence 2020-11-21 01:16:14 -08:00 committed by teor
parent 3f78476693
commit e0817d1747
15 changed files with 330 additions and 282 deletions

View File

@ -193,11 +193,19 @@ where
metrics::counter!("block.verified.block.count", 1); metrics::counter!("block.verified.block.count", 1);
// Finally, submit the block for contextual verification. // Finally, submit the block for contextual verification.
let new_outputs = Arc::try_unwrap(known_utxos)
.expect("all verification tasks using known_utxos are complete");
let prepared_block = zs::PreparedBlock {
block,
hash,
height,
new_outputs,
};
match state_service match state_service
.ready_and() .ready_and()
.await .await
.map_err(VerifyBlockError::Commit)? .map_err(VerifyBlockError::Commit)?
.call(zs::Request::CommitBlock { block }) .call(zs::Request::CommitBlock(prepared_block))
.await .await
.map_err(VerifyBlockError::Commit)? .map_err(VerifyBlockError::Commit)?
{ {

View File

@ -852,7 +852,7 @@ where
.ready_and() .ready_and()
.await .await
.expect("Verified checkpoints must be committed transactionally") .expect("Verified checkpoints must be committed transactionally")
.call(zs::Request::CommitFinalizedBlock { block }) .call(zs::Request::CommitFinalizedBlock(block.into()))
.await .await
.expect("Verified checkpoints must be committed transactionally") .expect("Verified checkpoints must be committed transactionally")
{ {

View File

@ -320,9 +320,9 @@ async fn continuous_blockchain(restart_height: Option<block::Height>) -> Result<
/// SPANDOC: Add block to the state {?height} /// SPANDOC: Add block to the state {?height}
ready_state_service ready_state_service
.call(zebra_state::Request::CommitFinalizedBlock { .call(zebra_state::Request::CommitFinalizedBlock(
block: block.clone(), block.clone().into(),
}) ))
.await .await
.map_err(|e| eyre!(e))?; .map_err(|e| eyre!(e))?;
} }

View File

@ -23,6 +23,6 @@ mod tests;
pub use config::Config; pub use config::Config;
pub use constants::MAX_BLOCK_REORG_HEIGHT; pub use constants::MAX_BLOCK_REORG_HEIGHT;
pub use error::{BoxError, CloneError, CommitBlockError, ValidateContextError}; pub use error::{BoxError, CloneError, CommitBlockError, ValidateContextError};
pub use request::{HashOrHeight, Request}; pub use request::{FinalizedBlock, HashOrHeight, PreparedBlock, Request};
pub use response::Response; pub use response::Response;
pub use service::init; pub use service::init;

View File

@ -1,4 +1,5 @@
use std::sync::Arc; use std::{collections::HashMap, sync::Arc};
use zebra_chain::{ use zebra_chain::{
block::{self, Block}, block::{self, Block},
transaction, transparent, transaction, transparent,
@ -47,6 +48,66 @@ impl From<block::Height> for HashOrHeight {
} }
} }
/// A block which has undergone semantic validation and has been prepared for
/// contextual validation.
///
/// It is the constructor's responsibility to perform semantic validation and to
/// ensure that all fields are consistent.
///
/// This structure contains data from contextual validation, which is computed in
/// the *service caller*'s task, not inside the service call itself. This allows
/// moving work out of the single-threaded state service.
#[derive(Clone, Debug, PartialEq, Eq)]
pub struct PreparedBlock {
/// The block to commit to the state.
pub block: Arc<Block>,
/// The hash of the block.
pub hash: block::Hash,
/// The height of the block.
pub 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 new_outputs: HashMap<transparent::OutPoint, transparent::Output>,
// TODO: add these parameters when we can compute anchors.
// sprout_anchor: sprout::tree::Root,
// sapling_anchor: sapling::tree::Root,
}
/// A finalized block, ready to be committed directly to the finalized state with
/// no checks.
///
/// This is exposed for use in checkpointing.
#[derive(Clone, Debug, PartialEq, Eq)]
pub struct FinalizedBlock {
// These are pub(crate) so we can add whatever db-format-dependent
// precomputation we want here without leaking internal details.
pub(crate) block: Arc<Block>,
pub(crate) hash: block::Hash,
pub(crate) height: block::Height,
}
// Doing precomputation in this From impl means that it will be done in
// the *service caller*'s task, not inside the service call itself.
// This allows moving work out of the single-threaded state service.
impl From<Arc<Block>> for FinalizedBlock {
fn from(block: Arc<Block>) -> Self {
let height = block
.coinbase_height()
.expect("finalized blocks must have a valid coinbase height");
let hash = block.hash();
Self {
block,
height,
hash,
}
}
}
#[derive(Clone, Debug, PartialEq, Eq)] #[derive(Clone, Debug, PartialEq, Eq)]
/// A query about or modification to the chain state. /// A query about or modification to the chain state.
pub enum Request { pub enum Request {
@ -65,13 +126,7 @@ pub enum Request {
/// future will have no effect on whether it is eventually processed. A /// future will have no effect on whether it is eventually processed. A
/// request to commit a block which has been queued internally but not yet /// request to commit a block which has been queued internally but not yet
/// committed will fail the older request and replace it with the newer request. /// committed will fail the older request and replace it with the newer request.
CommitBlock { CommitBlock(PreparedBlock),
/// The block to commit to the state.
block: Arc<Block>,
// TODO: add these parameters when we can compute anchors.
// sprout_anchor: sprout::tree::Root,
// sapling_anchor: sapling::tree::Root,
},
/// Commit a finalized block to the state, skipping all validation. /// Commit a finalized block to the state, skipping all validation.
/// ///
@ -87,13 +142,7 @@ pub enum Request {
/// future will have no effect on whether it is eventually processed. /// future will have no effect on whether it is eventually processed.
/// Duplicate requests should not be made, because it is the caller's /// Duplicate requests should not be made, because it is the caller's
/// responsibility to ensure that each block is valid and final. /// responsibility to ensure that each block is valid and final.
CommitFinalizedBlock { CommitFinalizedBlock(FinalizedBlock),
/// The block to commit to the state.
block: Arc<Block>,
// TODO: add these parameters when we can compute anchors.
// sprout_anchor: sprout::tree::Root,
// sapling_anchor: sapling::tree::Root,
},
/// Computes the depth in the current best chain of the block identified by the given hash. /// Computes the depth in the current best chain of the block identified by the given hash.
/// ///
@ -140,8 +189,8 @@ pub enum Request {
/// available if it is unknown. /// available if it is unknown.
/// ///
/// This request is purely informational, and there are no guarantees about /// This request is purely informational, and there are no guarantees about
/// whether the UTXO remains unspent or is on the best chain. Its purpose is /// whether the UTXO remains unspent or is on the best chain, or any chain.
/// to allow asynchronous script verification. /// Its purpose is to allow asynchronous script verification.
/// ///
/// Code making this request should apply a timeout layer to the service to /// Code making this request should apply a timeout layer to the service to
/// handle missing UTXOs. /// handle missing UTXOs.

View File

@ -3,8 +3,7 @@ use std::{
pin::Pin, pin::Pin,
sync::Arc, sync::Arc,
task::{Context, Poll}, task::{Context, Poll},
time::Duration, time::{Duration, Instant},
time::Instant,
}; };
use futures::future::FutureExt; use futures::future::FutureExt;
@ -21,8 +20,8 @@ use zebra_chain::{
}; };
use crate::{ use crate::{
request::HashOrHeight, BoxError, CommitBlockError, Config, Request, Response, request::HashOrHeight, BoxError, CommitBlockError, Config, FinalizedBlock, PreparedBlock,
ValidateContextError, Request, Response, ValidateContextError,
}; };
use self::finalized_state::FinalizedState; use self::finalized_state::FinalizedState;
@ -34,15 +33,14 @@ mod non_finalized_state;
mod tests; mod tests;
mod utxo; mod utxo;
// todo: put this somewhere pub type QueuedBlock = (
#[derive(Debug)] PreparedBlock,
pub struct QueuedBlock { oneshot::Sender<Result<block::Hash, BoxError>>,
pub block: Arc<Block>, );
// TODO: add these parameters when we can compute anchors. pub type QueuedFinalized = (
// sprout_anchor: sprout::tree::Root, FinalizedBlock,
// sapling_anchor: sapling::tree::Root, oneshot::Sender<Result<block::Hash, BoxError>>,
pub rsp_tx: oneshot::Sender<Result<block::Hash, BoxError>>, );
}
struct StateService { struct StateService {
/// Holds data relating to finalized chain state. /// Holds data relating to finalized chain state.
@ -85,15 +83,15 @@ impl StateService {
/// in RFC0005. /// in RFC0005.
/// ///
/// [1]: https://zebra.zfnd.org/dev/rfcs/0005-state-updates.html#committing-non-finalized-blocks /// [1]: https://zebra.zfnd.org/dev/rfcs/0005-state-updates.html#committing-non-finalized-blocks
#[instrument(skip(self, block))] #[instrument(skip(self, prepared))]
fn queue_and_commit_non_finalized( fn queue_and_commit_non_finalized(
&mut self, &mut self,
block: Arc<Block>, prepared: PreparedBlock,
) -> oneshot::Receiver<Result<block::Hash, BoxError>> { ) -> oneshot::Receiver<Result<block::Hash, BoxError>> {
let hash = block.hash(); let parent_hash = prepared.block.header.previous_block_hash;
let parent_hash = block.header.previous_block_hash;
if self.contains_committed_block(&block) { if self.mem.any_chain_contains(&prepared.hash) || self.disk.hash(prepared.height).is_some()
{
let (rsp_tx, rsp_rx) = oneshot::channel(); let (rsp_tx, rsp_rx) = oneshot::channel();
let _ = rsp_tx.send(Err("block is already committed to the state".into())); let _ = rsp_tx.send(Err("block is already committed to the state".into()));
return rsp_rx; return rsp_rx;
@ -102,15 +100,15 @@ impl StateService {
// Request::CommitBlock contract: a request to commit a block which has // Request::CommitBlock contract: a request to commit a block which has
// been queued but not yet committed to the state fails the older // been queued but not yet committed to the state fails the older
// request and replaces it with the newer request. // request and replaces it with the newer request.
let rsp_rx = if let Some(queued_block) = self.queued_blocks.get_mut(&hash) { let rsp_rx = if let Some((_, old_rsp_tx)) = self.queued_blocks.get_mut(&prepared.hash) {
tracing::debug!("replacing older queued request with new request"); tracing::debug!("replacing older queued request with new request");
let (mut rsp_tx, rsp_rx) = oneshot::channel(); let (mut rsp_tx, rsp_rx) = oneshot::channel();
std::mem::swap(&mut queued_block.rsp_tx, &mut rsp_tx); std::mem::swap(old_rsp_tx, &mut rsp_tx);
let _ = rsp_tx.send(Err("replaced by newer request".into())); let _ = rsp_tx.send(Err("replaced by newer request".into()));
rsp_rx rsp_rx
} else { } else {
let (rsp_tx, rsp_rx) = oneshot::channel(); let (rsp_tx, rsp_rx) = oneshot::channel();
self.queued_blocks.queue(QueuedBlock { block, rsp_tx }); self.queued_blocks.queue((prepared, rsp_tx));
rsp_rx rsp_rx
}; };
@ -138,16 +136,16 @@ impl StateService {
rsp_rx rsp_rx
} }
/// Run contextual validation on `block` and add it to the non-finalized /// Run contextual validation on the prepared block and add it to the
/// state if it is contextually valid. /// non-finalized state if it is contextually valid.
fn validate_and_commit(&mut self, block: Arc<Block>) -> Result<(), CommitBlockError> { fn validate_and_commit(&mut self, prepared: PreparedBlock) -> Result<(), CommitBlockError> {
self.check_contextual_validity(&block)?; self.check_contextual_validity(&prepared)?;
let parent_hash = block.header.previous_block_hash; let parent_hash = prepared.block.header.previous_block_hash;
if self.disk.finalized_tip_hash() == parent_hash { if self.disk.finalized_tip_hash() == parent_hash {
self.mem.commit_new_chain(block); self.mem.commit_new_chain(prepared);
} else { } else {
self.mem.commit_block(block); self.mem.commit_block(prepared);
} }
Ok(()) Ok(())
@ -158,17 +156,6 @@ impl StateService {
self.mem.any_chain_contains(hash) || &self.disk.finalized_tip_hash() == hash self.mem.any_chain_contains(hash) || &self.disk.finalized_tip_hash() == hash
} }
/// Returns true if the given hash has been committed to either the finalized
/// or non-finalized state.
fn contains_committed_block(&self, block: &Block) -> bool {
let hash = block.hash();
let height = block
.coinbase_height()
.expect("coinbase heights should be valid");
self.mem.any_chain_contains(&hash) || self.disk.hash(height) == Some(hash)
}
/// Attempt to validate and commit all queued blocks whose parents have /// Attempt to validate and commit all queued blocks whose parents have
/// recently arrived starting from `new_parent`, in breadth-first ordering. /// recently arrived starting from `new_parent`, in breadth-first ordering.
fn process_queued(&mut self, new_parent: block::Hash) { fn process_queued(&mut self, new_parent: block::Hash) {
@ -177,11 +164,11 @@ impl StateService {
while let Some(parent_hash) = new_parents.pop() { while let Some(parent_hash) = new_parents.pop() {
let queued_children = self.queued_blocks.dequeue_children(parent_hash); let queued_children = self.queued_blocks.dequeue_children(parent_hash);
for QueuedBlock { block, rsp_tx } in queued_children { for (child, rsp_tx) in queued_children {
let child_hash = block.hash(); let child_hash = child.hash.clone();
tracing::trace!(?child_hash, "validating queued child"); tracing::trace!(?child_hash, "validating queued child");
let result = self let result = self
.validate_and_commit(block) .validate_and_commit(child)
.map(|()| child_hash) .map(|()| child_hash)
.map_err(BoxError::from); .map_err(BoxError::from);
let _ = rsp_tx.send(result); let _ = rsp_tx.send(result);
@ -190,14 +177,17 @@ impl StateService {
} }
} }
/// Check that `block` is contextually valid for the configured network, /// Check that the prepared block is contextually valid for the configured
/// based on the committed finalized and non-finalized state. /// network, based on the committed finalized and non-finalized state.
fn check_contextual_validity(&mut self, block: &Block) -> Result<(), ValidateContextError> { fn check_contextual_validity(
&mut self,
prepared: &PreparedBlock,
) -> Result<(), ValidateContextError> {
check::block_is_contextually_valid( check::block_is_contextually_valid(
block, prepared,
self.network, self.network,
self.disk.finalized_tip_height(), self.disk.finalized_tip_height(),
self.chain(block.header.previous_block_hash), self.chain(prepared.block.header.previous_block_hash),
)?; )?;
Ok(()) Ok(())
@ -393,11 +383,11 @@ impl Service<Request> for StateService {
#[instrument(name = "state", skip(self, req))] #[instrument(name = "state", skip(self, req))]
fn call(&mut self, req: Request) -> Self::Future { fn call(&mut self, req: Request) -> Self::Future {
match req { match req {
Request::CommitBlock { block } => { Request::CommitBlock(prepared) => {
metrics::counter!("state.requests", 1, "type" => "commit_block"); metrics::counter!("state.requests", 1, "type" => "commit_block");
self.pending_utxos.check_block(&block); self.pending_utxos.check_against(&prepared.new_outputs);
let rsp_rx = self.queue_and_commit_non_finalized(block); let rsp_rx = self.queue_and_commit_non_finalized(prepared);
async move { async move {
rsp_rx rsp_rx
@ -408,14 +398,13 @@ impl Service<Request> for StateService {
} }
.boxed() .boxed()
} }
Request::CommitFinalizedBlock { block } => { Request::CommitFinalizedBlock(finalized) => {
metrics::counter!("state.requests", 1, "type" => "commit_finalized_block"); metrics::counter!("state.requests", 1, "type" => "commit_finalized_block");
let (rsp_tx, rsp_rx) = oneshot::channel(); let (rsp_tx, rsp_rx) = oneshot::channel();
self.pending_utxos.check_block(&block); self.pending_utxos.scan_block(&finalized.block);
self.disk self.disk.queue_and_commit_finalized((finalized, rsp_tx));
.queue_and_commit_finalized(QueuedBlock { block, rsp_tx });
async move { async move {
rsp_rx rsp_rx
@ -457,7 +446,7 @@ impl Service<Request> for StateService {
let fut = self.pending_utxos.queue(outpoint); let fut = self.pending_utxos.queue(outpoint);
if let Some(utxo) = self.utxo(&outpoint) { if let Some(utxo) = self.utxo(&outpoint) {
self.pending_utxos.respond(outpoint, utxo); self.pending_utxos.respond(&outpoint, utxo);
} }
fut.boxed() fut.boxed()

View File

@ -5,7 +5,7 @@ use zebra_chain::{
parameters::Network, parameters::Network,
}; };
use crate::ValidateContextError; use crate::{PreparedBlock, ValidateContextError};
use super::check; use super::check;
@ -14,8 +14,13 @@ use super::check;
/// ///
/// The relevant chain is an iterator over the ancestors of `block`, starting /// The relevant chain is an iterator over the ancestors of `block`, starting
/// with its parent block. /// with its parent block.
#[tracing::instrument(
name = "contextual_validation",
fields(?network),
skip(prepared, network, finalized_tip_height, relevant_chain)
)]
pub(crate) fn block_is_contextually_valid<C>( pub(crate) fn block_is_contextually_valid<C>(
block: &Block, prepared: &PreparedBlock,
network: Network, network: Network,
finalized_tip_height: Option<block::Height>, finalized_tip_height: Option<block::Height>,
relevant_chain: C, relevant_chain: C,
@ -24,22 +29,9 @@ where
C: IntoIterator, C: IntoIterator,
C::Item: AsRef<Block>, C::Item: AsRef<Block>,
{ {
let height = block
.coinbase_height()
.expect("semantically valid blocks have a coinbase height");
let hash = block.hash();
let span = tracing::info_span!(
"StateService::check_contextual_validity",
?height,
?network,
?hash
);
let _entered = span.enter();
let finalized_tip_height = finalized_tip_height let finalized_tip_height = finalized_tip_height
.expect("finalized state must contain at least one block to use the non-finalized state"); .expect("finalized state must contain at least one block to use the non-finalized state");
check::block_is_not_orphaned(finalized_tip_height, block)?; check::block_is_not_orphaned(finalized_tip_height, prepared.height)?;
let mut relevant_chain = relevant_chain.into_iter(); let mut relevant_chain = relevant_chain.into_iter();
let parent_block = relevant_chain let parent_block = relevant_chain
@ -49,7 +41,7 @@ where
let parent_height = parent_block let parent_height = parent_block
.coinbase_height() .coinbase_height()
.expect("valid blocks have a coinbase height"); .expect("valid blocks have a coinbase height");
check::height_one_more_than_parent_height(parent_height, block)?; check::height_one_more_than_parent_height(parent_height, prepared.height)?;
// TODO: validate difficulty adjustment // TODO: validate difficulty adjustment
// TODO: other contextual validation design and implelentation // TODO: other contextual validation design and implelentation
@ -60,13 +52,9 @@ where
/// block is less than or equal to the finalized tip height. /// block is less than or equal to the finalized tip height.
pub(super) fn block_is_not_orphaned( pub(super) fn block_is_not_orphaned(
finalized_tip_height: block::Height, finalized_tip_height: block::Height,
block: &Block, height: block::Height,
) -> Result<(), ValidateContextError> { ) -> Result<(), ValidateContextError> {
if block if height <= finalized_tip_height {
.coinbase_height()
.expect("valid blocks have a coinbase height")
<= finalized_tip_height
{
Err(ValidateContextError::OrphanedBlock) Err(ValidateContextError::OrphanedBlock)
} else { } else {
Ok(()) Ok(())
@ -77,12 +65,8 @@ pub(super) fn block_is_not_orphaned(
/// equal to the parent_height+1. /// equal to the parent_height+1.
pub(super) fn height_one_more_than_parent_height( pub(super) fn height_one_more_than_parent_height(
parent_height: block::Height, parent_height: block::Height,
block: &Block, height: block::Height,
) -> Result<(), ValidateContextError> { ) -> Result<(), ValidateContextError> {
let height = block
.coinbase_height()
.expect("valid blocks have a coinbase height");
if parent_height + 1 != Some(height) { if parent_height + 1 != Some(height) {
Err(ValidateContextError::NonSequentialBlock) Err(ValidateContextError::NonSequentialBlock)
} else { } else {
@ -102,16 +86,18 @@ mod tests {
fn test_orphan_consensus_check() { fn test_orphan_consensus_check() {
zebra_test::init(); zebra_test::init();
let block = zebra_test::vectors::BLOCK_MAINNET_347499_BYTES let height = zebra_test::vectors::BLOCK_MAINNET_347499_BYTES
.zcash_deserialize_into::<Arc<Block>>() .zcash_deserialize_into::<Arc<Block>>()
.unwrap()
.coinbase_height()
.unwrap(); .unwrap();
block_is_not_orphaned(block::Height(0), &block).expect("tip is lower so it should be fine"); block_is_not_orphaned(block::Height(0), height).expect("tip is lower so it should be fine");
block_is_not_orphaned(block::Height(347498), &block) block_is_not_orphaned(block::Height(347498), height)
.expect("tip is lower so it should be fine"); .expect("tip is lower so it should be fine");
block_is_not_orphaned(block::Height(347499), &block) block_is_not_orphaned(block::Height(347499), height)
.expect_err("tip is equal so it should error"); .expect_err("tip is equal so it should error");
block_is_not_orphaned(block::Height(500000), &block) block_is_not_orphaned(block::Height(500000), height)
.expect_err("tip is higher so it should error"); .expect_err("tip is higher so it should error");
} }
@ -119,21 +105,23 @@ mod tests {
fn test_sequential_height_check() { fn test_sequential_height_check() {
zebra_test::init(); zebra_test::init();
let block = zebra_test::vectors::BLOCK_MAINNET_347499_BYTES let height = zebra_test::vectors::BLOCK_MAINNET_347499_BYTES
.zcash_deserialize_into::<Arc<Block>>() .zcash_deserialize_into::<Arc<Block>>()
.unwrap()
.coinbase_height()
.unwrap(); .unwrap();
height_one_more_than_parent_height(block::Height(0), &block) height_one_more_than_parent_height(block::Height(0), height)
.expect_err("block is much lower, should panic"); .expect_err("block is much lower, should panic");
height_one_more_than_parent_height(block::Height(347497), &block) height_one_more_than_parent_height(block::Height(347497), height)
.expect_err("parent height is 2 less, should panic"); .expect_err("parent height is 2 less, should panic");
height_one_more_than_parent_height(block::Height(347498), &block) height_one_more_than_parent_height(block::Height(347498), height)
.expect("parent height is 1 less, should be good"); .expect("parent height is 1 less, should be good");
height_one_more_than_parent_height(block::Height(347499), &block) height_one_more_than_parent_height(block::Height(347499), height)
.expect_err("parent height is equal, should panic"); .expect_err("parent height is equal, should panic");
height_one_more_than_parent_height(block::Height(347500), &block) height_one_more_than_parent_height(block::Height(347500), height)
.expect_err("parent height is way more, should panic"); .expect_err("parent height is way more, should panic");
height_one_more_than_parent_height(block::Height(500000), &block) height_one_more_than_parent_height(block::Height(500000), height)
.expect_err("parent height is way more, should panic"); .expect_err("parent height is way more, should panic");
} }
} }

View File

@ -11,16 +11,16 @@ use zebra_chain::{
transaction::{self, Transaction}, transaction::{self, Transaction},
}; };
use crate::{BoxError, Config, HashOrHeight}; use crate::{BoxError, Config, FinalizedBlock, HashOrHeight};
use self::disk_format::{DiskDeserialize, DiskSerialize, FromDisk, IntoDisk, TransactionLocation}; use self::disk_format::{DiskDeserialize, DiskSerialize, FromDisk, IntoDisk, TransactionLocation};
use super::QueuedBlock; use super::QueuedFinalized;
/// The finalized part of the chain state, stored in the db. /// The finalized part of the chain state, stored in the db.
pub struct FinalizedState { pub struct FinalizedState {
/// Queued blocks that arrived out of order, indexed by their parent block hash. /// Queued blocks that arrived out of order, indexed by their parent block hash.
queued_by_prev_hash: HashMap<block::Hash, QueuedBlock>, queued_by_prev_hash: HashMap<block::Hash, QueuedFinalized>,
max_queued_height: i64, max_queued_height: i64,
db: rocksdb::DB, db: rocksdb::DB,
@ -92,10 +92,10 @@ impl FinalizedState {
/// ///
/// After queueing a finalized block, this method checks whether the newly /// After queueing a finalized block, this method checks whether the newly
/// queued block (and any of its descendants) can be committed to the state. /// queued block (and any of its descendants) can be committed to the state.
pub fn queue_and_commit_finalized(&mut self, queued_block: QueuedBlock) { pub fn queue_and_commit_finalized(&mut self, queued: QueuedFinalized) {
let prev_hash = queued_block.block.header.previous_block_hash; let prev_hash = queued.0.block.header.previous_block_hash;
let height = queued_block.block.coinbase_height().unwrap(); let height = queued.0.height;
self.queued_by_prev_hash.insert(prev_hash, queued_block); self.queued_by_prev_hash.insert(prev_hash, queued);
while let Some(queued_block) = self.queued_by_prev_hash.remove(&self.finalized_tip_hash()) { while let Some(queued_block) = self.queued_by_prev_hash.remove(&self.finalized_tip_hash()) {
self.commit_finalized(queued_block); self.commit_finalized(queued_block);
@ -111,7 +111,6 @@ impl FinalizedState {
} }
metrics::gauge!("state.finalized.queued.max.height", self.max_queued_height); metrics::gauge!("state.finalized.queued.max.height", self.max_queued_height);
metrics::gauge!( metrics::gauge!(
"state.finalized.queued.block.count", "state.finalized.queued.block.count",
self.queued_by_prev_hash.len() as _ self.queued_by_prev_hash.len() as _
@ -139,14 +138,18 @@ impl FinalizedState {
.valid() .valid()
} }
/// Immediately commit `block` to the finalized state. /// Immediately commit `finalized` to the finalized state.
pub fn commit_finalized_direct(&mut self, block: Arc<Block>) -> Result<block::Hash, BoxError> { pub fn commit_finalized_direct(
let height = block &mut self,
.coinbase_height() finalized: FinalizedBlock,
.expect("finalized blocks are valid and have a coinbase height"); ) -> Result<block::Hash, BoxError> {
let hash = block.hash(); block_precommit_metrics(&finalized);
block_precommit_metrics(&hash, height, &block); let FinalizedBlock {
block,
hash,
height,
} = 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();
let height_by_hash = self.db.cf_handle("height_by_hash").unwrap(); let height_by_hash = self.db.cf_handle("height_by_hash").unwrap();
@ -265,9 +268,9 @@ impl FinalizedState {
/// order. This function is called by [`queue`], which ensures order. /// order. This function is called by [`queue`], which ensures order.
/// It is intentionally not exposed as part of the public API of the /// It is intentionally not exposed as part of the public API of the
/// [`FinalizedState`]. /// [`FinalizedState`].
fn commit_finalized(&mut self, queued_block: QueuedBlock) { fn commit_finalized(&mut self, queued_block: QueuedFinalized) {
let QueuedBlock { block, rsp_tx } = queued_block; let (finalized, rsp_tx) = queued_block;
let result = self.commit_finalized_direct(block); let result = self.commit_finalized_direct(finalized);
let _ = rsp_tx.send(result.map_err(Into::into)); let _ = rsp_tx.send(result.map_err(Into::into));
} }
@ -342,7 +345,9 @@ impl Drop for FinalizedState {
} }
} }
fn block_precommit_metrics(hash: &block::Hash, height: block::Height, block: &Block) { fn block_precommit_metrics(finalized: &FinalizedBlock) {
let (hash, height, block) = (finalized.hash, finalized.height, finalized.block.as_ref());
let transaction_count = block.transactions.len(); let transaction_count = block.transactions.len();
let transparent_prevout_count = block let transparent_prevout_count = block
.transactions .transactions

View File

@ -15,7 +15,7 @@ use zebra_chain::{
transparent, transparent,
}; };
use crate::request::HashOrHeight; use crate::{FinalizedBlock, HashOrHeight, PreparedBlock};
use self::chain::Chain; use self::chain::Chain;
@ -31,7 +31,7 @@ pub struct NonFinalizedState {
impl NonFinalizedState { impl NonFinalizedState {
/// Finalize the lowest height block in the non-finalized portion of the best /// Finalize the lowest height block in the non-finalized portion of the best
/// chain and update all side-chains to match. /// chain and update all side-chains to match.
pub fn finalize(&mut self) -> Arc<Block> { pub fn finalize(&mut self) -> FinalizedBlock {
let chains = mem::take(&mut self.chain_set); let chains = mem::take(&mut self.chain_set);
let mut chains = chains.into_iter(); let mut chains = chains.into_iter();
@ -40,8 +40,8 @@ impl NonFinalizedState {
// extract the rest into side_chains so they can be mutated // extract the rest into side_chains so they can be mutated
let side_chains = chains; let side_chains = chains;
// remove the lowest height block from the best_chain as finalized_block // remove the lowest height block from the best_chain to be finalized
let finalized_block = best_chain.pop_root(); let finalizing = best_chain.pop_root();
// add best_chain back to `self.chain_set` // add best_chain back to `self.chain_set`
if !best_chain.is_empty() { if !best_chain.is_empty() {
@ -53,7 +53,7 @@ impl NonFinalizedState {
// remove the first block from `chain` // remove the first block from `chain`
let chain_start = chain.pop_root(); let chain_start = chain.pop_root();
// if block equals finalized_block // if block equals finalized_block
if !chain.is_empty() && chain_start == finalized_block { if !chain.is_empty() && chain_start.hash == finalizing.hash {
// add the chain back to `self.chain_set` // add the chain back to `self.chain_set`
self.chain_set.insert(chain); self.chain_set.insert(chain);
} else { } else {
@ -64,13 +64,17 @@ impl NonFinalizedState {
self.update_metrics_for_chains(); self.update_metrics_for_chains();
// return the finalized block // Construct a finalized block.
finalized_block FinalizedBlock {
block: finalizing.block,
hash: finalizing.hash,
height: finalizing.height,
}
} }
/// Commit block to the non-finalized state. /// Commit block to the non-finalized state.
pub fn commit_block(&mut self, block: Arc<Block>) { pub fn commit_block(&mut self, prepared: PreparedBlock) {
let parent_hash = block.header.previous_block_hash; let parent_hash = prepared.block.header.previous_block_hash;
let mut parent_chain = self let mut parent_chain = self
.take_chain_if(|chain| chain.non_finalized_tip_hash() == parent_hash) .take_chain_if(|chain| chain.non_finalized_tip_hash() == parent_hash)
@ -82,18 +86,20 @@ impl NonFinalizedState {
}) })
.expect("commit_block is only called with blocks that are ready to be commited"); .expect("commit_block is only called with blocks that are ready to be commited");
parent_chain.push(block.clone()); let (height, hash) = (prepared.height, prepared.hash);
parent_chain.push(prepared);
self.chain_set.insert(parent_chain); self.chain_set.insert(parent_chain);
self.update_metrics_for_committed_block(block); self.update_metrics_for_committed_block(height, hash);
} }
/// Commit block to the non-finalized state as a new chain where its parent /// Commit block to the non-finalized state as a new chain where its parent
/// is the finalized tip. /// is the finalized tip.
pub fn commit_new_chain(&mut self, block: Arc<Block>) { pub fn commit_new_chain(&mut self, prepared: PreparedBlock) {
let mut chain = Chain::default(); let mut chain = Chain::default();
chain.push(block.clone()); let (height, hash) = (prepared.height, prepared.hash);
chain.push(prepared);
self.chain_set.insert(Box::new(chain)); self.chain_set.insert(Box::new(chain));
self.update_metrics_for_committed_block(block); self.update_metrics_for_committed_block(height, hash);
} }
/// Returns the length of the non-finalized portion of the current best chain. /// Returns the length of the non-finalized portion of the current best chain.
@ -155,12 +161,12 @@ impl NonFinalizedState {
/// Returns the `block` with the given hash in the any chain. /// Returns the `block` with the given hash in the any chain.
pub fn block_by_hash(&self, hash: block::Hash) -> Option<Arc<Block>> { pub fn block_by_hash(&self, hash: block::Hash) -> Option<Arc<Block>> {
for chain in self.chain_set.iter().rev() { for chain in self.chain_set.iter().rev() {
if let Some(block) = chain if let Some(prepared) = chain
.height_by_hash .height_by_hash
.get(&hash) .get(&hash)
.and_then(|height| chain.blocks.get(height)) .and_then(|height| chain.blocks.get(height))
{ {
return Some(block.clone()); return Some(prepared.block.clone());
} }
} }
@ -173,7 +179,10 @@ impl NonFinalizedState {
let height = let height =
hash_or_height.height_or_else(|hash| best_chain.height_by_hash.get(&hash).cloned())?; hash_or_height.height_or_else(|hash| best_chain.height_by_hash.get(&hash).cloned())?;
best_chain.blocks.get(&height).cloned() best_chain
.blocks
.get(&height)
.map(|prepared| prepared.block.clone())
} }
/// Returns the hash for a given `block::Height` if it is present in the best chain. /// Returns the hash for a given `block::Height` if it is present in the best chain.
@ -211,10 +220,10 @@ impl NonFinalizedState {
/// Returns the given transaction if it exists in the best chain. /// Returns the given transaction if it exists in the best chain.
pub fn transaction(&self, hash: transaction::Hash) -> Option<Arc<Transaction>> { pub fn transaction(&self, hash: transaction::Hash) -> Option<Arc<Transaction>> {
let best_chain = self.best_chain()?; let best_chain = self.best_chain()?;
best_chain.tx_by_hash.get(&hash).map(|(height, index)| { best_chain
let block = &best_chain.blocks[height]; .tx_by_hash
block.transactions[*index].clone() .get(&hash)
}) .map(|(height, index)| best_chain.blocks[height].block.transactions[*index].clone())
} }
/// Return the non-finalized portion of the current best chain /// Return the non-finalized portion of the current best chain
@ -226,9 +235,7 @@ impl NonFinalizedState {
} }
/// Update the metrics after `block` is committed /// Update the metrics after `block` is committed
fn update_metrics_for_committed_block(&self, block: Arc<Block>) { fn update_metrics_for_committed_block(&self, height: block::Height, hash: block::Hash) {
let height = block.coinbase_height().unwrap();
metrics::counter!("state.memory.committed.block.count", 1); metrics::counter!("state.memory.committed.block.count", 1);
metrics::gauge!("state.memory.committed.block.height", height.0 as _); metrics::gauge!("state.memory.committed.block.height", height.0 as _);
@ -240,7 +247,8 @@ impl NonFinalizedState {
.next_back() .next_back()
.unwrap() .unwrap()
.1 .1
== &block .hash
== hash
{ {
metrics::counter!("state.memory.best.committed.block.count", 1); metrics::counter!("state.memory.best.committed.block.count", 1);
metrics::gauge!("state.memory.best.committed.block.height", height.0 as _); metrics::gauge!("state.memory.best.committed.block.height", height.0 as _);
@ -261,7 +269,7 @@ mod tests {
use zebra_chain::serialization::ZcashDeserializeInto; use zebra_chain::serialization::ZcashDeserializeInto;
use zebra_test::prelude::*; use zebra_test::prelude::*;
use crate::tests::FakeChainHelper; use crate::tests::{FakeChainHelper, Prepare};
use self::assert_eq; use self::assert_eq;
use super::*; use super::*;
@ -278,8 +286,8 @@ mod tests {
let expected_hash = block2.hash(); let expected_hash = block2.hash();
let mut state = NonFinalizedState::default(); let mut state = NonFinalizedState::default();
state.commit_new_chain(block2); state.commit_new_chain(block2.prepare());
state.commit_new_chain(child); state.commit_new_chain(child.prepare());
let best_chain = state.best_chain().unwrap(); let best_chain = state.best_chain().unwrap();
assert!(best_chain.height_by_hash.contains_key(&expected_hash)); assert!(best_chain.height_by_hash.contains_key(&expected_hash));
@ -297,15 +305,15 @@ mod tests {
let child = block1.make_fake_child().set_work(1); let child = block1.make_fake_child().set_work(1);
let mut state = NonFinalizedState::default(); let mut state = NonFinalizedState::default();
state.commit_new_chain(block1.clone()); state.commit_new_chain(block1.clone().prepare());
state.commit_block(block2.clone()); state.commit_block(block2.clone().prepare());
state.commit_block(child); state.commit_block(child.prepare());
let finalized = state.finalize(); let finalized = state.finalize();
assert_eq!(block1, finalized); assert_eq!(block1, finalized.block);
let finalized = state.finalize(); let finalized = state.finalize();
assert_eq!(block2, finalized); assert_eq!(block2, finalized.block);
assert!(state.best_chain().is_none()); assert!(state.best_chain().is_none());
@ -325,13 +333,13 @@ mod tests {
let mut state = NonFinalizedState::default(); let mut state = NonFinalizedState::default();
assert_eq!(0, state.chain_set.len()); assert_eq!(0, state.chain_set.len());
state.commit_new_chain(block1); state.commit_new_chain(block1.prepare());
assert_eq!(1, state.chain_set.len()); assert_eq!(1, state.chain_set.len());
state.commit_block(block2); state.commit_block(block2.prepare());
assert_eq!(1, state.chain_set.len()); assert_eq!(1, state.chain_set.len());
state.commit_block(child1); state.commit_block(child1.prepare());
assert_eq!(2, state.chain_set.len()); assert_eq!(2, state.chain_set.len());
state.commit_block(child2); state.commit_block(child2.prepare());
assert_eq!(2, state.chain_set.len()); assert_eq!(2, state.chain_set.len());
Ok(()) Ok(())
@ -350,10 +358,10 @@ mod tests {
let short_chain_block = block1.make_fake_child().set_work(3); let short_chain_block = block1.make_fake_child().set_work(3);
let mut state = NonFinalizedState::default(); let mut state = NonFinalizedState::default();
state.commit_new_chain(block1); state.commit_new_chain(block1.prepare());
state.commit_block(long_chain_block1); state.commit_block(long_chain_block1.prepare());
state.commit_block(long_chain_block2); state.commit_block(long_chain_block2.prepare());
state.commit_block(short_chain_block); state.commit_block(short_chain_block.prepare());
assert_eq!(2, state.chain_set.len()); assert_eq!(2, state.chain_set.len());
assert_eq!(2, state.best_chain_len()); assert_eq!(2, state.best_chain_len());
@ -376,12 +384,12 @@ mod tests {
let short_chain_block = block1.make_fake_child().set_work(3); let short_chain_block = block1.make_fake_child().set_work(3);
let mut state = NonFinalizedState::default(); let mut state = NonFinalizedState::default();
state.commit_new_chain(block1); state.commit_new_chain(block1.prepare());
state.commit_block(long_chain_block1); state.commit_block(long_chain_block1.prepare());
state.commit_block(long_chain_block2); state.commit_block(long_chain_block2.prepare());
state.commit_block(long_chain_block3); state.commit_block(long_chain_block3.prepare());
state.commit_block(long_chain_block4); state.commit_block(long_chain_block4.prepare());
state.commit_block(short_chain_block); state.commit_block(short_chain_block.prepare());
assert_eq!(2, state.chain_set.len()); assert_eq!(2, state.chain_set.len());
assert_eq!(5, state.best_chain_len()); assert_eq!(5, state.best_chain_len());
@ -401,9 +409,9 @@ mod tests {
let expected_hash = more_work_child.hash(); let expected_hash = more_work_child.hash();
let mut state = NonFinalizedState::default(); let mut state = NonFinalizedState::default();
state.commit_new_chain(block1); state.commit_new_chain(block1.prepare());
state.commit_block(less_work_child); state.commit_block(less_work_child.prepare());
state.commit_block(more_work_child); state.commit_block(more_work_child.prepare());
assert_eq!(2, state.chain_set.len()); assert_eq!(2, state.chain_set.len());
let tip_hash = state.tip().unwrap().1; let tip_hash = state.tip().unwrap().1;

View File

@ -2,20 +2,19 @@ use std::{
cmp::Ordering, cmp::Ordering,
collections::{BTreeMap, HashMap, HashSet}, collections::{BTreeMap, HashMap, HashSet},
ops::Deref, ops::Deref,
sync::Arc,
}; };
use tracing::{debug_span, instrument, trace}; use tracing::{debug_span, instrument, trace};
use zebra_chain::{ use zebra_chain::{
block::{self, Block}, block, primitives::Groth16Proof, sapling, sprout, transaction, transparent,
primitives::Groth16Proof,
sapling, sprout, transaction, transparent,
work::difficulty::PartialCumulativeWork, work::difficulty::PartialCumulativeWork,
}; };
use crate::PreparedBlock;
#[derive(Default, Clone)] #[derive(Default, Clone)]
pub struct Chain { pub struct Chain {
pub blocks: BTreeMap<block::Height, Arc<Block>>, pub blocks: BTreeMap<block::Height, PreparedBlock>,
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)>,
@ -30,20 +29,17 @@ pub struct Chain {
impl Chain { impl Chain {
/// Push a contextually valid non-finalized block into a chain as the new tip. /// Push a contextually valid non-finalized block into a chain as the new tip.
#[instrument(skip(self), fields(%block))] #[instrument(skip(self, block), fields(block = %block.block))]
pub fn push(&mut self, block: Arc<Block>) { pub fn push(&mut self, block: PreparedBlock) {
let block_height = block
.coinbase_height()
.expect("valid non-finalized blocks have a coinbase height");
// update cumulative data members // update cumulative data members
self.update_chain_state_with(&block); self.update_chain_state_with(&block);
self.blocks.insert(block_height, block); self.blocks.insert(block.height, block);
trace!("pushed block onto chain"); trace!("pushed block onto chain");
} }
/// Remove the lowest height block of the non-finalized portion of a chain. /// Remove the lowest height block of the non-finalized portion of a chain.
#[instrument(skip(self))] #[instrument(skip(self))]
pub fn pop_root(&mut self) -> Arc<Block> { pub fn pop_root(&mut self) -> PreparedBlock {
let block_height = self.lowest_height(); let block_height = self.lowest_height();
// remove the lowest height block from self.blocks // remove the lowest height block from self.blocks
@ -55,7 +51,7 @@ impl Chain {
// update cumulative data members // update cumulative data members
self.revert_chain_state_with(&block); self.revert_chain_state_with(&block);
// return the block // return the prepared block
block block
} }
@ -88,7 +84,7 @@ impl Chain {
.values() .values()
.next_back() .next_back()
.expect("only called while blocks is populated") .expect("only called while blocks is populated")
.hash() .hash
} }
/// Remove the highest height block of the non-finalized portion of a chain. /// Remove the highest height block of the non-finalized portion of a chain.
@ -138,15 +134,12 @@ trait UpdateWith<T> {
fn revert_chain_state_with(&mut self, _: &T); fn revert_chain_state_with(&mut self, _: &T);
} }
impl UpdateWith<Arc<Block>> for Chain { impl UpdateWith<PreparedBlock> for Chain {
fn update_chain_state_with(&mut self, block: &Arc<Block>) { fn update_chain_state_with(&mut self, prepared: &PreparedBlock) {
let block_height = block let (block, hash, height) = (prepared.block.as_ref(), prepared.hash, prepared.height);
.coinbase_height()
.expect("valid non-finalized blocks have a coinbase height");
let block_hash = block.hash();
// add hash to height_by_hash // add hash to height_by_hash
let prior_height = self.height_by_hash.insert(block_hash, block_height); let prior_height = self.height_by_hash.insert(hash, height);
assert!( assert!(
prior_height.is_none(), prior_height.is_none(),
"block heights must be unique within a single chain" "block heights must be unique within a single chain"
@ -179,7 +172,7 @@ impl UpdateWith<Arc<Block>> for Chain {
let transaction_hash = transaction.hash(); let transaction_hash = transaction.hash();
let prior_pair = self let prior_pair = self
.tx_by_hash .tx_by_hash
.insert(transaction_hash, (block_height, transaction_index)); .insert(transaction_hash, (height, transaction_index));
assert!( assert!(
prior_pair.is_none(), prior_pair.is_none(),
"transactions must be unique within a single chain" "transactions must be unique within a single chain"
@ -196,13 +189,13 @@ impl UpdateWith<Arc<Block>> for Chain {
} }
} }
#[instrument(skip(self), fields(%block))] #[instrument(skip(self, prepared), fields(block = %prepared.block))]
fn revert_chain_state_with(&mut self, block: &Arc<Block>) { fn revert_chain_state_with(&mut self, prepared: &PreparedBlock) {
let block_hash = block.hash(); let (block, hash) = (prepared.block.as_ref(), prepared.hash);
// remove the blocks hash from `height_by_hash` // remove the blocks hash from `height_by_hash`
assert!( assert!(
self.height_by_hash.remove(&block_hash).is_some(), self.height_by_hash.remove(&hash).is_some(),
"hash must be present if block was" "hash must be present if block was"
); );
@ -395,14 +388,14 @@ impl Ord for Chain {
.values() .values()
.last() .last()
.expect("always at least 1 element") .expect("always at least 1 element")
.hash(); .hash;
let other_hash = other let other_hash = other
.blocks .blocks
.values() .values()
.last() .last()
.expect("always at least 1 element") .expect("always at least 1 element")
.hash(); .hash;
// This comparison is a tie-breaker within the local node, so it does not need to // This comparison is a tie-breaker within the local node, so it does not need to
// be consistent with the ordering on `ExpandedDifficulty` and `block::Hash`. // be consistent with the ordering on `ExpandedDifficulty` and `block::Hash`.
@ -416,16 +409,17 @@ impl Ord for Chain {
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use std::{env, fmt}; use std::{env, fmt, sync::Arc};
use zebra_chain::serialization::ZcashDeserializeInto; use zebra_chain::serialization::ZcashDeserializeInto;
use zebra_chain::{ use zebra_chain::{
block::Block,
parameters::{Network, NetworkUpgrade}, parameters::{Network, NetworkUpgrade},
LedgerState, LedgerState,
}; };
use zebra_test::prelude::*; use zebra_test::prelude::*;
use crate::tests::FakeChainHelper; use crate::tests::{FakeChainHelper, Prepare};
use self::assert_eq; use self::assert_eq;
use super::*; use super::*;
@ -447,10 +441,11 @@ mod tests {
#[test] #[test]
fn construct_single() -> Result<()> { fn construct_single() -> Result<()> {
zebra_test::init(); zebra_test::init();
let block = zebra_test::vectors::BLOCK_MAINNET_434873_BYTES.zcash_deserialize_into()?; let block: Arc<Block> =
zebra_test::vectors::BLOCK_MAINNET_434873_BYTES.zcash_deserialize_into()?;
let mut chain = Chain::default(); let mut chain = Chain::default();
chain.push(block); chain.push(block.prepare());
assert_eq!(1, chain.blocks.len()); assert_eq!(1, chain.blocks.len());
@ -474,7 +469,7 @@ mod tests {
let mut chain = Chain::default(); let mut chain = Chain::default();
for block in blocks { for block in blocks {
chain.push(block); chain.push(block.prepare());
} }
assert_eq!(100, chain.blocks.len()); assert_eq!(100, chain.blocks.len());
@ -491,10 +486,10 @@ mod tests {
let more_block = less_block.clone().set_work(10); let more_block = less_block.clone().set_work(10);
let mut lesser_chain = Chain::default(); let mut lesser_chain = Chain::default();
lesser_chain.push(less_block); lesser_chain.push(less_block.prepare());
let mut bigger_chain = Chain::default(); let mut bigger_chain = Chain::default();
bigger_chain.push(more_block); bigger_chain.push(more_block.prepare());
assert!(bigger_chain > lesser_chain); assert!(bigger_chain > lesser_chain);
@ -529,11 +524,11 @@ mod tests {
let mut partial_chain = Chain::default(); let mut partial_chain = Chain::default();
for block in chain.iter().take(count) { for block in chain.iter().take(count) {
partial_chain.push(block.clone()); partial_chain.push(block.clone().prepare());
} }
for block in chain { for block in chain {
full_chain.push(block); full_chain.push(block.prepare());
} }
let forked = full_chain.fork(fork_tip_hash).expect("hash is present"); let forked = full_chain.fork(fork_tip_hash).expect("hash is present");
@ -560,11 +555,11 @@ mod tests {
let mut partial_chain = Chain::default(); let mut partial_chain = Chain::default();
for block in chain.iter().skip(finalized_count) { for block in chain.iter().skip(finalized_count) {
partial_chain.push(block.clone()); partial_chain.push(block.clone().prepare());
} }
for block in chain { for block in chain {
full_chain.push(block); full_chain.push(block.prepare());
} }
for _ in 0..finalized_count { for _ in 0..finalized_count {

View File

@ -28,27 +28,14 @@ impl QueuedBlocks {
/// ///
/// - if a block with the same `block::Hash` has already been queued. /// - if a block with the same `block::Hash` has already been queued.
pub fn queue(&mut self, new: QueuedBlock) { pub fn queue(&mut self, new: QueuedBlock) {
let new_hash = new.block.hash(); let new_hash = new.0.hash;
let new_height = new let new_height = new.0.height;
.block let parent_hash = new.0.block.header.previous_block_hash;
.coinbase_height()
.expect("validated non-finalized blocks have a coinbase height");
let parent_hash = new.block.header.previous_block_hash;
// XXX QueuedBlock should include this data // Track known UTXOs in queued blocks.
let prev_utxo_count = self.known_utxos.len(); for (outpoint, output) in new.0.new_outputs.iter() {
for transaction in &new.block.transactions { self.known_utxos.insert(*outpoint, output.clone());
let hash = transaction.hash();
for (index, output) in transaction.outputs().iter().cloned().enumerate() {
let index = index as u32;
self.known_utxos
.insert(transparent::OutPoint { hash, index }, output);
}
} }
tracing::trace!(
known_utxos = self.known_utxos.len(),
new = self.known_utxos.len() - prev_utxo_count
);
let replaced = self.blocks.insert(new_hash, new); let replaced = self.blocks.insert(new_hash, new);
assert!(replaced.is_none(), "hashes must be unique"); assert!(replaced.is_none(), "hashes must be unique");
@ -83,28 +70,15 @@ impl QueuedBlocks {
.blocks .blocks
.remove(&hash) .remove(&hash)
.expect("block is present if its hash is in by_parent"); .expect("block is present if its hash is in by_parent");
let prev_utxo_count = self.known_utxos.len();
for transaction in &queued.block.transactions {
let hash = transaction.hash();
for (index, _output) in transaction.outputs().iter().cloned().enumerate() {
let index = index as u32;
self.known_utxos
.remove(&transparent::OutPoint { hash, index });
}
}
tracing::trace!(
known_utxos = self.known_utxos.len(),
removed = prev_utxo_count - self.known_utxos.len()
);
queued queued
}) })
.collect::<Vec<_>>(); .collect::<Vec<_>>();
for queued in &queued_children { for queued in &queued_children {
let height = queued.block.coinbase_height().unwrap(); self.by_height.remove(&queued.0.height);
self.by_height.remove(&height); for outpoint in queued.0.new_outputs.keys() {
self.known_utxos.remove(outpoint);
}
} }
tracing::trace!( tracing::trace!(
@ -133,7 +107,7 @@ impl QueuedBlocks {
for hash in by_height.into_iter().flat_map(|(_, hashes)| hashes) { for hash in by_height.into_iter().flat_map(|(_, hashes)| hashes) {
let expired = self.blocks.remove(&hash).expect("block is present"); let expired = self.blocks.remove(&hash).expect("block is present");
let parent_hash = &expired.block.header.previous_block_hash; let parent_hash = &expired.0.block.header.previous_block_hash;
let parent_list = self let parent_list = self
.by_parent .by_parent
@ -191,7 +165,7 @@ mod tests {
use zebra_chain::{block::Block, serialization::ZcashDeserializeInto}; use zebra_chain::{block::Block, serialization::ZcashDeserializeInto};
use zebra_test::prelude::*; use zebra_test::prelude::*;
use crate::tests::FakeChainHelper; use crate::tests::{FakeChainHelper, Prepare};
use self::assert_eq; use self::assert_eq;
use super::*; use super::*;
@ -204,11 +178,7 @@ mod tests {
impl IntoQueued for Arc<Block> { impl IntoQueued for Arc<Block> {
fn into_queued(self) -> QueuedBlock { fn into_queued(self) -> QueuedBlock {
let (rsp_tx, _) = oneshot::channel(); let (rsp_tx, _) = oneshot::channel();
(self.prepare(), rsp_tx)
QueuedBlock {
block: self,
rsp_tx,
}
} }
} }
@ -253,22 +223,22 @@ mod tests {
// Dequeueing the first block removes 1 block from each list // Dequeueing the first block removes 1 block from each list
let children = queue.dequeue_children(parent); let children = queue.dequeue_children(parent);
assert_eq!(1, children.len()); assert_eq!(1, children.len());
assert_eq!(block1, children[0].block); assert_eq!(block1, children[0].0.block);
assert_eq!(2, queue.blocks.len()); assert_eq!(2, queue.blocks.len());
assert_eq!(1, queue.by_parent.len()); assert_eq!(1, queue.by_parent.len());
assert_eq!(1, queue.by_height.len()); assert_eq!(1, queue.by_height.len());
// Dequeueing the children of the first block removes both of the other // Dequeueing the children of the first block removes both of the other
// blocks, and empties all lists // blocks, and empties all lists
let parent = children[0].block.hash(); let parent = children[0].0.block.hash();
let children = queue.dequeue_children(parent); let children = queue.dequeue_children(parent);
assert_eq!(2, children.len()); assert_eq!(2, children.len());
assert!(children assert!(children
.iter() .iter()
.any(|QueuedBlock { block, .. }| block == &child1)); .any(|(block, _)| block.hash == child1.hash()));
assert!(children assert!(children
.iter() .iter()
.any(|QueuedBlock { block, .. }| block == &child2)); .any(|(block, _)| block.hash == child2.hash()));
assert_eq!(0, queue.blocks.len()); assert_eq!(0, queue.blocks.len());
assert_eq!(0, queue.by_parent.len()); assert_eq!(0, queue.by_parent.len());
assert_eq!(0, queue.by_height.len()); assert_eq!(0, queue.by_height.len());

View File

@ -17,7 +17,7 @@ async fn populated_state(
) -> BoxService<Request, Response, BoxError> { ) -> BoxService<Request, Response, BoxError> {
let requests = blocks let requests = blocks
.into_iter() .into_iter()
.map(|block| Request::CommitFinalizedBlock { block }); .map(|block| Request::CommitFinalizedBlock(block.into()));
let config = Config::ephemeral(); let config = Config::ephemeral();
let network = Network::Mainnet; let network = Network::Mainnet;

View File

@ -35,7 +35,7 @@ impl PendingUtxos {
/// Notify all utxo requests waiting for the `transparent::Output` pointed to /// Notify all utxo requests waiting for the `transparent::Output` pointed to
/// by the given `transparent::OutPoint` that the `Output` has arrived. /// by the given `transparent::OutPoint` that the `Output` has arrived.
pub fn respond(&mut self, outpoint: transparent::OutPoint, output: transparent::Output) { pub fn respond(&mut self, outpoint: &transparent::OutPoint, output: transparent::Output) {
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.
@ -44,9 +44,16 @@ impl PendingUtxos {
} }
} }
/// For each notifies waiting utxo requests for each `transparent::Output` in /// Check the list of pending UTXO requests against the supplied UTXO index.
/// `block` that the output has arrived. pub fn check_against(&mut self, utxos: &HashMap<transparent::OutPoint, transparent::Output>) {
pub fn check_block(&mut self, block: &Block) { for (outpoint, output) in utxos.iter() {
self.respond(outpoint, output.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() { if self.0.is_empty() {
return; return;
} }
@ -60,7 +67,7 @@ impl PendingUtxos {
index: index as _, index: index as _,
}; };
self.respond(outpoint, output.clone()); self.respond(&outpoint, output.clone());
} }
} }
} }

View File

@ -1,4 +1,4 @@
use std::{convert::TryFrom, mem, sync::Arc}; use std::{collections::HashMap, convert::TryFrom, mem, sync::Arc};
use primitive_types::U256; use primitive_types::U256;
use zebra_chain::{ use zebra_chain::{
@ -11,6 +11,35 @@ use zebra_chain::{
use super::*; use super::*;
/// Mocks computation done during semantic validation
pub trait Prepare {
fn prepare(self) -> PreparedBlock;
}
impl Prepare for Arc<Block> {
fn prepare(self) -> PreparedBlock {
let block = self;
let hash = block.hash();
let height = block.coinbase_height().unwrap();
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 {
block,
hash,
height,
new_outputs,
}
}
}
/// Helper trait for constructing "valid" looking chains of blocks /// Helper trait for constructing "valid" looking chains of blocks
pub trait FakeChainHelper { pub trait FakeChainHelper {
fn make_fake_child(&self) -> Arc<Block>; fn make_fake_child(&self) -> Arc<Block>;

View File

@ -17,7 +17,7 @@ static COMMIT_FINALIZED_BLOCK_MAINNET: Lazy<Vec<(Request, Result<Response, Trans
let hash = block.hash(); let hash = block.hash();
vec![ vec![
( (
Request::CommitFinalizedBlock { block }, Request::CommitFinalizedBlock(block.into()),
Ok(Response::Committed(hash)), Ok(Response::Committed(hash)),
), ),
( (
@ -37,7 +37,7 @@ static COMMIT_FINALIZED_BLOCK_TESTNET: Lazy<Vec<(Request, Result<Response, Trans
let hash = block.hash(); let hash = block.hash();
vec![ vec![
( (
Request::CommitFinalizedBlock { block }, Request::CommitFinalizedBlock(block.into()),
Ok(Response::Committed(hash)), Ok(Response::Committed(hash)),
), ),
( (