Move consensus verification errors (#1070)

* refactor block and tx validation errors

* rename errors module to error

* move NoTransactions to BlockError

* clarify some errors, use dbg format for hash in error

* mnake is_coinbase_first return BlockError

* add new error types for each consensus Service

Co-authored-by: Jane Lusby <jane@zfnd.org>
This commit is contained in:
Alfredo Garcia 2020-09-21 15:54:06 -03:00 committed by GitHub
parent 33afeb37cb
commit c616c7ffd2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 203 additions and 89 deletions

2
Cargo.lock generated
View File

@ -3275,6 +3275,7 @@ version = "3.0.0-alpha.0"
dependencies = [
"chrono",
"color-eyre",
"displaydoc",
"futures",
"futures-util",
"metrics",
@ -3283,6 +3284,7 @@ dependencies = [
"redjubjub",
"serde",
"spandoc",
"thiserror",
"tokio",
"tower",
"tower-batch",

View File

@ -16,6 +16,7 @@ serde = { version = "1", features = ["serde_derive"] }
futures = "0.3.5"
futures-util = "0.3.5"
metrics = "0.12"
thiserror = "1.0.20"
tokio = { version = "0.2.22", features = ["time", "sync", "stream", "tracing"] }
tower = "0.3"
tower-util = "0.3"
@ -26,6 +27,7 @@ tower-fallback = { path = "../tower-fallback/" }
tower-batch = { path = "../tower-batch/" }
zebra-chain = { path = "../zebra-chain" }
zebra-state = { path = "../zebra-state" }
displaydoc = "0.1.7"
[dev-dependencies]
rand = "0.7"

View File

@ -17,11 +17,16 @@ use std::{
use chrono::Utc;
use futures_util::FutureExt;
use thiserror::Error;
use tower::{Service, ServiceExt};
use zebra_chain::block::{self, Block};
use zebra_chain::{
block::{self, Block},
work::equihash,
};
use zebra_state as zs;
use crate::error::*;
use crate::BoxError;
mod check;
@ -39,6 +44,27 @@ where
state_service: S,
}
#[non_exhaustive]
#[derive(Debug, Error)]
pub enum VerifyBlockError {
#[error("unable to verify depth for block {hash} from chain state during block verification")]
Depth { source: BoxError, hash: block::Hash },
#[error(transparent)]
Block {
#[from]
source: BlockError,
},
#[error(transparent)]
Equihash {
#[from]
source: equihash::Error,
},
#[error(transparent)]
Time(BoxError),
#[error("unable to commit block after semantic verification")]
Commit(#[source] BoxError),
}
impl<S> BlockVerifier<S>
where
S: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
@ -55,7 +81,7 @@ where
S::Future: Send + 'static,
{
type Response = block::Hash;
type Error = BoxError;
type Error = VerifyBlockError;
type Future =
Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;
@ -74,15 +100,18 @@ where
let hash = block.hash();
// Check that this block is actually a new block.
match state_service.ready_and().await?.call(zs::Request::Depth(hash)).await? {
match state_service
.ready_and()
.await
.map_err(|source| VerifyBlockError::Depth { source, hash })?
.call(zs::Request::Depth(hash))
.await
.map_err(|source| VerifyBlockError::Depth { source, hash })?
{
zs::Response::Depth(Some(depth)) => {
return Err(format!(
"block {} is already in the chain at depth {:?}",
hash,
depth,
).into())
},
zs::Response::Depth(None) => {},
return Err(BlockError::AlreadyInChain(hash, depth).into())
}
zs::Response::Depth(None) => {}
_ => unreachable!("wrong response to Request::Depth"),
}
@ -90,13 +119,9 @@ where
// height for parsed blocks when we deserialize them.
let height = block
.coinbase_height()
.ok_or_else(|| format!("invalid block {:?}: missing block height",
hash))?;
.ok_or_else(|| BlockError::MissingHeight(hash))?;
if height > block::Height::MAX {
Err(format!("invalid block height {:?} in {:?}: greater than the maximum height {:?}",
height,
hash,
block::Height::MAX))?;
Err(BlockError::MaxHeight(height, hash, block::Height::MAX))?;
}
// Do the difficulty checks first, to raise the threshold for
@ -105,14 +130,13 @@ where
.header
.difficulty_threshold
.to_expanded()
.ok_or_else(|| format!("invalid difficulty threshold in block header {:?} {:?}",
height,
hash))?;
.ok_or_else(|| BlockError::InvalidDifficulty(height, hash))?;
if hash > difficulty_threshold {
Err(format!("block {:?} failed the difficulty filter: hash {:?} must be less than or equal to the difficulty threshold {:?}",
height,
hash,
difficulty_threshold))?;
Err(BlockError::DifficultyFilter(
height,
hash,
difficulty_threshold,
))?;
}
check::is_equihash_solution_valid(&block.header)?;
@ -121,24 +145,23 @@ where
// Field validity and structure checks
let now = Utc::now();
check::is_time_valid_at(&block.header, now)?;
check::is_time_valid_at(&block.header, now).map_err(VerifyBlockError::Time)?;
check::is_coinbase_first(&block)?;
// TODO: context-free header verification: merkle root
tracing::trace!("verified block");
metrics::gauge!(
"block.verified.block.height",
height.0 as _
);
metrics::gauge!("block.verified.block.height", height.0 as _);
metrics::counter!("block.verified.block.count", 1);
// Finally, submit the block for contextual verification.
match state_service
.ready_and()
.await?
.await
.map_err(VerifyBlockError::Commit)?
.call(zs::Request::CommitBlock { block })
.await?
.await
.map_err(VerifyBlockError::Commit)?
{
zs::Response::Committed(committed_hash) => {
assert_eq!(committed_hash, hash, "state must commit correct hash");

View File

@ -7,6 +7,7 @@ use zebra_chain::{
work::equihash,
};
use crate::error::*;
use crate::BoxError;
/// Check that there is exactly one coinbase transaction in `Block`, and that
@ -17,18 +18,19 @@ use crate::BoxError;
/// fees paid by transactions included in this block." [§3.10][3.10]
///
/// [3.10]: https://zips.z.cash/protocol/protocol.pdf#coinbasetransactions
pub fn is_coinbase_first(block: &Block) -> Result<(), BoxError> {
pub fn is_coinbase_first(block: &Block) -> Result<(), BlockError> {
let first = block
.transactions
.get(0)
.ok_or("block has no transactions")?;
.ok_or(BlockError::NoTransactions)?;
let mut rest = block.transactions.iter().skip(1);
if !first.is_coinbase() {
return Err("first transaction must be coinbase".into());
return Err(TransactionError::CoinbasePosition)?;
}
if rest.any(|tx| tx.contains_coinbase_input()) {
return Err("coinbase input found in non-coinbase transaction".into());
return Err(TransactionError::CoinbaseInputFound)?;
}
Ok(())
}

View File

@ -1,12 +1,15 @@
#[cfg(test)]
mod tests;
use displaydoc::Display;
use futures::{FutureExt, TryFutureExt};
use std::{
future::Future,
pin::Pin,
sync::Arc,
task::{Context, Poll},
};
use thiserror::Error;
use tower::{buffer::Buffer, util::BoxService, Service, ServiceExt};
use tracing::instrument;
@ -19,7 +22,8 @@ use zebra_state as zs;
use crate::{
block::BlockVerifier,
checkpoint::{CheckpointList, CheckpointVerifier},
block::VerifyBlockError,
checkpoint::{CheckpointList, CheckpointVerifier, VerifyCheckpointError},
BoxError, Config,
};
@ -41,20 +45,29 @@ where
last_block_height: Option<block::Height>,
}
#[derive(Debug, Display, Error)]
pub enum VerifyChainError {
/// block could not be checkpointed
Checkpoint(VerifyCheckpointError),
/// block could not be verified
Block(VerifyBlockError),
}
impl<S> Service<Arc<Block>> for ChainVerifier<S>
where
S: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
S::Future: Send + 'static,
{
type Response = block::Hash;
type Error = BoxError;
type Error = VerifyChainError;
type Future =
Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;
fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
match (self.checkpoint.poll_ready(cx), self.block.poll_ready(cx)) {
// First, fail if either service fails.
(Poll::Ready(Err(e)), _) | (_, Poll::Ready(Err(e))) => Poll::Ready(Err(e)),
(Poll::Ready(Err(e)), _) => Poll::Ready(Err(VerifyChainError::Checkpoint(e))),
(_, Poll::Ready(Err(e))) => Poll::Ready(Err(VerifyChainError::Block(e))),
// Second, we're unready if either service is unready.
(Poll::Pending, _) | (_, Poll::Pending) => Poll::Pending,
// Finally, we're ready if both services are ready and OK.
@ -99,9 +112,15 @@ where
// 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)
self.checkpoint
.call(block)
.map_err(VerifyChainError::Checkpoint)
.boxed()
} else {
self.block.call(block)
self.block
.call(block)
.map_err(VerifyChainError::Block)
.boxed()
}
}
}
@ -115,7 +134,7 @@ pub async fn init<S>(
config: Config,
network: Network,
mut state_service: S,
) -> Buffer<BoxService<Arc<Block>, block::Hash, BoxError>, Arc<Block>>
) -> Buffer<BoxService<Arc<Block>, block::Hash, VerifyChainError>, Arc<Block>>
where
S: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
S::Future: Send + 'static,

View File

@ -23,6 +23,7 @@ use std::{
};
use futures_util::FutureExt;
use thiserror::Error;
use tokio::sync::oneshot;
use tower::{Service, ServiceExt};
@ -52,7 +53,7 @@ struct QueuedBlock {
/// `block`'s cached header hash.
hash: block::Hash,
/// The transmitting end of the oneshot channel for this block's result.
tx: oneshot::Sender<Result<block::Hash, BoxError>>,
tx: oneshot::Sender<Result<block::Hash, VerifyCheckpointError>>,
}
/// A list of unverified blocks at a particular height.
@ -173,9 +174,9 @@ where
list: impl IntoIterator<Item = (block::Height, block::Hash)>,
initial_tip: Option<(block::Height, block::Hash)>,
state_service: S,
) -> Result<Self, BoxError> {
) -> Result<Self, VerifyCheckpointError> {
Ok(Self::from_checkpoint_list(
CheckpointList::from_list(list)?,
CheckpointList::from_list(list).map_err(VerifyCheckpointError::CheckpointList)?,
initial_tip,
state_service,
))
@ -347,9 +348,9 @@ where
/// - the block's height is less than or equal to the previously verified
/// checkpoint
/// - verification has finished
fn check_height(&self, height: block::Height) -> Result<(), BoxError> {
fn check_height(&self, height: block::Height) -> Result<(), VerifyCheckpointError> {
if height > self.checkpoint_list.max_height() {
Err("block is higher than the maximum checkpoint")?;
Err(VerifyCheckpointError::TooHigh)?;
}
match self.previous_checkpoint_height() {
@ -359,14 +360,11 @@ where
InitialTip(previous_height) | PreviousCheckpoint(previous_height)
if (height <= previous_height) =>
{
Err(format!(
"Block height has already been verified. {:?}",
height
))?
Err(VerifyCheckpointError::Duplicate { height })?
}
InitialTip(_) | PreviousCheckpoint(_) => {}
// We're finished, so no checkpoint height is valid
FinalCheckpoint => Err("verification has finished")?,
FinalCheckpoint => Err(VerifyCheckpointError::Finished)?,
};
Ok(())
@ -401,10 +399,10 @@ where
///
/// Returns an error if the block's height is invalid, see `check_height()`
/// for details.
fn check_block(&self, block: &Block) -> Result<block::Height, BoxError> {
fn check_block(&self, block: &Block) -> Result<block::Height, VerifyCheckpointError> {
let block_height = block
.coinbase_height()
.ok_or("the block does not have a coinbase height")?;
.ok_or(VerifyCheckpointError::CoinbaseHeight)?;
self.check_height(block_height)?;
Ok(block_height)
}
@ -420,7 +418,7 @@ where
fn queue_block(
&mut self,
block: Arc<Block>,
) -> oneshot::Receiver<Result<block::Hash, BoxError>> {
) -> oneshot::Receiver<Result<block::Hash, VerifyCheckpointError>> {
// Set up a oneshot channel to send results
let (tx, rx) = oneshot::channel();
@ -451,7 +449,7 @@ where
for qb in qblocks.iter_mut() {
if qb.hash == hash {
let old_tx = std::mem::replace(&mut qb.tx, tx);
let e = "rejected older of duplicate verification requests".into();
let e = VerifyCheckpointError::NewerRequest;
tracing::trace!(?e);
let _ = old_tx.send(Err(e));
return rx;
@ -460,7 +458,7 @@ where
// Memory DoS resistance: limit the queued blocks at each height
if qblocks.len() >= MAX_QUEUED_BLOCKS_PER_HEIGHT {
let e = "too many queued blocks at this height".into();
let e = VerifyCheckpointError::QueuedLimit;
tracing::warn!(?e);
let _ = tx.send(Err(e));
return rx;
@ -530,17 +528,18 @@ where
tracing::info!(?height, ?qblock.hash, ?expected_hash,
"Duplicate block at height in CheckpointVerifier");
// Reject duplicate blocks at the same height
let _ = qblock.tx.send(Err(
"duplicate valid blocks at this height, only one was chosen".into(),
));
let _ = qblock
.tx
.send(Err(VerifyCheckpointError::Duplicate { height }));
}
} else {
tracing::info!(?height, ?qblock.hash, ?expected_hash,
"Bad block hash at height in CheckpointVerifier");
// A bad block, that isn't part of the chain.
let _ = qblock.tx.send(Err(
"the block hash does not match the chained checkpoint hash".into(),
));
let _ = qblock.tx.send(Err(VerifyCheckpointError::UnexpectedHash {
found: qblock.hash,
expected: expected_hash,
}));
}
}
@ -735,14 +734,39 @@ where
.expect("each entry is only removed once");
for qblock in qblocks.drain(..) {
// Sending can fail, but there's nothing we can do about it.
let _ = qblock
.tx
.send(Err("checkpoint verifier was dropped".into()));
let _ = qblock.tx.send(Err(VerifyCheckpointError::Dropped));
}
}
}
}
#[derive(Debug, Error)]
pub enum VerifyCheckpointError {
#[error("checkpoint request after checkpointing finished")]
Finished,
#[error("block is higher than the maximum checkpoint")]
TooHigh,
#[error("block at {height:?} has already been verified")]
Duplicate { height: block::Height },
#[error("rejected older of duplicate verification requests")]
NewerRequest,
#[error("the block does not have a coinbase height")]
CoinbaseHeight,
#[error("checkpoint verifier was dropped")]
Dropped,
#[error(transparent)]
CommitFinalized(BoxError),
#[error(transparent)]
CheckpointList(BoxError),
#[error("too many queued blocks at this height")]
QueuedLimit,
#[error("the block hash does not match the chained checkpoint hash, expected {expected:?} found {found:?}")]
UnexpectedHash {
expected: block::Hash,
found: block::Hash,
},
}
/// The CheckpointVerifier service implementation.
///
/// After verification, the block futures resolve to their hashes.
@ -752,7 +776,7 @@ where
S::Future: Send + 'static,
{
type Response = block::Hash;
type Error = BoxError;
type Error = VerifyCheckpointError;
type Future =
Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;
@ -763,7 +787,7 @@ where
fn call(&mut self, block: Arc<Block>) -> Self::Future {
// Immediately reject all incoming blocks that arrive after we've finished.
if let FinalCheckpoint = self.previous_checkpoint_height() {
return async { Err("checkpoint request after checkpointing finished".into()) }.boxed();
return async { Err(VerifyCheckpointError::Finished) }.boxed();
}
// Queue the block for verification, until we receive all the blocks for
@ -791,9 +815,11 @@ where
match state_service
.ready_and()
.await?
.await
.map_err(VerifyCheckpointError::CommitFinalized)?
.call(zs::Request::CommitFinalizedBlock { block })
.await?
.await
.map_err(VerifyCheckpointError::CommitFinalized)?
{
zs::Response::Committed(committed_hash) => {
assert_eq!(committed_hash, hash, "state must commit correct hash");

View File

@ -8,11 +8,10 @@
#[cfg(test)]
mod tests;
use crate::parameters;
use crate::{parameters, BoxError};
use std::{
collections::{BTreeMap, HashSet},
error,
ops::RangeBounds,
str::FromStr,
};
@ -23,10 +22,6 @@ use zebra_chain::parameters::Network;
const MAINNET_CHECKPOINTS: &str = include_str!("main-checkpoints.txt");
const TESTNET_CHECKPOINTS: &str = include_str!("test-checkpoints.txt");
/// The inner error type for CheckpointVerifier.
// TODO(jlusby): Error = Report ?
type Error = Box<dyn error::Error + Send + Sync + 'static>;
/// A list of block height and hash checkpoints.
///
/// Checkpoints should be chosen to avoid forks or chain reorganizations,
@ -40,7 +35,7 @@ type Error = Box<dyn error::Error + Send + Sync + 'static>;
pub(crate) struct CheckpointList(BTreeMap<block::Height, block::Hash>);
impl FromStr for CheckpointList {
type Err = Error;
type Err = BoxError;
/// Parse a string into a CheckpointList.
///
@ -95,7 +90,7 @@ impl CheckpointList {
/// (All other checkpoints are optional.)
pub(crate) fn from_list(
list: impl IntoIterator<Item = (block::Height, block::Hash)>,
) -> Result<Self, Error> {
) -> Result<Self, BoxError> {
// BTreeMap silently ignores duplicates, so we count the checkpoints
// before adding them to the map
let original_checkpoints: Vec<(block::Height, block::Hash)> = list.into_iter().collect();

View File

@ -12,7 +12,7 @@ use zebra_chain::{
/// Make a checkpoint list containing only the genesis block
#[test]
fn checkpoint_list_genesis() -> Result<(), Error> {
fn checkpoint_list_genesis() -> Result<(), BoxError> {
zebra_test::init();
// Parse the genesis block
@ -35,7 +35,7 @@ fn checkpoint_list_genesis() -> Result<(), Error> {
/// Make a checkpoint list containing multiple blocks
#[test]
fn checkpoint_list_multiple() -> Result<(), Error> {
fn checkpoint_list_multiple() -> Result<(), BoxError> {
zebra_test::init();
// Parse all the blocks
@ -64,7 +64,7 @@ fn checkpoint_list_multiple() -> Result<(), Error> {
/// Make sure that an empty checkpoint list fails
#[test]
fn checkpoint_list_empty_fail() -> Result<(), Error> {
fn checkpoint_list_empty_fail() -> Result<(), BoxError> {
zebra_test::init();
let _ = CheckpointList::from_list(Vec::new()).expect_err("empty checkpoint lists should fail");
@ -74,7 +74,7 @@ fn checkpoint_list_empty_fail() -> Result<(), Error> {
/// Make sure a checkpoint list that doesn't contain the genesis block fails
#[test]
fn checkpoint_list_no_genesis_fail() -> Result<(), Error> {
fn checkpoint_list_no_genesis_fail() -> Result<(), BoxError> {
zebra_test::init();
// Parse a non-genesis block
@ -97,7 +97,7 @@ fn checkpoint_list_no_genesis_fail() -> Result<(), Error> {
/// Make sure a checkpoint list that contains a null hash fails
#[test]
fn checkpoint_list_null_hash_fail() -> Result<(), Error> {
fn checkpoint_list_null_hash_fail() -> Result<(), BoxError> {
zebra_test::init();
let checkpoint_data = vec![(block::Height(0), block::Hash([0; 32]))];
@ -113,7 +113,7 @@ fn checkpoint_list_null_hash_fail() -> Result<(), Error> {
/// Make sure a checkpoint list that contains an invalid block height fails
#[test]
fn checkpoint_list_bad_height_fail() -> Result<(), Error> {
fn checkpoint_list_bad_height_fail() -> Result<(), BoxError> {
zebra_test::init();
let checkpoint_data = vec![(
@ -141,7 +141,7 @@ fn checkpoint_list_bad_height_fail() -> Result<(), Error> {
/// Make sure that a checkpoint list containing duplicate blocks fails
#[test]
fn checkpoint_list_duplicate_blocks_fail() -> Result<(), Error> {
fn checkpoint_list_duplicate_blocks_fail() -> Result<(), BoxError> {
zebra_test::init();
// Parse some blocks twice
@ -169,7 +169,7 @@ fn checkpoint_list_duplicate_blocks_fail() -> Result<(), Error> {
/// Make sure that a checkpoint list containing duplicate heights
/// (with different hashes) fails
#[test]
fn checkpoint_list_duplicate_heights_fail() -> Result<(), Error> {
fn checkpoint_list_duplicate_heights_fail() -> Result<(), BoxError> {
zebra_test::init();
// Parse the genesis block
@ -197,7 +197,7 @@ fn checkpoint_list_duplicate_heights_fail() -> Result<(), Error> {
/// Make sure that a checkpoint list containing duplicate hashes
/// (at different heights) fails
#[test]
fn checkpoint_list_duplicate_hashes_fail() -> Result<(), Error> {
fn checkpoint_list_duplicate_hashes_fail() -> Result<(), BoxError> {
zebra_test::init();
// Parse the genesis block
@ -224,7 +224,7 @@ fn checkpoint_list_duplicate_hashes_fail() -> Result<(), Error> {
/// Parse and check the hard-coded Mainnet and Testnet lists
#[test]
fn checkpoint_list_load_hard_coded() -> Result<(), Error> {
fn checkpoint_list_load_hard_coded() -> Result<(), BoxError> {
zebra_test::init();
let _: CheckpointList = MAINNET_CHECKPOINTS
@ -241,17 +241,17 @@ fn checkpoint_list_load_hard_coded() -> Result<(), Error> {
}
#[test]
fn checkpoint_list_hard_coded_sapling_mainnet() -> Result<(), Error> {
fn checkpoint_list_hard_coded_sapling_mainnet() -> Result<(), BoxError> {
checkpoint_list_hard_coded_sapling(Mainnet)
}
#[test]
fn checkpoint_list_hard_coded_sapling_testnet() -> Result<(), Error> {
fn checkpoint_list_hard_coded_sapling_testnet() -> Result<(), BoxError> {
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> {
fn checkpoint_list_hard_coded_sapling(network: Network) -> Result<(), BoxError> {
zebra_test::init();
let sapling_activation = Sapling

View File

@ -0,0 +1,44 @@
//! Consensus validation errors
use thiserror::Error;
#[derive(Error, Debug)]
pub enum TransactionError {
#[error("first transaction must be coinbase")]
CoinbasePosition,
#[error("coinbase input found in non-coinbase transaction")]
CoinbaseInputFound,
}
#[derive(Error, Debug)]
pub enum BlockError {
#[error("block contains invalid transactions")]
Transaction(#[from] TransactionError),
#[error("block haves no transactions")]
NoTransactions,
#[error("block {0:?} is already in the chain at depth {1:?}")]
AlreadyInChain(zebra_chain::block::Hash, u32),
#[error("invalid block {0:?}: missing block height")]
MissingHeight(zebra_chain::block::Hash),
#[error("invalid block height {0:?} in {1:?}: greater than the maximum height {2:?}")]
MaxHeight(
zebra_chain::block::Height,
zebra_chain::block::Hash,
zebra_chain::block::Height,
),
#[error("invalid difficulty threshold in block header {0:?} {1:?}")]
InvalidDifficulty(zebra_chain::block::Height, zebra_chain::block::Hash),
#[error("block {0:?} failed the difficulty filter: hash {1:?} must be less than or equal to the difficulty threshold {2:?}")]
DifficultyFilter(
zebra_chain::block::Height,
zebra_chain::block::Hash,
zebra_chain::work::difficulty::ExpandedDifficulty,
),
}

View File

@ -20,6 +20,7 @@ pub mod block;
pub mod chain;
pub mod checkpoint;
pub mod config;
pub mod error;
pub mod mempool;
pub mod parameters;