zebra/zebra-state/src/service.rs

1390 lines
52 KiB
Rust
Raw Normal View History

//! [`tower::Service`]s for Zebra's cached chain state.
//!
//! Zebra provides cached state access via two main services:
//! - [`StateService`]: a read-write service that waits for queued blocks.
//! - [`ReadStateService`]: a read-only service that answers from the most
//! recent committed block.
//!
//! Most users should prefer [`ReadStateService`], unless they need to wait for
//! verified blocks to be committed. (For example, the syncer and mempool
//! tasks.)
//!
//! Zebra also provides access to the best chain tip via:
//! - [`LatestChainTip`]: a read-only channel that contains the latest committed
//! tip.
//! - [`ChainTipChange`]: a read-only channel that can asynchronously await
//! chain tip changes.
use std::{
convert,
future::Future,
pin::Pin,
sync::Arc,
task::{Context, Poll},
time::{Duration, Instant},
};
use futures::future::FutureExt;
use tokio::sync::{oneshot, watch};
use tower::{util::BoxService, Service, ServiceExt};
use tracing::{instrument, Instrument, Span};
#[cfg(any(test, feature = "proptest-impl"))]
use tower::buffer::Buffer;
use zebra_chain::{
block::{self, CountedHeader},
diagnostic::CodeTimer,
parameters::{Network, NetworkUpgrade},
transparent,
};
use crate::{
constants::{MAX_FIND_BLOCK_HASHES_RESULTS, MAX_FIND_BLOCK_HEADERS_RESULTS_FOR_ZEBRA},
service::{
chain_tip::{ChainTipBlock, ChainTipChange, ChainTipSender, LatestChainTip},
finalized_state::{FinalizedState, ZebraDb},
non_finalized_state::{Chain, NonFinalizedState, QueuedBlocks},
pending_utxos::PendingUtxos,
watch_receiver::WatchReceiver,
},
BoxError, CloneError, CommitBlockError, Config, FinalizedBlock, PreparedBlock, ReadRequest,
ReadResponse, Request, Response, ValidateContextError,
};
pub mod block_iter;
pub mod chain_tip;
pub mod watch_receiver;
pub(crate) mod check;
2020-11-16 16:05:35 -08:00
mod finalized_state;
mod non_finalized_state;
mod pending_utxos;
pub(crate) mod read;
#[cfg(any(test, feature = "proptest-impl"))]
pub mod arbitrary;
#[cfg(test)]
mod tests;
pub use finalized_state::{OutputIndex, OutputLocation, TransactionLocation};
pub type QueuedBlock = (
PreparedBlock,
oneshot::Sender<Result<block::Hash, BoxError>>,
);
pub type QueuedFinalized = (
FinalizedBlock,
oneshot::Sender<Result<block::Hash, BoxError>>,
);
/// A read-write service for Zebra's cached blockchain state.
///
/// This service modifies and provides access to:
/// - the non-finalized state: the ~100 most recent blocks.
/// Zebra allows chain forks in the non-finalized state,
/// stores it in memory, and re-downloads it when restarted.
/// - the finalized state: older blocks that have many confirmations.
/// Zebra stores the single best chain in the finalized state,
/// and re-loads it from disk when restarted.
///
/// Read requests to this service are buffered, then processed concurrently.
/// Block write requests are buffered, then queued, then processed in order by a separate task.
///
/// Most state users can get faster read responses using the [`ReadStateService`],
/// because its requests do not share a [`tower::buffer::Buffer`] with block write requests.
///
/// To quickly get the latest block, use [`LatestChainTip`] or [`ChainTipChange`].
/// They can read the latest block directly, without queueing any requests.
#[derive(Debug)]
pub(crate) struct StateService {
// Configuration
//
/// The configured Zcash network.
network: Network,
// Exclusively Writeable State
//
/// The finalized chain state, including its on-disk database.
pub(crate) disk: FinalizedState,
/// The non-finalized chain state, including its in-memory chain forks.
mem: NonFinalizedState,
// Queued Non-Finalized Blocks
//
/// Blocks awaiting their parent blocks for contextual verification.
queued_blocks: QueuedBlocks,
// Pending UTXO Request Tracking
//
/// The set of outpoints with pending requests for their associated transparent::Output.
pending_utxos: PendingUtxos,
/// Instant tracking the last time `pending_utxos` was pruned.
last_prune: Instant,
// Concurrently Readable State
//
/// A sender channel used to update the current best chain tip for
/// [`LatestChainTip`] and [`ChainTipChange`].
chain_tip_sender: ChainTipSender,
/// A sender channel used to update the current best non-finalized chain for [`ReadStateService`].
best_chain_sender: watch::Sender<Option<Arc<Chain>>>,
/// A cloneable [`ReadStateService`], used to answer concurrent read requests.
///
/// TODO: move concurrent read requests to [`ReadRequest`], and remove `read_service`.
read_service: ReadStateService,
}
/// A read-only service for accessing Zebra's cached blockchain state.
///
/// This service provides read-only access to:
/// - the non-finalized state: the ~100 most recent blocks.
/// - the finalized state: older blocks that have many confirmations.
///
/// Requests to this service are processed in parallel,
/// ignoring any blocks queued by the read-write [`StateService`].
///
/// This quick response behavior is better for most state users.
/// It allows other async tasks to make progress while concurrently reading data from disk.
#[derive(Clone, Debug)]
pub struct ReadStateService {
// Configuration
//
/// The configured Zcash network.
network: Network,
// Shared Concurrently Readable State
//
/// The shared inner on-disk database for the finalized state.
///
/// RocksDB allows reads and writes via a shared reference,
/// but [`ZebraDb`] doesn't expose any write methods or types.
///
/// This chain is updated concurrently with requests,
/// so it might include some block data that is also in `best_mem`.
db: ZebraDb,
/// A watch channel for the current best in-memory chain.
///
/// This chain is only updated between requests,
/// so it might include some block data that is also on `disk`.
best_chain_receiver: WatchReceiver<Option<Arc<Chain>>>,
}
impl StateService {
const PRUNE_INTERVAL: Duration = Duration::from_secs(30);
/// Create a new read-write state service.
/// Returns the read-write and read-only state services,
/// and read-only watch channels for its best chain tip.
pub fn new(
config: Config,
network: Network,
) -> (Self, ReadStateService, LatestChainTip, ChainTipChange) {
let timer = CodeTimer::start();
let disk = FinalizedState::new(&config, network);
timer.finish(module_path!(), line!(), "opening finalized state database");
let timer = CodeTimer::start();
let initial_tip = disk
.db()
.tip_block()
.map(FinalizedBlock::from)
.map(ChainTipBlock::from);
timer.finish(module_path!(), line!(), "fetching database tip");
let timer = CodeTimer::start();
let (chain_tip_sender, latest_chain_tip, chain_tip_change) =
ChainTipSender::new(initial_tip, network);
Reject connections from outdated peers (#2519) * Simplify state service initialization in test Use the test helper function to remove redundant code. * Create `BestTipHeight` helper type This type abstracts away the calculation of the best tip height based on the finalized block height and the best non-finalized chain's tip. * Add `best_tip_height` field to `StateService` The receiver endpoint is currently ignored. * Return receiver endpoint from service constructor Make it available so that the best tip height can be watched. * Update finalized height after finalizing blocks After blocks from the queue are finalized and committed to disk, update the finalized block height. * Update best non-finalized height after validation Update the value of the best non-finalized chain tip block height after a new block is committed to the non-finalized state. * Update finalized height after loading from disk When `FinalizedState` is first created, it loads the state from persistent storage, and the finalized tip height is updated. Therefore, the `best_tip_height` must be notified of the initial value. * Update the finalized height on checkpoint commit When a checkpointed block is commited, it bypasses the non-finalized state, so there's an extra place where the finalized height has to be updated. * Add `best_tip_height` to `Handshake` service It can be configured using the `Builder::with_best_tip_height`. It's currently not used, but it will be used to determine if a connection to a remote peer should be rejected or not based on that peer's protocol version. * Require best tip height to init. `zebra_network` Without it the handshake service can't properly enforce the minimum network protocol version from peers. Zebrad obtains the best tip height endpoint from `zebra_state`, and the test vectors simply use a dummy endpoint that's fixed at the genesis height. * Pass `best_tip_height` to proto. ver. negotiation The protocol version negotiation code will reject connections to peers if they are using an old protocol version. An old version is determined based on the current known best chain tip height. * Handle an optional height in `Version` Fallback to the genesis height in `None` is specified. * Reject connections to peers on old proto. versions Avoid connecting to peers that are on protocol versions that don't recognize a network update. * Document why peers on old versions are rejected Describe why it's a security issue above the check. * Test if `BestTipHeight` starts with `None` Check if initially there is no best tip height. * Test if best tip height is max. of latest values After applying a list of random updates where each one either sets the finalized height or the non-finalized height, check that the best tip height is the maximum of the most recently set finalized height and the most recently set non-finalized height. * Add `queue_and_commit_finalized` method A small refactor to make testing easier. The handling of requests for committing non-finalized and finalized blocks is now more consistent. * Add `assert_block_can_be_validated` helper Refactor to move into a separate method some assertions that are done before a block is validated. This is to allow moving these assertions more easily to simplify testing. * Remove redundant PoW block assertion It's also checked in `zebra_state::service::check::block_is_contextually_valid`, and it was getting in the way of tests that received a gossiped block before finalizing enough blocks. * Create a test strategy for test vector chain Splits a chain loaded from the test vectors in two parts, containing the blocks to finalize and the blocks to keep in the non-finalized state. * Test committing blocks update best tip height Create a mock blockchain state, with a chain of finalized blocks and a chain of non-finalized blocks. Commit all the blocks appropriately, and verify that the best tip height is updated. Co-authored-by: teor <teor@riseup.net>
2021-08-08 16:52:52 -07:00
let mem = NonFinalizedState::new(network);
let (read_service, best_chain_sender) = ReadStateService::new(&disk);
let queued_blocks = QueuedBlocks::default();
let pending_utxos = PendingUtxos::default();
let state = Self {
network,
disk,
mem,
queued_blocks,
pending_utxos,
last_prune: Instant::now(),
chain_tip_sender,
best_chain_sender,
read_service: read_service.clone(),
};
timer.finish(module_path!(), line!(), "initializing state service");
tracing::info!("starting legacy chain check");
let timer = CodeTimer::start();
if let Some(tip) = state.best_tip() {
let nu5_activation_height = NetworkUpgrade::Nu5
.activation_height(network)
.expect("NU5 activation height is set");
if let Err(error) = check::legacy_chain(
nu5_activation_height,
state.any_ancestor_blocks(tip.1),
state.network,
) {
let legacy_db_path = state.disk.path().to_path_buf();
panic!(
"Cached state contains a legacy chain.\n\
An outdated Zebra version did not know about a recent network upgrade,\n\
so it followed a legacy chain using outdated consensus branch rules.\n\
Hint: Delete your database, and restart Zebra to do a full sync.\n\
Database path: {legacy_db_path:?}\n\
Error: {error:?}",
);
}
}
tracing::info!("cached state consensus branch is valid: no legacy chain found");
timer.finish(module_path!(), line!(), "legacy chain check");
(state, read_service, latest_chain_tip, chain_tip_change)
Reject connections from outdated peers (#2519) * Simplify state service initialization in test Use the test helper function to remove redundant code. * Create `BestTipHeight` helper type This type abstracts away the calculation of the best tip height based on the finalized block height and the best non-finalized chain's tip. * Add `best_tip_height` field to `StateService` The receiver endpoint is currently ignored. * Return receiver endpoint from service constructor Make it available so that the best tip height can be watched. * Update finalized height after finalizing blocks After blocks from the queue are finalized and committed to disk, update the finalized block height. * Update best non-finalized height after validation Update the value of the best non-finalized chain tip block height after a new block is committed to the non-finalized state. * Update finalized height after loading from disk When `FinalizedState` is first created, it loads the state from persistent storage, and the finalized tip height is updated. Therefore, the `best_tip_height` must be notified of the initial value. * Update the finalized height on checkpoint commit When a checkpointed block is commited, it bypasses the non-finalized state, so there's an extra place where the finalized height has to be updated. * Add `best_tip_height` to `Handshake` service It can be configured using the `Builder::with_best_tip_height`. It's currently not used, but it will be used to determine if a connection to a remote peer should be rejected or not based on that peer's protocol version. * Require best tip height to init. `zebra_network` Without it the handshake service can't properly enforce the minimum network protocol version from peers. Zebrad obtains the best tip height endpoint from `zebra_state`, and the test vectors simply use a dummy endpoint that's fixed at the genesis height. * Pass `best_tip_height` to proto. ver. negotiation The protocol version negotiation code will reject connections to peers if they are using an old protocol version. An old version is determined based on the current known best chain tip height. * Handle an optional height in `Version` Fallback to the genesis height in `None` is specified. * Reject connections to peers on old proto. versions Avoid connecting to peers that are on protocol versions that don't recognize a network update. * Document why peers on old versions are rejected Describe why it's a security issue above the check. * Test if `BestTipHeight` starts with `None` Check if initially there is no best tip height. * Test if best tip height is max. of latest values After applying a list of random updates where each one either sets the finalized height or the non-finalized height, check that the best tip height is the maximum of the most recently set finalized height and the most recently set non-finalized height. * Add `queue_and_commit_finalized` method A small refactor to make testing easier. The handling of requests for committing non-finalized and finalized blocks is now more consistent. * Add `assert_block_can_be_validated` helper Refactor to move into a separate method some assertions that are done before a block is validated. This is to allow moving these assertions more easily to simplify testing. * Remove redundant PoW block assertion It's also checked in `zebra_state::service::check::block_is_contextually_valid`, and it was getting in the way of tests that received a gossiped block before finalizing enough blocks. * Create a test strategy for test vector chain Splits a chain loaded from the test vectors in two parts, containing the blocks to finalize and the blocks to keep in the non-finalized state. * Test committing blocks update best tip height Create a mock blockchain state, with a chain of finalized blocks and a chain of non-finalized blocks. Commit all the blocks appropriately, and verify that the best tip height is updated. Co-authored-by: teor <teor@riseup.net>
2021-08-08 16:52:52 -07:00
}
/// Queue a finalized block for verification and storage in the finalized state.
fn queue_and_commit_finalized(
&mut self,
finalized: FinalizedBlock,
) -> oneshot::Receiver<Result<block::Hash, BoxError>> {
let (rsp_tx, rsp_rx) = oneshot::channel();
// TODO: move this code into the state block commit task:
// - queue_and_commit_finalized()'s commit_finalized() call becomes a send to the block commit channel
// - run commit_finalized() in the state block commit task
// - run the metrics update in queue_and_commit_finalized() in the block commit task
// - run the set_finalized_tip() in this function in the state block commit task
// - move all that code to the inner service
let tip_block = self
.disk
.queue_and_commit_finalized((finalized, rsp_tx))
.map(ChainTipBlock::from);
self.chain_tip_sender.set_finalized_tip(tip_block);
Reject connections from outdated peers (#2519) * Simplify state service initialization in test Use the test helper function to remove redundant code. * Create `BestTipHeight` helper type This type abstracts away the calculation of the best tip height based on the finalized block height and the best non-finalized chain's tip. * Add `best_tip_height` field to `StateService` The receiver endpoint is currently ignored. * Return receiver endpoint from service constructor Make it available so that the best tip height can be watched. * Update finalized height after finalizing blocks After blocks from the queue are finalized and committed to disk, update the finalized block height. * Update best non-finalized height after validation Update the value of the best non-finalized chain tip block height after a new block is committed to the non-finalized state. * Update finalized height after loading from disk When `FinalizedState` is first created, it loads the state from persistent storage, and the finalized tip height is updated. Therefore, the `best_tip_height` must be notified of the initial value. * Update the finalized height on checkpoint commit When a checkpointed block is commited, it bypasses the non-finalized state, so there's an extra place where the finalized height has to be updated. * Add `best_tip_height` to `Handshake` service It can be configured using the `Builder::with_best_tip_height`. It's currently not used, but it will be used to determine if a connection to a remote peer should be rejected or not based on that peer's protocol version. * Require best tip height to init. `zebra_network` Without it the handshake service can't properly enforce the minimum network protocol version from peers. Zebrad obtains the best tip height endpoint from `zebra_state`, and the test vectors simply use a dummy endpoint that's fixed at the genesis height. * Pass `best_tip_height` to proto. ver. negotiation The protocol version negotiation code will reject connections to peers if they are using an old protocol version. An old version is determined based on the current known best chain tip height. * Handle an optional height in `Version` Fallback to the genesis height in `None` is specified. * Reject connections to peers on old proto. versions Avoid connecting to peers that are on protocol versions that don't recognize a network update. * Document why peers on old versions are rejected Describe why it's a security issue above the check. * Test if `BestTipHeight` starts with `None` Check if initially there is no best tip height. * Test if best tip height is max. of latest values After applying a list of random updates where each one either sets the finalized height or the non-finalized height, check that the best tip height is the maximum of the most recently set finalized height and the most recently set non-finalized height. * Add `queue_and_commit_finalized` method A small refactor to make testing easier. The handling of requests for committing non-finalized and finalized blocks is now more consistent. * Add `assert_block_can_be_validated` helper Refactor to move into a separate method some assertions that are done before a block is validated. This is to allow moving these assertions more easily to simplify testing. * Remove redundant PoW block assertion It's also checked in `zebra_state::service::check::block_is_contextually_valid`, and it was getting in the way of tests that received a gossiped block before finalizing enough blocks. * Create a test strategy for test vector chain Splits a chain loaded from the test vectors in two parts, containing the blocks to finalize and the blocks to keep in the non-finalized state. * Test committing blocks update best tip height Create a mock blockchain state, with a chain of finalized blocks and a chain of non-finalized blocks. Commit all the blocks appropriately, and verify that the best tip height is updated. Co-authored-by: teor <teor@riseup.net>
2021-08-08 16:52:52 -07:00
rsp_rx
}
/// Queue a non finalized block for verification and check if any queued
/// blocks are ready to be verified and committed to the state.
///
/// This function encodes the logic for [committing non-finalized blocks][1]
/// in RFC0005.
///
/// [1]: https://zebra.zfnd.org/dev/rfcs/0005-state-updates.html#committing-non-finalized-blocks
#[instrument(level = "debug", skip(self, prepared))]
fn queue_and_commit_non_finalized(
&mut self,
prepared: PreparedBlock,
) -> oneshot::Receiver<Result<block::Hash, BoxError>> {
tracing::debug!(block = %prepared.block, "queueing block for contextual verification");
let parent_hash = prepared.block.header.previous_block_hash;
if self.mem.any_chain_contains(&prepared.hash)
|| self.disk.db().hash(prepared.height).is_some()
{
let (rsp_tx, rsp_rx) = oneshot::channel();
let _ = rsp_tx.send(Err("block is already committed to the state".into()));
return rsp_rx;
}
// Request::CommitBlock contract: a request to commit a block which has
// been queued but not yet committed to the state fails the older
// request and replaces it with the newer request.
let rsp_rx = if let Some((_, old_rsp_tx)) = self.queued_blocks.get_mut(&prepared.hash) {
tracing::debug!("replacing older queued request with new request");
let (mut rsp_tx, rsp_rx) = oneshot::channel();
std::mem::swap(old_rsp_tx, &mut rsp_tx);
let _ = rsp_tx.send(Err("replaced by newer request".into()));
rsp_rx
} else {
let (rsp_tx, rsp_rx) = oneshot::channel();
self.queued_blocks.queue((prepared, rsp_tx));
rsp_rx
};
if !self.can_fork_chain_at(&parent_hash) {
tracing::trace!("unready to verify, returning early");
return rsp_rx;
}
// TODO: move this code into the state block commit task:
// - process_queued()'s validate_and_commit() call becomes a send to the block commit channel
// - run validate_and_commit() in the state block commit task
// - run all the rest of the code in this function in the state block commit task
// - move all that code to the inner service
self.process_queued(parent_hash);
while self.mem.best_chain_len() > crate::constants::MAX_BLOCK_REORG_HEIGHT {
tracing::trace!("finalizing block past the reorg limit");
let finalized_with_trees = self.mem.finalize();
self.disk
.commit_finalized_direct(finalized_with_trees, "best non-finalized chain root")
.expect(
"expected that errors would not occur when writing to disk or updating note commitment and history trees",
);
}
let finalized_tip_height = self.disk.db().finalized_tip_height().expect(
"Finalized state must have at least one block before committing non-finalized state",
Reject connections from outdated peers (#2519) * Simplify state service initialization in test Use the test helper function to remove redundant code. * Create `BestTipHeight` helper type This type abstracts away the calculation of the best tip height based on the finalized block height and the best non-finalized chain's tip. * Add `best_tip_height` field to `StateService` The receiver endpoint is currently ignored. * Return receiver endpoint from service constructor Make it available so that the best tip height can be watched. * Update finalized height after finalizing blocks After blocks from the queue are finalized and committed to disk, update the finalized block height. * Update best non-finalized height after validation Update the value of the best non-finalized chain tip block height after a new block is committed to the non-finalized state. * Update finalized height after loading from disk When `FinalizedState` is first created, it loads the state from persistent storage, and the finalized tip height is updated. Therefore, the `best_tip_height` must be notified of the initial value. * Update the finalized height on checkpoint commit When a checkpointed block is commited, it bypasses the non-finalized state, so there's an extra place where the finalized height has to be updated. * Add `best_tip_height` to `Handshake` service It can be configured using the `Builder::with_best_tip_height`. It's currently not used, but it will be used to determine if a connection to a remote peer should be rejected or not based on that peer's protocol version. * Require best tip height to init. `zebra_network` Without it the handshake service can't properly enforce the minimum network protocol version from peers. Zebrad obtains the best tip height endpoint from `zebra_state`, and the test vectors simply use a dummy endpoint that's fixed at the genesis height. * Pass `best_tip_height` to proto. ver. negotiation The protocol version negotiation code will reject connections to peers if they are using an old protocol version. An old version is determined based on the current known best chain tip height. * Handle an optional height in `Version` Fallback to the genesis height in `None` is specified. * Reject connections to peers on old proto. versions Avoid connecting to peers that are on protocol versions that don't recognize a network update. * Document why peers on old versions are rejected Describe why it's a security issue above the check. * Test if `BestTipHeight` starts with `None` Check if initially there is no best tip height. * Test if best tip height is max. of latest values After applying a list of random updates where each one either sets the finalized height or the non-finalized height, check that the best tip height is the maximum of the most recently set finalized height and the most recently set non-finalized height. * Add `queue_and_commit_finalized` method A small refactor to make testing easier. The handling of requests for committing non-finalized and finalized blocks is now more consistent. * Add `assert_block_can_be_validated` helper Refactor to move into a separate method some assertions that are done before a block is validated. This is to allow moving these assertions more easily to simplify testing. * Remove redundant PoW block assertion It's also checked in `zebra_state::service::check::block_is_contextually_valid`, and it was getting in the way of tests that received a gossiped block before finalizing enough blocks. * Create a test strategy for test vector chain Splits a chain loaded from the test vectors in two parts, containing the blocks to finalize and the blocks to keep in the non-finalized state. * Test committing blocks update best tip height Create a mock blockchain state, with a chain of finalized blocks and a chain of non-finalized blocks. Commit all the blocks appropriately, and verify that the best tip height is updated. Co-authored-by: teor <teor@riseup.net>
2021-08-08 16:52:52 -07:00
);
self.queued_blocks.prune_by_height(finalized_tip_height);
let tip_block_height = self.update_latest_chain_channels();
Make block metrics more accurate (#2835) * Check for state errors before updating metrics Previously, the metrics would be updated for some rejected blocks. * Clarify and expand block verification metrics Rename checkpoint-specific metrics to clarify their purpose. Add metrics for: - finalized blocks on disk - blocks verified using the full block verifier (this metric was previously incorrectly called `zcash_chain_verified_block_height`) * Update dashboard metric names Also: - add some extra block height metrics - fix a dashboard name * Add exact block heights to Grafana dashboards * Add a missing comment * grafana: use 0 decimals for metrics Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * grafana: show the entire height instead of abbreviated Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * grafana: show the entire height instead of abbreviated Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Fix typo in metric name Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Move height gauges to the state, so they are correct If we update height gauges in futures, they can execute out of order, so the metrics can be incorrect. Instead: - move the height gauges to the state, and update them based on the best tip - move the verified block counts to the state - continue to include all verified blocks on all non-finalized chains (not just the best chain) * Show exact checkpoint heights in the dashboard Co-authored-by: Deirdre Connolly <deirdre@zfnd.org> Co-authored-by: Conrado Gouvea <conrado@zfnd.org>
2021-10-07 06:42:38 -07:00
// update metrics using the best non-finalized tip
if let Some(tip_block_height) = tip_block_height {
Make block metrics more accurate (#2835) * Check for state errors before updating metrics Previously, the metrics would be updated for some rejected blocks. * Clarify and expand block verification metrics Rename checkpoint-specific metrics to clarify their purpose. Add metrics for: - finalized blocks on disk - blocks verified using the full block verifier (this metric was previously incorrectly called `zcash_chain_verified_block_height`) * Update dashboard metric names Also: - add some extra block height metrics - fix a dashboard name * Add exact block heights to Grafana dashboards * Add a missing comment * grafana: use 0 decimals for metrics Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * grafana: show the entire height instead of abbreviated Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * grafana: show the entire height instead of abbreviated Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Fix typo in metric name Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Move height gauges to the state, so they are correct If we update height gauges in futures, they can execute out of order, so the metrics can be incorrect. Instead: - move the height gauges to the state, and update them based on the best tip - move the verified block counts to the state - continue to include all verified blocks on all non-finalized chains (not just the best chain) * Show exact checkpoint heights in the dashboard Co-authored-by: Deirdre Connolly <deirdre@zfnd.org> Co-authored-by: Conrado Gouvea <conrado@zfnd.org>
2021-10-07 06:42:38 -07:00
metrics::gauge!(
"state.full_verifier.committed.block.height",
tip_block_height.0 as f64,
Make block metrics more accurate (#2835) * Check for state errors before updating metrics Previously, the metrics would be updated for some rejected blocks. * Clarify and expand block verification metrics Rename checkpoint-specific metrics to clarify their purpose. Add metrics for: - finalized blocks on disk - blocks verified using the full block verifier (this metric was previously incorrectly called `zcash_chain_verified_block_height`) * Update dashboard metric names Also: - add some extra block height metrics - fix a dashboard name * Add exact block heights to Grafana dashboards * Add a missing comment * grafana: use 0 decimals for metrics Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * grafana: show the entire height instead of abbreviated Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * grafana: show the entire height instead of abbreviated Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Fix typo in metric name Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Move height gauges to the state, so they are correct If we update height gauges in futures, they can execute out of order, so the metrics can be incorrect. Instead: - move the height gauges to the state, and update them based on the best tip - move the verified block counts to the state - continue to include all verified blocks on all non-finalized chains (not just the best chain) * Show exact checkpoint heights in the dashboard Co-authored-by: Deirdre Connolly <deirdre@zfnd.org> Co-authored-by: Conrado Gouvea <conrado@zfnd.org>
2021-10-07 06:42:38 -07:00
);
// This height gauge is updated for both fully verified and checkpoint blocks.
// These updates can't conflict, because the state makes sure that blocks
// are committed in order.
metrics::gauge!(
"zcash.chain.verified.block.height",
tip_block_height.0 as f64,
);
Make block metrics more accurate (#2835) * Check for state errors before updating metrics Previously, the metrics would be updated for some rejected blocks. * Clarify and expand block verification metrics Rename checkpoint-specific metrics to clarify their purpose. Add metrics for: - finalized blocks on disk - blocks verified using the full block verifier (this metric was previously incorrectly called `zcash_chain_verified_block_height`) * Update dashboard metric names Also: - add some extra block height metrics - fix a dashboard name * Add exact block heights to Grafana dashboards * Add a missing comment * grafana: use 0 decimals for metrics Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * grafana: show the entire height instead of abbreviated Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * grafana: show the entire height instead of abbreviated Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Fix typo in metric name Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Move height gauges to the state, so they are correct If we update height gauges in futures, they can execute out of order, so the metrics can be incorrect. Instead: - move the height gauges to the state, and update them based on the best tip - move the verified block counts to the state - continue to include all verified blocks on all non-finalized chains (not just the best chain) * Show exact checkpoint heights in the dashboard Co-authored-by: Deirdre Connolly <deirdre@zfnd.org> Co-authored-by: Conrado Gouvea <conrado@zfnd.org>
2021-10-07 06:42:38 -07:00
}
tracing::trace!("finished processing queued block");
rsp_rx
}
/// Update the [`LatestChainTip`], [`ChainTipChange`], and `best_chain_sender`
/// channels with the latest non-finalized [`ChainTipBlock`] and
/// [`Chain`][1].
///
/// Returns the latest non-finalized chain tip height, or `None` if the
/// non-finalized state is empty.
///
/// [1]: non_finalized_state::Chain
#[instrument(level = "debug", skip(self))]
fn update_latest_chain_channels(&mut self) -> Option<block::Height> {
let best_chain = self.mem.best_chain();
let tip_block = best_chain
.and_then(|chain| chain.tip_block())
.cloned()
.map(ChainTipBlock::from);
let tip_block_height = tip_block.as_ref().map(|block| block.height);
// The RPC service uses the ReadStateService, but it is not turned on by default.
if self.best_chain_sender.receiver_count() > 0 {
// If the final receiver was just dropped, ignore the error.
let _ = self.best_chain_sender.send(best_chain.cloned());
}
self.chain_tip_sender.set_best_non_finalized_tip(tip_block);
tip_block_height
}
/// Run contextual validation on the prepared block and add it to the
/// non-finalized state if it is contextually valid.
2021-11-30 08:05:35 -08:00
#[tracing::instrument(level = "debug", skip(self, prepared))]
fn validate_and_commit(&mut self, prepared: PreparedBlock) -> Result<(), CommitBlockError> {
self.check_contextual_validity(&prepared)?;
let parent_hash = prepared.block.header.previous_block_hash;
if self.disk.db().finalized_tip_hash() == parent_hash {
self.mem.commit_new_chain(prepared, self.disk.db())?;
} else {
self.mem.commit_block(prepared, self.disk.db())?;
}
Ok(())
}
/// Returns `true` if `hash` is a valid previous block hash for new non-finalized blocks.
fn can_fork_chain_at(&self, hash: &block::Hash) -> bool {
self.mem.any_chain_contains(hash) || &self.disk.db().finalized_tip_hash() == hash
}
/// Attempt to validate and commit all queued blocks whose parents have
/// recently arrived starting from `new_parent`, in breadth-first ordering.
2021-11-30 08:05:35 -08:00
#[tracing::instrument(level = "debug", skip(self, new_parent))]
fn process_queued(&mut self, new_parent: block::Hash) {
let mut new_parents: Vec<(block::Hash, Result<(), CloneError>)> =
vec![(new_parent, Ok(()))];
while let Some((parent_hash, parent_result)) = new_parents.pop() {
let queued_children = self.queued_blocks.dequeue_children(parent_hash);
for (child, rsp_tx) in queued_children {
2020-11-22 19:38:25 -08:00
let child_hash = child.hash;
let result;
// If the block is invalid, reject any descendant blocks.
//
// At this point, we know that the block and all its descendants
// are invalid, because we checked all the consensus rules before
// committing the block to the non-finalized state.
// (These checks also bind the transaction data to the block
// header, using the transaction merkle tree and authorizing data
// commitment.)
if let Err(ref parent_error) = parent_result {
tracing::trace!(
?child_hash,
?parent_error,
"rejecting queued child due to parent error"
);
result = Err(parent_error.clone());
} else {
tracing::trace!(?child_hash, "validating queued child");
result = self.validate_and_commit(child).map_err(CloneError::from);
Make block metrics more accurate (#2835) * Check for state errors before updating metrics Previously, the metrics would be updated for some rejected blocks. * Clarify and expand block verification metrics Rename checkpoint-specific metrics to clarify their purpose. Add metrics for: - finalized blocks on disk - blocks verified using the full block verifier (this metric was previously incorrectly called `zcash_chain_verified_block_height`) * Update dashboard metric names Also: - add some extra block height metrics - fix a dashboard name * Add exact block heights to Grafana dashboards * Add a missing comment * grafana: use 0 decimals for metrics Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * grafana: show the entire height instead of abbreviated Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * grafana: show the entire height instead of abbreviated Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Fix typo in metric name Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Move height gauges to the state, so they are correct If we update height gauges in futures, they can execute out of order, so the metrics can be incorrect. Instead: - move the height gauges to the state, and update them based on the best tip - move the verified block counts to the state - continue to include all verified blocks on all non-finalized chains (not just the best chain) * Show exact checkpoint heights in the dashboard Co-authored-by: Deirdre Connolly <deirdre@zfnd.org> Co-authored-by: Conrado Gouvea <conrado@zfnd.org>
2021-10-07 06:42:38 -07:00
if result.is_ok() {
// Update the metrics if semantic and contextual validation passes
metrics::counter!("state.full_verifier.committed.block.count", 1);
metrics::counter!("zcash.chain.verified.block.total", 1);
}
}
let _ = rsp_tx.send(result.clone().map(|()| child_hash).map_err(BoxError::from));
new_parents.push((child_hash, result));
}
}
}
/// Check that the prepared block is contextually valid for the configured
/// network, based on the committed finalized and non-finalized state.
///
/// Note: some additional contextual validity checks are performed by the
/// non-finalized [`Chain`].
fn check_contextual_validity(
&mut self,
prepared: &PreparedBlock,
) -> Result<(), ValidateContextError> {
let relevant_chain = self.any_ancestor_blocks(prepared.block.header.previous_block_hash);
// Security: check proof of work before any other checks
check::block_is_valid_for_recent_chain(
prepared,
self.network,
self.disk.db().finalized_tip_height(),
relevant_chain,
)?;
check::nullifier::no_duplicates_in_finalized_chain(prepared, self.disk.db())?;
Ok(())
}
/// Return the tip of the current best chain.
pub fn best_tip(&self) -> Option<(block::Height, block::Hash)> {
self.mem.best_tip().or_else(|| self.disk.db().tip())
}
/// Return the height for the block at `hash` in any chain.
pub fn any_height_by_hash(&self, hash: block::Hash) -> Option<block::Height> {
self.mem
.any_height_by_hash(hash)
.or_else(|| self.disk.db().height(hash))
}
/// Return the [`transparent::Utxo`] pointed to by `outpoint`, if it exists
/// in any chain, or in any pending block.
///
/// Some of the returned UTXOs may be invalid, because:
/// - they are not in the best chain, or
/// - their block fails contextual validation.
pub fn any_utxo(&self, outpoint: &transparent::OutPoint) -> Option<transparent::Utxo> {
// We ignore any UTXOs in FinalizedState.queued_by_prev_hash,
// because it is only used during checkpoint verification.
self.mem
.any_utxo(outpoint)
.or_else(|| self.queued_blocks.utxo(outpoint))
.or_else(|| {
self.disk
.db()
.utxo(outpoint)
.map(|ordered_utxo| ordered_utxo.utxo)
})
}
/// Return an iterator over the relevant chain of the block identified by
/// `hash`, in order from the largest height to the genesis block.
///
/// The block identified by `hash` is included in the chain of blocks yielded
/// by the iterator. `hash` can come from any chain.
pub fn any_ancestor_blocks(&self, hash: block::Hash) -> block_iter::Iter<'_> {
block_iter::Iter {
service: self,
state: block_iter::IterState::NonFinalized(hash),
}
}
Reject connections from outdated peers (#2519) * Simplify state service initialization in test Use the test helper function to remove redundant code. * Create `BestTipHeight` helper type This type abstracts away the calculation of the best tip height based on the finalized block height and the best non-finalized chain's tip. * Add `best_tip_height` field to `StateService` The receiver endpoint is currently ignored. * Return receiver endpoint from service constructor Make it available so that the best tip height can be watched. * Update finalized height after finalizing blocks After blocks from the queue are finalized and committed to disk, update the finalized block height. * Update best non-finalized height after validation Update the value of the best non-finalized chain tip block height after a new block is committed to the non-finalized state. * Update finalized height after loading from disk When `FinalizedState` is first created, it loads the state from persistent storage, and the finalized tip height is updated. Therefore, the `best_tip_height` must be notified of the initial value. * Update the finalized height on checkpoint commit When a checkpointed block is commited, it bypasses the non-finalized state, so there's an extra place where the finalized height has to be updated. * Add `best_tip_height` to `Handshake` service It can be configured using the `Builder::with_best_tip_height`. It's currently not used, but it will be used to determine if a connection to a remote peer should be rejected or not based on that peer's protocol version. * Require best tip height to init. `zebra_network` Without it the handshake service can't properly enforce the minimum network protocol version from peers. Zebrad obtains the best tip height endpoint from `zebra_state`, and the test vectors simply use a dummy endpoint that's fixed at the genesis height. * Pass `best_tip_height` to proto. ver. negotiation The protocol version negotiation code will reject connections to peers if they are using an old protocol version. An old version is determined based on the current known best chain tip height. * Handle an optional height in `Version` Fallback to the genesis height in `None` is specified. * Reject connections to peers on old proto. versions Avoid connecting to peers that are on protocol versions that don't recognize a network update. * Document why peers on old versions are rejected Describe why it's a security issue above the check. * Test if `BestTipHeight` starts with `None` Check if initially there is no best tip height. * Test if best tip height is max. of latest values After applying a list of random updates where each one either sets the finalized height or the non-finalized height, check that the best tip height is the maximum of the most recently set finalized height and the most recently set non-finalized height. * Add `queue_and_commit_finalized` method A small refactor to make testing easier. The handling of requests for committing non-finalized and finalized blocks is now more consistent. * Add `assert_block_can_be_validated` helper Refactor to move into a separate method some assertions that are done before a block is validated. This is to allow moving these assertions more easily to simplify testing. * Remove redundant PoW block assertion It's also checked in `zebra_state::service::check::block_is_contextually_valid`, and it was getting in the way of tests that received a gossiped block before finalizing enough blocks. * Create a test strategy for test vector chain Splits a chain loaded from the test vectors in two parts, containing the blocks to finalize and the blocks to keep in the non-finalized state. * Test committing blocks update best tip height Create a mock blockchain state, with a chain of finalized blocks and a chain of non-finalized blocks. Commit all the blocks appropriately, and verify that the best tip height is updated. Co-authored-by: teor <teor@riseup.net>
2021-08-08 16:52:52 -07:00
/// Assert some assumptions about the prepared `block` before it is validated.
fn assert_block_can_be_validated(&self, block: &PreparedBlock) {
// required by validate_and_commit, moved here to make testing easier
assert!(
block.height > self.network.mandatory_checkpoint_height(),
"invalid non-finalized block height: the canopy checkpoint is mandatory, pre-canopy \
blocks, and the canopy activation block, must be committed to the state as finalized \
blocks"
);
}
}
impl ReadStateService {
/// Creates a new read-only state service, using the provided finalized state.
///
/// Returns the newly created service,
/// and a watch channel for updating its best non-finalized chain.
pub(crate) fn new(disk: &FinalizedState) -> (Self, watch::Sender<Option<Arc<Chain>>>) {
let (best_chain_sender, best_chain_receiver) = watch::channel(None);
let read_service = Self {
network: disk.network(),
db: disk.db().clone(),
best_chain_receiver: WatchReceiver::new(best_chain_receiver),
};
tracing::info!("created new read-only state service");
(read_service, best_chain_sender)
}
}
impl Service<Request> for StateService {
type Response = Response;
type Error = BoxError;
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>> {
let now = Instant::now();
if self.last_prune + Self::PRUNE_INTERVAL < now {
let tip = self.best_tip();
let old_len = self.pending_utxos.len();
self.pending_utxos.prune();
self.last_prune = now;
let new_len = self.pending_utxos.len();
let prune_count = old_len
.checked_sub(new_len)
.expect("prune does not add any utxo requests");
if prune_count > 0 {
tracing::debug!(
?old_len,
?new_len,
?prune_count,
?tip,
"pruned utxo requests"
);
} else {
tracing::debug!(len = ?old_len, ?tip, "no utxo requests needed pruning");
}
}
Poll::Ready(Ok(()))
}
#[instrument(name = "state", skip(self, req))]
fn call(&mut self, req: Request) -> Self::Future {
match req {
// Uses queued_blocks and pending_utxos in the StateService
// Accesses shared writeable state in the StateService, NonFinalizedState, and ZebraDb.
Request::CommitBlock(prepared) => {
metrics::counter!(
"state.requests",
1,
"service" => "state",
"type" => "commit_block",
);
let timer = CodeTimer::start();
Reject connections from outdated peers (#2519) * Simplify state service initialization in test Use the test helper function to remove redundant code. * Create `BestTipHeight` helper type This type abstracts away the calculation of the best tip height based on the finalized block height and the best non-finalized chain's tip. * Add `best_tip_height` field to `StateService` The receiver endpoint is currently ignored. * Return receiver endpoint from service constructor Make it available so that the best tip height can be watched. * Update finalized height after finalizing blocks After blocks from the queue are finalized and committed to disk, update the finalized block height. * Update best non-finalized height after validation Update the value of the best non-finalized chain tip block height after a new block is committed to the non-finalized state. * Update finalized height after loading from disk When `FinalizedState` is first created, it loads the state from persistent storage, and the finalized tip height is updated. Therefore, the `best_tip_height` must be notified of the initial value. * Update the finalized height on checkpoint commit When a checkpointed block is commited, it bypasses the non-finalized state, so there's an extra place where the finalized height has to be updated. * Add `best_tip_height` to `Handshake` service It can be configured using the `Builder::with_best_tip_height`. It's currently not used, but it will be used to determine if a connection to a remote peer should be rejected or not based on that peer's protocol version. * Require best tip height to init. `zebra_network` Without it the handshake service can't properly enforce the minimum network protocol version from peers. Zebrad obtains the best tip height endpoint from `zebra_state`, and the test vectors simply use a dummy endpoint that's fixed at the genesis height. * Pass `best_tip_height` to proto. ver. negotiation The protocol version negotiation code will reject connections to peers if they are using an old protocol version. An old version is determined based on the current known best chain tip height. * Handle an optional height in `Version` Fallback to the genesis height in `None` is specified. * Reject connections to peers on old proto. versions Avoid connecting to peers that are on protocol versions that don't recognize a network update. * Document why peers on old versions are rejected Describe why it's a security issue above the check. * Test if `BestTipHeight` starts with `None` Check if initially there is no best tip height. * Test if best tip height is max. of latest values After applying a list of random updates where each one either sets the finalized height or the non-finalized height, check that the best tip height is the maximum of the most recently set finalized height and the most recently set non-finalized height. * Add `queue_and_commit_finalized` method A small refactor to make testing easier. The handling of requests for committing non-finalized and finalized blocks is now more consistent. * Add `assert_block_can_be_validated` helper Refactor to move into a separate method some assertions that are done before a block is validated. This is to allow moving these assertions more easily to simplify testing. * Remove redundant PoW block assertion It's also checked in `zebra_state::service::check::block_is_contextually_valid`, and it was getting in the way of tests that received a gossiped block before finalizing enough blocks. * Create a test strategy for test vector chain Splits a chain loaded from the test vectors in two parts, containing the blocks to finalize and the blocks to keep in the non-finalized state. * Test committing blocks update best tip height Create a mock blockchain state, with a chain of finalized blocks and a chain of non-finalized blocks. Commit all the blocks appropriately, and verify that the best tip height is updated. Co-authored-by: teor <teor@riseup.net>
2021-08-08 16:52:52 -07:00
self.assert_block_can_be_validated(&prepared);
self.pending_utxos
.check_against_ordered(&prepared.new_outputs);
// # Performance
//
// Allow other async tasks to make progress while blocks are being verified
// and written to disk. But wait for the blocks to finish committing,
// so that `StateService` multi-block queries always observe a consistent state.
//
// Since each block is spawned into its own task,
// there shouldn't be any other code running in the same task,
// so we don't need to worry about blocking it:
// https://docs.rs/tokio/latest/tokio/task/fn.block_in_place.html
let span = Span::current();
let rsp_rx = tokio::task::block_in_place(move || {
span.in_scope(|| self.queue_and_commit_non_finalized(prepared))
});
// The work is all done, the future just waits on a channel for the result
timer.finish(module_path!(), line!(), "CommitBlock");
let span = Span::current();
async move {
rsp_rx
.await
.map_err(|_recv_error| {
BoxError::from("block was dropped from the state CommitBlock queue")
})
// TODO: replace with Result::flatten once it stabilises
// https://github.com/rust-lang/rust/issues/70142
.and_then(convert::identity)
.map(Response::Committed)
.map_err(Into::into)
}
.instrument(span)
.boxed()
}
// Uses queued_by_prev_hash in the FinalizedState and pending_utxos in the StateService.
// Accesses shared writeable state in the StateService, FinalizedState, and ZebraDb.
Request::CommitFinalizedBlock(finalized) => {
metrics::counter!(
"state.requests",
1,
"service" => "state",
"type" => "commit_finalized_block",
);
let timer = CodeTimer::start();
// # Consensus
//
// A non-finalized block verification could have called AwaitUtxo
// before this finalized block arrived in the state.
// So we need to check for pending UTXOs here for non-finalized blocks,
// even though it is redundant for most finalized blocks.
// (Finalized blocks are verified using block hash checkpoints
// and transaction merkle tree block header commitments.)
self.pending_utxos.check_against(&finalized.new_outputs);
// # Performance
//
// Allow other async tasks to make progress while blocks are being verified
// and written to disk.
//
// See the note in `CommitBlock` for more details.
let span = Span::current();
let rsp_rx = tokio::task::block_in_place(move || {
span.in_scope(|| self.queue_and_commit_finalized(finalized))
});
// The work is all done, the future just waits on a channel for the result
timer.finish(module_path!(), line!(), "CommitFinalizedBlock");
let span = Span::current();
async move {
rsp_rx
.await
.map_err(|_recv_error| {
BoxError::from(
"block was dropped from the state CommitFinalizedBlock queue",
)
})
// TODO: replace with Result::flatten once it stabilises
// https://github.com/rust-lang/rust/issues/70142
.and_then(convert::identity)
2020-09-10 10:52:51 -07:00
.map(Response::Committed)
.map_err(Into::into)
}
.instrument(span)
.boxed()
}
// TODO: add a name() method to Request, and combine all the read requests
//
// Runs concurrently using the ReadStateService
Request::Depth(_) => {
metrics::counter!(
"state.requests",
1,
"service" => "state",
"type" => "depth",
);
// Redirect the request to the concurrent ReadStateService
let read_service = self.read_service.clone();
async move {
let req = req
.try_into()
.expect("ReadRequest conversion should not fail");
let rsp = read_service.oneshot(req).await?;
let rsp = rsp.try_into().expect("Response conversion should not fail");
Ok(rsp)
}
.boxed()
}
// Runs concurrently using the ReadStateService
Request::Tip => {
metrics::counter!(
"state.requests",
1,
"service" => "state",
"type" => "tip",
);
// Redirect the request to the concurrent ReadStateService
let read_service = self.read_service.clone();
async move {
let req = req
.try_into()
.expect("ReadRequest conversion should not fail");
let rsp = read_service.oneshot(req).await?;
let rsp = rsp.try_into().expect("Response conversion should not fail");
Ok(rsp)
}
.boxed()
}
// Runs concurrently using the ReadStateService
Request::BlockLocator => {
metrics::counter!(
"state.requests",
1,
"service" => "state",
"type" => "block_locator",
);
// Redirect the request to the concurrent ReadStateService
let read_service = self.read_service.clone();
async move {
let req = req
.try_into()
.expect("ReadRequest conversion should not fail");
let rsp = read_service.oneshot(req).await?;
let rsp = rsp.try_into().expect("Response conversion should not fail");
Ok(rsp)
}
.boxed()
}
// Runs concurrently using the ReadStateService
Request::Transaction(_) => {
metrics::counter!(
"state.requests",
1,
"service" => "state",
"type" => "transaction",
);
// Redirect the request to the concurrent ReadStateService
let read_service = self.read_service.clone();
async move {
let req = req
.try_into()
.expect("ReadRequest conversion should not fail");
let rsp = read_service.oneshot(req).await?;
let rsp = rsp.try_into().expect("Response conversion should not fail");
Ok(rsp)
}
.boxed()
}
// Runs concurrently using the ReadStateService
Request::Block(_) => {
metrics::counter!(
"state.requests",
1,
"service" => "state",
"type" => "block",
);
// Redirect the request to the concurrent ReadStateService
let read_service = self.read_service.clone();
async move {
let req = req
.try_into()
.expect("ReadRequest conversion should not fail");
let rsp = read_service.oneshot(req).await?;
let rsp = rsp.try_into().expect("Response conversion should not fail");
Ok(rsp)
}
.boxed()
}
// Uses pending_utxos and queued_blocks in the StateService.
// Accesses shared writeable state in the StateService.
Request::AwaitUtxo(outpoint) => {
metrics::counter!(
"state.requests",
1,
"service" => "state",
"type" => "await_utxo",
);
let timer = CodeTimer::start();
let span = Span::current();
let fut = self.pending_utxos.queue(outpoint);
// TODO: move disk reads (in `any_utxo()`) to a blocking thread (#2188)
if let Some(utxo) = self.any_utxo(&outpoint) {
self.pending_utxos.respond(&outpoint, utxo);
}
// The future waits on a channel for a response.
timer.finish(module_path!(), line!(), "AwaitUtxo");
fut.instrument(span).boxed()
}
// Runs concurrently using the ReadStateService
Request::FindBlockHashes { .. } => {
metrics::counter!(
"state.requests",
1,
"service" => "state",
"type" => "find_block_hashes",
);
// Redirect the request to the concurrent ReadStateService
let read_service = self.read_service.clone();
async move {
let req = req
.try_into()
.expect("ReadRequest conversion should not fail");
let rsp = read_service.oneshot(req).await?;
let rsp = rsp.try_into().expect("Response conversion should not fail");
Ok(rsp)
}
.boxed()
}
// Runs concurrently using the ReadStateService
Request::FindBlockHeaders { .. } => {
metrics::counter!(
"state.requests",
1,
"service" => "state",
"type" => "find_block_headers",
);
// Redirect the request to the concurrent ReadStateService
let read_service = self.read_service.clone();
async move {
let req = req
.try_into()
.expect("ReadRequest conversion should not fail");
let rsp = read_service.oneshot(req).await?;
let rsp = rsp.try_into().expect("Response conversion should not fail");
Ok(rsp)
}
.boxed()
}
}
}
}
impl Service<ReadRequest> for ReadStateService {
type Response = ReadResponse;
type Error = BoxError;
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>> {
Poll::Ready(Ok(()))
}
#[instrument(name = "read_state", skip(self))]
fn call(&mut self, req: ReadRequest) -> Self::Future {
match req {
// Used by the StateService.
ReadRequest::Tip => {
metrics::counter!(
"state.requests",
1,
"service" => "read_state",
"type" => "tip",
);
let timer = CodeTimer::start();
let state = self.clone();
let span = Span::current();
tokio::task::spawn_blocking(move || {
span.in_scope(move || {
let tip = state
.best_chain_receiver
.with_watch_data(|best_chain| read::tip(best_chain, &state.db));
// The work is done in the future.
timer.finish(module_path!(), line!(), "ReadRequest::Tip");
Ok(ReadResponse::Tip(tip))
})
})
.map(|join_result| join_result.expect("panic in ReadRequest::Tip"))
.boxed()
}
// Used by the StateService.
ReadRequest::Depth(hash) => {
metrics::counter!(
"state.requests",
1,
"service" => "read_state",
"type" => "depth",
);
let timer = CodeTimer::start();
let state = self.clone();
let span = Span::current();
tokio::task::spawn_blocking(move || {
span.in_scope(move || {
let depth = state
.best_chain_receiver
.with_watch_data(|best_chain| read::depth(best_chain, &state.db, hash));
// The work is done in the future.
timer.finish(module_path!(), line!(), "ReadRequest::Depth");
Ok(ReadResponse::Depth(depth))
})
})
.map(|join_result| join_result.expect("panic in ReadRequest::Tip"))
.boxed()
}
// Used by get_block RPC.
ReadRequest::Block(hash_or_height) => {
metrics::counter!(
"state.requests",
1,
"service" => "read_state",
"type" => "block",
);
let timer = CodeTimer::start();
let state = self.clone();
let span = Span::current();
tokio::task::spawn_blocking(move || {
span.in_scope(move || {
let block = state.best_chain_receiver.with_watch_data(|best_chain| {
read::block(best_chain, &state.db, hash_or_height)
});
// The work is done in the future.
timer.finish(module_path!(), line!(), "ReadRequest::Block");
Ok(ReadResponse::Block(block))
})
})
.map(|join_result| join_result.expect("panic in ReadRequest::Block"))
.boxed()
}
// For the get_raw_transaction RPC.
ReadRequest::Transaction(hash) => {
metrics::counter!(
"state.requests",
1,
"service" => "read_state",
"type" => "transaction",
);
let timer = CodeTimer::start();
let state = self.clone();
let span = Span::current();
tokio::task::spawn_blocking(move || {
span.in_scope(move || {
let transaction_and_height =
state.best_chain_receiver.with_watch_data(|best_chain| {
read::transaction(best_chain, &state.db, hash)
});
// The work is done in the future.
timer.finish(module_path!(), line!(), "ReadRequest::Transaction");
Ok(ReadResponse::Transaction(transaction_and_height))
})
})
.map(|join_result| join_result.expect("panic in ReadRequest::Transaction"))
.boxed()
}
// Used by the StateService.
ReadRequest::BlockLocator => {
metrics::counter!(
"state.requests",
1,
"service" => "read_state",
"type" => "block_locator",
);
let timer = CodeTimer::start();
let state = self.clone();
let span = Span::current();
tokio::task::spawn_blocking(move || {
span.in_scope(move || {
let block_locator =
state.best_chain_receiver.with_watch_data(|best_chain| {
read::block_locator(best_chain, &state.db)
});
// The work is done in the future.
timer.finish(module_path!(), line!(), "ReadRequest::BlockLocator");
Ok(ReadResponse::BlockLocator(
block_locator.unwrap_or_default(),
))
})
})
.map(|join_result| join_result.expect("panic in ReadRequest::Tip"))
.boxed()
}
// Used by the StateService.
ReadRequest::FindBlockHashes { known_blocks, stop } => {
metrics::counter!(
"state.requests",
1,
"service" => "read_state",
"type" => "find_block_hashes",
);
let timer = CodeTimer::start();
let state = self.clone();
let span = Span::current();
tokio::task::spawn_blocking(move || {
span.in_scope(move || {
let block_hashes =
state.best_chain_receiver.with_watch_data(|best_chain| {
read::find_chain_hashes(
best_chain,
&state.db,
known_blocks,
stop,
MAX_FIND_BLOCK_HASHES_RESULTS,
)
});
// The work is done in the future.
timer.finish(module_path!(), line!(), "ReadRequest::FindBlockHashes");
Ok(ReadResponse::BlockHashes(block_hashes))
})
})
.map(|join_result| join_result.expect("panic in ReadRequest::Tip"))
.boxed()
}
// Used by the StateService.
ReadRequest::FindBlockHeaders { known_blocks, stop } => {
metrics::counter!(
"state.requests",
1,
"service" => "read_state",
"type" => "find_block_headers",
);
let timer = CodeTimer::start();
let state = self.clone();
let span = Span::current();
tokio::task::spawn_blocking(move || {
span.in_scope(move || {
let block_headers =
state.best_chain_receiver.with_watch_data(|best_chain| {
read::find_chain_headers(
best_chain,
&state.db,
known_blocks,
stop,
MAX_FIND_BLOCK_HEADERS_RESULTS_FOR_ZEBRA,
)
});
let block_headers = block_headers
.into_iter()
.map(|header| CountedHeader { header })
.collect();
// The work is done in the future.
timer.finish(module_path!(), line!(), "ReadRequest::FindBlockHeaders");
Ok(ReadResponse::BlockHeaders(block_headers))
})
})
.map(|join_result| join_result.expect("panic in ReadRequest::Tip"))
.boxed()
}
feat(rpc): Implement `z_gettreestate` RPC (#3990) * Impl the elementary structure of the `z_gettreestate` RPC * Fix merging bugs * Fix a merge bug * Fix a merge bug * Move a derive attribute Co-authored-by: teor <teor@riseup.net> * Clarify the support of negative heights * Add Orchard note commitment trees to the response * Add the time to the response * Finalize the `z_gettreestate` RPC * Add a note that verified blocks have coinbase height * Refactor `from_str` for `HashOrHeight` * Fix a mistake in the docs Co-authored-by: teor <teor@riseup.net> * Clarify request types Co-authored-by: teor <teor@riseup.net> * Simplify `hash_or_height` conversion to height Co-authored-by: teor <teor@riseup.net> * Add a TODO about optimization Co-authored-by: teor <teor@riseup.net> * Add a doc comment * Make sure Sapling & Orchard trees don't get mixed up * Serialize Sapling commitment trees * Refactor some comments * Serialize Orchard commitment trees * Serialize block heights * Simplify the serialization of commitment trees * Remove the block time from the RPC response * Simplify the serialization of block heights * Put Sapling & Orchard requests together * Remove a redundant TODO * Add block times to the RPC response * Derive `Clone, Debug, Eq, PartialEq` for `GetTreestate` Co-authored-by: teor <teor@riseup.net> * Derive `Clone`, `Debug`, `Eq` and `PartialEq` for `SerializedTree` * Document the fields of `GetTreestate` * Skip the serialization of empty trees This ensures compatibility with `zcashd` in the `z_gettreestate` RPC. * Document the `impl` of `merkle_tree::Hashable` for nodes * Make the structure of the JSON response consistent with `zcashd` * Derive `Eq` for nodes Co-authored-by: teor <teor@riseup.net> * Convert Sapling commitment trees to a format compatible with zcashd * Refactor the conversion of Sapling commitment trees * Refactor some comments * Refactor comments * Add a description of the conversion Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Fix comment indenting * Document the conversion between the dense and sparse formats Co-authored-by: teor <teor@riseup.net> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> Co-authored-by: Conrado Gouvea <conrado@zfnd.org>
2022-05-12 00:00:12 -07:00
ReadRequest::SaplingTree(hash_or_height) => {
metrics::counter!(
"state.requests",
1,
"service" => "read_state",
"type" => "sapling_tree",
);
let timer = CodeTimer::start();
feat(rpc): Implement `z_gettreestate` RPC (#3990) * Impl the elementary structure of the `z_gettreestate` RPC * Fix merging bugs * Fix a merge bug * Fix a merge bug * Move a derive attribute Co-authored-by: teor <teor@riseup.net> * Clarify the support of negative heights * Add Orchard note commitment trees to the response * Add the time to the response * Finalize the `z_gettreestate` RPC * Add a note that verified blocks have coinbase height * Refactor `from_str` for `HashOrHeight` * Fix a mistake in the docs Co-authored-by: teor <teor@riseup.net> * Clarify request types Co-authored-by: teor <teor@riseup.net> * Simplify `hash_or_height` conversion to height Co-authored-by: teor <teor@riseup.net> * Add a TODO about optimization Co-authored-by: teor <teor@riseup.net> * Add a doc comment * Make sure Sapling & Orchard trees don't get mixed up * Serialize Sapling commitment trees * Refactor some comments * Serialize Orchard commitment trees * Serialize block heights * Simplify the serialization of commitment trees * Remove the block time from the RPC response * Simplify the serialization of block heights * Put Sapling & Orchard requests together * Remove a redundant TODO * Add block times to the RPC response * Derive `Clone, Debug, Eq, PartialEq` for `GetTreestate` Co-authored-by: teor <teor@riseup.net> * Derive `Clone`, `Debug`, `Eq` and `PartialEq` for `SerializedTree` * Document the fields of `GetTreestate` * Skip the serialization of empty trees This ensures compatibility with `zcashd` in the `z_gettreestate` RPC. * Document the `impl` of `merkle_tree::Hashable` for nodes * Make the structure of the JSON response consistent with `zcashd` * Derive `Eq` for nodes Co-authored-by: teor <teor@riseup.net> * Convert Sapling commitment trees to a format compatible with zcashd * Refactor the conversion of Sapling commitment trees * Refactor some comments * Refactor comments * Add a description of the conversion Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Fix comment indenting * Document the conversion between the dense and sparse formats Co-authored-by: teor <teor@riseup.net> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> Co-authored-by: Conrado Gouvea <conrado@zfnd.org>
2022-05-12 00:00:12 -07:00
let state = self.clone();
let span = Span::current();
tokio::task::spawn_blocking(move || {
span.in_scope(move || {
let sapling_tree =
state.best_chain_receiver.with_watch_data(|best_chain| {
read::sapling_tree(best_chain, &state.db, hash_or_height)
});
// The work is done in the future.
timer.finish(module_path!(), line!(), "ReadRequest::SaplingTree");
feat(rpc): Implement `z_gettreestate` RPC (#3990) * Impl the elementary structure of the `z_gettreestate` RPC * Fix merging bugs * Fix a merge bug * Fix a merge bug * Move a derive attribute Co-authored-by: teor <teor@riseup.net> * Clarify the support of negative heights * Add Orchard note commitment trees to the response * Add the time to the response * Finalize the `z_gettreestate` RPC * Add a note that verified blocks have coinbase height * Refactor `from_str` for `HashOrHeight` * Fix a mistake in the docs Co-authored-by: teor <teor@riseup.net> * Clarify request types Co-authored-by: teor <teor@riseup.net> * Simplify `hash_or_height` conversion to height Co-authored-by: teor <teor@riseup.net> * Add a TODO about optimization Co-authored-by: teor <teor@riseup.net> * Add a doc comment * Make sure Sapling & Orchard trees don't get mixed up * Serialize Sapling commitment trees * Refactor some comments * Serialize Orchard commitment trees * Serialize block heights * Simplify the serialization of commitment trees * Remove the block time from the RPC response * Simplify the serialization of block heights * Put Sapling & Orchard requests together * Remove a redundant TODO * Add block times to the RPC response * Derive `Clone, Debug, Eq, PartialEq` for `GetTreestate` Co-authored-by: teor <teor@riseup.net> * Derive `Clone`, `Debug`, `Eq` and `PartialEq` for `SerializedTree` * Document the fields of `GetTreestate` * Skip the serialization of empty trees This ensures compatibility with `zcashd` in the `z_gettreestate` RPC. * Document the `impl` of `merkle_tree::Hashable` for nodes * Make the structure of the JSON response consistent with `zcashd` * Derive `Eq` for nodes Co-authored-by: teor <teor@riseup.net> * Convert Sapling commitment trees to a format compatible with zcashd * Refactor the conversion of Sapling commitment trees * Refactor some comments * Refactor comments * Add a description of the conversion Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Fix comment indenting * Document the conversion between the dense and sparse formats Co-authored-by: teor <teor@riseup.net> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> Co-authored-by: Conrado Gouvea <conrado@zfnd.org>
2022-05-12 00:00:12 -07:00
Ok(ReadResponse::SaplingTree(sapling_tree))
})
})
.map(|join_result| join_result.expect("panic in ReadRequest::SaplingTree"))
feat(rpc): Implement `z_gettreestate` RPC (#3990) * Impl the elementary structure of the `z_gettreestate` RPC * Fix merging bugs * Fix a merge bug * Fix a merge bug * Move a derive attribute Co-authored-by: teor <teor@riseup.net> * Clarify the support of negative heights * Add Orchard note commitment trees to the response * Add the time to the response * Finalize the `z_gettreestate` RPC * Add a note that verified blocks have coinbase height * Refactor `from_str` for `HashOrHeight` * Fix a mistake in the docs Co-authored-by: teor <teor@riseup.net> * Clarify request types Co-authored-by: teor <teor@riseup.net> * Simplify `hash_or_height` conversion to height Co-authored-by: teor <teor@riseup.net> * Add a TODO about optimization Co-authored-by: teor <teor@riseup.net> * Add a doc comment * Make sure Sapling & Orchard trees don't get mixed up * Serialize Sapling commitment trees * Refactor some comments * Serialize Orchard commitment trees * Serialize block heights * Simplify the serialization of commitment trees * Remove the block time from the RPC response * Simplify the serialization of block heights * Put Sapling & Orchard requests together * Remove a redundant TODO * Add block times to the RPC response * Derive `Clone, Debug, Eq, PartialEq` for `GetTreestate` Co-authored-by: teor <teor@riseup.net> * Derive `Clone`, `Debug`, `Eq` and `PartialEq` for `SerializedTree` * Document the fields of `GetTreestate` * Skip the serialization of empty trees This ensures compatibility with `zcashd` in the `z_gettreestate` RPC. * Document the `impl` of `merkle_tree::Hashable` for nodes * Make the structure of the JSON response consistent with `zcashd` * Derive `Eq` for nodes Co-authored-by: teor <teor@riseup.net> * Convert Sapling commitment trees to a format compatible with zcashd * Refactor the conversion of Sapling commitment trees * Refactor some comments * Refactor comments * Add a description of the conversion Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Fix comment indenting * Document the conversion between the dense and sparse formats Co-authored-by: teor <teor@riseup.net> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> Co-authored-by: Conrado Gouvea <conrado@zfnd.org>
2022-05-12 00:00:12 -07:00
.boxed()
}
ReadRequest::OrchardTree(hash_or_height) => {
metrics::counter!(
"state.requests",
1,
"service" => "read_state",
"type" => "orchard_tree",
);
let timer = CodeTimer::start();
feat(rpc): Implement `z_gettreestate` RPC (#3990) * Impl the elementary structure of the `z_gettreestate` RPC * Fix merging bugs * Fix a merge bug * Fix a merge bug * Move a derive attribute Co-authored-by: teor <teor@riseup.net> * Clarify the support of negative heights * Add Orchard note commitment trees to the response * Add the time to the response * Finalize the `z_gettreestate` RPC * Add a note that verified blocks have coinbase height * Refactor `from_str` for `HashOrHeight` * Fix a mistake in the docs Co-authored-by: teor <teor@riseup.net> * Clarify request types Co-authored-by: teor <teor@riseup.net> * Simplify `hash_or_height` conversion to height Co-authored-by: teor <teor@riseup.net> * Add a TODO about optimization Co-authored-by: teor <teor@riseup.net> * Add a doc comment * Make sure Sapling & Orchard trees don't get mixed up * Serialize Sapling commitment trees * Refactor some comments * Serialize Orchard commitment trees * Serialize block heights * Simplify the serialization of commitment trees * Remove the block time from the RPC response * Simplify the serialization of block heights * Put Sapling & Orchard requests together * Remove a redundant TODO * Add block times to the RPC response * Derive `Clone, Debug, Eq, PartialEq` for `GetTreestate` Co-authored-by: teor <teor@riseup.net> * Derive `Clone`, `Debug`, `Eq` and `PartialEq` for `SerializedTree` * Document the fields of `GetTreestate` * Skip the serialization of empty trees This ensures compatibility with `zcashd` in the `z_gettreestate` RPC. * Document the `impl` of `merkle_tree::Hashable` for nodes * Make the structure of the JSON response consistent with `zcashd` * Derive `Eq` for nodes Co-authored-by: teor <teor@riseup.net> * Convert Sapling commitment trees to a format compatible with zcashd * Refactor the conversion of Sapling commitment trees * Refactor some comments * Refactor comments * Add a description of the conversion Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Fix comment indenting * Document the conversion between the dense and sparse formats Co-authored-by: teor <teor@riseup.net> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> Co-authored-by: Conrado Gouvea <conrado@zfnd.org>
2022-05-12 00:00:12 -07:00
let state = self.clone();
let span = Span::current();
tokio::task::spawn_blocking(move || {
span.in_scope(move || {
let orchard_tree =
state.best_chain_receiver.with_watch_data(|best_chain| {
read::orchard_tree(best_chain, &state.db, hash_or_height)
});
feat(rpc): Implement `z_gettreestate` RPC (#3990) * Impl the elementary structure of the `z_gettreestate` RPC * Fix merging bugs * Fix a merge bug * Fix a merge bug * Move a derive attribute Co-authored-by: teor <teor@riseup.net> * Clarify the support of negative heights * Add Orchard note commitment trees to the response * Add the time to the response * Finalize the `z_gettreestate` RPC * Add a note that verified blocks have coinbase height * Refactor `from_str` for `HashOrHeight` * Fix a mistake in the docs Co-authored-by: teor <teor@riseup.net> * Clarify request types Co-authored-by: teor <teor@riseup.net> * Simplify `hash_or_height` conversion to height Co-authored-by: teor <teor@riseup.net> * Add a TODO about optimization Co-authored-by: teor <teor@riseup.net> * Add a doc comment * Make sure Sapling & Orchard trees don't get mixed up * Serialize Sapling commitment trees * Refactor some comments * Serialize Orchard commitment trees * Serialize block heights * Simplify the serialization of commitment trees * Remove the block time from the RPC response * Simplify the serialization of block heights * Put Sapling & Orchard requests together * Remove a redundant TODO * Add block times to the RPC response * Derive `Clone, Debug, Eq, PartialEq` for `GetTreestate` Co-authored-by: teor <teor@riseup.net> * Derive `Clone`, `Debug`, `Eq` and `PartialEq` for `SerializedTree` * Document the fields of `GetTreestate` * Skip the serialization of empty trees This ensures compatibility with `zcashd` in the `z_gettreestate` RPC. * Document the `impl` of `merkle_tree::Hashable` for nodes * Make the structure of the JSON response consistent with `zcashd` * Derive `Eq` for nodes Co-authored-by: teor <teor@riseup.net> * Convert Sapling commitment trees to a format compatible with zcashd * Refactor the conversion of Sapling commitment trees * Refactor some comments * Refactor comments * Add a description of the conversion Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Fix comment indenting * Document the conversion between the dense and sparse formats Co-authored-by: teor <teor@riseup.net> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> Co-authored-by: Conrado Gouvea <conrado@zfnd.org>
2022-05-12 00:00:12 -07:00
// The work is done in the future.
timer.finish(module_path!(), line!(), "ReadRequest::OrchardTree");
Ok(ReadResponse::OrchardTree(orchard_tree))
})
})
.map(|join_result| join_result.expect("panic in ReadRequest::OrchardTree"))
feat(rpc): Implement `z_gettreestate` RPC (#3990) * Impl the elementary structure of the `z_gettreestate` RPC * Fix merging bugs * Fix a merge bug * Fix a merge bug * Move a derive attribute Co-authored-by: teor <teor@riseup.net> * Clarify the support of negative heights * Add Orchard note commitment trees to the response * Add the time to the response * Finalize the `z_gettreestate` RPC * Add a note that verified blocks have coinbase height * Refactor `from_str` for `HashOrHeight` * Fix a mistake in the docs Co-authored-by: teor <teor@riseup.net> * Clarify request types Co-authored-by: teor <teor@riseup.net> * Simplify `hash_or_height` conversion to height Co-authored-by: teor <teor@riseup.net> * Add a TODO about optimization Co-authored-by: teor <teor@riseup.net> * Add a doc comment * Make sure Sapling & Orchard trees don't get mixed up * Serialize Sapling commitment trees * Refactor some comments * Serialize Orchard commitment trees * Serialize block heights * Simplify the serialization of commitment trees * Remove the block time from the RPC response * Simplify the serialization of block heights * Put Sapling & Orchard requests together * Remove a redundant TODO * Add block times to the RPC response * Derive `Clone, Debug, Eq, PartialEq` for `GetTreestate` Co-authored-by: teor <teor@riseup.net> * Derive `Clone`, `Debug`, `Eq` and `PartialEq` for `SerializedTree` * Document the fields of `GetTreestate` * Skip the serialization of empty trees This ensures compatibility with `zcashd` in the `z_gettreestate` RPC. * Document the `impl` of `merkle_tree::Hashable` for nodes * Make the structure of the JSON response consistent with `zcashd` * Derive `Eq` for nodes Co-authored-by: teor <teor@riseup.net> * Convert Sapling commitment trees to a format compatible with zcashd * Refactor the conversion of Sapling commitment trees * Refactor some comments * Refactor comments * Add a description of the conversion Co-authored-by: Conrado Gouvea <conrado@zfnd.org> * Fix comment indenting * Document the conversion between the dense and sparse formats Co-authored-by: teor <teor@riseup.net> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> Co-authored-by: Conrado Gouvea <conrado@zfnd.org>
2022-05-12 00:00:12 -07:00
.boxed()
}
// For the get_address_balance RPC.
ReadRequest::AddressBalance(addresses) => {
metrics::counter!(
"state.requests",
1,
"service" => "read_state",
"type" => "address_balance",
);
let timer = CodeTimer::start();
let state = self.clone();
let span = Span::current();
tokio::task::spawn_blocking(move || {
span.in_scope(move || {
let balance = state.best_chain_receiver.with_watch_data(|best_chain| {
read::transparent_balance(best_chain, &state.db, addresses)
})?;
// The work is done in the future.
timer.finish(module_path!(), line!(), "ReadRequest::AddressBalance");
Ok(ReadResponse::AddressBalance(balance))
})
})
.map(|join_result| join_result.expect("panic in ReadRequest::AddressBalance"))
.boxed()
}
feat(rpc): Implement `getaddressbalance` RPC (#4138) * Add `Amount::serialize_as_string` helper method A helper method that makes it easier to serialize an `Amount` as a string. This is needed for the response type of the `getaccountbalance` RPC. * Implement state service call for address balance Add `Read{Request,Response}::AddressBalance` variants and implement the handler that calls the query function. * Create an `AddressBalance` response type Only contains the `balance` field which is needed by `lightwalletd`. That field is serialized as a string, following the RPC specification. * Implement `get_address_balance` RPC Query the read-only state service for the information, and wrap it in an `AddressBalance` response type so that it is serialized correctly. * Run `rustfmt` inside `proptest!` block Fix some minor formatting details. * Test `get_address_balance` with valid addresses Check that the RPC leads to a query to the mocked state service for a balance amount. * Test `get_address_balance` with invalid addresses An error message should be returned by the RPC. * Rename metric to `address_balance` Keep it consistent with how it's named in other places. Co-authored-by: teor <teor@riseup.net> * Revert "Add `Amount::serialize_as_string` helper method" This reverts commit 01b432e3d2ac2313a90d55d06b3fa855c0b71330. * Serialize amount as an integer This is different from what the documentation says, but it's what lightwalletd expects. * Add reference to RPC documentation Make sure it is linked to for easy access. * Create an `AddressStrings` type To be used as the input for the `get_address_balance` RPC method. * Use `AddressStrings` in `get_address_balance` RPC Fix the input parameter so that the list of address strings is placed inside a JSON map. * Update property tests to use `AddressStrings` Make sure the proper input type is created. Co-authored-by: teor <teor@riseup.net>
2022-04-20 11:27:00 -07:00
// For the get_address_tx_ids RPC.
ReadRequest::TransactionIdsByAddresses {
addresses,
height_range,
} => {
feat(rpc): Implement `getaddressbalance` RPC (#4138) * Add `Amount::serialize_as_string` helper method A helper method that makes it easier to serialize an `Amount` as a string. This is needed for the response type of the `getaccountbalance` RPC. * Implement state service call for address balance Add `Read{Request,Response}::AddressBalance` variants and implement the handler that calls the query function. * Create an `AddressBalance` response type Only contains the `balance` field which is needed by `lightwalletd`. That field is serialized as a string, following the RPC specification. * Implement `get_address_balance` RPC Query the read-only state service for the information, and wrap it in an `AddressBalance` response type so that it is serialized correctly. * Run `rustfmt` inside `proptest!` block Fix some minor formatting details. * Test `get_address_balance` with valid addresses Check that the RPC leads to a query to the mocked state service for a balance amount. * Test `get_address_balance` with invalid addresses An error message should be returned by the RPC. * Rename metric to `address_balance` Keep it consistent with how it's named in other places. Co-authored-by: teor <teor@riseup.net> * Revert "Add `Amount::serialize_as_string` helper method" This reverts commit 01b432e3d2ac2313a90d55d06b3fa855c0b71330. * Serialize amount as an integer This is different from what the documentation says, but it's what lightwalletd expects. * Add reference to RPC documentation Make sure it is linked to for easy access. * Create an `AddressStrings` type To be used as the input for the `get_address_balance` RPC method. * Use `AddressStrings` in `get_address_balance` RPC Fix the input parameter so that the list of address strings is placed inside a JSON map. * Update property tests to use `AddressStrings` Make sure the proper input type is created. Co-authored-by: teor <teor@riseup.net>
2022-04-20 11:27:00 -07:00
metrics::counter!(
"state.requests",
1,
"service" => "read_state",
"type" => "transaction_ids_by_addresses",
feat(rpc): Implement `getaddressbalance` RPC (#4138) * Add `Amount::serialize_as_string` helper method A helper method that makes it easier to serialize an `Amount` as a string. This is needed for the response type of the `getaccountbalance` RPC. * Implement state service call for address balance Add `Read{Request,Response}::AddressBalance` variants and implement the handler that calls the query function. * Create an `AddressBalance` response type Only contains the `balance` field which is needed by `lightwalletd`. That field is serialized as a string, following the RPC specification. * Implement `get_address_balance` RPC Query the read-only state service for the information, and wrap it in an `AddressBalance` response type so that it is serialized correctly. * Run `rustfmt` inside `proptest!` block Fix some minor formatting details. * Test `get_address_balance` with valid addresses Check that the RPC leads to a query to the mocked state service for a balance amount. * Test `get_address_balance` with invalid addresses An error message should be returned by the RPC. * Rename metric to `address_balance` Keep it consistent with how it's named in other places. Co-authored-by: teor <teor@riseup.net> * Revert "Add `Amount::serialize_as_string` helper method" This reverts commit 01b432e3d2ac2313a90d55d06b3fa855c0b71330. * Serialize amount as an integer This is different from what the documentation says, but it's what lightwalletd expects. * Add reference to RPC documentation Make sure it is linked to for easy access. * Create an `AddressStrings` type To be used as the input for the `get_address_balance` RPC method. * Use `AddressStrings` in `get_address_balance` RPC Fix the input parameter so that the list of address strings is placed inside a JSON map. * Update property tests to use `AddressStrings` Make sure the proper input type is created. Co-authored-by: teor <teor@riseup.net>
2022-04-20 11:27:00 -07:00
);
let timer = CodeTimer::start();
feat(rpc): Implement `getaddressbalance` RPC (#4138) * Add `Amount::serialize_as_string` helper method A helper method that makes it easier to serialize an `Amount` as a string. This is needed for the response type of the `getaccountbalance` RPC. * Implement state service call for address balance Add `Read{Request,Response}::AddressBalance` variants and implement the handler that calls the query function. * Create an `AddressBalance` response type Only contains the `balance` field which is needed by `lightwalletd`. That field is serialized as a string, following the RPC specification. * Implement `get_address_balance` RPC Query the read-only state service for the information, and wrap it in an `AddressBalance` response type so that it is serialized correctly. * Run `rustfmt` inside `proptest!` block Fix some minor formatting details. * Test `get_address_balance` with valid addresses Check that the RPC leads to a query to the mocked state service for a balance amount. * Test `get_address_balance` with invalid addresses An error message should be returned by the RPC. * Rename metric to `address_balance` Keep it consistent with how it's named in other places. Co-authored-by: teor <teor@riseup.net> * Revert "Add `Amount::serialize_as_string` helper method" This reverts commit 01b432e3d2ac2313a90d55d06b3fa855c0b71330. * Serialize amount as an integer This is different from what the documentation says, but it's what lightwalletd expects. * Add reference to RPC documentation Make sure it is linked to for easy access. * Create an `AddressStrings` type To be used as the input for the `get_address_balance` RPC method. * Use `AddressStrings` in `get_address_balance` RPC Fix the input parameter so that the list of address strings is placed inside a JSON map. * Update property tests to use `AddressStrings` Make sure the proper input type is created. Co-authored-by: teor <teor@riseup.net>
2022-04-20 11:27:00 -07:00
let state = self.clone();
let span = Span::current();
tokio::task::spawn_blocking(move || {
span.in_scope(move || {
let tx_ids = state.best_chain_receiver.with_watch_data(|best_chain| {
read::transparent_tx_ids(best_chain, &state.db, addresses, height_range)
});
// The work is done in the future.
timer.finish(
module_path!(),
line!(),
"ReadRequest::TransactionIdsByAddresses",
);
feat(rpc): Implement `getaddressbalance` RPC (#4138) * Add `Amount::serialize_as_string` helper method A helper method that makes it easier to serialize an `Amount` as a string. This is needed for the response type of the `getaccountbalance` RPC. * Implement state service call for address balance Add `Read{Request,Response}::AddressBalance` variants and implement the handler that calls the query function. * Create an `AddressBalance` response type Only contains the `balance` field which is needed by `lightwalletd`. That field is serialized as a string, following the RPC specification. * Implement `get_address_balance` RPC Query the read-only state service for the information, and wrap it in an `AddressBalance` response type so that it is serialized correctly. * Run `rustfmt` inside `proptest!` block Fix some minor formatting details. * Test `get_address_balance` with valid addresses Check that the RPC leads to a query to the mocked state service for a balance amount. * Test `get_address_balance` with invalid addresses An error message should be returned by the RPC. * Rename metric to `address_balance` Keep it consistent with how it's named in other places. Co-authored-by: teor <teor@riseup.net> * Revert "Add `Amount::serialize_as_string` helper method" This reverts commit 01b432e3d2ac2313a90d55d06b3fa855c0b71330. * Serialize amount as an integer This is different from what the documentation says, but it's what lightwalletd expects. * Add reference to RPC documentation Make sure it is linked to for easy access. * Create an `AddressStrings` type To be used as the input for the `get_address_balance` RPC method. * Use `AddressStrings` in `get_address_balance` RPC Fix the input parameter so that the list of address strings is placed inside a JSON map. * Update property tests to use `AddressStrings` Make sure the proper input type is created. Co-authored-by: teor <teor@riseup.net>
2022-04-20 11:27:00 -07:00
tx_ids.map(ReadResponse::AddressesTransactionIds)
})
})
.map(|join_result| {
join_result.expect("panic in ReadRequest::TransactionIdsByAddresses")
})
feat(rpc): Implement `getaddressbalance` RPC (#4138) * Add `Amount::serialize_as_string` helper method A helper method that makes it easier to serialize an `Amount` as a string. This is needed for the response type of the `getaccountbalance` RPC. * Implement state service call for address balance Add `Read{Request,Response}::AddressBalance` variants and implement the handler that calls the query function. * Create an `AddressBalance` response type Only contains the `balance` field which is needed by `lightwalletd`. That field is serialized as a string, following the RPC specification. * Implement `get_address_balance` RPC Query the read-only state service for the information, and wrap it in an `AddressBalance` response type so that it is serialized correctly. * Run `rustfmt` inside `proptest!` block Fix some minor formatting details. * Test `get_address_balance` with valid addresses Check that the RPC leads to a query to the mocked state service for a balance amount. * Test `get_address_balance` with invalid addresses An error message should be returned by the RPC. * Rename metric to `address_balance` Keep it consistent with how it's named in other places. Co-authored-by: teor <teor@riseup.net> * Revert "Add `Amount::serialize_as_string` helper method" This reverts commit 01b432e3d2ac2313a90d55d06b3fa855c0b71330. * Serialize amount as an integer This is different from what the documentation says, but it's what lightwalletd expects. * Add reference to RPC documentation Make sure it is linked to for easy access. * Create an `AddressStrings` type To be used as the input for the `get_address_balance` RPC method. * Use `AddressStrings` in `get_address_balance` RPC Fix the input parameter so that the list of address strings is placed inside a JSON map. * Update property tests to use `AddressStrings` Make sure the proper input type is created. Co-authored-by: teor <teor@riseup.net>
2022-04-20 11:27:00 -07:00
.boxed()
}
// For the get_address_utxos RPC.
ReadRequest::UtxosByAddresses(addresses) => {
metrics::counter!(
"state.requests",
1,
"service" => "read_state",
"type" => "utxos_by_addresses",
);
let timer = CodeTimer::start();
let state = self.clone();
let span = Span::current();
tokio::task::spawn_blocking(move || {
span.in_scope(move || {
let utxos = state.best_chain_receiver.with_watch_data(|best_chain| {
read::transparent_utxos(state.network, best_chain, &state.db, addresses)
});
// The work is done in the future.
timer.finish(module_path!(), line!(), "ReadRequest::UtxosByAddresses");
utxos.map(ReadResponse::Utxos)
})
})
.map(|join_result| join_result.expect("panic in ReadRequest::UtxosByAddresses"))
.boxed()
}
}
}
}
2020-09-09 17:51:08 -07:00
/// Initialize a state service from the provided [`Config`].
/// Returns a boxed state service, a read-only state service,
/// and receivers for state chain tip updates.
///
/// Each `network` has its own separate on-disk database.
///
/// To share access to the state, wrap the returned service in a `Buffer`,
/// or clone the returned [`ReadStateService`].
///
/// It's possible to construct multiple state services in the same application (as
/// long as they, e.g., use different storage locations), but doing so is
/// probably not what you want.
Reject connections from outdated peers (#2519) * Simplify state service initialization in test Use the test helper function to remove redundant code. * Create `BestTipHeight` helper type This type abstracts away the calculation of the best tip height based on the finalized block height and the best non-finalized chain's tip. * Add `best_tip_height` field to `StateService` The receiver endpoint is currently ignored. * Return receiver endpoint from service constructor Make it available so that the best tip height can be watched. * Update finalized height after finalizing blocks After blocks from the queue are finalized and committed to disk, update the finalized block height. * Update best non-finalized height after validation Update the value of the best non-finalized chain tip block height after a new block is committed to the non-finalized state. * Update finalized height after loading from disk When `FinalizedState` is first created, it loads the state from persistent storage, and the finalized tip height is updated. Therefore, the `best_tip_height` must be notified of the initial value. * Update the finalized height on checkpoint commit When a checkpointed block is commited, it bypasses the non-finalized state, so there's an extra place where the finalized height has to be updated. * Add `best_tip_height` to `Handshake` service It can be configured using the `Builder::with_best_tip_height`. It's currently not used, but it will be used to determine if a connection to a remote peer should be rejected or not based on that peer's protocol version. * Require best tip height to init. `zebra_network` Without it the handshake service can't properly enforce the minimum network protocol version from peers. Zebrad obtains the best tip height endpoint from `zebra_state`, and the test vectors simply use a dummy endpoint that's fixed at the genesis height. * Pass `best_tip_height` to proto. ver. negotiation The protocol version negotiation code will reject connections to peers if they are using an old protocol version. An old version is determined based on the current known best chain tip height. * Handle an optional height in `Version` Fallback to the genesis height in `None` is specified. * Reject connections to peers on old proto. versions Avoid connecting to peers that are on protocol versions that don't recognize a network update. * Document why peers on old versions are rejected Describe why it's a security issue above the check. * Test if `BestTipHeight` starts with `None` Check if initially there is no best tip height. * Test if best tip height is max. of latest values After applying a list of random updates where each one either sets the finalized height or the non-finalized height, check that the best tip height is the maximum of the most recently set finalized height and the most recently set non-finalized height. * Add `queue_and_commit_finalized` method A small refactor to make testing easier. The handling of requests for committing non-finalized and finalized blocks is now more consistent. * Add `assert_block_can_be_validated` helper Refactor to move into a separate method some assertions that are done before a block is validated. This is to allow moving these assertions more easily to simplify testing. * Remove redundant PoW block assertion It's also checked in `zebra_state::service::check::block_is_contextually_valid`, and it was getting in the way of tests that received a gossiped block before finalizing enough blocks. * Create a test strategy for test vector chain Splits a chain loaded from the test vectors in two parts, containing the blocks to finalize and the blocks to keep in the non-finalized state. * Test committing blocks update best tip height Create a mock blockchain state, with a chain of finalized blocks and a chain of non-finalized blocks. Commit all the blocks appropriately, and verify that the best tip height is updated. Co-authored-by: teor <teor@riseup.net>
2021-08-08 16:52:52 -07:00
pub fn init(
config: Config,
network: Network,
) -> (
BoxService<Request, Response, BoxError>,
ReadStateService,
LatestChainTip,
ChainTipChange,
) {
let (state_service, read_only_state_service, latest_chain_tip, chain_tip_change) =
StateService::new(config, network);
(
BoxService::new(state_service),
read_only_state_service,
latest_chain_tip,
chain_tip_change,
)
}
/// 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`].
#[cfg(any(test, feature = "proptest-impl"))]
pub fn init_test(network: Network) -> Buffer<BoxService<Request, Response, BoxError>, Request> {
let (state_service, _, _, _) = StateService::new(Config::ephemeral(), network);
Buffer::new(BoxService::new(state_service), 1)
}
/// Initializes a state service with an ephemeral [`Config`] and a buffer with a single slot,
/// then returns the read-write service, read-only service, and tip watch channels.
///
/// This can be used to create a state service for testing. See also [`init`].
#[cfg(any(test, feature = "proptest-impl"))]
pub fn init_test_services(
network: Network,
) -> (
Buffer<BoxService<Request, Response, BoxError>, Request>,
ReadStateService,
LatestChainTip,
ChainTipChange,
) {
let (state_service, read_state_service, latest_chain_tip, chain_tip_change) =
StateService::new(Config::ephemeral(), network);
let state_service = Buffer::new(BoxService::new(state_service), 1);
(
state_service,
read_state_service,
latest_chain_tip,
chain_tip_change,
)
}