fix: Avoid creating a CheckpointVerifier if it is not needed

Also:
* make sure the checkpoints cover all pre-Sapling blocks
* do some minor refactoring to reduce duplicate code
This commit is contained in:
teor 2020-07-29 20:51:26 +10:00
parent fbeae99328
commit 5b9703f95a
6 changed files with 168 additions and 85 deletions

View File

@ -14,7 +14,8 @@
#[cfg(test)] #[cfg(test)]
mod tests; mod tests;
use crate::checkpoint::CheckpointVerifier; use crate::checkpoint::{CheckpointList, CheckpointVerifier};
use crate::parameters::NetworkUpgrade::Sapling;
use futures_util::FutureExt; use futures_util::FutureExt;
use std::{ use std::{
@ -36,23 +37,39 @@ use zebra_chain::Network;
/// Used to identify unexpected high blocks. /// Used to identify unexpected high blocks.
const MAX_EXPECTED_BLOCK_GAP: u32 = 100_000; const MAX_EXPECTED_BLOCK_GAP: u32 = 100_000;
struct ChainVerifier<BV, S> { struct ChainVerifier<BV, S>
where
BV: Service<Arc<Block>, Response = BlockHeaderHash, Error = Error> + Send + Clone + 'static,
BV::Future: Send + 'static,
S: Service<zebra_state::Request, Response = zebra_state::Response, Error = Error>
+ Send
+ Clone
+ 'static,
S::Future: Send + 'static,
{
/// The underlying `BlockVerifier`, possibly wrapped in other services. /// The underlying `BlockVerifier`, possibly wrapped in other services.
block_verifier: BV, block_verifier: BV,
/// The underlying `CheckpointVerifier`, wrapped in a buffer, so we can /// The underlying `CheckpointVerifier`, wrapped in a buffer, so we can
/// clone and share it with futures. /// clone and share it with futures.
checkpoint_verifier: Buffer<CheckpointVerifier, Arc<Block>>, ///
/// None if all the checkpoints have been verified.
checkpoint_verifier: Option<Buffer<CheckpointVerifier, Arc<Block>>>,
/// The maximum checkpoint height for `checkpoint_verifier`. /// The maximum checkpoint height for `checkpoint_verifier`.
max_checkpoint_height: BlockHeight, ///
/// None if all the checkpoints have been verified.
max_checkpoint_height: Option<BlockHeight>,
/// The underlying `ZebraState`, possibly wrapped in other services. /// The underlying `ZebraState`, possibly wrapped in other services.
state_service: S, state_service: S,
/// The last block height. Used for debugging. /// 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.
/// Not updated for unexpected high blocks. /// Not updated for unexpected high blocks.
last_block_height: BlockHeight, last_block_height: Option<BlockHeight>,
} }
/// The error type for the ChainVerifier Service. /// The error type for the ChainVerifier Service.
@ -86,30 +103,31 @@ where
fn call(&mut self, block: Arc<Block>) -> Self::Future { fn call(&mut self, block: Arc<Block>) -> Self::Future {
// TODO(jlusby): Error = Report, handle errors from state_service. // TODO(jlusby): Error = Report, handle errors from state_service.
let mut block_verifier = self.block_verifier.clone();
let mut checkpoint_verifier = self.checkpoint_verifier.clone();
let mut state_service = self.state_service.clone();
let max_checkpoint_height = self.max_checkpoint_height;
let span = tracing::debug_span!( let span = tracing::debug_span!(
"block_verify", "block_verify",
height = ?block.coinbase_height(), height = ?block.coinbase_height(),
hash = ?BlockHeaderHash::from(block.as_ref()) hash = ?block.hash()
); );
let height = block.coinbase_height(); let block_height = block.coinbase_height();
let mut block_verifier = self.block_verifier.clone();
let mut state_service = self.state_service.clone();
let checkpoint_verifier = self.checkpoint_verifier.clone();
let max_checkpoint_height = self.max_checkpoint_height;
// Log an info-level message on unexpected high blocks // Log an info-level message on unexpected high blocks
let is_unexpected_high_block = match height { let is_unexpected_high_block = match (block_height, self.last_block_height) {
Some(BlockHeight(height)) (Some(BlockHeight(block_height)), Some(BlockHeight(last_block_height)))
if (height > self.last_block_height.0 + MAX_EXPECTED_BLOCK_GAP) => if (block_height > last_block_height + MAX_EXPECTED_BLOCK_GAP) =>
{ {
true true
} }
Some(height) => { (Some(block_height), _) => {
// Update the last height if the block height was expected // Update the last height if the block height was expected
self.last_block_height = height; self.last_block_height = Some(block_height);
false false
} }
// The other cases are covered by the verifiers
_ => false, _ => false,
}; };
@ -118,33 +136,27 @@ where
// to use BlockVerifier, CheckpointVerifier, or both. // to use BlockVerifier, CheckpointVerifier, or both.
// Call a verifier based on the block height and checkpoints. // Call a verifier based on the block height and checkpoints.
match height { if is_higher_than_max_checkpoint(block_height, max_checkpoint_height) {
Some(height) if (height <= max_checkpoint_height) => { // Log an info-level message on early high blocks.
checkpoint_verifier // The sync service rejects most of these blocks, but we
.ready_and() // still want to know if a large number get through.
.await? if is_unexpected_high_block {
.call(block.clone()) tracing::info!(?block_height, "unexpected high block");
.await?
} }
_ => {
// Log an info-level message on early high blocks.
// The sync service rejects most of these blocks, but we
// still want to know if a large number get through.
if is_unexpected_high_block {
tracing::info!(?height, "unexpected high block");
}
block_verifier block_verifier
.ready_and() .ready_and()
.await? .await?
.call(block.clone()) .call(block.clone())
.await? .await?;
} } else {
}; checkpoint_verifier
.expect("Missing checkpoint verifier: verifier must be Some if max checkpoint height is Some")
// TODO(teor): .ready_and()
// - handle chain reorgs .await?
// - adjust state_service "unique block height" conditions .call(block.clone())
.await?;
}
let add_block = state_service let add_block = state_service
.ready_and() .ready_and()
@ -161,10 +173,25 @@ where
} }
} }
/// 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<BlockHeight>,
max_checkpoint_height: Option<BlockHeight>,
) -> 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 `network` and `state_service`. /// Return a chain verification service, using `network` and `state_service`.
/// ///
/// Gets the initial tip from the state service, and uses it to create a block /// Gets the initial tip from the state service, and uses it to create a block
/// verifier and checkpoint verifier. /// verifier and checkpoint verifier (if needed).
/// ///
/// This function should only be called once for a particular state service. If /// 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 /// you need shared block or checkpoint verfiers, create them yourself, and pass
@ -191,26 +218,24 @@ where
+ 'static, + 'static,
S::Future: Send + 'static, S::Future: Send + 'static,
{ {
let initial_tip = zebra_state::initial_tip(state_service.clone()) let initial_tip = zebra_state::current_tip(state_service.clone())
.await .await
.expect("State service poll_ready is Ok"); .expect("State service poll_ready is Ok");
let height = initial_tip.clone().map(|b| b.coinbase_height()).flatten();
let hash = initial_tip.clone().map(|b| b.hash());
tracing::debug!(
?network,
?height,
?hash,
"initialising ChainVerifier with network and initial tip"
);
let block_verifier = crate::block::init(state_service.clone()); let block_verifier = crate::block::init(state_service.clone());
let checkpoint_verifier = CheckpointVerifier::new(network, initial_tip); let checkpoint_list = CheckpointList::new(network);
init_from_verifiers(block_verifier, checkpoint_verifier, state_service, height) init_from_verifiers(
network,
block_verifier,
Some(checkpoint_list),
state_service,
initial_tip,
)
} }
/// Return a chain verification service, using the provided verifier, state /// Return a chain verification service, using the provided block verifier,
/// services, and initial tip height. /// checkpoint list, and state service.
/// ///
/// The chain verifier holds a state service of type `S`, used as context for /// The chain verifier holds a state service of type `S`, used as context for
/// block validation and to which newly verified blocks will be committed. This /// block validation and to which newly verified blocks will be committed. This
@ -221,14 +246,22 @@ where
/// verification service. /// verification service.
/// ///
/// This function should only be called once for a particular state service and /// This function should only be called once for a particular state service and
/// verifiers (and the result be shared, cloning if needed). Constructing /// block verifier (and the result be shared, cloning if needed). Constructing
/// multiple services from the same underlying state might cause synchronisation /// multiple services from the same underlying state might cause synchronisation
/// bugs. /// bugs.
pub fn init_from_verifiers<BV, S>( ///
/// Panics:
///
/// Panics if the `checkpoint_verifier` 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<BV, S>(
network: Network,
block_verifier: BV, block_verifier: BV,
checkpoint_verifier: CheckpointVerifier, checkpoint_list: Option<CheckpointList>,
state_service: S, state_service: S,
initial_tip_height: Option<BlockHeight>, initial_tip: Option<Arc<Block>>,
) -> impl Service< ) -> impl Service<
Arc<Block>, Arc<Block>,
Response = BlockHeaderHash, Response = BlockHeaderHash,
@ -246,14 +279,36 @@ where
+ 'static, + 'static,
S::Future: Send + 'static, S::Future: Send + 'static,
{ {
let max_checkpoint_height = checkpoint_verifier.list().max_height(); let max_checkpoint_height = checkpoint_list.clone().map(|c| c.max_height());
tracing::debug!( 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, ?max_checkpoint_height,
"initialising ChainVerifier with max checkpoint height" ?initial_height,
?initial_hash,
"initialising ChainVerifier"
); );
// Wrap the checkpoint verifier in a buffer, so we can share it let sapling_activation = Sapling
let checkpoint_verifier = Buffer::new(checkpoint_verifier, 1); .activation_height(network)
.expect("Unexpected network upgrade info: Sapling must have an activation height");
let (checkpoint_verifier, max_checkpoint_height) = 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, None),
// No list, no checkpoint verifier
(_, None, _) => (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(_)) => (Some(Buffer::new(CheckpointVerifier::from_checkpoint_list(list, initial_tip), 1)), max_checkpoint_height),
};
Buffer::new( Buffer::new(
ChainVerifier { ChainVerifier {
@ -261,10 +316,7 @@ where
checkpoint_verifier, checkpoint_verifier,
max_checkpoint_height, max_checkpoint_height,
state_service, state_service,
// We might not have the genesis block yet, but that's ok, last_block_height: initial_height,
// because this field is only used for debugging unexpected high
// blocks.
last_block_height: initial_tip_height.unwrap_or(BlockHeight(0)),
}, },
1, 1,
) )

