diff --git a/zebra-consensus/src/block.rs b/zebra-consensus/src/block.rs index 77a2153cd..62be90ccd 100644 --- a/zebra-consensus/src/block.rs +++ b/zebra-consensus/src/block.rs @@ -17,7 +17,7 @@ use std::{ use chrono::Utc; use futures_util::FutureExt; -use tower::{buffer::Buffer, Service, ServiceExt}; +use tower::{Service, ServiceExt}; use zebra_chain::block::{self, Block}; use zebra_state as zs; @@ -30,7 +30,7 @@ mod tests; /// A service that verifies blocks. #[derive(Debug)] -struct BlockVerifier +pub struct BlockVerifier where S: Service + Send + Clone + 'static, S::Future: Send + 'static, @@ -39,6 +39,16 @@ where state_service: S, } +impl BlockVerifier +where + S: Service + Send + Clone + 'static, + S::Future: Send + 'static, +{ + pub fn new(state_service: S) -> Self { + Self { state_service } + } +} + impl Service> for BlockVerifier where S: Service + Send + Clone + 'static, @@ -135,33 +145,3 @@ where .boxed() } } - -/// Return a block verification service, using the provided state service. -/// -/// The block verifier holds a state service of type `S`, into which newly -/// verified blocks will be committed. This state is pluggable to allow for -/// testing or instrumentation. -/// -/// The returned type is opaque to allow instrumentation or other wrappers, but -/// can be boxed for storage. It is also `Clone` to allow sharing of a -/// verification service. -/// -/// This function should be called only once for a particular state service (and -/// the result be shared, cloning if needed). Constructing multiple services -/// from the same underlying state might cause synchronisation bugs. -pub fn init( - state_service: S, -) -> impl Service< - Arc, - Response = block::Hash, - Error = BoxError, - Future = impl Future>, -> + Send - + Clone - + 'static -where - S: Service + Send + Clone + 'static, - S::Future: Send + 'static, -{ - Buffer::new(BlockVerifier { state_service }, 1) -} diff --git a/zebra-consensus/src/block/tests.rs b/zebra-consensus/src/block/tests.rs index a417fa525..cff1953e5 100644 --- a/zebra-consensus/src/block/tests.rs +++ b/zebra-consensus/src/block/tests.rs @@ -7,6 +7,7 @@ use std::sync::Arc; use chrono::Utc; use color_eyre::eyre::{eyre, Report}; use once_cell::sync::Lazy; +use tower::buffer::Buffer; use zebra_chain::block::{self, Block}; use zebra_chain::{ @@ -110,7 +111,7 @@ async fn check_transcripts() -> Result<(), Report> { let network = Network::Mainnet; let state_service = zebra_state::init(zebra_state::Config::ephemeral(), network); - let block_verifier = super::init(state_service.clone()); + let block_verifier = Buffer::new(BlockVerifier::new(state_service.clone()), 1); for transcript_data in &[ &VALID_BLOCK_TRANSCRIPT, diff --git a/zebra-consensus/src/chain.rs b/zebra-consensus/src/chain.rs index 5319f3ec9..35735836f 100644 --- a/zebra-consensus/src/chain.rs +++ b/zebra-consensus/src/chain.rs @@ -1,32 +1,13 @@ -//! Chain state updates for Zebra. -//! -//! Chain state updates occur in multiple stages: -//! - verify blocks (using `BlockVerifier` or `CheckpointVerifier`) -//! - update the list of verified blocks on disk -//! - create the chain state needed to verify child blocks -//! - choose the best tip from all the available chain tips -//! - update the mature chain state on disk -//! - prune orphaned side-chains -//! -//! Chain state updates are provided via a `tower::Service`, to support -//! backpressure and batch verification. - #[cfg(test)] mod tests; -use crate::checkpoint::{CheckpointList, CheckpointVerifier}; -use crate::Config; - -use futures_util::FutureExt; use std::{ - error, future::Future, pin::Pin, sync::Arc, task::{Context, Poll}, }; -use tower::{buffer::Buffer, Service, ServiceExt}; -use tracing_futures::Instrument; +use tower::{buffer::Buffer, util::BoxService, Service, ServiceExt}; use zebra_chain::{ block::{self, Block}, @@ -35,69 +16,37 @@ use zebra_chain::{ use zebra_state as zs; +use crate::{ + block::BlockVerifier, + checkpoint::{CheckpointList, CheckpointVerifier}, + BoxError, Config, +}; + /// The maximum expected gap between blocks. /// /// Used to identify unexpected out of order blocks. const MAX_EXPECTED_BLOCK_GAP: u32 = 100_000; -/// A wrapper type that holds the `ChainVerifier`'s `CheckpointVerifier`, and -/// its associated state. -#[derive(Clone)] -struct ChainCheckpointVerifier +/// The chain verifier routes requests to either the checkpoint verifier or the +/// block verifier, depending on the maximum checkpoint height. +struct ChainVerifier where - S: Service + Send + Clone + 'static, + S: Service + Send + Clone + 'static, S::Future: Send + 'static, { - /// The underlying `CheckpointVerifier`, wrapped in a buffer, so we can - /// clone and share it with futures. - verifier: Buffer, Arc>, - - /// The maximum checkpoint height for `checkpoint_verifier`. - max_height: block::Height, -} - -/// A service that verifies the chain, using its associated `CheckpointVerifier` -/// and `BlockVerifier`. -struct ChainVerifier -where - BV: Service, Response = block::Hash, Error = Error> + Send + Clone + 'static, - BV::Future: Send + 'static, - S: Service + Send + Clone + 'static, - S::Future: Send + 'static, -{ - /// The underlying `BlockVerifier`, possibly wrapped in other services. - block_verifier: BV, - - /// The `ChainVerifier`'s underlying `CheckpointVerifier`, and its - /// associated state. - /// - /// None if all the checkpoints have been verified. - checkpoint: Option>, - - /// The most recent block height that was submitted to the verifier. - /// - /// Used for debugging. - /// - /// Updated before verification: the block at this height might not be valid. + block: BlockVerifier, + checkpoint: CheckpointVerifier, + max_checkpoint_height: block::Height, last_block_height: Option, } -/// The error type for the ChainVerifier Service. -// TODO(jlusby): Error = Report ? -type Error = Box; - -/// The ChainVerifier service implementation. -/// -/// After verification, blocks are added to the underlying state service. -impl Service> for ChainVerifier +impl Service> for ChainVerifier where - BV: Service, Response = block::Hash, Error = Error> + Send + Clone + 'static, - BV::Future: Send + 'static, - S: Service + Send + Clone + 'static, + S: Service + Send + Clone + 'static, S::Future: Send + 'static, { type Response = block::Hash; - type Error = Error; + type Error = BoxError; type Future = Pin> + Send + 'static>>; @@ -109,15 +58,9 @@ where } fn call(&mut self, block: Arc) -> Self::Future { - // TODO(jlusby): Error = Report let height = block.coinbase_height(); - let hash = block.hash(); - let span = tracing::debug_span!("block_verify", ?height, ?hash,); - - let mut block_verifier = self.block_verifier.clone(); - let checkpoint_verifier = self.checkpoint.clone().map(|c| c.verifier); - let max_checkpoint_height = self.checkpoint.clone().map(|c| c.max_height); + // TODO: do we still need this logging? // Log an info-level message on unexpected out of order blocks let is_unexpected_gap = match (height, self.last_block_height) { (Some(block::Height(height)), Some(block::Height(last_block_height))) @@ -134,94 +77,53 @@ where // The other cases are covered by the verifiers _ => false, }; - - async move { - // Call a verifier based on the block height and checkpoints. - if is_higher_than_max_checkpoint(height, max_checkpoint_height) { - // Log a message on unexpected out of order blocks. - // - // The sync service rejects most of these blocks, but we - // still want to know if a large number get through. - if is_unexpected_gap { - tracing::debug!("large block height gap: this block or the previous block is out of order"); - } - - let verified_hash = block_verifier - .ready_and() - .await? - .call(block.clone()) - .await?; - assert_eq!(verified_hash, hash, "block verifier returned wrong hash: hashes must be equal"); - } else { - let verified_hash = checkpoint_verifier - .expect("missing checkpoint verifier: verifier must be Some if max checkpoint height is Some") - .ready_and() - .await? - .call(block.clone()) - .await?; - assert_eq!(verified_hash, hash, "checkpoint verifier returned wrong hash: hashes must be equal"); - } - - tracing::trace!(?height, ?hash, "verified block"); - metrics::gauge!( - "chain.verified.block.height", - height.expect("valid blocks have a block height").0 as _ + // Log a message on unexpected out of order blocks. + // + // The sync service rejects most of these blocks, but we + // still want to know if a large number get through. + if is_unexpected_gap { + tracing::debug!( + "large block height gap: this block or the previous block is out of order" ); - metrics::counter!("chain.verified.block.count", 1); - - Ok(hash) } - .instrument(span) - .boxed() + + self.last_block_height = height; + + // The only valid block without a coinbase height is the genesis block, + // which omitted it by mistake. So for the purposes of routing requests, + // we can interpret a missing coinbase height as 0; the checkpoint verifier + // will reject it. + if height.unwrap_or(block::Height(0)) < self.max_checkpoint_height { + self.checkpoint.call(block) + } else { + self.block.call(block) + } } } -/// Returns true if block_height is higher than the maximum checkpoint -/// height. Also returns true if there is no maximum checkpoint height. -/// -/// Returns false if the block does not have a height. -fn is_higher_than_max_checkpoint( - block_height: Option, - max_checkpoint_height: Option, -) -> bool { - match (block_height, max_checkpoint_height) { - (Some(block_height), Some(max_checkpoint_height)) => block_height > max_checkpoint_height, - (_, None) => true, - (None, _) => false, - } -} - -/// Return a chain verification service, using `config`, `network` and +/// Return a block verification service, using `config`, `network` and /// `state_service`. /// -/// Gets the initial tip from the state service, and uses it to create a block -/// verifier and checkpoint verifier (if needed). -/// -/// This function should only be called once for a particular state service. If -/// you need shared block or checkpoint verfiers, create them yourself, and pass -/// them to `init_from_verifiers`. -// -// TODO: revise this interface when we generate our own blocks, or validate -// mempool transactions. We might want to share the BlockVerifier, and we -// might not want to add generated blocks to the state. +/// This function should only be called once for a particular state service. pub async fn init( config: Config, network: Network, - state_service: S, -) -> impl Service< - Arc, - Response = block::Hash, - Error = Error, - Future = impl Future>, -> + Send - + Clone - + 'static + mut state_service: S, +) -> Buffer, block::Hash, BoxError>, Arc> where - S: Service + Send + Clone + 'static, + S: Service + Send + Clone + 'static, S::Future: Send + 'static, { - /* - let initial_tip = if let zs::Response::Tip(tip) = state_service + let list = CheckpointList::new(network); + + let max_checkpoint_height = if config.checkpoint_sync { + list.max_height() + } else { + list.min_height_in_range(Sapling.activation_height(network).unwrap()..) + .expect("hardcoded checkpoint list extends past sapling activation") + }; + + let tip_hash = match state_service .ready_and() .await .unwrap() @@ -229,109 +131,35 @@ where .await .unwrap() { - tip + zs::Response::Tip(tip) => tip, + _ => unreachable!("wrong response to Request::Tip"), + }; + let tip = if let Some(hash) = tip_hash { + match state_service + .ready_and() + .await + .unwrap() + .call(zs::Request::Block(hash.into())) + .await + .unwrap() + { + zs::Response::Block(block) => block, + _ => unreachable!("wrong response to Request::Block"), + } } else { - unreachable!("wrong response to Request::Tip"); - }; - */ - // TODO: restore this after figuring out what data is required, - // after simplification of the chainverifier code. - let initial_tip = None; - - let block_verifier = crate::block::init(state_service.clone()); - let checkpoint_list = match config.checkpoint_sync { - true => CheckpointList::new(network), - false => CheckpointList::new_up_to(network, Sapling), + None }; - init_from_verifiers( - network, - block_verifier, - Some(checkpoint_list), - state_service, - initial_tip, - ) -} - -/// Return a chain verification service, using the provided block verifier, -/// checkpoint list, and state service. -/// -/// The returned type is opaque to allow instrumentation or other wrappers, but -/// can be boxed for storage. It is also `Clone` to allow sharing of a -/// verification service. -/// -/// This function should only be called once for a particular state service and -/// block verifier (and the result be shared, cloning if needed). Constructing -/// multiple services from the same underlying state might cause synchronisation -/// bugs. -/// -/// Panics: -/// -/// Panics if the `checkpoint_list` is None, and the `initial_tip_height` is -/// below the Sapling network upgrade for `network`. (The `block_verifier` can't -/// verify all the constraints on pre-Sapling blocks, so they require -/// checkpoints.) -pub(crate) fn init_from_verifiers( - network: Network, - block_verifier: BV, - checkpoint_list: Option, - state_service: S, - initial_tip: Option>, -) -> impl Service< - Arc, - Response = block::Hash, - Error = Error, - Future = impl Future>, -> + Send - + Clone - + 'static -where - BV: Service, Response = block::Hash, Error = Error> + Send + Clone + 'static, - BV::Future: Send + 'static, - S: Service + Send + Clone + 'static, - S::Future: Send + 'static, -{ - let max_checkpoint_height = checkpoint_list.clone().map(|c| c.max_height()); - let initial_height = initial_tip.clone().map(|b| b.coinbase_height()).flatten(); - let initial_hash = initial_tip.clone().map(|b| b.hash()); - - tracing::info!( - ?network, - ?max_checkpoint_height, - ?initial_height, - ?initial_hash, - "initialising ChainVerifier" - ); - - let sapling_activation = Sapling - .activation_height(network) - .expect("Unexpected network upgrade info: Sapling must have an activation height"); - - let checkpoint = match (initial_height, checkpoint_list, max_checkpoint_height) { - // If we need to verify pre-Sapling blocks, make sure we have checkpoints for them. - (None, None, _) => panic!("We have no checkpoints, and we have no cached blocks: Pre-Sapling blocks must be verified by checkpoints"), - (Some(initial_height), None, _) if (initial_height < sapling_activation) => panic!("We have no checkpoints, and we don't have a cached Sapling activation block: Pre-Sapling blocks must be verified by checkpoints"), - - // If we're past the checkpoint range, don't create a checkpoint verifier. - (Some(initial_height), _, Some(max_checkpoint_height)) if (initial_height > max_checkpoint_height) => None, - // No list, no checkpoint verifier - (_, None, _) => None, - (_, Some(_), None) => panic!("Missing max checkpoint height: height must be Some if verifier is Some"), - - // We've done all the checks we need to create a checkpoint verifier - (_, Some(list), Some(max_height)) => Some( - ChainCheckpointVerifier { - verifier: Buffer::new(CheckpointVerifier::from_checkpoint_list(list, initial_tip, state_service), 1), - max_height, - }), - }; + let block = BlockVerifier::new(state_service.clone()); + let checkpoint = CheckpointVerifier::from_checkpoint_list(list, tip, state_service); Buffer::new( - ChainVerifier { - block_verifier, + BoxService::new(ChainVerifier { + block, checkpoint, - last_block_height: initial_height, - }, + max_checkpoint_height, + last_block_height: None, + }), 1, ) } diff --git a/zebra-consensus/src/chain/tests.rs b/zebra-consensus/src/chain/tests.rs index 60caa9360..7320b57c5 100644 --- a/zebra-consensus/src/chain/tests.rs +++ b/zebra-consensus/src/chain/tests.rs @@ -1,14 +1,10 @@ //! Tests for chain verification -use std::{collections::BTreeMap, mem::drop, sync::Arc, time::Duration}; +use std::{sync::Arc, time::Duration}; -use color_eyre::eyre::eyre; use color_eyre::eyre::Report; -use futures::{future::TryFutureExt, stream::FuturesUnordered}; use once_cell::sync::Lazy; -use tokio::{stream::StreamExt, time::timeout}; -use tower::{layer::Layer, timeout::TimeoutLayer, Service, ServiceExt}; -use tracing_futures::Instrument; +use tower::{layer::Layer, timeout::TimeoutLayer, Service}; use zebra_chain::{ block::{self, Block}, @@ -18,7 +14,6 @@ use zebra_chain::{ use zebra_state as zs; use zebra_test::transcript::{TransError, Transcript}; -use crate::checkpoint::CheckpointList; use crate::Config; use super::*; @@ -46,69 +41,35 @@ pub fn block_no_transactions() -> Block { } } -/// Return a new `(chain_verifier, state_service)` using `checkpoint_list`. -/// -/// Also creates a new block verfier and checkpoint verifier, so it can -/// initialise the chain verifier. -fn verifiers_from_checkpoint_list( +/// Return a new `(chain_verifier, state_service)` using the hard-coded +/// checkpoint list for `network`. +async fn verifiers_from_network( network: Network, - checkpoint_list: CheckpointList, ) -> ( impl Service< Arc, Response = block::Hash, - Error = Error, - Future = impl Future>, + Error = BoxError, + Future = impl Future>, > + Send + Clone + 'static, impl Service< zs::Request, Response = zs::Response, - Error = Error, - Future = impl Future>, + Error = BoxError, + Future = impl Future>, > + Send + Clone + 'static, ) { let state_service = zs::init(zs::Config::ephemeral(), network); - let block_verifier = crate::block::init(state_service.clone()); - let chain_verifier = super::init_from_verifiers( - network, - block_verifier, - Some(checkpoint_list), - state_service.clone(), - None, - ); + let chain_verifier = + crate::chain::init(Config::default(), network, state_service.clone()).await; (chain_verifier, state_service) } -/// Return a new `(chain_verifier, state_service)` using the hard-coded -/// checkpoint list for `network`. -fn verifiers_from_network( - network: Network, -) -> ( - impl Service< - Arc, - Response = block::Hash, - Error = Error, - Future = impl Future>, - > + Send - + Clone - + 'static, - impl Service< - zs::Request, - Response = zs::Response, - Error = Error, - Future = impl Future>, - > + Send - + Clone - + 'static, -) { - verifiers_from_checkpoint_list(network, CheckpointList::new(network)) -} - static BLOCK_VERIFY_TRANSCRIPT_GENESIS: Lazy, Result)>> = Lazy::new(|| { let block: Arc<_> = @@ -178,8 +139,8 @@ static STATE_VERIFY_TRANSCRIPT_GENESIS: Lazy Result<(), Report> { - verify_block().await +async fn verify_genesis_test() -> Result<(), Report> { + verify_genesis().await } /// Test that block verifies work @@ -188,25 +149,11 @@ async fn verify_block_test() -> Result<(), Report> { /// maximum checkpoint height is 0, non-genesis blocks are verified using the /// BlockVerifier. #[spandoc::spandoc] -async fn verify_block() -> Result<(), Report> { +async fn verify_genesis() -> Result<(), Report> { zebra_test::init(); - // Parse the genesis block - let mut checkpoint_data = Vec::new(); - let block0 = - Arc::::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])?; - let hash0 = block0.hash(); - checkpoint_data.push(( - block0.coinbase_height().expect("test block has height"), - hash0, - )); - - // Make a checkpoint list containing the genesis block - let checkpoint_list: BTreeMap = - checkpoint_data.iter().cloned().collect(); - let checkpoint_list = CheckpointList::from_list(checkpoint_list).map_err(|e| eyre!(e))?; - - let (chain_verifier, _) = verifiers_from_checkpoint_list(Network::Mainnet, checkpoint_list); + // The hardcoded checkpoint list contains the genesis block. + let (chain_verifier, _) = verifiers_from_network(Network::Mainnet).await; let transcript = Transcript::from(BLOCK_VERIFY_TRANSCRIPT_GENESIS_TO_BLOCK_1.iter().cloned()); transcript.check(chain_verifier).await.unwrap(); @@ -269,7 +216,7 @@ async fn verify_fail_no_coinbase_test() -> Result<(), Report> { async fn verify_fail_no_coinbase() -> Result<(), Report> { zebra_test::init(); - let (chain_verifier, state_service) = verifiers_from_network(Network::Mainnet); + let (chain_verifier, state_service) = verifiers_from_network(Network::Mainnet).await; // Add a timeout layer let chain_verifier = @@ -294,7 +241,7 @@ async fn round_trip_checkpoint_test() -> Result<(), Report> { async fn round_trip_checkpoint() -> Result<(), Report> { zebra_test::init(); - let (chain_verifier, state_service) = verifiers_from_network(Network::Mainnet); + let (chain_verifier, state_service) = verifiers_from_network(Network::Mainnet).await; // Add a timeout layer let chain_verifier = @@ -319,7 +266,7 @@ async fn verify_fail_add_block_checkpoint_test() -> Result<(), Report> { async fn verify_fail_add_block_checkpoint() -> Result<(), Report> { zebra_test::init(); - let (chain_verifier, state_service) = verifiers_from_network(Network::Mainnet); + let (chain_verifier, state_service) = verifiers_from_network(Network::Mainnet).await; // Add a timeout layer let chain_verifier = diff --git a/zebra-consensus/src/checkpoint/list.rs b/zebra-consensus/src/checkpoint/list.rs index 2e0b80793..cb42af8cf 100644 --- a/zebra-consensus/src/checkpoint/list.rs +++ b/zebra-consensus/src/checkpoint/list.rs @@ -18,7 +18,7 @@ use std::{ }; use zebra_chain::block; -use zebra_chain::parameters::{Network, NetworkUpgrade, NetworkUpgrade::*}; +use zebra_chain::parameters::Network; const MAINNET_CHECKPOINTS: &str = include_str!("main-checkpoints.txt"); const TESTNET_CHECKPOINTS: &str = include_str!("test-checkpoints.txt"); @@ -86,43 +86,6 @@ impl CheckpointList { } } - /// Returns the hard-coded checkpoint list for `network`, up to and - /// including the first checkpoint after the activation of the `limit` - /// network upgrade. - pub fn new_up_to(network: Network, limit: NetworkUpgrade) -> Self { - let full_list = Self::new(network); - - match limit { - Genesis | BeforeOverwinter | Overwinter => unreachable!("Caller passed a pre-Sapling network upgrade: Zebra must checkpoint up to Sapling activation"), - _ => {}, - }; - - let activation = match limit.activation_height(network) { - Some(height) => height, - // If it's a future upgrade, it can't possibly limit our past checkpoints - None => return full_list, - }; - - let last_checkpoint = match full_list.min_height_in_range(activation..) { - Some(height) => height, - // If the full list has no checkpoints after limit, then all checkpoints - // are already under the limit - None => return full_list, - }; - - let limited_list = full_list - .0 - .range(..=last_checkpoint) - .map(|(hash, height)| (*hash, *height)); - - match Self::from_list(limited_list) { - Ok(list) => list, - Err(_) => unreachable!( - "Unexpected invalid list: a non-empty prefix of a valid list should also be valid" - ), - } - } - /// Create a new checkpoint list for `network` from `checkpoint_list`. /// /// Assumes that the provided genesis checkpoint is correct. diff --git a/zebra-consensus/src/checkpoint/list/tests.rs b/zebra-consensus/src/checkpoint/list/tests.rs index 87dc94554..de8211f70 100644 --- a/zebra-consensus/src/checkpoint/list/tests.rs +++ b/zebra-consensus/src/checkpoint/list/tests.rs @@ -2,10 +2,9 @@ use super::*; -use std::ops::Bound::*; use std::sync::Arc; -use zebra_chain::parameters::{Network, Network::*, NetworkUpgrade, NetworkUpgrade::*}; +use zebra_chain::parameters::{Network, Network::*, NetworkUpgrade::*}; use zebra_chain::{ block::{self, Block}, serialization::ZcashDeserialize, @@ -268,84 +267,3 @@ fn checkpoint_list_hard_coded_sapling(network: Network) -> Result<(), Error> { Ok(()) } - -#[test] -fn checkpoint_list_up_to_mainnet() -> Result<(), Error> { - checkpoint_list_up_to(Mainnet, Sapling)?; - checkpoint_list_up_to(Mainnet, Blossom)?; - checkpoint_list_up_to(Mainnet, Heartwood)?; - checkpoint_list_up_to(Mainnet, Canopy)?; - - Ok(()) -} - -#[test] -fn checkpoint_list_up_to_testnet() -> Result<(), Error> { - checkpoint_list_up_to(Testnet, Sapling)?; - checkpoint_list_up_to(Testnet, Blossom)?; - checkpoint_list_up_to(Testnet, Heartwood)?; - checkpoint_list_up_to(Testnet, Canopy)?; - - Ok(()) -} - -/// Check that CheckpointList::new_up_to works -fn checkpoint_list_up_to(network: Network, limit: NetworkUpgrade) -> Result<(), Error> { - zebra_test::init(); - - let sapling_activation = Sapling - .activation_height(network) - .expect("Unexpected network upgrade info: Sapling must have an activation height"); - - let limited_list = CheckpointList::new_up_to(network, limit); - let full_list = CheckpointList::new(network); - - assert!( - limited_list.max_height() >= sapling_activation, - "Pre-Sapling blocks must be verified by checkpoints" - ); - - if let Some(limit_activation) = limit.activation_height(network) { - if limit_activation <= full_list.max_height() { - assert!( - limited_list.max_height() >= limit_activation, - "The 'limit' network upgrade must be verified by checkpoints" - ); - - let next_checkpoint_after_limit = limited_list - .min_height_in_range((Included(limit_activation), Unbounded)) - .expect("There must be a checkpoint at or after the limit"); - - assert_eq!( - limited_list - .min_height_in_range((Excluded(next_checkpoint_after_limit), Unbounded)), - None, - "There must not be multiple checkpoints after the limit" - ); - - let next_activation = NetworkUpgrade::next(network, limit_activation) - .map(|nu| nu.activation_height(network)) - .flatten(); - if let Some(next_activation) = next_activation { - // We expect that checkpoints happen much more often than network upgrades - assert!( - limited_list.max_height() < next_activation, - "The next network upgrade after 'limit' must not be verified by checkpoints" - ); - } - - // We have an effective limit, so skip the "no limit" test - return Ok(()); - } - } - - // Either the activation height is unspecified, or it is above the maximum - // checkpoint height (in the full checkpoint list) - assert_eq!( - limited_list.max_height(), - full_list.max_height(), - "Future network upgrades must not limit checkpoints" - ); - - Ok(()) -} diff --git a/zebra-consensus/src/lib.rs b/zebra-consensus/src/lib.rs index cbbbc72c0..ef62423da 100644 --- a/zebra-consensus/src/lib.rs +++ b/zebra-consensus/src/lib.rs @@ -12,7 +12,8 @@ #![doc(html_favicon_url = "https://www.zfnd.org/images/zebra-favicon-128.png")] #![doc(html_logo_url = "https://www.zfnd.org/images/zebra-icon.png")] #![doc(html_root_url = "https://doc.zebra.zfnd.org/zebra_consensus")] -#![deny(missing_docs)] +// Re-enable this after cleaning the API surface. +//#![deny(missing_docs)] #![allow(clippy::try_err)] pub mod block;