zebra/zebra-consensus/src/block.rs

289 lines
10 KiB
Rust
Raw Normal View History

//! Consensus-based block verification.
//!
//! In contrast to checkpoint verification, which only checks hardcoded
//! hashes, block verification checks all Zcash consensus rules.
//!
//! The block verifier performs all of the semantic validation checks.
//! If accepted, the block is sent to the state service for contextual
//! verification, where it may be accepted or rejected.
use std::{
future::Future,
pin::Pin,
sync::Arc,
task::{Context, Poll},
};
use chrono::Utc;
use futures::stream::FuturesUnordered;
use futures_util::FutureExt;
use thiserror::Error;
use tower::{Service, ServiceExt};
use tracing::Instrument;
use zebra_chain::{
amount::Amount,
block::{self, Block},
parameters::Network,
transparent,
work::equihash,
};
use zebra_state as zs;
use crate::{error::*, transaction as tx, BoxError};
pub mod check;
mod subsidy;
#[cfg(test)]
mod tests;
/// Asynchronous block verification.
#[derive(Debug)]
pub struct BlockVerifier<S, V> {
/// The network to be verified.
network: Network,
state_service: S,
transaction_verifier: V,
}
// TODO: dedupe with crate::error::BlockError
#[non_exhaustive]
#[allow(missing_docs)]
#[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(zebra_chain::block::BlockTimeError),
#[error("unable to commit block after semantic verification")]
Commit(#[source] BoxError),
#[error("invalid transaction")]
Transaction(#[from] TransactionError),
}
/// The maximum allowed number of legacy signature check operations in a block.
///
/// This consensus rule is not documented, so Zebra follows the `zcashd` implementation.
/// We re-use some `zcashd` C++ script code via `zebra-script` and `zcash_script`.
///
/// See:
/// <https://github.com/zcash/zcash/blob/bad7f7eadbbb3466bebe3354266c7f69f607fcfd/src/consensus/consensus.h#L30>
pub const MAX_BLOCK_SIGOPS: u64 = 20_000;
impl<S, V> BlockVerifier<S, V>
where
S: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
S::Future: Send + 'static,
V: Service<tx::Request, Response = tx::Response, Error = BoxError> + Send + Clone + 'static,
V::Future: Send + 'static,
{
pub fn new(network: Network, state_service: S, transaction_verifier: V) -> Self {
Self {
network,
state_service,
transaction_verifier,
}
}
}
impl<S, V> Service<Arc<Block>> for BlockVerifier<S, V>
where
S: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
S::Future: Send + 'static,
V: Service<tx::Request, Response = tx::Response, Error = BoxError> + Send + Clone + 'static,
V::Future: Send + 'static,
{
type Response = block::Hash;
type Error = VerifyBlockError;
type Future =
Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;
fn poll_ready(&mut self, _: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
// We use the state for contextual verification, and we expect those
// queries to be fast. So we don't need to call
// `state_service.poll_ready()` here.
Poll::Ready(Ok(()))
}
fn call(&mut self, block: Arc<Block>) -> Self::Future {
let mut state_service = self.state_service.clone();
let mut transaction_verifier = self.transaction_verifier.clone();
let network = self.network;
// We don't include the block hash, because it's likely already in a parent span
let span = tracing::debug_span!("block", height = ?block.coinbase_height());
async move {
let hash = block.hash();
// Check that this block is actually a new block.
tracing::trace!("checking that block is not already in state");
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
.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(BlockError::AlreadyInChain(hash, depth).into())
}
zs::Response::Depth(None) => {}
_ => unreachable!("wrong response to Request::Depth"),
}
tracing::trace!("performing block checks");
let height = block
.coinbase_height()
.ok_or(BlockError::MissingHeight(hash))?;
// Zebra does not support heights greater than
// [`block::Height::MAX`].
if height > block::Height::MAX {
Err(BlockError::MaxHeight(height, hash, block::Height::MAX))?;
}
// Do the difficulty checks first, to raise the threshold for
// attacks that use any other fields.
check::difficulty_is_valid(&block.header, network, &height, &hash)?;
check::equihash_solution_is_valid(&block.header)?;
// Next, check the Merkle root validity, to ensure that
// the header binds to the transactions in the blocks.
// Precomputing this avoids duplicating transaction hash computations.
let transaction_hashes: Arc<[_]> =
block.transactions.iter().map(|t| t.hash()).collect();
check::merkle_root_validity(network, &block, &transaction_hashes)?;
// Since errors cause an early exit, try to do the
// quick checks first.
// Quick field validity and structure checks
let now = Utc::now();
check::time_is_valid_at(&block.header, now, &height, &hash)
.map_err(VerifyBlockError::Time)?;
let coinbase_tx = check::coinbase_is_first(&block)?;
check::subsidy_is_valid(&block, network)?;
// Now do the slower checks
// Check compatibility with ZIP-212 shielded Sapling and Orchard coinbase output decryption
tx::check::coinbase_outputs_are_decryptable(&coinbase_tx, network, height)?;
// Send transactions to the transaction verifier to be checked
let mut async_checks = FuturesUnordered::new();
let known_utxos = Arc::new(transparent::new_ordered_outputs(
&block,
&transaction_hashes,
));
for transaction in &block.transactions {
let rsp = transaction_verifier
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
.expect("transaction verifier is always ready")
.call(tx::Request::Block {
transaction: transaction.clone(),
known_utxos: known_utxos.clone(),
height,
Validate transaction lock times (#3060) * Create a `LockTime::unlocked` helper constructor Returns a `LockTime` that is unlocked at the genesis block. * Return `Option<LockTime>` from `lock_time` method Prepare to return `None` for when a transaction has its lock time disabled. * Return `None` instead of zero `LockTime` Because a zero lock time means that the transaction was unlocked at the genesis block, so it was never actually locked. * Rephrase zero lock time check comment Clarify that the check is not redundant, and is necessary for the genesis transaction. Co-authored-by: teor <teor@riseup.net> * Add a `transparent::Input::sequence` getter method Retrieve a transparent input's sequence number. * Check if lock time is enabled by a sequence number Validate the consensus rule that the lock time is only enabled if at least one transparent input has a value different from `u32::MAX` as its sequence number. * Add more Zcash specific details to comment Explain the Zcash specific lock time behaviors. Co-authored-by: teor <teor@riseup.net> * Add `time` field to `Request::Block` variant The block time to use to check if the transaction was unlocked and allowed to be included in the block. * Add `Request::block_time` getter Returns the block time for the block that owns the transaction being validated or the current time plus a tolerance for mempool transactions. * Validate transaction lock times If they are enabled by a transaction's transparent input sequence numbers, make sure that they are in the past. * Add comments with consensus rule parts Make it easier to map what part of the consensus rule each match arm is responsible for. Co-authored-by: teor <teor@riseup.net>
2021-11-22 21:53:53 -08:00
time: block.header.time,
});
async_checks.push(rsp);
}
tracing::trace!(len = async_checks.len(), "built async tx checks");
// Get the transaction results back from the transaction verifier.
// Sum up some block totals from the transaction responses.
let mut legacy_sigop_count = 0;
let mut block_miner_fees = Ok(Amount::zero());
use futures::StreamExt;
while let Some(result) = async_checks.next().await {
tracing::trace!(?result, remaining = async_checks.len());
let response = result
.map_err(Into::into)
.map_err(VerifyBlockError::Transaction)?;
assert!(
matches!(response, tx::Response::Block { .. }),
"unexpected response from transaction verifier: {:?}",
response
);
legacy_sigop_count += response.legacy_sigop_count();
// Coinbase transactions consume the miner fee,
// so they don't add any value to the block's total miner fee.
if let Some(miner_fee) = response.miner_fee() {
block_miner_fees += miner_fee;
}
}
// Check the summed block totals
if legacy_sigop_count > MAX_BLOCK_SIGOPS {
Err(BlockError::TooManyTransparentSignatureOperations {
height,
hash,
legacy_sigop_count,
})?;
}
2020-10-26 06:11:52 -07:00
let block_miner_fees =
block_miner_fees.map_err(|amount_error| BlockError::SummingMinerFees {
height,
hash,
source: amount_error,
})?;
check::miner_fees_are_valid(&block, network, block_miner_fees)?;
// Finally, submit the block for contextual verification.
let new_outputs = Arc::try_unwrap(known_utxos)
.expect("all verification tasks using known_utxos are complete");
let prepared_block = zs::PreparedBlock {
block,
hash,
height,
new_outputs,
transaction_hashes,
};
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
.map_err(VerifyBlockError::Commit)?
.call(zs::Request::CommitBlock(prepared_block))
.await
.map_err(VerifyBlockError::Commit)?
{
zs::Response::Committed(committed_hash) => {
assert_eq!(committed_hash, hash, "state must commit correct hash");
Ok(hash)
}
_ => unreachable!("wrong response for CommitBlock"),
}
}
.instrument(span)
.boxed()
}
}