View File

@ -44,6 +44,7 @@ pub fn block_no_transactions() -> Block {
/// Also creates a new block verfier and checkpoint verifier, so it can /// Also creates a new block verfier and checkpoint verifier, so it can
/// initialise the chain verifier. /// initialise the chain verifier.
fn verifiers_from_checkpoint_list( fn verifiers_from_checkpoint_list(
network: Network,
checkpoint_list: CheckpointList, checkpoint_list: CheckpointList,
) -> ( ) -> (
impl Service< impl Service<
@ -65,11 +66,10 @@ fn verifiers_from_checkpoint_list(
) { ) {
let state_service = zebra_state::in_memory::init(); let state_service = zebra_state::in_memory::init();
let block_verifier = crate::block::init(state_service.clone()); let block_verifier = crate::block::init(state_service.clone());
let checkpoint_verifier =
crate::checkpoint::CheckpointVerifier::from_checkpoint_list(checkpoint_list, None);
let chain_verifier = super::init_from_verifiers( let chain_verifier = super::init_from_verifiers(
network,
block_verifier, block_verifier,
checkpoint_verifier, Some(checkpoint_list),
state_service.clone(), state_service.clone(),
None, None,
); );
@ -99,7 +99,7 @@ fn verifiers_from_network(
+ Clone + Clone
+ 'static, + 'static,
) { ) {
verifiers_from_checkpoint_list(CheckpointList::new(network)) verifiers_from_checkpoint_list(network, CheckpointList::new(network))
} }
#[tokio::test] #[tokio::test]
@ -131,7 +131,7 @@ async fn verify_block() -> Result<(), Report> {
checkpoint_data.iter().cloned().collect(); checkpoint_data.iter().cloned().collect();
let checkpoint_list = CheckpointList::from_list(checkpoint_list).map_err(|e| eyre!(e))?; let checkpoint_list = CheckpointList::from_list(checkpoint_list).map_err(|e| eyre!(e))?;
let (mut chain_verifier, _) = verifiers_from_checkpoint_list(checkpoint_list); let (mut chain_verifier, _) = verifiers_from_checkpoint_list(Mainnet, checkpoint_list);
/// SPANDOC: Make sure the verifier service is ready for block 0 /// SPANDOC: Make sure the verifier service is ready for block 0
let ready_verifier_service = chain_verifier.ready_and().await.map_err(|e| eyre!(e))?; let ready_verifier_service = chain_verifier.ready_and().await.map_err(|e| eyre!(e))?;
@ -434,7 +434,7 @@ async fn continuous_blockchain() -> Result<(), Report> {
.collect(); .collect();
let checkpoint_list = CheckpointList::from_list(checkpoint_list).map_err(|e| eyre!(e))?; let checkpoint_list = CheckpointList::from_list(checkpoint_list).map_err(|e| eyre!(e))?;
let (mut chain_verifier, _) = verifiers_from_checkpoint_list(checkpoint_list); let (mut chain_verifier, _) = verifiers_from_checkpoint_list(Mainnet, checkpoint_list);
let mut handles = FuturesUnordered::new(); let mut handles = FuturesUnordered::new();

View File

@ -196,6 +196,8 @@ impl CheckpointVerifier {
} }
} }
/// Return the checkpoint list for this verifier.
#[allow(dead_code)]
pub(crate) fn list(&self) -> &CheckpointList { pub(crate) fn list(&self) -> &CheckpointList {
&self.checkpoint_list &self.checkpoint_list
} }

View File

@ -34,7 +34,7 @@ type Error = Box<dyn error::Error + Send + Sync + 'static>;
/// ///
/// This is actually a bijective map, but since it is read-only, we use a /// This is actually a bijective map, but since it is read-only, we use a
/// BTreeMap, and do the value uniqueness check on initialisation. /// BTreeMap, and do the value uniqueness check on initialisation.
#[derive(Debug)] #[derive(Clone, Debug, Eq, Hash, PartialEq)]
pub(crate) struct CheckpointList(BTreeMap<BlockHeight, BlockHeaderHash>); pub(crate) struct CheckpointList(BTreeMap<BlockHeight, BlockHeaderHash>);
impl FromStr for CheckpointList { impl FromStr for CheckpointList {
@ -176,6 +176,8 @@ impl CheckpointList {
mod tests { mod tests {
use super::*; use super::*;
use crate::parameters::NetworkUpgrade::Sapling;
use std::sync::Arc; use std::sync::Arc;
use zebra_chain::{block::Block, serialization::ZcashDeserialize}; use zebra_chain::{block::Block, serialization::ZcashDeserialize};
@ -394,9 +396,9 @@ mod tests {
Ok(()) Ok(())
} }
/// Parse the hard-coded Mainnet and Testnet lists /// Parse and check the hard-coded Mainnet and Testnet lists
#[test] #[test]
fn checkpoint_list_hard_coded() -> Result<(), Error> { fn checkpoint_list_load_hard_coded() -> Result<(), Error> {
zebra_test::init(); zebra_test::init();
let _: CheckpointList = MAINNET_CHECKPOINTS let _: CheckpointList = MAINNET_CHECKPOINTS
@ -411,4 +413,32 @@ mod tests {
Ok(()) Ok(())
} }
#[test]
fn checkpoint_list_hard_coded_sapling_mainnet() -> Result<(), Error> {
checkpoint_list_hard_coded_sapling(Mainnet)
}
#[test]
fn checkpoint_list_hard_coded_sapling_testnet() -> Result<(), Error> {
checkpoint_list_hard_coded_sapling(Testnet)
}
/// Check that the hard-coded lists cover the Sapling network upgrade
fn checkpoint_list_hard_coded_sapling(network: Network) -> Result<(), Error> {
zebra_test::init();
let sapling_activation = Sapling
.activation_height(network)
.expect("Unexpected network upgrade info: Sapling must have an activation height");
let list = CheckpointList::new(network);
assert!(
list.max_height() >= sapling_activation,
"Pre-Sapling blocks must be verified by checkpoints"
);
Ok(())
}
} }

View File

@ -732,8 +732,7 @@ async fn hard_coded_mainnet() -> Result<(), Report> {
checkpoint_verifier.target_checkpoint_height(), checkpoint_verifier.target_checkpoint_height(),
WaitingForBlocks WaitingForBlocks
); );
// The lists will get bigger over time, so we just pick a recent height assert!(checkpoint_verifier.checkpoint_list.max_height() > BlockHeight(0));
assert!(checkpoint_verifier.checkpoint_list.max_height() > BlockHeight(900_000));
/// SPANDOC: Make sure the verifier service is ready /// SPANDOC: Make sure the verifier service is ready
let ready_verifier_service = checkpoint_verifier let ready_verifier_service = checkpoint_verifier

View File

@ -155,12 +155,12 @@ type Error = Box<dyn error::Error + Send + Sync + 'static>;
/// ///
/// If there is no tip, returns `Ok(None)`. /// If there is no tip, returns `Ok(None)`.
/// Returns an error if `state.poll_ready` errors. /// Returns an error if `state.poll_ready` errors.
pub async fn initial_tip<S>(state: S) -> Result<Option<Arc<Block>>, Report> pub async fn current_tip<S>(state: S) -> Result<Option<Arc<Block>>, Report>
where where
S: Service<Request, Response = Response, Error = Error> + Send + Clone + 'static, S: Service<Request, Response = Response, Error = Error> + Send + Clone + 'static,
S::Future: Send + 'static, S::Future: Send + 'static,
{ {
let initial_tip_hash = state let current_tip_hash = state
.clone() .clone()
.ready_and() .ready_and()
.await .await
@ -173,7 +173,7 @@ where
}) })
.ok(); .ok();
let initial_tip_block = match initial_tip_hash { let current_tip_block = match current_tip_hash {
Some(hash) => state Some(hash) => state
.clone() .clone()
.ready_and() .ready_and()
@ -189,7 +189,7 @@ where
None => None, None => None,
}; };
Ok(initial_tip_block) Ok(current_tip_block)
} }
#[cfg(test)] #[cfg(test)]