diff --git a/zebra-consensus/src/block.rs b/zebra-consensus/src/block.rs index acdd5653..564ae804 100644 --- a/zebra-consensus/src/block.rs +++ b/zebra-consensus/src/block.rs @@ -193,11 +193,19 @@ where metrics::counter!("block.verified.block.count", 1); // 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 .ready_and() .await .map_err(VerifyBlockError::Commit)? - .call(zs::Request::CommitBlock { block }) + .call(zs::Request::CommitBlock(prepared_block)) .await .map_err(VerifyBlockError::Commit)? { diff --git a/zebra-consensus/src/checkpoint.rs b/zebra-consensus/src/checkpoint.rs index 0e1422b8..ec98e0ad 100644 --- a/zebra-consensus/src/checkpoint.rs +++ b/zebra-consensus/src/checkpoint.rs @@ -852,7 +852,7 @@ where .ready_and() .await .expect("Verified checkpoints must be committed transactionally") - .call(zs::Request::CommitFinalizedBlock { block }) + .call(zs::Request::CommitFinalizedBlock(block.into())) .await .expect("Verified checkpoints must be committed transactionally") { diff --git a/zebra-consensus/src/checkpoint/tests.rs b/zebra-consensus/src/checkpoint/tests.rs index 535d9461..d29d5069 100644 --- a/zebra-consensus/src/checkpoint/tests.rs +++ b/zebra-consensus/src/checkpoint/tests.rs @@ -320,9 +320,9 @@ async fn continuous_blockchain(restart_height: Option) -> Result< /// SPANDOC: Add block to the state {?height} ready_state_service - .call(zebra_state::Request::CommitFinalizedBlock { - block: block.clone(), - }) + .call(zebra_state::Request::CommitFinalizedBlock( + block.clone().into(), + )) .await .map_err(|e| eyre!(e))?; } diff --git a/zebra-state/src/lib.rs b/zebra-state/src/lib.rs index cdf6c339..f97a82c2 100644 --- a/zebra-state/src/lib.rs +++ b/zebra-state/src/lib.rs @@ -23,6 +23,6 @@ mod tests; pub use config::Config; pub use constants::MAX_BLOCK_REORG_HEIGHT; 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 service::init; diff --git a/zebra-state/src/request.rs b/zebra-state/src/request.rs index 76167735..7c78112a 100644 --- a/zebra-state/src/request.rs +++ b/zebra-state/src/request.rs @@ -1,4 +1,5 @@ -use std::sync::Arc; +use std::{collections::HashMap, sync::Arc}; + use zebra_chain::{ block::{self, Block}, transaction, transparent, @@ -47,6 +48,66 @@ impl From 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, + /// 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, + // 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, + 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> for FinalizedBlock { + fn from(block: Arc) -> 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)] /// A query about or modification to the chain state. pub enum Request { @@ -65,13 +126,7 @@ pub enum Request { /// 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 /// committed will fail the older request and replace it with the newer request. - CommitBlock { - /// The block to commit to the state. - block: Arc, - // TODO: add these parameters when we can compute anchors. - // sprout_anchor: sprout::tree::Root, - // sapling_anchor: sapling::tree::Root, - }, + CommitBlock(PreparedBlock), /// 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. /// Duplicate requests should not be made, because it is the caller's /// responsibility to ensure that each block is valid and final. - CommitFinalizedBlock { - /// The block to commit to the state. - block: Arc, - // TODO: add these parameters when we can compute anchors. - // sprout_anchor: sprout::tree::Root, - // sapling_anchor: sapling::tree::Root, - }, + CommitFinalizedBlock(FinalizedBlock), /// 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. /// /// 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 - /// to allow asynchronous script verification. + /// whether the UTXO remains unspent or is on the best chain, or any chain. + /// Its purpose is to allow asynchronous script verification. /// /// Code making this request should apply a timeout layer to the service to /// handle missing UTXOs. diff --git a/zebra-state/src/service.rs b/zebra-state/src/service.rs index da559f7e..0b3d1351 100644 --- a/zebra-state/src/service.rs +++ b/zebra-state/src/service.rs @@ -3,8 +3,7 @@ use std::{ pin::Pin, sync::Arc, task::{Context, Poll}, - time::Duration, - time::Instant, + time::{Duration, Instant}, }; use futures::future::FutureExt; @@ -21,8 +20,8 @@ use zebra_chain::{ }; use crate::{ - request::HashOrHeight, BoxError, CommitBlockError, Config, Request, Response, - ValidateContextError, + request::HashOrHeight, BoxError, CommitBlockError, Config, FinalizedBlock, PreparedBlock, + Request, Response, ValidateContextError, }; use self::finalized_state::FinalizedState; @@ -34,15 +33,14 @@ mod non_finalized_state; mod tests; mod utxo; -// todo: put this somewhere -#[derive(Debug)] -pub struct QueuedBlock { - pub block: Arc, - // TODO: add these parameters when we can compute anchors. - // sprout_anchor: sprout::tree::Root, - // sapling_anchor: sapling::tree::Root, - pub rsp_tx: oneshot::Sender>, -} +pub type QueuedBlock = ( + PreparedBlock, + oneshot::Sender>, +); +pub type QueuedFinalized = ( + FinalizedBlock, + oneshot::Sender>, +); struct StateService { /// Holds data relating to finalized chain state. @@ -85,15 +83,15 @@ impl StateService { /// in RFC0005. /// /// [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( &mut self, - block: Arc, + prepared: PreparedBlock, ) -> oneshot::Receiver> { - let hash = block.hash(); - let parent_hash = block.header.previous_block_hash; + let parent_hash = prepared.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.send(Err("block is already committed to the state".into())); return rsp_rx; @@ -102,15 +100,15 @@ impl StateService { // Request::CommitBlock contract: a request to commit a block which has // been queued but not yet committed to the state fails the older // 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"); 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())); rsp_rx } else { let (rsp_tx, rsp_rx) = oneshot::channel(); - self.queued_blocks.queue(QueuedBlock { block, rsp_tx }); + self.queued_blocks.queue((prepared, rsp_tx)); rsp_rx }; @@ -138,16 +136,16 @@ impl StateService { rsp_rx } - /// Run contextual validation on `block` and add it to the non-finalized - /// state if it is contextually valid. - fn validate_and_commit(&mut self, block: Arc) -> Result<(), CommitBlockError> { - self.check_contextual_validity(&block)?; - let parent_hash = block.header.previous_block_hash; + /// Run contextual validation on the prepared block and add it to the + /// non-finalized state if it is contextually valid. + fn validate_and_commit(&mut self, prepared: PreparedBlock) -> Result<(), CommitBlockError> { + self.check_contextual_validity(&prepared)?; + let parent_hash = prepared.block.header.previous_block_hash; if self.disk.finalized_tip_hash() == parent_hash { - self.mem.commit_new_chain(block); + self.mem.commit_new_chain(prepared); } else { - self.mem.commit_block(block); + self.mem.commit_block(prepared); } Ok(()) @@ -158,17 +156,6 @@ impl StateService { 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 /// recently arrived starting from `new_parent`, in breadth-first ordering. fn process_queued(&mut self, new_parent: block::Hash) { @@ -177,11 +164,11 @@ impl StateService { while let Some(parent_hash) = new_parents.pop() { let queued_children = self.queued_blocks.dequeue_children(parent_hash); - for QueuedBlock { block, rsp_tx } in queued_children { - let child_hash = block.hash(); + for (child, rsp_tx) in queued_children { + let child_hash = child.hash.clone(); tracing::trace!(?child_hash, "validating queued child"); let result = self - .validate_and_commit(block) + .validate_and_commit(child) .map(|()| child_hash) .map_err(BoxError::from); let _ = rsp_tx.send(result); @@ -190,14 +177,17 @@ impl StateService { } } - /// Check that `block` is contextually valid for the configured network, - /// based on the committed finalized and non-finalized state. - fn check_contextual_validity(&mut self, block: &Block) -> Result<(), ValidateContextError> { + /// Check that the prepared block is contextually valid for the configured + /// network, based on the committed finalized and non-finalized state. + fn check_contextual_validity( + &mut self, + prepared: &PreparedBlock, + ) -> Result<(), ValidateContextError> { check::block_is_contextually_valid( - block, + prepared, self.network, self.disk.finalized_tip_height(), - self.chain(block.header.previous_block_hash), + self.chain(prepared.block.header.previous_block_hash), )?; Ok(()) @@ -393,11 +383,11 @@ impl Service for StateService { #[instrument(name = "state", skip(self, req))] fn call(&mut self, req: Request) -> Self::Future { match req { - Request::CommitBlock { block } => { + Request::CommitBlock(prepared) => { metrics::counter!("state.requests", 1, "type" => "commit_block"); - self.pending_utxos.check_block(&block); - let rsp_rx = self.queue_and_commit_non_finalized(block); + self.pending_utxos.check_against(&prepared.new_outputs); + let rsp_rx = self.queue_and_commit_non_finalized(prepared); async move { rsp_rx @@ -408,14 +398,13 @@ impl Service for StateService { } .boxed() } - Request::CommitFinalizedBlock { block } => { + Request::CommitFinalizedBlock(finalized) => { metrics::counter!("state.requests", 1, "type" => "commit_finalized_block"); let (rsp_tx, rsp_rx) = oneshot::channel(); - self.pending_utxos.check_block(&block); - self.disk - .queue_and_commit_finalized(QueuedBlock { block, rsp_tx }); + self.pending_utxos.scan_block(&finalized.block); + self.disk.queue_and_commit_finalized((finalized, rsp_tx)); async move { rsp_rx @@ -457,7 +446,7 @@ impl Service for StateService { let fut = self.pending_utxos.queue(outpoint); if let Some(utxo) = self.utxo(&outpoint) { - self.pending_utxos.respond(outpoint, utxo); + self.pending_utxos.respond(&outpoint, utxo); } fut.boxed() diff --git a/zebra-state/src/service/check.rs b/zebra-state/src/service/check.rs index b7a6a18c..cdc0439e 100644 --- a/zebra-state/src/service/check.rs +++ b/zebra-state/src/service/check.rs @@ -5,7 +5,7 @@ use zebra_chain::{ parameters::Network, }; -use crate::ValidateContextError; +use crate::{PreparedBlock, ValidateContextError}; use super::check; @@ -14,8 +14,13 @@ use super::check; /// /// The relevant chain is an iterator over the ancestors of `block`, starting /// 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( - block: &Block, + prepared: &PreparedBlock, network: Network, finalized_tip_height: Option, relevant_chain: C, @@ -24,22 +29,9 @@ where C: IntoIterator, C::Item: AsRef, { - 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 .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 parent_block = relevant_chain @@ -49,7 +41,7 @@ where let parent_height = parent_block .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: other contextual validation design and implelentation @@ -60,13 +52,9 @@ where /// block is less than or equal to the finalized tip height. pub(super) fn block_is_not_orphaned( finalized_tip_height: block::Height, - block: &Block, + height: block::Height, ) -> Result<(), ValidateContextError> { - if block - .coinbase_height() - .expect("valid blocks have a coinbase height") - <= finalized_tip_height - { + if height <= finalized_tip_height { Err(ValidateContextError::OrphanedBlock) } else { Ok(()) @@ -77,12 +65,8 @@ pub(super) fn block_is_not_orphaned( /// equal to the parent_height+1. pub(super) fn height_one_more_than_parent_height( parent_height: block::Height, - block: &Block, + height: block::Height, ) -> Result<(), ValidateContextError> { - let height = block - .coinbase_height() - .expect("valid blocks have a coinbase height"); - if parent_height + 1 != Some(height) { Err(ValidateContextError::NonSequentialBlock) } else { @@ -102,16 +86,18 @@ mod tests { fn test_orphan_consensus_check() { zebra_test::init(); - let block = zebra_test::vectors::BLOCK_MAINNET_347499_BYTES + let height = zebra_test::vectors::BLOCK_MAINNET_347499_BYTES .zcash_deserialize_into::>() + .unwrap() + .coinbase_height() .unwrap(); - block_is_not_orphaned(block::Height(0), &block).expect("tip is lower so it should be fine"); - block_is_not_orphaned(block::Height(347498), &block) + block_is_not_orphaned(block::Height(0), height).expect("tip is lower so it should be fine"); + block_is_not_orphaned(block::Height(347498), height) .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"); - 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"); } @@ -119,21 +105,23 @@ mod tests { fn test_sequential_height_check() { zebra_test::init(); - let block = zebra_test::vectors::BLOCK_MAINNET_347499_BYTES + let height = zebra_test::vectors::BLOCK_MAINNET_347499_BYTES .zcash_deserialize_into::>() + .unwrap() + .coinbase_height() .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"); - 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"); - 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"); - 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"); - 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"); - 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"); } } diff --git a/zebra-state/src/service/finalized_state.rs b/zebra-state/src/service/finalized_state.rs index dc3e2b0c..e801b5a3 100644 --- a/zebra-state/src/service/finalized_state.rs +++ b/zebra-state/src/service/finalized_state.rs @@ -11,16 +11,16 @@ use zebra_chain::{ transaction::{self, Transaction}, }; -use crate::{BoxError, Config, HashOrHeight}; +use crate::{BoxError, Config, FinalizedBlock, HashOrHeight}; 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. pub struct FinalizedState { /// Queued blocks that arrived out of order, indexed by their parent block hash. - queued_by_prev_hash: HashMap, + queued_by_prev_hash: HashMap, max_queued_height: i64, db: rocksdb::DB, @@ -92,10 +92,10 @@ impl FinalizedState { /// /// After queueing a finalized block, this method checks whether the newly /// queued block (and any of its descendants) can be committed to the state. - pub fn queue_and_commit_finalized(&mut self, queued_block: QueuedBlock) { - let prev_hash = queued_block.block.header.previous_block_hash; - let height = queued_block.block.coinbase_height().unwrap(); - self.queued_by_prev_hash.insert(prev_hash, queued_block); + pub fn queue_and_commit_finalized(&mut self, queued: QueuedFinalized) { + let prev_hash = queued.0.block.header.previous_block_hash; + let height = queued.0.height; + self.queued_by_prev_hash.insert(prev_hash, queued); while let Some(queued_block) = self.queued_by_prev_hash.remove(&self.finalized_tip_hash()) { 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.block.count", self.queued_by_prev_hash.len() as _ @@ -139,14 +138,18 @@ impl FinalizedState { .valid() } - /// Immediately commit `block` to the finalized state. - pub fn commit_finalized_direct(&mut self, block: Arc) -> Result { - let height = block - .coinbase_height() - .expect("finalized blocks are valid and have a coinbase height"); - let hash = block.hash(); + /// Immediately commit `finalized` to the finalized state. + pub fn commit_finalized_direct( + &mut self, + finalized: FinalizedBlock, + ) -> Result { + 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 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. /// It is intentionally not exposed as part of the public API of the /// [`FinalizedState`]. - fn commit_finalized(&mut self, queued_block: QueuedBlock) { - let QueuedBlock { block, rsp_tx } = queued_block; - let result = self.commit_finalized_direct(block); + fn commit_finalized(&mut self, queued_block: QueuedFinalized) { + let (finalized, rsp_tx) = queued_block; + let result = self.commit_finalized_direct(finalized); 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 transparent_prevout_count = block .transactions diff --git a/zebra-state/src/service/non_finalized_state.rs b/zebra-state/src/service/non_finalized_state.rs index 2444e7e1..bf849cf8 100644 --- a/zebra-state/src/service/non_finalized_state.rs +++ b/zebra-state/src/service/non_finalized_state.rs @@ -15,7 +15,7 @@ use zebra_chain::{ transparent, }; -use crate::request::HashOrHeight; +use crate::{FinalizedBlock, HashOrHeight, PreparedBlock}; use self::chain::Chain; @@ -31,7 +31,7 @@ pub struct NonFinalizedState { impl NonFinalizedState { /// Finalize the lowest height block in the non-finalized portion of the best /// chain and update all side-chains to match. - pub fn finalize(&mut self) -> Arc { + pub fn finalize(&mut self) -> FinalizedBlock { let chains = mem::take(&mut self.chain_set); let mut chains = chains.into_iter(); @@ -40,8 +40,8 @@ impl NonFinalizedState { // extract the rest into side_chains so they can be mutated let side_chains = chains; - // remove the lowest height block from the best_chain as finalized_block - let finalized_block = best_chain.pop_root(); + // remove the lowest height block from the best_chain to be finalized + let finalizing = best_chain.pop_root(); // add best_chain back to `self.chain_set` if !best_chain.is_empty() { @@ -53,7 +53,7 @@ impl NonFinalizedState { // remove the first block from `chain` let chain_start = chain.pop_root(); // 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` self.chain_set.insert(chain); } else { @@ -64,13 +64,17 @@ impl NonFinalizedState { self.update_metrics_for_chains(); - // return the finalized block - finalized_block + // Construct a finalized block. + FinalizedBlock { + block: finalizing.block, + hash: finalizing.hash, + height: finalizing.height, + } } /// Commit block to the non-finalized state. - pub fn commit_block(&mut self, block: Arc) { - let parent_hash = block.header.previous_block_hash; + pub fn commit_block(&mut self, prepared: PreparedBlock) { + let parent_hash = prepared.block.header.previous_block_hash; let mut parent_chain = self .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"); - parent_chain.push(block.clone()); + let (height, hash) = (prepared.height, prepared.hash); + parent_chain.push(prepared); 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 /// is the finalized tip. - pub fn commit_new_chain(&mut self, block: Arc) { + pub fn commit_new_chain(&mut self, prepared: PreparedBlock) { 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.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. @@ -155,12 +161,12 @@ impl NonFinalizedState { /// Returns the `block` with the given hash in the any chain. pub fn block_by_hash(&self, hash: block::Hash) -> Option> { for chain in self.chain_set.iter().rev() { - if let Some(block) = chain + if let Some(prepared) = chain .height_by_hash .get(&hash) .and_then(|height| chain.blocks.get(height)) { - return Some(block.clone()); + return Some(prepared.block.clone()); } } @@ -173,7 +179,10 @@ impl NonFinalizedState { let height = 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. @@ -211,10 +220,10 @@ impl NonFinalizedState { /// Returns the given transaction if it exists in the best chain. pub fn transaction(&self, hash: transaction::Hash) -> Option> { let best_chain = self.best_chain()?; - best_chain.tx_by_hash.get(&hash).map(|(height, index)| { - let block = &best_chain.blocks[height]; - block.transactions[*index].clone() - }) + best_chain + .tx_by_hash + .get(&hash) + .map(|(height, index)| best_chain.blocks[height].block.transactions[*index].clone()) } /// Return the non-finalized portion of the current best chain @@ -226,9 +235,7 @@ impl NonFinalizedState { } /// Update the metrics after `block` is committed - fn update_metrics_for_committed_block(&self, block: Arc) { - let height = block.coinbase_height().unwrap(); - + fn update_metrics_for_committed_block(&self, height: block::Height, hash: block::Hash) { metrics::counter!("state.memory.committed.block.count", 1); metrics::gauge!("state.memory.committed.block.height", height.0 as _); @@ -240,7 +247,8 @@ impl NonFinalizedState { .next_back() .unwrap() .1 - == &block + .hash + == hash { metrics::counter!("state.memory.best.committed.block.count", 1); metrics::gauge!("state.memory.best.committed.block.height", height.0 as _); @@ -261,7 +269,7 @@ mod tests { use zebra_chain::serialization::ZcashDeserializeInto; use zebra_test::prelude::*; - use crate::tests::FakeChainHelper; + use crate::tests::{FakeChainHelper, Prepare}; use self::assert_eq; use super::*; @@ -278,8 +286,8 @@ mod tests { let expected_hash = block2.hash(); let mut state = NonFinalizedState::default(); - state.commit_new_chain(block2); - state.commit_new_chain(child); + state.commit_new_chain(block2.prepare()); + state.commit_new_chain(child.prepare()); let best_chain = state.best_chain().unwrap(); 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 mut state = NonFinalizedState::default(); - state.commit_new_chain(block1.clone()); - state.commit_block(block2.clone()); - state.commit_block(child); + state.commit_new_chain(block1.clone().prepare()); + state.commit_block(block2.clone().prepare()); + state.commit_block(child.prepare()); let finalized = state.finalize(); - assert_eq!(block1, finalized); + assert_eq!(block1, finalized.block); let finalized = state.finalize(); - assert_eq!(block2, finalized); + assert_eq!(block2, finalized.block); assert!(state.best_chain().is_none()); @@ -325,13 +333,13 @@ mod tests { let mut state = NonFinalizedState::default(); 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()); - state.commit_block(block2); + state.commit_block(block2.prepare()); assert_eq!(1, state.chain_set.len()); - state.commit_block(child1); + state.commit_block(child1.prepare()); assert_eq!(2, state.chain_set.len()); - state.commit_block(child2); + state.commit_block(child2.prepare()); assert_eq!(2, state.chain_set.len()); Ok(()) @@ -350,10 +358,10 @@ mod tests { let short_chain_block = block1.make_fake_child().set_work(3); let mut state = NonFinalizedState::default(); - state.commit_new_chain(block1); - state.commit_block(long_chain_block1); - state.commit_block(long_chain_block2); - state.commit_block(short_chain_block); + state.commit_new_chain(block1.prepare()); + state.commit_block(long_chain_block1.prepare()); + state.commit_block(long_chain_block2.prepare()); + state.commit_block(short_chain_block.prepare()); assert_eq!(2, state.chain_set.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 mut state = NonFinalizedState::default(); - state.commit_new_chain(block1); - state.commit_block(long_chain_block1); - state.commit_block(long_chain_block2); - state.commit_block(long_chain_block3); - state.commit_block(long_chain_block4); - state.commit_block(short_chain_block); + state.commit_new_chain(block1.prepare()); + state.commit_block(long_chain_block1.prepare()); + state.commit_block(long_chain_block2.prepare()); + state.commit_block(long_chain_block3.prepare()); + state.commit_block(long_chain_block4.prepare()); + state.commit_block(short_chain_block.prepare()); assert_eq!(2, state.chain_set.len()); assert_eq!(5, state.best_chain_len()); @@ -401,9 +409,9 @@ mod tests { let expected_hash = more_work_child.hash(); let mut state = NonFinalizedState::default(); - state.commit_new_chain(block1); - state.commit_block(less_work_child); - state.commit_block(more_work_child); + state.commit_new_chain(block1.prepare()); + state.commit_block(less_work_child.prepare()); + state.commit_block(more_work_child.prepare()); assert_eq!(2, state.chain_set.len()); let tip_hash = state.tip().unwrap().1; diff --git a/zebra-state/src/service/non_finalized_state/chain.rs b/zebra-state/src/service/non_finalized_state/chain.rs index f3e61a87..bb4e2bb3 100644 --- a/zebra-state/src/service/non_finalized_state/chain.rs +++ b/zebra-state/src/service/non_finalized_state/chain.rs @@ -2,20 +2,19 @@ use std::{ cmp::Ordering, collections::{BTreeMap, HashMap, HashSet}, ops::Deref, - sync::Arc, }; use tracing::{debug_span, instrument, trace}; use zebra_chain::{ - block::{self, Block}, - primitives::Groth16Proof, - sapling, sprout, transaction, transparent, + block, primitives::Groth16Proof, sapling, sprout, transaction, transparent, work::difficulty::PartialCumulativeWork, }; +use crate::PreparedBlock; + #[derive(Default, Clone)] pub struct Chain { - pub blocks: BTreeMap>, + pub blocks: BTreeMap, pub height_by_hash: HashMap, pub tx_by_hash: HashMap, @@ -30,20 +29,17 @@ pub struct Chain { impl Chain { /// Push a contextually valid non-finalized block into a chain as the new tip. - #[instrument(skip(self), fields(%block))] - pub fn push(&mut self, block: Arc) { - let block_height = block - .coinbase_height() - .expect("valid non-finalized blocks have a coinbase height"); + #[instrument(skip(self, block), fields(block = %block.block))] + pub fn push(&mut self, block: PreparedBlock) { // update cumulative data members self.update_chain_state_with(&block); - self.blocks.insert(block_height, block); + self.blocks.insert(block.height, block); trace!("pushed block onto chain"); } /// Remove the lowest height block of the non-finalized portion of a chain. #[instrument(skip(self))] - pub fn pop_root(&mut self) -> Arc { + pub fn pop_root(&mut self) -> PreparedBlock { let block_height = self.lowest_height(); // remove the lowest height block from self.blocks @@ -55,7 +51,7 @@ impl Chain { // update cumulative data members self.revert_chain_state_with(&block); - // return the block + // return the prepared block block } @@ -88,7 +84,7 @@ impl Chain { .values() .next_back() .expect("only called while blocks is populated") - .hash() + .hash } /// Remove the highest height block of the non-finalized portion of a chain. @@ -138,15 +134,12 @@ trait UpdateWith { fn revert_chain_state_with(&mut self, _: &T); } -impl UpdateWith> for Chain { - fn update_chain_state_with(&mut self, block: &Arc) { - let block_height = block - .coinbase_height() - .expect("valid non-finalized blocks have a coinbase height"); - let block_hash = block.hash(); +impl UpdateWith for Chain { + fn update_chain_state_with(&mut self, prepared: &PreparedBlock) { + let (block, hash, height) = (prepared.block.as_ref(), prepared.hash, prepared.height); // 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!( prior_height.is_none(), "block heights must be unique within a single chain" @@ -179,7 +172,7 @@ impl UpdateWith> for Chain { let transaction_hash = transaction.hash(); let prior_pair = self .tx_by_hash - .insert(transaction_hash, (block_height, transaction_index)); + .insert(transaction_hash, (height, transaction_index)); assert!( prior_pair.is_none(), "transactions must be unique within a single chain" @@ -196,13 +189,13 @@ impl UpdateWith> for Chain { } } - #[instrument(skip(self), fields(%block))] - fn revert_chain_state_with(&mut self, block: &Arc) { - let block_hash = block.hash(); + #[instrument(skip(self, prepared), fields(block = %prepared.block))] + fn revert_chain_state_with(&mut self, prepared: &PreparedBlock) { + let (block, hash) = (prepared.block.as_ref(), prepared.hash); // remove the blocks hash from `height_by_hash` 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" ); @@ -395,14 +388,14 @@ impl Ord for Chain { .values() .last() .expect("always at least 1 element") - .hash(); + .hash; let other_hash = other .blocks .values() .last() .expect("always at least 1 element") - .hash(); + .hash; // 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`. @@ -416,16 +409,17 @@ impl Ord for Chain { #[cfg(test)] mod tests { - use std::{env, fmt}; + use std::{env, fmt, sync::Arc}; use zebra_chain::serialization::ZcashDeserializeInto; use zebra_chain::{ + block::Block, parameters::{Network, NetworkUpgrade}, LedgerState, }; use zebra_test::prelude::*; - use crate::tests::FakeChainHelper; + use crate::tests::{FakeChainHelper, Prepare}; use self::assert_eq; use super::*; @@ -447,10 +441,11 @@ mod tests { #[test] fn construct_single() -> Result<()> { zebra_test::init(); - let block = zebra_test::vectors::BLOCK_MAINNET_434873_BYTES.zcash_deserialize_into()?; + let block: Arc = + zebra_test::vectors::BLOCK_MAINNET_434873_BYTES.zcash_deserialize_into()?; let mut chain = Chain::default(); - chain.push(block); + chain.push(block.prepare()); assert_eq!(1, chain.blocks.len()); @@ -474,7 +469,7 @@ mod tests { let mut chain = Chain::default(); for block in blocks { - chain.push(block); + chain.push(block.prepare()); } assert_eq!(100, chain.blocks.len()); @@ -491,10 +486,10 @@ mod tests { let more_block = less_block.clone().set_work(10); let mut lesser_chain = Chain::default(); - lesser_chain.push(less_block); + lesser_chain.push(less_block.prepare()); let mut bigger_chain = Chain::default(); - bigger_chain.push(more_block); + bigger_chain.push(more_block.prepare()); assert!(bigger_chain > lesser_chain); @@ -529,11 +524,11 @@ mod tests { let mut partial_chain = Chain::default(); for block in chain.iter().take(count) { - partial_chain.push(block.clone()); + partial_chain.push(block.clone().prepare()); } 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"); @@ -560,11 +555,11 @@ mod tests { let mut partial_chain = Chain::default(); for block in chain.iter().skip(finalized_count) { - partial_chain.push(block.clone()); + partial_chain.push(block.clone().prepare()); } for block in chain { - full_chain.push(block); + full_chain.push(block.prepare()); } for _ in 0..finalized_count { diff --git a/zebra-state/src/service/non_finalized_state/queued_blocks.rs b/zebra-state/src/service/non_finalized_state/queued_blocks.rs index 790c1238..0bea77fd 100644 --- a/zebra-state/src/service/non_finalized_state/queued_blocks.rs +++ b/zebra-state/src/service/non_finalized_state/queued_blocks.rs @@ -28,27 +28,14 @@ impl QueuedBlocks { /// /// - if a block with the same `block::Hash` has already been queued. pub fn queue(&mut self, new: QueuedBlock) { - let new_hash = new.block.hash(); - let new_height = new - .block - .coinbase_height() - .expect("validated non-finalized blocks have a coinbase height"); - let parent_hash = new.block.header.previous_block_hash; + let new_hash = new.0.hash; + let new_height = new.0.height; + let parent_hash = new.0.block.header.previous_block_hash; - // XXX QueuedBlock should include this data - let prev_utxo_count = self.known_utxos.len(); - for transaction in &new.block.transactions { - 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); - } + // Track known UTXOs in queued blocks. + for (outpoint, output) in new.0.new_outputs.iter() { + self.known_utxos.insert(*outpoint, output.clone()); } - tracing::trace!( - known_utxos = self.known_utxos.len(), - new = self.known_utxos.len() - prev_utxo_count - ); let replaced = self.blocks.insert(new_hash, new); assert!(replaced.is_none(), "hashes must be unique"); @@ -83,28 +70,15 @@ impl QueuedBlocks { .blocks .remove(&hash) .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 }) .collect::>(); for queued in &queued_children { - let height = queued.block.coinbase_height().unwrap(); - self.by_height.remove(&height); + self.by_height.remove(&queued.0.height); + for outpoint in queued.0.new_outputs.keys() { + self.known_utxos.remove(outpoint); + } } tracing::trace!( @@ -133,7 +107,7 @@ impl QueuedBlocks { for hash in by_height.into_iter().flat_map(|(_, hashes)| hashes) { 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 .by_parent @@ -191,7 +165,7 @@ mod tests { use zebra_chain::{block::Block, serialization::ZcashDeserializeInto}; use zebra_test::prelude::*; - use crate::tests::FakeChainHelper; + use crate::tests::{FakeChainHelper, Prepare}; use self::assert_eq; use super::*; @@ -204,11 +178,7 @@ mod tests { impl IntoQueued for Arc { fn into_queued(self) -> QueuedBlock { let (rsp_tx, _) = oneshot::channel(); - - QueuedBlock { - block: self, - rsp_tx, - } + (self.prepare(), rsp_tx) } } @@ -253,22 +223,22 @@ mod tests { // Dequeueing the first block removes 1 block from each list let children = queue.dequeue_children(parent); 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!(1, queue.by_parent.len()); assert_eq!(1, queue.by_height.len()); // Dequeueing the children of the first block removes both of the other // blocks, and empties all lists - let parent = children[0].block.hash(); + let parent = children[0].0.block.hash(); let children = queue.dequeue_children(parent); assert_eq!(2, children.len()); assert!(children .iter() - .any(|QueuedBlock { block, .. }| block == &child1)); + .any(|(block, _)| block.hash == child1.hash())); assert!(children .iter() - .any(|QueuedBlock { block, .. }| block == &child2)); + .any(|(block, _)| block.hash == child2.hash())); assert_eq!(0, queue.blocks.len()); assert_eq!(0, queue.by_parent.len()); assert_eq!(0, queue.by_height.len()); diff --git a/zebra-state/src/service/tests.rs b/zebra-state/src/service/tests.rs index 1acf1590..3442cad4 100644 --- a/zebra-state/src/service/tests.rs +++ b/zebra-state/src/service/tests.rs @@ -17,7 +17,7 @@ async fn populated_state( ) -> BoxService { let requests = blocks .into_iter() - .map(|block| Request::CommitFinalizedBlock { block }); + .map(|block| Request::CommitFinalizedBlock(block.into())); let config = Config::ephemeral(); let network = Network::Mainnet; diff --git a/zebra-state/src/service/utxo.rs b/zebra-state/src/service/utxo.rs index 4ade4c53..099dee24 100644 --- a/zebra-state/src/service/utxo.rs +++ b/zebra-state/src/service/utxo.rs @@ -35,7 +35,7 @@ impl PendingUtxos { /// Notify all utxo requests waiting for the `transparent::Output` pointed to /// 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) { // Adding the outpoint as a field lets us crossreference // 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 - /// `block` that the output has arrived. - pub fn check_block(&mut self, block: &Block) { + /// Check the list of pending UTXO requests against the supplied UTXO index. + pub fn check_against(&mut self, utxos: &HashMap) { + 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() { return; } @@ -60,7 +67,7 @@ impl PendingUtxos { index: index as _, }; - self.respond(outpoint, output.clone()); + self.respond(&outpoint, output.clone()); } } } diff --git a/zebra-state/src/tests.rs b/zebra-state/src/tests.rs index 86ea3323..dee35a69 100644 --- a/zebra-state/src/tests.rs +++ b/zebra-state/src/tests.rs @@ -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 zebra_chain::{ @@ -11,6 +11,35 @@ use zebra_chain::{ use super::*; +/// Mocks computation done during semantic validation +pub trait Prepare { + fn prepare(self) -> PreparedBlock; +} + +impl Prepare for Arc { + 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 pub trait FakeChainHelper { fn make_fake_child(&self) -> Arc; diff --git a/zebra-state/tests/basic.rs b/zebra-state/tests/basic.rs index 27b93cb4..6a7a6f67 100644 --- a/zebra-state/tests/basic.rs +++ b/zebra-state/tests/basic.rs @@ -17,7 +17,7 @@ static COMMIT_FINALIZED_BLOCK_MAINNET: Lazy