feature: Implement a basic ChainVerifier service
The ChainVerifier service chooses between CheckpointVerifier and BlockVerifier, based on the block's height.
This commit is contained in:
parent
f1a0036824
commit
b8b1239ac4
|
@ -0,0 +1,178 @@
|
||||||
|
//! 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::CheckpointVerifier;
|
||||||
|
|
||||||
|
use futures_util::FutureExt;
|
||||||
|
use std::{
|
||||||
|
error,
|
||||||
|
future::Future,
|
||||||
|
pin::Pin,
|
||||||
|
sync::Arc,
|
||||||
|
task::{Context, Poll},
|
||||||
|
};
|
||||||
|
use tower::{buffer::Buffer, Service, ServiceExt};
|
||||||
|
|
||||||
|
use zebra_chain::block::{Block, BlockHeaderHash};
|
||||||
|
use zebra_chain::types::BlockHeight;
|
||||||
|
|
||||||
|
struct ChainVerifier<BV, S> {
|
||||||
|
/// The underlying `BlockVerifier`, possibly wrapped in other services.
|
||||||
|
block_verifier: BV,
|
||||||
|
|
||||||
|
/// The underlying `CheckpointVerifier`, wrapped in a buffer, so we can
|
||||||
|
/// clone and share it with futures.
|
||||||
|
checkpoint_verifier: Buffer<CheckpointVerifier, Arc<Block>>,
|
||||||
|
/// The maximum checkpoint height for `checkpoint_verifier`.
|
||||||
|
max_checkpoint_height: BlockHeight,
|
||||||
|
|
||||||
|
/// The underlying `ZebraState`, possibly wrapped in other services.
|
||||||
|
state_service: S,
|
||||||
|
}
|
||||||
|
|
||||||
|
/// The error type for the ChainVerifier Service.
|
||||||
|
// TODO(jlusby): Error = Report ?
|
||||||
|
type Error = Box<dyn error::Error + Send + Sync + 'static>;
|
||||||
|
|
||||||
|
/// The ChainVerifier service implementation.
|
||||||
|
///
|
||||||
|
/// After verification, blocks are added to the underlying state service.
|
||||||
|
impl<BV, S> Service<Arc<Block>> for 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,
|
||||||
|
{
|
||||||
|
type Response = BlockHeaderHash;
|
||||||
|
type Error = Error;
|
||||||
|
type Future =
|
||||||
|
Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;
|
||||||
|
|
||||||
|
fn poll_ready(&mut self, _: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
|
||||||
|
// We don't expect the state or verifiers to exert backpressure on our
|
||||||
|
// users, so we don't need to call `state_service.poll_ready()` here.
|
||||||
|
// (And we don't know which verifier to choose at this point, anyway.)
|
||||||
|
Poll::Ready(Ok(()))
|
||||||
|
}
|
||||||
|
|
||||||
|
fn call(&mut self, block: Arc<Block>) -> Self::Future {
|
||||||
|
// 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;
|
||||||
|
|
||||||
|
async move {
|
||||||
|
// Call a verifier based on the block height and checkpoints
|
||||||
|
//
|
||||||
|
// TODO(teor): for post-sapling checkpoint blocks, allow callers
|
||||||
|
// to use BlockVerifier, CheckpointVerifier, or both.
|
||||||
|
match block.coinbase_height() {
|
||||||
|
Some(height) if (height <= max_checkpoint_height) => {
|
||||||
|
checkpoint_verifier
|
||||||
|
.ready_and()
|
||||||
|
.await?
|
||||||
|
.call(block.clone())
|
||||||
|
.await?
|
||||||
|
}
|
||||||
|
Some(_) => {
|
||||||
|
block_verifier
|
||||||
|
.ready_and()
|
||||||
|
.await?
|
||||||
|
.call(block.clone())
|
||||||
|
.await?
|
||||||
|
}
|
||||||
|
None => return Err("Invalid block: must have a coinbase height".into()),
|
||||||
|
};
|
||||||
|
|
||||||
|
// TODO(teor):
|
||||||
|
// - handle chain reorgs
|
||||||
|
// - adjust state_service "unique block height" conditions
|
||||||
|
|
||||||
|
// `Tower::Buffer` requires a 1:1 relationship between `poll()`s
|
||||||
|
// and `call()`s, because it reserves a buffer slot in each
|
||||||
|
// `call()`.
|
||||||
|
let add_block = state_service
|
||||||
|
.ready_and()
|
||||||
|
.await?
|
||||||
|
.call(zebra_state::Request::AddBlock { block });
|
||||||
|
|
||||||
|
match add_block.await? {
|
||||||
|
zebra_state::Response::Added { hash } => Ok(hash),
|
||||||
|
_ => Err("adding block to zebra-state failed".into()),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
.boxed()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Return a chain verification service, using the provided verifier and state
|
||||||
|
/// services.
|
||||||
|
///
|
||||||
|
/// 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
|
||||||
|
/// 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 only be called once for a particular state service and
|
||||||
|
/// verifiers (and the result be shared, cloning if needed). Constructing
|
||||||
|
/// multiple services from the same underlying state might cause synchronisation
|
||||||
|
/// bugs.
|
||||||
|
//
|
||||||
|
// Only used by tests and other modules
|
||||||
|
#[allow(dead_code)]
|
||||||
|
pub fn init<BV, S>(
|
||||||
|
block_verifier: BV,
|
||||||
|
checkpoint_verifier: CheckpointVerifier,
|
||||||
|
state_service: S,
|
||||||
|
) -> impl Service<
|
||||||
|
Arc<Block>,
|
||||||
|
Response = BlockHeaderHash,
|
||||||
|
Error = Error,
|
||||||
|
Future = impl Future<Output = Result<BlockHeaderHash, Error>>,
|
||||||
|
> + Send
|
||||||
|
+ Clone
|
||||||
|
+ 'static
|
||||||
|
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,
|
||||||
|
{
|
||||||
|
let max_checkpoint_height = checkpoint_verifier.list().max_height();
|
||||||
|
// Wrap the checkpoint verifier in a buffer, so we can share it
|
||||||
|
let checkpoint_verifier = Buffer::new(checkpoint_verifier, 1);
|
||||||
|
|
||||||
|
Buffer::new(
|
||||||
|
ChainVerifier {
|
||||||
|
block_verifier,
|
||||||
|
checkpoint_verifier,
|
||||||
|
max_checkpoint_height,
|
||||||
|
state_service,
|
||||||
|
},
|
||||||
|
1,
|
||||||
|
)
|
||||||
|
}
|
|
@ -0,0 +1,368 @@
|
||||||
|
//! Tests for chain verification
|
||||||
|
|
||||||
|
use super::*;
|
||||||
|
|
||||||
|
use crate::checkpoint::CheckpointList;
|
||||||
|
|
||||||
|
use color_eyre::eyre::Report;
|
||||||
|
use color_eyre::eyre::{bail, eyre};
|
||||||
|
use futures::future::TryFutureExt;
|
||||||
|
use std::mem::drop;
|
||||||
|
use std::{collections::BTreeMap, sync::Arc, time::Duration};
|
||||||
|
use tokio::time::timeout;
|
||||||
|
use tower::{util::ServiceExt, Service};
|
||||||
|
|
||||||
|
use zebra_chain::block::{Block, BlockHeader};
|
||||||
|
use zebra_chain::serialization::ZcashDeserialize;
|
||||||
|
use zebra_chain::Network::{self, *};
|
||||||
|
|
||||||
|
/// The timeout we apply to each verify future during testing.
|
||||||
|
///
|
||||||
|
/// The checkpoint verifier uses `tokio::sync::oneshot` channels as futures.
|
||||||
|
/// If the verifier doesn't send a message on the channel, any tests that
|
||||||
|
/// await the channel future will hang.
|
||||||
|
///
|
||||||
|
/// This value is set to a large value, to avoid spurious failures due to
|
||||||
|
/// high system load.
|
||||||
|
const VERIFY_TIMEOUT_SECONDS: u64 = 10;
|
||||||
|
|
||||||
|
/// Generate a block with no transactions (not even a coinbase transaction).
|
||||||
|
///
|
||||||
|
/// The generated block should fail validation.
|
||||||
|
pub fn block_no_transactions() -> Block {
|
||||||
|
Block {
|
||||||
|
header: BlockHeader::zcash_deserialize(&zebra_test::vectors::DUMMY_HEADER[..]).unwrap(),
|
||||||
|
transactions: Vec::new(),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// 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(
|
||||||
|
checkpoint_list: CheckpointList,
|
||||||
|
) -> (
|
||||||
|
impl Service<
|
||||||
|
Arc<Block>,
|
||||||
|
Response = BlockHeaderHash,
|
||||||
|
Error = Error,
|
||||||
|
Future = impl Future<Output = Result<BlockHeaderHash, Error>>,
|
||||||
|
> + Send
|
||||||
|
+ Clone
|
||||||
|
+ 'static,
|
||||||
|
impl Service<
|
||||||
|
zebra_state::Request,
|
||||||
|
Response = zebra_state::Response,
|
||||||
|
Error = Error,
|
||||||
|
Future = impl Future<Output = Result<zebra_state::Response, Error>>,
|
||||||
|
> + Send
|
||||||
|
+ Clone
|
||||||
|
+ 'static,
|
||||||
|
) {
|
||||||
|
let state_service = zebra_state::in_memory::init();
|
||||||
|
let block_verifier = crate::block::init(state_service.clone());
|
||||||
|
let checkpoint_verifier =
|
||||||
|
crate::checkpoint::CheckpointVerifier::from_checkpoint_list(checkpoint_list);
|
||||||
|
let chain_verifier = super::init(block_verifier, checkpoint_verifier, state_service.clone());
|
||||||
|
|
||||||
|
(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<Block>,
|
||||||
|
Response = BlockHeaderHash,
|
||||||
|
Error = Error,
|
||||||
|
Future = impl Future<Output = Result<BlockHeaderHash, Error>>,
|
||||||
|
> + Send
|
||||||
|
+ Clone
|
||||||
|
+ 'static,
|
||||||
|
impl Service<
|
||||||
|
zebra_state::Request,
|
||||||
|
Response = zebra_state::Response,
|
||||||
|
Error = Error,
|
||||||
|
Future = impl Future<Output = Result<zebra_state::Response, Error>>,
|
||||||
|
> + Send
|
||||||
|
+ Clone
|
||||||
|
+ 'static,
|
||||||
|
) {
|
||||||
|
verifiers_from_checkpoint_list(CheckpointList::new(network))
|
||||||
|
}
|
||||||
|
|
||||||
|
#[tokio::test]
|
||||||
|
async fn verify_block_test() -> Result<(), Report> {
|
||||||
|
verify_block().await
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Test that block verifies work
|
||||||
|
///
|
||||||
|
/// Uses a custom checkpoint list, containing only the genesis block. Since the
|
||||||
|
/// maximum checkpoint height is 0, non-genesis blocks are verified using the
|
||||||
|
/// BlockVerifier.
|
||||||
|
#[spandoc::spandoc]
|
||||||
|
async fn verify_block() -> Result<(), Report> {
|
||||||
|
zebra_test::init();
|
||||||
|
|
||||||
|
// Parse the genesis block
|
||||||
|
let mut checkpoint_data = Vec::new();
|
||||||
|
let block0 =
|
||||||
|
Arc::<Block>::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])?;
|
||||||
|
let hash0: BlockHeaderHash = block0.as_ref().into();
|
||||||
|
checkpoint_data.push((
|
||||||
|
block0.coinbase_height().expect("test block has height"),
|
||||||
|
hash0,
|
||||||
|
));
|
||||||
|
|
||||||
|
// Make a checkpoint list containing the genesis block
|
||||||
|
let checkpoint_list: BTreeMap<BlockHeight, BlockHeaderHash> =
|
||||||
|
checkpoint_data.iter().cloned().collect();
|
||||||
|
let checkpoint_list = CheckpointList::from_list(checkpoint_list).map_err(|e| eyre!(e))?;
|
||||||
|
|
||||||
|
let (mut chain_verifier, _) = verifiers_from_checkpoint_list(checkpoint_list);
|
||||||
|
|
||||||
|
let block1 = Arc::<Block>::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_1_BYTES[..])?;
|
||||||
|
let hash1: BlockHeaderHash = block1.as_ref().into();
|
||||||
|
|
||||||
|
/// SPANDOC: Make sure the verifier service is ready
|
||||||
|
let ready_verifier_service = chain_verifier.ready_and().await.map_err(|e| eyre!(e))?;
|
||||||
|
/// SPANDOC: Set up the future
|
||||||
|
let verify_future = timeout(
|
||||||
|
Duration::from_secs(VERIFY_TIMEOUT_SECONDS),
|
||||||
|
ready_verifier_service.call(block1.clone()),
|
||||||
|
);
|
||||||
|
/// SPANDOC: Verify the block
|
||||||
|
// TODO(teor || jlusby): check error kind
|
||||||
|
let verify_response = verify_future
|
||||||
|
.map_err(|e| eyre!(e))
|
||||||
|
.await
|
||||||
|
.expect("timeout should not happen")
|
||||||
|
.expect("block should verify");
|
||||||
|
|
||||||
|
assert_eq!(verify_response, hash1);
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
|
||||||
|
#[tokio::test]
|
||||||
|
async fn verify_checkpoint_test() -> Result<(), Report> {
|
||||||
|
verify_checkpoint().await
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Test that checkpoint verifies work
|
||||||
|
#[spandoc::spandoc]
|
||||||
|
async fn verify_checkpoint() -> Result<(), Report> {
|
||||||
|
zebra_test::init();
|
||||||
|
|
||||||
|
let block =
|
||||||
|
Arc::<Block>::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])?;
|
||||||
|
let hash: BlockHeaderHash = block.as_ref().into();
|
||||||
|
|
||||||
|
let (mut chain_verifier, _) = verifiers_from_network(Mainnet);
|
||||||
|
|
||||||
|
/// SPANDOC: Make sure the verifier service is ready
|
||||||
|
let ready_verifier_service = chain_verifier.ready_and().await.map_err(|e| eyre!(e))?;
|
||||||
|
/// SPANDOC: Set up the future
|
||||||
|
let verify_future = timeout(
|
||||||
|
Duration::from_secs(VERIFY_TIMEOUT_SECONDS),
|
||||||
|
ready_verifier_service.call(block.clone()),
|
||||||
|
);
|
||||||
|
/// SPANDOC: Verify the block
|
||||||
|
// TODO(teor || jlusby): check error kind
|
||||||
|
let verify_response = verify_future
|
||||||
|
.map_err(|e| eyre!(e))
|
||||||
|
.await
|
||||||
|
.expect("timeout should not happen")
|
||||||
|
.expect("block should verify");
|
||||||
|
|
||||||
|
assert_eq!(verify_response, hash);
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
|
||||||
|
#[tokio::test]
|
||||||
|
async fn verify_fail_no_coinbase_test() -> Result<(), Report> {
|
||||||
|
verify_fail_no_coinbase().await
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Test that blocks with no coinbase height are rejected by the ChainVerifier
|
||||||
|
///
|
||||||
|
/// ChainVerifier uses the block height to decide between the CheckpointVerifier
|
||||||
|
/// and BlockVerifier. This is the error case, where there is no height.
|
||||||
|
#[spandoc::spandoc]
|
||||||
|
async fn verify_fail_no_coinbase() -> Result<(), Report> {
|
||||||
|
zebra_test::init();
|
||||||
|
|
||||||
|
let block = block_no_transactions();
|
||||||
|
let hash: BlockHeaderHash = (&block).into();
|
||||||
|
|
||||||
|
let (mut chain_verifier, mut state_service) = verifiers_from_network(Mainnet);
|
||||||
|
|
||||||
|
/// SPANDOC: Make sure the verifier service is ready
|
||||||
|
let ready_verifier_service = chain_verifier.ready_and().await.map_err(|e| eyre!(e))?;
|
||||||
|
/// SPANDOC: Set up the future to verify the block
|
||||||
|
let verify_future = timeout(
|
||||||
|
Duration::from_secs(VERIFY_TIMEOUT_SECONDS),
|
||||||
|
ready_verifier_service.call(block.into()),
|
||||||
|
);
|
||||||
|
/// SPANDOC: Verify the block
|
||||||
|
// TODO(teor || jlusby): check error kind
|
||||||
|
let _ = verify_future
|
||||||
|
.map_err(|e| eyre!(e))
|
||||||
|
.await
|
||||||
|
.expect("timeout should not happen")
|
||||||
|
.unwrap_err();
|
||||||
|
|
||||||
|
/// SPANDOC: Make sure the state service is ready
|
||||||
|
let ready_state_service = state_service.ready_and().await.map_err(|e| eyre!(e))?;
|
||||||
|
/// SPANDOC: The state should not contain failed blocks
|
||||||
|
let _ = ready_state_service
|
||||||
|
.call(zebra_state::Request::GetBlock { hash })
|
||||||
|
.await
|
||||||
|
.expect_err("failed block should not be in state");
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
|
||||||
|
#[tokio::test]
|
||||||
|
async fn round_trip_checkpoint_test() -> Result<(), Report> {
|
||||||
|
round_trip_checkpoint().await
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Test that state updates work
|
||||||
|
#[spandoc::spandoc]
|
||||||
|
async fn round_trip_checkpoint() -> Result<(), Report> {
|
||||||
|
zebra_test::init();
|
||||||
|
|
||||||
|
let block =
|
||||||
|
Arc::<Block>::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])?;
|
||||||
|
let hash: BlockHeaderHash = block.as_ref().into();
|
||||||
|
|
||||||
|
let (mut chain_verifier, mut state_service) = verifiers_from_network(Mainnet);
|
||||||
|
|
||||||
|
/// SPANDOC: Make sure the verifier service is ready
|
||||||
|
let ready_verifier_service = chain_verifier.ready_and().await.map_err(|e| eyre!(e))?;
|
||||||
|
/// SPANDOC: Set up the future
|
||||||
|
let verify_future = timeout(
|
||||||
|
Duration::from_secs(VERIFY_TIMEOUT_SECONDS),
|
||||||
|
ready_verifier_service.call(block.clone()),
|
||||||
|
);
|
||||||
|
/// SPANDOC: Verify the block
|
||||||
|
// TODO(teor || jlusby): check error kind
|
||||||
|
let verify_response = verify_future
|
||||||
|
.map_err(|e| eyre!(e))
|
||||||
|
.await
|
||||||
|
.expect("timeout should not happen")
|
||||||
|
.expect("block should verify");
|
||||||
|
|
||||||
|
assert_eq!(verify_response, hash);
|
||||||
|
|
||||||
|
/// SPANDOC: Make sure the state service is ready
|
||||||
|
let ready_state_service = state_service.ready_and().await.map_err(|e| eyre!(e))?;
|
||||||
|
/// SPANDOC: Make sure the block was added to the state
|
||||||
|
let state_response = ready_state_service
|
||||||
|
.call(zebra_state::Request::GetBlock { hash })
|
||||||
|
.await
|
||||||
|
.map_err(|e| eyre!(e))?;
|
||||||
|
|
||||||
|
if let zebra_state::Response::Block {
|
||||||
|
block: returned_block,
|
||||||
|
} = state_response
|
||||||
|
{
|
||||||
|
assert_eq!(block, returned_block);
|
||||||
|
} else {
|
||||||
|
bail!("unexpected response kind: {:?}", state_response);
|
||||||
|
}
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
|
||||||
|
#[tokio::test]
|
||||||
|
async fn verify_fail_add_block_checkpoint_test() -> Result<(), Report> {
|
||||||
|
verify_fail_add_block_checkpoint().await
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Test that the state rejects duplicate block adds
|
||||||
|
#[spandoc::spandoc]
|
||||||
|
async fn verify_fail_add_block_checkpoint() -> Result<(), Report> {
|
||||||
|
zebra_test::init();
|
||||||
|
|
||||||
|
let block =
|
||||||
|
Arc::<Block>::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])?;
|
||||||
|
let hash: BlockHeaderHash = block.as_ref().into();
|
||||||
|
|
||||||
|
let (mut chain_verifier, mut state_service) = verifiers_from_network(Mainnet);
|
||||||
|
|
||||||
|
/// SPANDOC: Make sure the verifier service is ready (1/2)
|
||||||
|
let ready_verifier_service = chain_verifier.ready_and().await.map_err(|e| eyre!(e))?;
|
||||||
|
/// SPANDOC: Set up the future to verify the block for the first time
|
||||||
|
let verify_future = timeout(
|
||||||
|
Duration::from_secs(VERIFY_TIMEOUT_SECONDS),
|
||||||
|
ready_verifier_service.call(block.clone()),
|
||||||
|
);
|
||||||
|
/// SPANDOC: Verify the block for the first time
|
||||||
|
// TODO(teor || jlusby): check error kind
|
||||||
|
let verify_response = verify_future
|
||||||
|
.map_err(|e| eyre!(e))
|
||||||
|
.await
|
||||||
|
.expect("timeout should not happen")
|
||||||
|
.expect("block should verify");
|
||||||
|
|
||||||
|
assert_eq!(verify_response, hash);
|
||||||
|
|
||||||
|
/// SPANDOC: Make sure the state service is ready (1/2)
|
||||||
|
let ready_state_service = state_service.ready_and().await.map_err(|e| eyre!(e))?;
|
||||||
|
/// SPANDOC: Make sure the block was added to the state
|
||||||
|
let state_response = ready_state_service
|
||||||
|
.call(zebra_state::Request::GetBlock { hash })
|
||||||
|
.await
|
||||||
|
.map_err(|e| eyre!(e))?;
|
||||||
|
|
||||||
|
if let zebra_state::Response::Block {
|
||||||
|
block: returned_block,
|
||||||
|
} = state_response
|
||||||
|
{
|
||||||
|
assert_eq!(block, returned_block);
|
||||||
|
} else {
|
||||||
|
bail!("unexpected response kind: {:?}", state_response);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// SPANDOC: Make sure the verifier service is ready (2/2)
|
||||||
|
let ready_verifier_service = chain_verifier.ready_and().await.map_err(|e| eyre!(e))?;
|
||||||
|
/// SPANDOC: Set up the future to verify the block for the first time
|
||||||
|
let verify_future = timeout(
|
||||||
|
Duration::from_secs(VERIFY_TIMEOUT_SECONDS),
|
||||||
|
ready_verifier_service.call(block.clone()),
|
||||||
|
);
|
||||||
|
/// SPANDOC: Verify the block for the first time
|
||||||
|
// TODO(teor): ignore duplicate block verifies?
|
||||||
|
// TODO(teor || jlusby): check error kind
|
||||||
|
let _ = verify_future
|
||||||
|
.map_err(|e| eyre!(e))
|
||||||
|
.await
|
||||||
|
.expect("timeout should not happen")
|
||||||
|
.unwrap_err();
|
||||||
|
|
||||||
|
/// SPANDOC: Make sure the state service is ready (2/2)
|
||||||
|
let ready_state_service = state_service.ready_and().await.map_err(|e| eyre!(e))?;
|
||||||
|
/// SPANDOC: But the state should still return the original block we added
|
||||||
|
let state_response = ready_state_service
|
||||||
|
.call(zebra_state::Request::GetBlock { hash })
|
||||||
|
.await
|
||||||
|
.map_err(|e| eyre!(e))?;
|
||||||
|
|
||||||
|
if let zebra_state::Response::Block {
|
||||||
|
block: returned_block,
|
||||||
|
} = state_response
|
||||||
|
{
|
||||||
|
assert_eq!(block, returned_block);
|
||||||
|
} else {
|
||||||
|
bail!("unexpected response kind: {:?}", state_response);
|
||||||
|
}
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
}
|
|
@ -13,13 +13,13 @@
|
||||||
//! Verification is provided via a `tower::Service`, to support backpressure and batch
|
//! Verification is provided via a `tower::Service`, to support backpressure and batch
|
||||||
//! verification.
|
//! verification.
|
||||||
|
|
||||||
mod list;
|
pub(crate) mod list;
|
||||||
mod types;
|
mod types;
|
||||||
|
|
||||||
#[cfg(test)]
|
#[cfg(test)]
|
||||||
mod tests;
|
mod tests;
|
||||||
|
|
||||||
use list::CheckpointList;
|
pub(crate) use list::CheckpointList;
|
||||||
use types::{Progress, Progress::*};
|
use types::{Progress, Progress::*};
|
||||||
use types::{Target, Target::*};
|
use types::{Target, Target::*};
|
||||||
|
|
||||||
|
@ -82,7 +82,7 @@ pub const MAX_QUEUED_BLOCKS_PER_HEIGHT: usize = 4;
|
||||||
/// Verifies blocks using a supplied list of checkpoints. There must be at
|
/// Verifies blocks using a supplied list of checkpoints. There must be at
|
||||||
/// least one checkpoint for the genesis block.
|
/// least one checkpoint for the genesis block.
|
||||||
#[derive(Debug)]
|
#[derive(Debug)]
|
||||||
struct CheckpointVerifier {
|
pub struct CheckpointVerifier {
|
||||||
// Inputs
|
// Inputs
|
||||||
//
|
//
|
||||||
/// The checkpoint list for this verifier.
|
/// The checkpoint list for this verifier.
|
||||||
|
@ -158,6 +158,10 @@ impl CheckpointVerifier {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
pub(crate) fn list(&self) -> &CheckpointList {
|
||||||
|
&self.checkpoint_list
|
||||||
|
}
|
||||||
|
|
||||||
/// Return the current verifier's progress.
|
/// Return the current verifier's progress.
|
||||||
///
|
///
|
||||||
/// If verification has not started yet, returns `BeforeGenesis`.
|
/// If verification has not started yet, returns `BeforeGenesis`.
|
||||||
|
|
|
@ -32,7 +32,7 @@ type Error = Box<dyn error::Error + Send + Sync + 'static>;
|
||||||
/// (zcashd allows chain reorganizations up to 99 blocks, and prunes
|
/// (zcashd allows chain reorganizations up to 99 blocks, and prunes
|
||||||
/// orphaned side-chains after 288 blocks.)
|
/// orphaned side-chains after 288 blocks.)
|
||||||
#[derive(Debug)]
|
#[derive(Debug)]
|
||||||
pub struct CheckpointList(BTreeMap<BlockHeight, BlockHeaderHash>);
|
pub(crate) struct CheckpointList(BTreeMap<BlockHeight, BlockHeaderHash>);
|
||||||
|
|
||||||
impl FromStr for CheckpointList {
|
impl FromStr for CheckpointList {
|
||||||
type Err = Error;
|
type Err = Error;
|
||||||
|
|
|
@ -16,6 +16,7 @@
|
||||||
#![allow(clippy::try_err)]
|
#![allow(clippy::try_err)]
|
||||||
|
|
||||||
pub mod block;
|
pub mod block;
|
||||||
|
pub mod chain;
|
||||||
pub mod checkpoint;
|
pub mod checkpoint;
|
||||||
pub mod mempool;
|
pub mod mempool;
|
||||||
pub mod parameters;
|
pub mod parameters;
|
||||||
|
|
Loading…
Reference in New Issue