fix(consensus): Check that Zebra's state contains the social consensus chain on startup (#6163)
* Allow missing docs directly on derived error types * Make Request::BestChainBlockHash redirect to the ReadStateService * Re-write the checkpoint_sync documentation based on the latest consensus rules * Expose the underlying iterator for CheckpointList * Validate existing state block hashes at startup, but ignore the result * Monitor state block hash checkpoint task in the start command * Fix indentation * Make logging consistent * Explain the config needed for full security * Tidy required checkpoints docs, expand other docs * Add security and deprecation changelog entries * Replace task handle vector with a struct * Add a test that this consensus-critical code actually runs and finishes * Make some state methods and types available in tests * Add missing docs --------- Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
This commit is contained in:
parent
83d038c067
commit
4daedbc789
20
CHANGELOG.md
20
CHANGELOG.md
|
@ -4,6 +4,26 @@ All notable changes to Zebra are documented in this file.
|
|||
|
||||
The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), and this project adheres to [Semantic Versioning](https://semver.org).
|
||||
|
||||
## [Zebra 1.0.0-rc.5](https://github.com/ZcashFoundation/zebra/releases/tag/v1.0.0-rc.5) - 2023-02-TODO INSERT DATE HERE
|
||||
|
||||
In this release we ... (TODO)
|
||||
We also check that Zebra is following the consensus chain each time it starts up.
|
||||
|
||||
### Security
|
||||
- Check that Zebra's state contains the consensus chain each time it starts up. This implements
|
||||
the "settled network upgrade" consensus rule using all of Zebra's checkpoints ([#6163](https://github.com/ZcashFoundation/zebra/pull/6163)).
|
||||
*User action required:*
|
||||
- If your config is based on an old version of Zebra, or you have manually edited it,
|
||||
make sure `consensus.checkpoint_sync = true`.
|
||||
This option has been true by default since March 2022.
|
||||
|
||||
### Deprecated
|
||||
- The `consensus.checkpoint_sync` config in `zebrad.toml` is deprecated. It might be ignored or
|
||||
removed in a future release. ([#6163](https://github.com/ZcashFoundation/zebra/pull/6163))
|
||||
|
||||
TODO: add other changes here
|
||||
|
||||
|
||||
## [Zebra 1.0.0-rc.4](https://github.com/ZcashFoundation/zebra/releases/tag/v1.0.0-rc.4) - 2023-01-30
|
||||
|
||||
In this release we fixed bugs and inconsistencies between zcashd and zebrad in the output of the `getblocktemplate` RPC method. In addition, we added block proposal mode to the `getblocktemplate` RPC, while we continue the effort of adding and testing mining pool RPC methods.
|
||||
|
|
|
@ -21,9 +21,9 @@ use std::{
|
|||
use displaydoc::Display;
|
||||
use futures::{FutureExt, TryFutureExt};
|
||||
use thiserror::Error;
|
||||
use tokio::task::{spawn_blocking, JoinHandle};
|
||||
use tokio::task::JoinHandle;
|
||||
use tower::{buffer::Buffer, util::BoxService, Service, ServiceExt};
|
||||
use tracing::{instrument, Span};
|
||||
use tracing::{instrument, Instrument, Span};
|
||||
|
||||
use zebra_chain::{
|
||||
block::{self, Height},
|
||||
|
@ -237,22 +237,22 @@ pub async fn init<S>(
|
|||
BoxService<transaction::Request, transaction::Response, TransactionError>,
|
||||
transaction::Request,
|
||||
>,
|
||||
JoinHandle<()>,
|
||||
BackgroundTaskHandles,
|
||||
Height,
|
||||
)
|
||||
where
|
||||
S: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
|
||||
S::Future: Send + 'static,
|
||||
{
|
||||
// Pre-download Groth16 parameters in a separate thread.
|
||||
|
||||
// Give other tasks priority, before spawning the download task.
|
||||
// Give other tasks priority before spawning the download and checkpoint tasks.
|
||||
tokio::task::yield_now().await;
|
||||
|
||||
// Pre-download Groth16 parameters in a separate thread.
|
||||
|
||||
// The parameter download thread must be launched before initializing any verifiers.
|
||||
// Otherwise, the download might happen on the startup thread.
|
||||
let span = Span::current();
|
||||
let groth16_download_handle = spawn_blocking(move || {
|
||||
let groth16_download_handle = tokio::task::spawn_blocking(move || {
|
||||
span.in_scope(|| {
|
||||
if !debug_skip_parameter_preload {
|
||||
// The lazy static initializer does the download, if needed,
|
||||
|
@ -262,6 +262,79 @@ where
|
|||
})
|
||||
});
|
||||
|
||||
// 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 = CheckpointList::new(network);
|
||||
|
||||
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) => {
|
||||
tracing::warn!(
|
||||
"unexpected error: {e:?} in state request while verifying previous \
|
||||
state checkpoints"
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
tracing::info!("finished state checkpoint validation");
|
||||
}
|
||||
.instrument(Span::current()),
|
||||
);
|
||||
|
||||
// transaction verification
|
||||
|
||||
let transaction = transaction::Verifier::new(network, state_service.clone());
|
||||
|
@ -293,18 +366,18 @@ where
|
|||
|
||||
let chain = Buffer::new(BoxService::new(chain), VERIFIER_BUFFER_BOUND);
|
||||
|
||||
(
|
||||
chain,
|
||||
transaction,
|
||||
let task_handles = BackgroundTaskHandles {
|
||||
groth16_download_handle,
|
||||
max_checkpoint_height,
|
||||
)
|
||||
state_checkpoint_verify_handle,
|
||||
};
|
||||
|
||||
(chain, transaction, task_handles, max_checkpoint_height)
|
||||
}
|
||||
|
||||
/// 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 twice at startup.
|
||||
// TODO: Zebra parses the checkpoint list three times at startup.
|
||||
// Instead, cache the checkpoint list for each `network`.
|
||||
let list = CheckpointList::new(network);
|
||||
|
||||
|
@ -317,3 +390,15 @@ pub fn init_checkpoint_list(config: Config, network: Network) -> (CheckpointList
|
|||
|
||||
(list, max_checkpoint_height)
|
||||
}
|
||||
|
||||
/// The background task handles for `zebra-consensus` verifier initialization.
|
||||
#[derive(Debug)]
|
||||
pub struct BackgroundTaskHandles {
|
||||
/// A handle to the Groth16 parameter download task.
|
||||
/// Finishes when the parameters are downloaded and their checksums verified.
|
||||
pub groth16_download_handle: JoinHandle<()>,
|
||||
|
||||
/// 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<()>,
|
||||
}
|
||||
|
|
|
@ -196,4 +196,9 @@ impl CheckpointList {
|
|||
{
|
||||
self.0.range(range).map(|(height, _)| *height).next_back()
|
||||
}
|
||||
|
||||
/// Returns an iterator over all the checkpoints, in increasing height order.
|
||||
pub fn iter(&self) -> impl Iterator<Item = (&block::Height, &block::Hash)> {
|
||||
self.0.iter()
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,20 +1,34 @@
|
|||
//! Configuration for semantic verification which is run in parallel.
|
||||
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
/// Consensus configuration.
|
||||
/// Configuration for parallel semantic verification:
|
||||
/// <https://zebra.zfnd.org/dev/rfcs/0002-parallel-verification.html#definitions>
|
||||
#[derive(Clone, Debug, Deserialize, Serialize)]
|
||||
#[serde(deny_unknown_fields, default)]
|
||||
pub struct Config {
|
||||
/// Should Zebra use its optional checkpoints to sync?
|
||||
/// Should Zebra make sure that it follows the consensus chain while syncing?
|
||||
/// This is a developer-only option.
|
||||
///
|
||||
/// This option is `true` by default, and allows for faster chain synchronization.
|
||||
/// # Security
|
||||
///
|
||||
/// Zebra requires some checkpoints to validate legacy network upgrades.
|
||||
/// But it also ships with optional checkpoints, which can be used instead of full block validation.
|
||||
/// Disabling this option leaves your node vulnerable to some kinds of chain-based attacks.
|
||||
/// Zebra regularly updates its checkpoints to ensure nodes are following the best chain.
|
||||
///
|
||||
/// Disabling this option makes Zebra start full validation as soon as possible.
|
||||
/// This helps developers debug Zebra, by running full validation on more blocks.
|
||||
/// # Details
|
||||
///
|
||||
/// Future versions of Zebra may change the required and optional checkpoints.
|
||||
/// This option is `true` by default, because it prevents some kinds of chain attacks.
|
||||
///
|
||||
/// Disabling this option makes Zebra start full validation earlier.
|
||||
/// It is slower and less secure.
|
||||
///
|
||||
/// Zebra requires some checkpoints to simplify validation of legacy network upgrades.
|
||||
/// Required checkpoints are always active, even when this option is `false`.
|
||||
///
|
||||
/// # Deprecation
|
||||
///
|
||||
/// For security reasons, this option might be deprecated or ignored in a future Zebra
|
||||
/// release.
|
||||
pub checkpoint_sync: bool,
|
||||
|
||||
/// Skip the pre-download of Groth16 parameters if this option is true.
|
||||
|
|
|
@ -20,6 +20,7 @@ use proptest_derive::Arbitrary;
|
|||
const MAX_EXPIRY_HEIGHT: block::Height = block::Height::MAX_EXPIRY_HEIGHT;
|
||||
|
||||
#[derive(Error, Copy, Clone, Debug, PartialEq, Eq)]
|
||||
#[allow(missing_docs)]
|
||||
pub enum SubsidyError {
|
||||
#[error("no coinbase transaction in block")]
|
||||
NoCoinbase,
|
||||
|
@ -36,6 +37,7 @@ pub enum SubsidyError {
|
|||
|
||||
#[derive(Error, Clone, Debug, PartialEq, Eq)]
|
||||
#[cfg_attr(any(test, feature = "proptest-impl"), derive(Arbitrary))]
|
||||
#[allow(missing_docs)]
|
||||
pub enum TransactionError {
|
||||
#[error("first transaction must be coinbase")]
|
||||
CoinbasePosition,
|
||||
|
@ -226,6 +228,7 @@ impl From<BoxError> for TransactionError {
|
|||
}
|
||||
|
||||
#[derive(Error, Clone, Debug, PartialEq, Eq)]
|
||||
#[allow(missing_docs)]
|
||||
pub enum BlockError {
|
||||
#[error("block contains invalid transactions")]
|
||||
Transaction(#[from] TransactionError),
|
||||
|
|
|
@ -40,11 +40,10 @@ mod config;
|
|||
mod parameters;
|
||||
mod primitives;
|
||||
mod script;
|
||||
pub mod transaction;
|
||||
|
||||
pub mod chain;
|
||||
#[allow(missing_docs)]
|
||||
pub mod error;
|
||||
pub mod transaction;
|
||||
|
||||
pub use block::{
|
||||
subsidy::{
|
||||
|
|
|
@ -15,11 +15,12 @@
|
|||
#[macro_use]
|
||||
extern crate tracing;
|
||||
|
||||
pub mod constants;
|
||||
|
||||
#[cfg(any(test, feature = "proptest-impl"))]
|
||||
pub mod arbitrary;
|
||||
|
||||
mod config;
|
||||
pub mod constants;
|
||||
mod error;
|
||||
mod request;
|
||||
mod response;
|
||||
|
@ -32,8 +33,6 @@ pub use config::{check_and_delete_old_databases, Config};
|
|||
pub use constants::MAX_BLOCK_REORG_HEIGHT;
|
||||
pub use error::{BoxError, CloneError, CommitBlockError, ValidateContextError};
|
||||
pub use request::{FinalizedBlock, HashOrHeight, PreparedBlock, ReadRequest, Request};
|
||||
#[cfg(feature = "getblocktemplate-rpcs")]
|
||||
pub use response::GetBlockTemplateChainInfo;
|
||||
pub use response::{ReadResponse, Response};
|
||||
pub use service::{
|
||||
chain_tip::{ChainTipChange, LatestChainTip, TipAction},
|
||||
|
@ -42,11 +41,15 @@ pub use service::{
|
|||
OutputIndex, OutputLocation, TransactionLocation,
|
||||
};
|
||||
|
||||
#[cfg(feature = "getblocktemplate-rpcs")]
|
||||
pub use response::GetBlockTemplateChainInfo;
|
||||
|
||||
#[cfg(any(test, feature = "proptest-impl"))]
|
||||
pub use service::{
|
||||
arbitrary::{populated_state, CHAIN_TIP_UPDATE_WAIT_LIMIT},
|
||||
chain_tip::{ChainTipBlock, ChainTipSender},
|
||||
init_test, init_test_services,
|
||||
finalized_state::{DiskWriteBatch, WriteDisk},
|
||||
init_test, init_test_services, ReadStateService,
|
||||
};
|
||||
|
||||
pub(crate) use request::ContextuallyValidBlock;
|
||||
|
|
|
@ -599,6 +599,14 @@ pub enum Request {
|
|||
/// Returns [`Response::BestChainNextMedianTimePast`] when successful.
|
||||
BestChainNextMedianTimePast,
|
||||
|
||||
/// Looks up a block hash by height in the current best chain.
|
||||
///
|
||||
/// Returns
|
||||
///
|
||||
/// * [`Response::BlockHash(Some(hash))`](Response::BlockHash) if the block is in the best chain;
|
||||
/// * [`Response::BlockHash(None)`](Response::BlockHash) otherwise.
|
||||
BestChainBlockHash(block::Height),
|
||||
|
||||
#[cfg(feature = "getblocktemplate-rpcs")]
|
||||
/// Performs contextual validation of the given block, but does not commit it to the state.
|
||||
///
|
||||
|
@ -625,6 +633,7 @@ impl Request {
|
|||
"best_chain_tip_nullifiers_anchors"
|
||||
}
|
||||
Request::BestChainNextMedianTimePast => "best_chain_next_median_time_past",
|
||||
Request::BestChainBlockHash(_) => "best_chain_block_hash",
|
||||
#[cfg(feature = "getblocktemplate-rpcs")]
|
||||
Request::CheckBlockProposalValidity(_) => "check_block_proposal_validity",
|
||||
}
|
||||
|
@ -910,6 +919,7 @@ impl TryFrom<Request> for ReadRequest {
|
|||
Request::Tip => Ok(ReadRequest::Tip),
|
||||
Request::Depth(hash) => Ok(ReadRequest::Depth(hash)),
|
||||
Request::BestChainNextMedianTimePast => Ok(ReadRequest::BestChainNextMedianTimePast),
|
||||
Request::BestChainBlockHash(hash) => Ok(ReadRequest::BestChainBlockHash(hash)),
|
||||
|
||||
Request::Block(hash_or_height) => Ok(ReadRequest::Block(hash_or_height)),
|
||||
Request::Transaction(tx_hash) => Ok(ReadRequest::Transaction(tx_hash)),
|
||||
|
@ -933,14 +943,14 @@ impl TryFrom<Request> for ReadRequest {
|
|||
Err("ReadService does not write blocks")
|
||||
}
|
||||
|
||||
Request::AwaitUtxo(_) => Err("ReadService does not track pending UTXOs. \
|
||||
Manually convert the request to ReadRequest::AnyChainUtxo, \
|
||||
and handle pending UTXOs"),
|
||||
|
||||
#[cfg(feature = "getblocktemplate-rpcs")]
|
||||
Request::CheckBlockProposalValidity(prepared) => {
|
||||
Ok(ReadRequest::CheckBlockProposalValidity(prepared))
|
||||
}
|
||||
|
||||
Request::AwaitUtxo(_) => Err("ReadService does not track pending UTXOs. \
|
||||
Manually convert the request to ReadRequest::AnyChainUtxo, \
|
||||
and handle pending UTXOs"),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -65,15 +65,19 @@ pub enum Response {
|
|||
/// Contains the median-time-past for the *next* block on the best chain.
|
||||
BestChainNextMedianTimePast(DateTime32),
|
||||
|
||||
/// Response to [`Request::BestChainBlockHash`](Request::BestChainBlockHash) with the
|
||||
/// specified block hash.
|
||||
BlockHash(Option<block::Hash>),
|
||||
|
||||
#[cfg(feature = "getblocktemplate-rpcs")]
|
||||
/// Response to [`Request::CheckBlockProposalValidity`](crate::Request::CheckBlockProposalValidity)
|
||||
/// Response to [`Request::CheckBlockProposalValidity`](Request::CheckBlockProposalValidity)
|
||||
ValidBlockProposal,
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug, PartialEq, Eq)]
|
||||
/// A response to a read-only
|
||||
/// [`ReadStateService`](crate::service::ReadStateService)'s
|
||||
/// [`ReadRequest`](crate::ReadRequest).
|
||||
/// [`ReadRequest`](ReadRequest).
|
||||
pub enum ReadResponse {
|
||||
/// Response to [`ReadRequest::Tip`] with the current best chain tip.
|
||||
Tip(Option<(block::Height, block::Hash)>),
|
||||
|
@ -137,21 +141,21 @@ pub enum ReadResponse {
|
|||
/// Contains the median-time-past for the *next* block on the best chain.
|
||||
BestChainNextMedianTimePast(DateTime32),
|
||||
|
||||
/// Response to [`ReadRequest::BestChainBlockHash`](crate::ReadRequest::BestChainBlockHash) with the
|
||||
/// Response to [`ReadRequest::BestChainBlockHash`](ReadRequest::BestChainBlockHash) with the
|
||||
/// specified block hash.
|
||||
BlockHash(Option<block::Hash>),
|
||||
|
||||
#[cfg(feature = "getblocktemplate-rpcs")]
|
||||
/// Response to [`ReadRequest::ChainInfo`](crate::ReadRequest::ChainInfo) with the state
|
||||
/// Response to [`ReadRequest::ChainInfo`](ReadRequest::ChainInfo) with the state
|
||||
/// information needed by the `getblocktemplate` RPC method.
|
||||
ChainInfo(GetBlockTemplateChainInfo),
|
||||
|
||||
#[cfg(feature = "getblocktemplate-rpcs")]
|
||||
/// Response to [`ReadRequest::SolutionRate`](crate::ReadRequest::SolutionRate)
|
||||
/// Response to [`ReadRequest::SolutionRate`](ReadRequest::SolutionRate)
|
||||
SolutionRate(Option<u128>),
|
||||
|
||||
#[cfg(feature = "getblocktemplate-rpcs")]
|
||||
/// Response to [`ReadRequest::CheckBlockProposalValidity`](crate::ReadRequest::CheckBlockProposalValidity)
|
||||
/// Response to [`ReadRequest::CheckBlockProposalValidity`](ReadRequest::CheckBlockProposalValidity)
|
||||
ValidBlockProposal,
|
||||
}
|
||||
|
||||
|
@ -204,6 +208,7 @@ impl TryFrom<ReadResponse> for Response {
|
|||
ReadResponse::Tip(height_and_hash) => Ok(Response::Tip(height_and_hash)),
|
||||
ReadResponse::Depth(depth) => Ok(Response::Depth(depth)),
|
||||
ReadResponse::BestChainNextMedianTimePast(median_time_past) => Ok(Response::BestChainNextMedianTimePast(median_time_past)),
|
||||
ReadResponse::BlockHash(hash) => Ok(Response::BlockHash(hash)),
|
||||
|
||||
ReadResponse::Block(block) => Ok(Response::Block(block)),
|
||||
ReadResponse::Transaction(tx_and_height) => {
|
||||
|
@ -230,10 +235,6 @@ impl TryFrom<ReadResponse> for Response {
|
|||
Err("there is no corresponding Response for this ReadResponse")
|
||||
}
|
||||
|
||||
ReadResponse::BlockHash(_) => {
|
||||
Err("there is no corresponding Response for this ReadResponse")
|
||||
}
|
||||
|
||||
#[cfg(feature = "getblocktemplate-rpcs")]
|
||||
ReadResponse::ValidBlockProposal => Ok(Response::ValidBlockProposal),
|
||||
|
||||
|
|
|
@ -809,6 +809,13 @@ impl ReadStateService {
|
|||
fn latest_best_chain(&self) -> Option<Arc<Chain>> {
|
||||
self.latest_non_finalized_state().best_chain().cloned()
|
||||
}
|
||||
|
||||
/// Test-only access to the inner database.
|
||||
/// Can be used to modify the database without doing any consensus checks.
|
||||
#[cfg(any(test, feature = "proptest-impl"))]
|
||||
pub fn db(&self) -> &ZebraDb {
|
||||
&self.db
|
||||
}
|
||||
}
|
||||
|
||||
impl Service<Request> for StateService {
|
||||
|
@ -1032,6 +1039,7 @@ impl Service<Request> for StateService {
|
|||
Request::Tip
|
||||
| Request::Depth(_)
|
||||
| Request::BestChainNextMedianTimePast
|
||||
| Request::BestChainBlockHash(_)
|
||||
| Request::BlockLocator
|
||||
| Request::Transaction(_)
|
||||
| Request::UnspentBestChainUtxo(_)
|
||||
|
@ -1874,9 +1882,7 @@ pub fn spawn_init(
|
|||
|
||||
/// Returns a [`StateService`] with an ephemeral [`Config`] and a buffer with a single slot.
|
||||
///
|
||||
/// This can be used to create a state service for testing.
|
||||
///
|
||||
/// See also [`init`].
|
||||
/// This can be used to create a state service for testing. See also [`init`].
|
||||
#[cfg(any(test, feature = "proptest-impl"))]
|
||||
pub fn init_test(network: Network) -> Buffer<BoxService<Request, Response, BoxError>, Request> {
|
||||
// TODO: pass max_checkpoint_height and checkpoint_verify_concurrency limit
|
||||
|
|
|
@ -42,6 +42,9 @@ pub use disk_format::{OutputIndex, OutputLocation, TransactionLocation};
|
|||
|
||||
pub(super) use zebra_db::ZebraDb;
|
||||
|
||||
#[cfg(any(test, feature = "proptest-impl"))]
|
||||
pub use disk_db::{DiskWriteBatch, WriteDisk};
|
||||
|
||||
/// The finalized part of the chain state, stored in the db.
|
||||
///
|
||||
/// `rocksdb` allows concurrent writes through a shared reference,
|
||||
|
|
|
@ -67,7 +67,7 @@ impl ZebraDb {
|
|||
/// It should only be used in debugging or test code, immediately before a manual shutdown.
|
||||
///
|
||||
/// See [`DiskDb::shutdown`] for details.
|
||||
pub(crate) fn shutdown(&mut self, force: bool) {
|
||||
pub fn shutdown(&mut self, force: bool) {
|
||||
self.check_max_on_disk_tip_height();
|
||||
|
||||
self.db.shutdown(force);
|
||||
|
|
|
@ -1,7 +1,5 @@
|
|||
//! Arbitrary value generation and test harnesses for high-level typed database access.
|
||||
|
||||
#![allow(dead_code)]
|
||||
|
||||
use std::ops::Deref;
|
||||
|
||||
use zebra_chain::{
|
||||
|
|
|
@ -74,6 +74,7 @@ use tokio::{pin, select, sync::oneshot};
|
|||
use tower::{builder::ServiceBuilder, util::BoxService};
|
||||
use tracing_futures::Instrument;
|
||||
|
||||
use zebra_consensus::chain::BackgroundTaskHandles;
|
||||
use zebra_rpc::server::RpcServer;
|
||||
|
||||
use crate::{
|
||||
|
@ -140,7 +141,7 @@ impl StartCmd {
|
|||
zebra_network::init(config.network.clone(), inbound, latest_chain_tip.clone()).await;
|
||||
|
||||
info!("initializing verifiers");
|
||||
let (chain_verifier, tx_verifier, mut groth16_download_handle, max_checkpoint_height) =
|
||||
let (chain_verifier, tx_verifier, consensus_task_handles, max_checkpoint_height) =
|
||||
zebra_consensus::chain::init(
|
||||
config.consensus.clone(),
|
||||
config.network.network,
|
||||
|
@ -252,8 +253,17 @@ impl StartCmd {
|
|||
pin!(progress_task_handle);
|
||||
|
||||
// startup tasks
|
||||
let BackgroundTaskHandles {
|
||||
mut groth16_download_handle,
|
||||
mut state_checkpoint_verify_handle,
|
||||
} = consensus_task_handles;
|
||||
|
||||
let groth16_download_handle_fused = (&mut groth16_download_handle).fuse();
|
||||
pin!(groth16_download_handle_fused);
|
||||
|
||||
let state_checkpoint_verify_handle_fused = (&mut state_checkpoint_verify_handle).fuse();
|
||||
pin!(state_checkpoint_verify_handle_fused);
|
||||
|
||||
let old_databases_task_handle_fused = (&mut old_databases_task_handle).fuse();
|
||||
pin!(old_databases_task_handle_fused);
|
||||
|
||||
|
@ -319,7 +329,17 @@ impl StartCmd {
|
|||
Ok(())
|
||||
}
|
||||
|
||||
// The same for the old databases task, we expect it to finish while Zebra is running.
|
||||
// We also expect the state checkpoint verify task to finish.
|
||||
state_checkpoint_verify_result = &mut state_checkpoint_verify_handle_fused => {
|
||||
state_checkpoint_verify_result
|
||||
.unwrap_or_else(|_| panic!(
|
||||
"unexpected panic checking previous state followed the best chain"));
|
||||
|
||||
exit_when_task_finishes = false;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
// And the old databases task should finish while Zebra is running.
|
||||
old_databases_result = &mut old_databases_task_handle_fused => {
|
||||
old_databases_result
|
||||
.unwrap_or_else(|_| panic!(
|
||||
|
@ -355,6 +375,7 @@ impl StartCmd {
|
|||
|
||||
// startup tasks
|
||||
groth16_download_handle.abort();
|
||||
state_checkpoint_verify_handle.abort();
|
||||
old_databases_task_handle.abort();
|
||||
|
||||
// Wait until the RPC server shuts down.
|
||||
|
|
|
@ -314,6 +314,10 @@ pub fn check_sync_logs_until(
|
|||
if check_legacy_chain {
|
||||
zebrad.expect_stdout_line_matches("starting legacy chain check")?;
|
||||
zebrad.expect_stdout_line_matches("no legacy chain found")?;
|
||||
|
||||
zebrad.expect_stdout_line_matches("starting state checkpoint validation")?;
|
||||
// TODO: what if the mempool is enabled for debugging before this finishes?
|
||||
zebrad.expect_stdout_line_matches("finished state checkpoint validation")?;
|
||||
}
|
||||
|
||||
// before the stop regex, expect mempool activation
|
||||
|
|
Loading…
Reference in New Issue