zebra/zebra-consensus/src/router.rs

385 lines
15 KiB
Rust
Raw Normal View History

//! Top-level semantic block verification for Zebra.
//!
//! Verifies blocks using the [`CheckpointVerifier`] or full [`SemanticBlockVerifier`],
//! depending on the config and block height.
//!
//! # Correctness
//!
//! Block and transaction verification requests should be wrapped in a timeout, because:
//! - checkpoint verification waits for previous blocks, and
//! - full block and transaction verification wait for UTXOs from previous blocks.
//!
//! Otherwise, verification of out-of-order and invalid blocks and transactions can hang
//! indefinitely.
use std::{
future::Future,
pin::Pin,
task::{Context, Poll},
};
use displaydoc::Display;
use futures::{FutureExt, TryFutureExt};
use thiserror::Error;
use tokio::task::JoinHandle;
use tower::{buffer::Buffer, util::BoxService, Service, ServiceExt};
use tracing::{instrument, Instrument, Span};
use zebra_chain::{
change(rpc): Add proposal capability to getblocktemplate (#5870) * adds ValidateBlock request to state * adds `Request` enum in block verifier skips solution check for BlockProposal requests calls CheckBlockValidity instead of Commit block for BlockProposal requests * uses new Request in references to chain verifier * adds getblocktemplate proposal mode response type * makes getblocktemplate-rpcs feature in zebra-consensus select getblocktemplate-rpcs in zebra-state * Adds PR review revisions * adds info log in CheckBlockProposalValidity * Reverts replacement of match statement * adds `GetBlockTemplate::capabilities` fn * conditions calling checkpoint verifier on !request.is_proposal * updates references to validate_and_commit_non_finalized * adds snapshot test, updates test vectors * adds `should_count_metrics` to NonFinalizedState * Returns an error from chain verifier for block proposal requests below checkpoint height adds feature flags * adds "proposal" to GET_BLOCK_TEMPLATE_CAPABILITIES_FIELD * adds back block::Request to zebra-consensus lib * updates snapshots * Removes unnecessary network arg * skips req in tracing intstrument for read state * Moves out block proposal validation to its own fn * corrects `difficulty_threshold_is_valid` docs adds/fixes some comments, adds TODOs general cleanup from a self-review. * Update zebra-state/src/service.rs * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * Update zebra-rpc/src/methods/get_block_template_rpcs.rs Co-authored-by: teor <teor@riseup.net> * check best chain tip * Update zebra-state/src/service.rs Co-authored-by: teor <teor@riseup.net> * Applies cleanup suggestions from code review Co-authored-by: teor <teor@riseup.net> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2023-01-11 15:39:51 -08:00
block::{self, Height},
parameters::Network,
};
use zebra_state as zs;
use crate::{
block::{Request, SemanticBlockVerifier, VerifyBlockError},
checkpoint::{CheckpointList, CheckpointVerifier, VerifyCheckpointError},
error::TransactionError,
transaction, BoxError, Config, ParameterCheckpoint as _,
};
#[cfg(test)]
mod tests;
/// The bound for the chain verifier and transaction verifier buffers.
///
/// We choose the verifier buffer bound based on the maximum number of
/// concurrent verifier users, to avoid contention:
/// - the `ChainSync` block download and verify stream
/// - the `Inbound` block download and verify stream
/// - the `Mempool` transaction download and verify stream
/// - a block miner component, which we might add in future, and
/// - 1 extra slot to avoid contention.
///
/// We deliberately add extra slots, because they only cost a small amount of
/// memory, but missing slots can significantly slow down Zebra.
const VERIFIER_BUFFER_BOUND: usize = 5;
/// The block verifier router routes requests to either the checkpoint verifier or the
/// semantic block verifier, depending on the maximum checkpoint height.
///
/// # Correctness
///
/// Block verification requests should be wrapped in a timeout, so that
/// out-of-order and invalid requests do not hang indefinitely. See the [`router`](`crate::router`)
/// module documentation for details.
struct BlockVerifierRouter<S, V>
where
S: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
S::Future: Send + 'static,
V: Service<transaction::Request, Response = transaction::Response, Error = BoxError>
+ Send
+ Clone
+ 'static,
V::Future: Send + 'static,
{
/// The checkpointing block verifier.
///
/// Always used for blocks before `Canopy`, optionally used for the entire checkpoint list.
checkpoint: CheckpointVerifier<S>,
/// The highest permitted checkpoint block.
///
/// This height must be in the `checkpoint` verifier's checkpoint list.
max_checkpoint_height: block::Height,
/// The full semantic block verifier, used for blocks after `max_checkpoint_height`.
block: SemanticBlockVerifier<S, V>,
}
/// An error while semantically verifying a block.
//
// One or both of these error variants are at least 140 bytes
#[derive(Debug, Display, Error)]
#[allow(missing_docs)]
pub enum RouterError {
/// Block could not be checkpointed
Checkpoint { source: Box<VerifyCheckpointError> },
/// Block could not be full-verified
Block { source: Box<VerifyBlockError> },
}
impl From<VerifyCheckpointError> for RouterError {
fn from(err: VerifyCheckpointError) -> Self {
RouterError::Checkpoint {
source: Box::new(err),
}
}
}
impl From<VerifyBlockError> for RouterError {
fn from(err: VerifyBlockError) -> Self {
RouterError::Block {
source: Box::new(err),
}
}
}
impl RouterError {
/// Returns `true` if this is definitely a duplicate request.
/// Some duplicate requests might not be detected, and therefore return `false`.
pub fn is_duplicate_request(&self) -> bool {
match self {
RouterError::Checkpoint { source, .. } => source.is_duplicate_request(),
RouterError::Block { source, .. } => source.is_duplicate_request(),
}
}
}
impl<S, V> Service<Request> for BlockVerifierRouter<S, V>
where
S: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
S::Future: Send + 'static,
V: Service<transaction::Request, Response = transaction::Response, Error = BoxError>
+ Send
+ Clone
+ 'static,
V::Future: Send + 'static,
{
type Response = block::Hash;
type Error = RouterError;
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>> {
// CORRECTNESS
//
// The current task must be scheduled for wakeup every time we return
// `Poll::Pending`.
//
// If either verifier is unready, this task is scheduled for wakeup when it becomes
// ready.
//
// We acquire checkpoint readiness before block readiness, to avoid an unlikely
// hang during the checkpoint to block verifier transition. If the checkpoint and
// block verifiers are contending for the same buffer/batch, we want the checkpoint
// verifier to win, so that checkpoint verification completes, and block verification
// can start. (Buffers and batches have multiple slots, so this contention is unlikely.)
use futures::ready;
// The chain verifier holds one slot in each verifier, for each concurrent task.
// Therefore, any shared buffers or batches polled by these verifiers should double
// their bounds. (For example, the state service buffer.)
ready!(self.checkpoint.poll_ready(cx))?;
ready!(self.block.poll_ready(cx))?;
Poll::Ready(Ok(()))
}
change(rpc): Add proposal capability to getblocktemplate (#5870) * adds ValidateBlock request to state * adds `Request` enum in block verifier skips solution check for BlockProposal requests calls CheckBlockValidity instead of Commit block for BlockProposal requests * uses new Request in references to chain verifier * adds getblocktemplate proposal mode response type * makes getblocktemplate-rpcs feature in zebra-consensus select getblocktemplate-rpcs in zebra-state * Adds PR review revisions * adds info log in CheckBlockProposalValidity * Reverts replacement of match statement * adds `GetBlockTemplate::capabilities` fn * conditions calling checkpoint verifier on !request.is_proposal * updates references to validate_and_commit_non_finalized * adds snapshot test, updates test vectors * adds `should_count_metrics` to NonFinalizedState * Returns an error from chain verifier for block proposal requests below checkpoint height adds feature flags * adds "proposal" to GET_BLOCK_TEMPLATE_CAPABILITIES_FIELD * adds back block::Request to zebra-consensus lib * updates snapshots * Removes unnecessary network arg * skips req in tracing intstrument for read state * Moves out block proposal validation to its own fn * corrects `difficulty_threshold_is_valid` docs adds/fixes some comments, adds TODOs general cleanup from a self-review. * Update zebra-state/src/service.rs * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * Update zebra-rpc/src/methods/get_block_template_rpcs.rs Co-authored-by: teor <teor@riseup.net> * check best chain tip * Update zebra-state/src/service.rs Co-authored-by: teor <teor@riseup.net> * Applies cleanup suggestions from code review Co-authored-by: teor <teor@riseup.net> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2023-01-11 15:39:51 -08:00
fn call(&mut self, request: Request) -> Self::Future {
let block = request.block();
match block.coinbase_height() {
change(rpc): Add proposal capability to getblocktemplate (#5870) * adds ValidateBlock request to state * adds `Request` enum in block verifier skips solution check for BlockProposal requests calls CheckBlockValidity instead of Commit block for BlockProposal requests * uses new Request in references to chain verifier * adds getblocktemplate proposal mode response type * makes getblocktemplate-rpcs feature in zebra-consensus select getblocktemplate-rpcs in zebra-state * Adds PR review revisions * adds info log in CheckBlockProposalValidity * Reverts replacement of match statement * adds `GetBlockTemplate::capabilities` fn * conditions calling checkpoint verifier on !request.is_proposal * updates references to validate_and_commit_non_finalized * adds snapshot test, updates test vectors * adds `should_count_metrics` to NonFinalizedState * Returns an error from chain verifier for block proposal requests below checkpoint height adds feature flags * adds "proposal" to GET_BLOCK_TEMPLATE_CAPABILITIES_FIELD * adds back block::Request to zebra-consensus lib * updates snapshots * Removes unnecessary network arg * skips req in tracing intstrument for read state * Moves out block proposal validation to its own fn * corrects `difficulty_threshold_is_valid` docs adds/fixes some comments, adds TODOs general cleanup from a self-review. * Update zebra-state/src/service.rs * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * Update zebra-rpc/src/methods/get_block_template_rpcs.rs Co-authored-by: teor <teor@riseup.net> * check best chain tip * Update zebra-state/src/service.rs Co-authored-by: teor <teor@riseup.net> * Applies cleanup suggestions from code review Co-authored-by: teor <teor@riseup.net> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2023-01-11 15:39:51 -08:00
#[cfg(feature = "getblocktemplate-rpcs")]
// There's currently no known use case for block proposals below the checkpoint height,
// so it's okay to immediately return an error here.
Some(height) if height <= self.max_checkpoint_height && request.is_proposal() => {
async {
// TODO: Add a `ValidateProposalError` enum with a `BelowCheckpoint` variant?
Err(VerifyBlockError::ValidateProposal(
"block proposals must be above checkpoint height".into(),
))?
}
.boxed()
}
Some(height) if height <= self.max_checkpoint_height => {
self.checkpoint.call(block).map_err(Into::into).boxed()
}
// This also covers blocks with no height, which the block verifier
// will reject immediately.
change(rpc): Add proposal capability to getblocktemplate (#5870) * adds ValidateBlock request to state * adds `Request` enum in block verifier skips solution check for BlockProposal requests calls CheckBlockValidity instead of Commit block for BlockProposal requests * uses new Request in references to chain verifier * adds getblocktemplate proposal mode response type * makes getblocktemplate-rpcs feature in zebra-consensus select getblocktemplate-rpcs in zebra-state * Adds PR review revisions * adds info log in CheckBlockProposalValidity * Reverts replacement of match statement * adds `GetBlockTemplate::capabilities` fn * conditions calling checkpoint verifier on !request.is_proposal * updates references to validate_and_commit_non_finalized * adds snapshot test, updates test vectors * adds `should_count_metrics` to NonFinalizedState * Returns an error from chain verifier for block proposal requests below checkpoint height adds feature flags * adds "proposal" to GET_BLOCK_TEMPLATE_CAPABILITIES_FIELD * adds back block::Request to zebra-consensus lib * updates snapshots * Removes unnecessary network arg * skips req in tracing intstrument for read state * Moves out block proposal validation to its own fn * corrects `difficulty_threshold_is_valid` docs adds/fixes some comments, adds TODOs general cleanup from a self-review. * Update zebra-state/src/service.rs * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * Update zebra-rpc/src/methods/get_block_template_rpcs.rs Co-authored-by: teor <teor@riseup.net> * check best chain tip * Update zebra-state/src/service.rs Co-authored-by: teor <teor@riseup.net> * Applies cleanup suggestions from code review Co-authored-by: teor <teor@riseup.net> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2023-01-11 15:39:51 -08:00
_ => self.block.call(request).map_err(Into::into).boxed(),
}
}
}
/// Initialize block and transaction verification services.
Download Zcash Sapling parameters and load them from cached files (#3057) * Replace Zcash parameters crates with pre-downloaded local parameter files * Download Zcash parameters using the `zcashd` script in CI and Docker * Add a zcash_proofs dependency to zebra-consensus * Download Sapling parameters using zcash_proofs, rather than fetch-params.sh * Add a new `zebrad download` subcommand This command isn't required for nomrmal usage. But it's useful when testing, or launching multiple Zebra instances. * Use `zebrad download` in CI to pre-download parameters * Log a helpful hint if downloading fails * Allow some duplicate dependencies currently hidden by orchard * Spawn a separate task to download Groth16 parameters * Run the parameter download with code coverage This avoids re-compining Zebra with and without coverage. * Update Cargo.lock after rebase * Try to pass `download` as an argument to `zebrad` in coverage CI * Fix copy and paste comment typos * Add path and download examples, like zcash_proofs * Download params in CI just like zcash_proofs does * Delete a redundant build step * Implement graceful shutdown for zebrad start * Send coverage summary to /dev/null when getting the params path * Use the correct parameters path and download commands in CI * Explain pre-downloads * Avoid calling params_folder twice * Rename parameter types and methods for consistency ```sh fastmod SaplingParams SaplingParameters zebra* fastmod Groth16Params Groth16Parameters zebra* fastmod PARAMS GROTH16_PARAMETERS zebra* fastmod params_folder directory zebra* ``` And a manual variable name tweak. * rustfmt * Remove a redundant coverage step Co-authored-by: Janito Vaqueiro Ferreira Filho <janito.vff@gmail.com>
2021-11-19 15:02:56 -08:00
///
/// Returns a block verifier, transaction verifier,
/// a [`BackgroundTaskHandles`] with the state checkpoint verify task,
/// and the maximum configured checkpoint verification height.
///
/// The consensus configuration is specified by `config`, and the Zcash network
/// to verify blocks for is specified by `network`.
///
/// The block verification service asynchronously performs semantic verification
/// checks. Blocks that pass semantic verification are submitted to the supplied
/// `state_service` for contextual verification before being committed to the chain.
///
/// The transaction verification service asynchronously performs semantic verification
/// checks. Transactions that pass semantic verification return an `Ok` result to the caller.
///
/// This function should only be called once for a particular state service.
///
/// Dropped requests are cancelled on a best-effort basis, but may continue to be processed.
///
/// # Correctness
///
/// Block and transaction verification requests should be wrapped in a timeout,
/// so that out-of-order and invalid requests do not hang indefinitely.
/// See the [`router`](`crate::router`) module documentation for details.
#[instrument(skip(state_service))]
pub async fn init<S>(
config: Config,
network: Network,
mut state_service: S,
) -> (
Buffer<BoxService<Request, block::Hash, RouterError>, Request>,
Buffer<
BoxService<transaction::Request, transaction::Response, TransactionError>,
transaction::Request,
>,
BackgroundTaskHandles,
Height,
)
where
S: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
S::Future: Send + 'static,
{
// Give other tasks priority before spawning the checkpoint task.
tokio::task::yield_now().await;
Download Zcash Sapling parameters and load them from cached files (#3057) * Replace Zcash parameters crates with pre-downloaded local parameter files * Download Zcash parameters using the `zcashd` script in CI and Docker * Add a zcash_proofs dependency to zebra-consensus * Download Sapling parameters using zcash_proofs, rather than fetch-params.sh * Add a new `zebrad download` subcommand This command isn't required for nomrmal usage. But it's useful when testing, or launching multiple Zebra instances. * Use `zebrad download` in CI to pre-download parameters * Log a helpful hint if downloading fails * Allow some duplicate dependencies currently hidden by orchard * Spawn a separate task to download Groth16 parameters * Run the parameter download with code coverage This avoids re-compining Zebra with and without coverage. * Update Cargo.lock after rebase * Try to pass `download` as an argument to `zebrad` in coverage CI * Fix copy and paste comment typos * Add path and download examples, like zcash_proofs * Download params in CI just like zcash_proofs does * Delete a redundant build step * Implement graceful shutdown for zebrad start * Send coverage summary to /dev/null when getting the params path * Use the correct parameters path and download commands in CI * Explain pre-downloads * Avoid calling params_folder twice * Rename parameter types and methods for consistency ```sh fastmod SaplingParams SaplingParameters zebra* fastmod Groth16Params Groth16Parameters zebra* fastmod PARAMS GROTH16_PARAMETERS zebra* fastmod params_folder directory zebra* ``` And a manual variable name tweak. * rustfmt * Remove a redundant coverage step Co-authored-by: Janito Vaqueiro Ferreira Filho <janito.vff@gmail.com>
2021-11-19 15:02:56 -08:00
// Make sure the state contains the known best chain checkpoints, in a separate thread.
let checkpoint_state_service = state_service.clone();
let checkpoint_sync = config.checkpoint_sync;
let state_checkpoint_verify_handle = tokio::task::spawn(
// TODO: move this into an async function?
async move {
tracing::info!("starting state checkpoint validation");
// # Consensus
//
// We want to verify all available checkpoints, even if the node is not configured
// to use them for syncing. Zebra's checkpoints are updated with every release,
// which makes sure they include the latest settled network upgrade.
//
// > A network upgrade is settled on a given network when there is a social
// > consensus that it has activated with a given activation block hash.
// > A full validator that potentially risks Mainnet funds or displays Mainnet
// > transaction information to a user MUST do so only for a block chain that
// > includes the activation block of the most recent settled network upgrade,
// > with the corresponding activation block hash. Currently, there is social
// > consensus that NU5 has activated on the Zcash Mainnet and Testnet with the
// > activation block hashes given in § 3.12 Mainnet and Testnet on p. 20.
//
// <https://zips.z.cash/protocol/protocol.pdf#blockchain>
let full_checkpoints = network.checkpoint_list();
let mut already_warned = false;
for (height, checkpoint_hash) in full_checkpoints.iter() {
let checkpoint_state_service = checkpoint_state_service.clone();
let request = zebra_state::Request::BestChainBlockHash(*height);
match checkpoint_state_service.oneshot(request).await {
Ok(zebra_state::Response::BlockHash(Some(state_hash))) => assert_eq!(
*checkpoint_hash, state_hash,
"invalid block in state: a previous Zebra instance followed an \
incorrect chain. Delete and re-sync your state to use the best chain"
),
Ok(zebra_state::Response::BlockHash(None)) => {
if checkpoint_sync {
tracing::info!(
"state is not fully synced yet, remaining checkpoints will be \
verified during syncing"
);
} else {
tracing::warn!(
"state is not fully synced yet, remaining checkpoints will be \
verified next time Zebra starts up. Zebra will be less secure \
until it is restarted. Use consensus.checkpoint_sync = true \
in zebrad.toml to make sure you are following a valid chain"
);
}
break;
}
Ok(response) => {
unreachable!("unexpected response type: {response:?} from state request")
}
Err(e) => {
// This error happens a lot in some tests, and it could happen to users.
if !already_warned {
tracing::warn!(
"unexpected error: {e:?} in state request while verifying previous \
state checkpoints. Is Zebra shutting down?"
);
already_warned = true;
}
}
}
}
tracing::info!("finished state checkpoint validation");
}
.instrument(Span::current()),
);
// transaction verification
let transaction = transaction::Verifier::new(network, state_service.clone());
let transaction = Buffer::new(BoxService::new(transaction), VERIFIER_BUFFER_BOUND);
// block verification
change(state): Write non-finalized blocks to the state in a separate thread, to avoid network and RPC hangs (#5257) * Add a new block commit task and channels, that don't do anything yet * Add last_block_hash_sent to the state service, to avoid database accesses * Update last_block_hash_sent regardless of commit errors * Rename a field to StateService.max_queued_finalized_height * Commit finalized blocks to the state in a separate task * Check for panics in the block write task * Wait for the block commit task in tests, and check for errors * Always run a proptest that sleeps once * Add extra debugging to state shutdowns * Work around a RocksDB shutdown bug * Close the finalized block channel when we're finished with it * Only reset state queue once per error * Update some TODOs * Add a module doc comment * Drop channels and check for closed channels in the block commit task * Close state channels and tasks on drop * Remove some duplicate fields across StateService and ReadStateService * Try tweaking the shutdown steps * Update and clarify some comments * Clarify another comment * Don't try to cancel RocksDB background work on drop * Fix up some comments * Remove some duplicate code * Remove redundant workarounds for shutdown issues * Remode a redundant channel close in the block commit task * Remove a mistaken `!force` shutdown condition * Remove duplicate force-shutdown code and explain it better * Improve RPC error logging * Wait for chain tip updates in the RPC tests * Wait 2 seconds for chain tip updates before skipping them * Remove an unnecessary block_in_place() * Fix some test error messages that were changed by earlier fixes * Expand some comments, fix typos Co-authored-by: Marek <mail@marek.onl> * Actually drop children of failed blocks * Explain why we drop descendants of failed blocks * Clarify a comment * Wait for chain tip updates in a failing test on macOS * Clean duplicate finalized blocks when the non-finalized state activates * Send an error when receiving a duplicate finalized block * Update checkpoint block behaviour, document its consensus rule * Wait for chain tip changes in inbound_block_height_lookahead_limit test * Wait for the genesis block to commit in the fake peer set mempool tests * Disable unreliable mempool verification check in the send transaction test * Appease rustfmt * Use clear_finalized_block_queue() everywhere that blocks are dropped * Document how Finalized and NonFinalized clones are different * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * updates comments, renames send_process_queued, other minor cleanup * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * updates comments, renames send_process_queued, other minor cleanup * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * removes duplicate field definitions on StateService that were a result of a bad merge * update NotReadyToBeCommitted error message * Appear rustfmt * Fix doc links * Rename a function to initial_contextual_validity() * Do error tasks on Err, and success tasks on Ok * Simplify parent_error_map truncation * Rewrite best_tip() to use tip() * Rename latest_mem() to latest_non_finalized_state() ```sh fastmod latest_mem latest_non_finalized_state zebra* cargo fmt --all ``` * Simplify latest_non_finalized_state() using a new WatchReceiver API * Expand some error messages * Send the result after updating the channels, and document why * wait for chain_tip_update before cancelling download in mempool_cancel_mined * adds `sent_non_finalized_block_hashes` field to StateService * adds batched sent_hash insertions and checks sent hashes in queue_and_commit_non_finalized before adding a block to the queue * check that the `curr_buf` in SentHashes is not empty before pushing it to the `sent_bufs` * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * Fix rustfmt * Check for finalized block heights using zs_contains() * adds known_utxos field to SentHashes * updates comment on SentHashes.add method * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * return early when there's a duplicate hash in QueuedBlocks.queue instead of panicking * Make finalized UTXOs near the final checkpoint available for full block verification * Replace a checkpoint height literal with the actual config * Update mainnet and testnet checkpoints - 7 October 2022 * Fix some state service init arguments * Allow more lookahead in the downloader, but less lookahead in the syncer * Add the latest config to the tests, and fix the latest config check * Increase the number of finalized blocks checked for non-finalized block UTXO spends * fix(log): reduce verbose logs for block commits (#5348) * Remove some verbose block write channel logs * Only warn about tracing endpoint if the address is actually set * Use CloneError instead of formatting a non-cloneable error Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> * Increase block verify timeout * Work around a known block timeout bug by using a shorter timeout Co-authored-by: teor <teor@riseup.net> Co-authored-by: Marek <mail@marek.onl> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2022-10-11 12:25:45 -07:00
let (list, max_checkpoint_height) = init_checkpoint_list(config, network);
let tip = match state_service
Update to Tokio 1.13.0 (#2994) * Update `tower` to version `0.4.9` Update to latest version to add support for Tokio version 1. * Replace usage of `ServiceExt::ready_and` It was deprecated in favor of `ServiceExt::ready`. * Update Tokio dependency to version `1.13.0` This will break the build because the code isn't ready for the update, but future commits will fix the issues. * Replace import of `tokio::stream::StreamExt` Use `futures::stream::StreamExt` instead, because newer versions of Tokio don't have the `stream` feature. * Use `IntervalStream` in `zebra-network` In newer versions of Tokio `Interval` doesn't implement `Stream`, so the wrapper types from `tokio-stream` have to be used instead. * Use `IntervalStream` in `inventory_registry` In newer versions of Tokio the `Interval` type doesn't implement `Stream`, so `tokio_stream::wrappers::IntervalStream` has to be used instead. * Use `BroadcastStream` in `inventory_registry` In newer versions of Tokio `broadcast::Receiver` doesn't implement `Stream`, so `tokio_stream::wrappers::BroadcastStream` instead. This also requires changing the error type that is used. * Handle `Semaphore::acquire` error in `tower-batch` Newer versions of Tokio can return an error if the semaphore is closed. This shouldn't happen in `tower-batch` because the semaphore is never closed. * Handle `Semaphore::acquire` error in `zebrad` test On newer versions of Tokio `Semaphore::acquire` can return an error if the semaphore is closed. This shouldn't happen in the test because the semaphore is never closed. * Update some `zebra-network` dependencies Use versions compatible with Tokio version 1. * Upgrade Hyper to version 0.14 Use a version that supports Tokio version 1. * Update `metrics` dependency to version 0.17 And also update the `metrics-exporter-prometheus` to version 0.6.1. These updates are to make sure Tokio 1 is supported. * Use `f64` as the histogram data type `u64` isn't supported as the histogram data type in newer versions of `metrics`. * Update the initialization of the metrics component Make it compatible with the new version of `metrics`. * Simplify build version counter Remove all constants and use the new `metrics::incement_counter!` macro. * Change metrics output line to match on The snapshot string isn't included in the newer version of `metrics-exporter-prometheus`. * Update `sentry` to version 0.23.0 Use a version compatible with Tokio version 1. * Remove usage of `TracingIntegration` This seems to not be available from `sentry-tracing` anymore, so it needs to be replaced. * Add sentry layer to tracing initialization This seems like the replacement for `TracingIntegration`. * Remove unnecessary conversion Suggested by a Clippy lint. * Update Cargo lock file Apply all of the updates to dependencies. * Ban duplicate tokio dependencies Also ban git sources for tokio dependencies. * Stop allowing sentry-tracing git repository in `deny.toml` * Allow remaining duplicates after the tokio upgrade * Use C: drive for CI build output on Windows GitHub Actions uses a Windows image with two disk drives, and the default D: drive is smaller than the C: drive. Zebra currently uses a lot of space to build, so it has to use the C: drive to avoid CI build failures because of insufficient space. Co-authored-by: teor <teor@riseup.net>
2021-11-02 11:46:57 -07:00
.ready()
.await
.unwrap()
.call(zs::Request::Tip)
.await
.unwrap()
{
zs::Response::Tip(tip) => tip,
_ => unreachable!("wrong response to Request::Tip"),
};
tracing::info!(
?tip,
?max_checkpoint_height,
"initializing block verifier router"
);
let block = SemanticBlockVerifier::new(network, state_service.clone(), transaction.clone());
let checkpoint = CheckpointVerifier::from_checkpoint_list(list, network, tip, state_service);
let router = BlockVerifierRouter {
checkpoint,
max_checkpoint_height,
block,
};
let router = Buffer::new(BoxService::new(router), VERIFIER_BUFFER_BOUND);
let task_handles = BackgroundTaskHandles {
state_checkpoint_verify_handle,
};
(router, transaction, task_handles, max_checkpoint_height)
}
change(state): Write non-finalized blocks to the state in a separate thread, to avoid network and RPC hangs (#5257) * Add a new block commit task and channels, that don't do anything yet * Add last_block_hash_sent to the state service, to avoid database accesses * Update last_block_hash_sent regardless of commit errors * Rename a field to StateService.max_queued_finalized_height * Commit finalized blocks to the state in a separate task * Check for panics in the block write task * Wait for the block commit task in tests, and check for errors * Always run a proptest that sleeps once * Add extra debugging to state shutdowns * Work around a RocksDB shutdown bug * Close the finalized block channel when we're finished with it * Only reset state queue once per error * Update some TODOs * Add a module doc comment * Drop channels and check for closed channels in the block commit task * Close state channels and tasks on drop * Remove some duplicate fields across StateService and ReadStateService * Try tweaking the shutdown steps * Update and clarify some comments * Clarify another comment * Don't try to cancel RocksDB background work on drop * Fix up some comments * Remove some duplicate code * Remove redundant workarounds for shutdown issues * Remode a redundant channel close in the block commit task * Remove a mistaken `!force` shutdown condition * Remove duplicate force-shutdown code and explain it better * Improve RPC error logging * Wait for chain tip updates in the RPC tests * Wait 2 seconds for chain tip updates before skipping them * Remove an unnecessary block_in_place() * Fix some test error messages that were changed by earlier fixes * Expand some comments, fix typos Co-authored-by: Marek <mail@marek.onl> * Actually drop children of failed blocks * Explain why we drop descendants of failed blocks * Clarify a comment * Wait for chain tip updates in a failing test on macOS * Clean duplicate finalized blocks when the non-finalized state activates * Send an error when receiving a duplicate finalized block * Update checkpoint block behaviour, document its consensus rule * Wait for chain tip changes in inbound_block_height_lookahead_limit test * Wait for the genesis block to commit in the fake peer set mempool tests * Disable unreliable mempool verification check in the send transaction test * Appease rustfmt * Use clear_finalized_block_queue() everywhere that blocks are dropped * Document how Finalized and NonFinalized clones are different * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * updates comments, renames send_process_queued, other minor cleanup * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * updates comments, renames send_process_queued, other minor cleanup * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * removes duplicate field definitions on StateService that were a result of a bad merge * update NotReadyToBeCommitted error message * Appear rustfmt * Fix doc links * Rename a function to initial_contextual_validity() * Do error tasks on Err, and success tasks on Ok * Simplify parent_error_map truncation * Rewrite best_tip() to use tip() * Rename latest_mem() to latest_non_finalized_state() ```sh fastmod latest_mem latest_non_finalized_state zebra* cargo fmt --all ``` * Simplify latest_non_finalized_state() using a new WatchReceiver API * Expand some error messages * Send the result after updating the channels, and document why * wait for chain_tip_update before cancelling download in mempool_cancel_mined * adds `sent_non_finalized_block_hashes` field to StateService * adds batched sent_hash insertions and checks sent hashes in queue_and_commit_non_finalized before adding a block to the queue * check that the `curr_buf` in SentHashes is not empty before pushing it to the `sent_bufs` * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * Fix rustfmt * Check for finalized block heights using zs_contains() * adds known_utxos field to SentHashes * updates comment on SentHashes.add method * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * return early when there's a duplicate hash in QueuedBlocks.queue instead of panicking * Make finalized UTXOs near the final checkpoint available for full block verification * Replace a checkpoint height literal with the actual config * Update mainnet and testnet checkpoints - 7 October 2022 * Fix some state service init arguments * Allow more lookahead in the downloader, but less lookahead in the syncer * Add the latest config to the tests, and fix the latest config check * Increase the number of finalized blocks checked for non-finalized block UTXO spends * fix(log): reduce verbose logs for block commits (#5348) * Remove some verbose block write channel logs * Only warn about tracing endpoint if the address is actually set * Use CloneError instead of formatting a non-cloneable error Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> * Increase block verify timeout * Work around a known block timeout bug by using a shorter timeout Co-authored-by: teor <teor@riseup.net> Co-authored-by: Marek <mail@marek.onl> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2022-10-11 12:25:45 -07:00
/// Parses the checkpoint list for `network` and `config`.
/// Returns the checkpoint list and maximum checkpoint height.
pub fn init_checkpoint_list(config: Config, network: Network) -> (CheckpointList, Height) {
// TODO: Zebra parses the checkpoint list three times at startup.
change(state): Write non-finalized blocks to the state in a separate thread, to avoid network and RPC hangs (#5257) * Add a new block commit task and channels, that don't do anything yet * Add last_block_hash_sent to the state service, to avoid database accesses * Update last_block_hash_sent regardless of commit errors * Rename a field to StateService.max_queued_finalized_height * Commit finalized blocks to the state in a separate task * Check for panics in the block write task * Wait for the block commit task in tests, and check for errors * Always run a proptest that sleeps once * Add extra debugging to state shutdowns * Work around a RocksDB shutdown bug * Close the finalized block channel when we're finished with it * Only reset state queue once per error * Update some TODOs * Add a module doc comment * Drop channels and check for closed channels in the block commit task * Close state channels and tasks on drop * Remove some duplicate fields across StateService and ReadStateService * Try tweaking the shutdown steps * Update and clarify some comments * Clarify another comment * Don't try to cancel RocksDB background work on drop * Fix up some comments * Remove some duplicate code * Remove redundant workarounds for shutdown issues * Remode a redundant channel close in the block commit task * Remove a mistaken `!force` shutdown condition * Remove duplicate force-shutdown code and explain it better * Improve RPC error logging * Wait for chain tip updates in the RPC tests * Wait 2 seconds for chain tip updates before skipping them * Remove an unnecessary block_in_place() * Fix some test error messages that were changed by earlier fixes * Expand some comments, fix typos Co-authored-by: Marek <mail@marek.onl> * Actually drop children of failed blocks * Explain why we drop descendants of failed blocks * Clarify a comment * Wait for chain tip updates in a failing test on macOS * Clean duplicate finalized blocks when the non-finalized state activates * Send an error when receiving a duplicate finalized block * Update checkpoint block behaviour, document its consensus rule * Wait for chain tip changes in inbound_block_height_lookahead_limit test * Wait for the genesis block to commit in the fake peer set mempool tests * Disable unreliable mempool verification check in the send transaction test * Appease rustfmt * Use clear_finalized_block_queue() everywhere that blocks are dropped * Document how Finalized and NonFinalized clones are different * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * updates comments, renames send_process_queued, other minor cleanup * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * updates comments, renames send_process_queued, other minor cleanup * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * removes duplicate field definitions on StateService that were a result of a bad merge * update NotReadyToBeCommitted error message * Appear rustfmt * Fix doc links * Rename a function to initial_contextual_validity() * Do error tasks on Err, and success tasks on Ok * Simplify parent_error_map truncation * Rewrite best_tip() to use tip() * Rename latest_mem() to latest_non_finalized_state() ```sh fastmod latest_mem latest_non_finalized_state zebra* cargo fmt --all ``` * Simplify latest_non_finalized_state() using a new WatchReceiver API * Expand some error messages * Send the result after updating the channels, and document why * wait for chain_tip_update before cancelling download in mempool_cancel_mined * adds `sent_non_finalized_block_hashes` field to StateService * adds batched sent_hash insertions and checks sent hashes in queue_and_commit_non_finalized before adding a block to the queue * check that the `curr_buf` in SentHashes is not empty before pushing it to the `sent_bufs` * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * Fix rustfmt * Check for finalized block heights using zs_contains() * adds known_utxos field to SentHashes * updates comment on SentHashes.add method * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * return early when there's a duplicate hash in QueuedBlocks.queue instead of panicking * Make finalized UTXOs near the final checkpoint available for full block verification * Replace a checkpoint height literal with the actual config * Update mainnet and testnet checkpoints - 7 October 2022 * Fix some state service init arguments * Allow more lookahead in the downloader, but less lookahead in the syncer * Add the latest config to the tests, and fix the latest config check * Increase the number of finalized blocks checked for non-finalized block UTXO spends * fix(log): reduce verbose logs for block commits (#5348) * Remove some verbose block write channel logs * Only warn about tracing endpoint if the address is actually set * Use CloneError instead of formatting a non-cloneable error Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> * Increase block verify timeout * Work around a known block timeout bug by using a shorter timeout Co-authored-by: teor <teor@riseup.net> Co-authored-by: Marek <mail@marek.onl> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2022-10-11 12:25:45 -07:00
// Instead, cache the checkpoint list for each `network`.
let list = network.checkpoint_list();
change(state): Write non-finalized blocks to the state in a separate thread, to avoid network and RPC hangs (#5257) * Add a new block commit task and channels, that don't do anything yet * Add last_block_hash_sent to the state service, to avoid database accesses * Update last_block_hash_sent regardless of commit errors * Rename a field to StateService.max_queued_finalized_height * Commit finalized blocks to the state in a separate task * Check for panics in the block write task * Wait for the block commit task in tests, and check for errors * Always run a proptest that sleeps once * Add extra debugging to state shutdowns * Work around a RocksDB shutdown bug * Close the finalized block channel when we're finished with it * Only reset state queue once per error * Update some TODOs * Add a module doc comment * Drop channels and check for closed channels in the block commit task * Close state channels and tasks on drop * Remove some duplicate fields across StateService and ReadStateService * Try tweaking the shutdown steps * Update and clarify some comments * Clarify another comment * Don't try to cancel RocksDB background work on drop * Fix up some comments * Remove some duplicate code * Remove redundant workarounds for shutdown issues * Remode a redundant channel close in the block commit task * Remove a mistaken `!force` shutdown condition * Remove duplicate force-shutdown code and explain it better * Improve RPC error logging * Wait for chain tip updates in the RPC tests * Wait 2 seconds for chain tip updates before skipping them * Remove an unnecessary block_in_place() * Fix some test error messages that were changed by earlier fixes * Expand some comments, fix typos Co-authored-by: Marek <mail@marek.onl> * Actually drop children of failed blocks * Explain why we drop descendants of failed blocks * Clarify a comment * Wait for chain tip updates in a failing test on macOS * Clean duplicate finalized blocks when the non-finalized state activates * Send an error when receiving a duplicate finalized block * Update checkpoint block behaviour, document its consensus rule * Wait for chain tip changes in inbound_block_height_lookahead_limit test * Wait for the genesis block to commit in the fake peer set mempool tests * Disable unreliable mempool verification check in the send transaction test * Appease rustfmt * Use clear_finalized_block_queue() everywhere that blocks are dropped * Document how Finalized and NonFinalized clones are different * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * updates comments, renames send_process_queued, other minor cleanup * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * sends non-finalized blocks to the block write task * passes ZebraDb to commit_new_chain, commit_block, and no_duplicates_in_finalized_chain instead of FinalizedState * updates comments, renames send_process_queued, other minor cleanup * Update zebra-state/src/service/write.rs Co-authored-by: teor <teor@riseup.net> * update assert_block_can_be_validated comment * removes `mem` field from StateService * removes `disk` field from StateService and updates block_iter to use `ZebraDb` instead of the finalized state * updates tests that use the disk to use read_service.db instead * moves best_tip to a read fn and returns finalized & non-finalized states from setup instead of the state service * changes `contextual_validity` to get the network from the finalized_state instead of another param * swaps out StateService with FinalizedState and NonFinalizedState in tests * adds NotReadyToBeCommitted error and returns it from validate_and_commit when a blocks parent hash is not in any chain * removes NonFinalizedWriteCmd and calls, moves update_latest_channels above rsp_tx.send * makes parent_errors_map an indexmap * clears non-finalized block queue when the receiver is dropped and when the StateService is being dropped * removes duplicate field definitions on StateService that were a result of a bad merge * update NotReadyToBeCommitted error message * Appear rustfmt * Fix doc links * Rename a function to initial_contextual_validity() * Do error tasks on Err, and success tasks on Ok * Simplify parent_error_map truncation * Rewrite best_tip() to use tip() * Rename latest_mem() to latest_non_finalized_state() ```sh fastmod latest_mem latest_non_finalized_state zebra* cargo fmt --all ``` * Simplify latest_non_finalized_state() using a new WatchReceiver API * Expand some error messages * Send the result after updating the channels, and document why * wait for chain_tip_update before cancelling download in mempool_cancel_mined * adds `sent_non_finalized_block_hashes` field to StateService * adds batched sent_hash insertions and checks sent hashes in queue_and_commit_non_finalized before adding a block to the queue * check that the `curr_buf` in SentHashes is not empty before pushing it to the `sent_bufs` * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * Fix rustfmt * Check for finalized block heights using zs_contains() * adds known_utxos field to SentHashes * updates comment on SentHashes.add method * Apply suggestions from code review Co-authored-by: teor <teor@riseup.net> * return early when there's a duplicate hash in QueuedBlocks.queue instead of panicking * Make finalized UTXOs near the final checkpoint available for full block verification * Replace a checkpoint height literal with the actual config * Update mainnet and testnet checkpoints - 7 October 2022 * Fix some state service init arguments * Allow more lookahead in the downloader, but less lookahead in the syncer * Add the latest config to the tests, and fix the latest config check * Increase the number of finalized blocks checked for non-finalized block UTXO spends * fix(log): reduce verbose logs for block commits (#5348) * Remove some verbose block write channel logs * Only warn about tracing endpoint if the address is actually set * Use CloneError instead of formatting a non-cloneable error Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> * Increase block verify timeout * Work around a known block timeout bug by using a shorter timeout Co-authored-by: teor <teor@riseup.net> Co-authored-by: Marek <mail@marek.onl> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
2022-10-11 12:25:45 -07:00
let max_checkpoint_height = if config.checkpoint_sync {
list.max_height()
} else {
list.min_height_in_range(network.mandatory_checkpoint_height()..)
.expect("hardcoded checkpoint list extends past canopy activation")
};
(list, max_checkpoint_height)
}
/// The background task handles for `zebra-consensus` verifier initialization.
#[derive(Debug)]
pub struct BackgroundTaskHandles {
/// A handle to the state checkpoint verify task.
/// Finishes when all the checkpoints are verified, or when the state tip is reached.
pub state_checkpoint_verify_handle: JoinHandle<()>,
}