7. feat(state): add a read-only state service stub (#3835)

* Add state service module docs and cleanup

* Move and add finalized state methods

* Add chain and non-finalized state methods

* Cleanup methods and imports

* Create a ReadStateService type

* Add a stub service implementation

* Add a TODO

* Update ReadStateService request stubs with RPC names and tickets

* Documentation updates

* Make RPC State generic bounds accept a buffered state and a read-only state

* Doc updates

* Add missing proptest-impl feature in RPC dev dependencies
This commit is contained in:
teor 2022-03-11 23:58:22 +10:00 committed by GitHub
parent f39ac48c59
commit 9ad47d1081
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 322 additions and 90 deletions

View File

@ -37,4 +37,5 @@ thiserror = "1.0.30"
tokio = { version = "1.16.1", features = ["full", "test-util"] }
zebra-chain = { path = "../zebra-chain", features = ["proptest-impl"] }
zebra-state = { path = "../zebra-state", features = ["proptest-impl"] }
zebra-test = { path = "../zebra-test/" }

View File

@ -119,24 +119,23 @@ where
/// A handle to the mempool service.
mempool: Buffer<Mempool, mempool::Request>,
/// A handle to the state service.
state: Buffer<State, zebra_state::Request>,
state: State,
}
impl<Mempool, State> RpcImpl<Mempool, State>
where
Mempool: Service<mempool::Request, Response = mempool::Response, Error = BoxError>,
State: Service<
zebra_state::Request,
Response = zebra_state::Response,
Error = zebra_state::BoxError,
> + 'static,
State::Future: Send,
zebra_state::Request,
Response = zebra_state::Response,
Error = zebra_state::BoxError,
>,
{
/// Create a new instance of the RPC handler.
pub fn new(
app_version: String,
mempool: Buffer<Mempool, mempool::Request>,
state: Buffer<State, zebra_state::Request>,
state: State,
) -> Self {
RpcImpl {
app_version,
@ -155,7 +154,10 @@ where
zebra_state::Request,
Response = zebra_state::Response,
Error = zebra_state::BoxError,
> + 'static,
> + Clone
+ Send
+ Sync
+ 'static,
State::Future: Send,
{
fn get_info(&self) -> Result<GetInfo> {

View File

@ -8,6 +8,8 @@ use zebra_chain::{
transaction::{Transaction, UnminedTx},
};
use zebra_node_services::mempool;
use zebra_state::BoxError;
use zebra_test::mock_service::MockService;
use super::super::{Rpc, RpcImpl, SentTransactionHash};
@ -20,7 +22,7 @@ proptest! {
runtime.block_on(async move {
let mut mempool = MockService::build().for_prop_tests();
let mut state = MockService::build().for_prop_tests();
let mut state: MockService<_, _, _, BoxError> = MockService::build().for_prop_tests();
let rpc = RpcImpl::new("RPC test".to_owned(), Buffer::new(mempool.clone(), 1), Buffer::new(state.clone(), 1));
let hash = SentTransactionHash(transaction.hash());
@ -61,7 +63,7 @@ proptest! {
runtime.block_on(async move {
let mut mempool = MockService::build().for_prop_tests();
let mut state = MockService::build().for_prop_tests();
let mut state: MockService<_, _, _, BoxError> = MockService::build().for_prop_tests();
let rpc = RpcImpl::new("RPC test".to_owned(), Buffer::new(mempool.clone(), 1), Buffer::new(state.clone(), 1));
@ -109,7 +111,7 @@ proptest! {
runtime.block_on(async move {
let mut mempool = MockService::build().for_prop_tests();
let mut state = MockService::build().for_prop_tests();
let mut state: MockService<_, _, _, BoxError> = MockService::build().for_prop_tests();
let rpc = RpcImpl::new("RPC test".to_owned(), Buffer::new(mempool.clone(), 1), Buffer::new(state.clone(), 1));
@ -164,7 +166,7 @@ proptest! {
runtime.block_on(async move {
let mut mempool = MockService::build().for_prop_tests();
let mut state = MockService::build().for_prop_tests();
let mut state: MockService<_, _, _, BoxError> = MockService::build().for_prop_tests();
let rpc = RpcImpl::new("RPC test".to_owned(), Buffer::new(mempool.clone(), 1), Buffer::new(state.clone(), 1));
@ -208,7 +210,7 @@ proptest! {
runtime.block_on(async move {
let mut mempool = MockService::build().for_prop_tests();
let mut state = MockService::build().for_prop_tests();
let mut state: MockService<_, _, _, BoxError> = MockService::build().for_prop_tests();
let rpc = RpcImpl::new("RPC test".to_owned(), Buffer::new(mempool.clone(), 1), Buffer::new(state.clone(), 1));

View File

@ -7,6 +7,7 @@ use tower::buffer::Buffer;
use zebra_chain::{block::Block, parameters::Network, serialization::ZcashDeserializeInto};
use zebra_network::constants::USER_AGENT;
use zebra_node_services::BoxError;
use zebra_test::mock_service::MockService;
use super::super::*;
@ -19,7 +20,7 @@ async fn rpc_getinfo() {
zebra_test::init();
let mut mempool: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
let mut state = MockService::build().for_unit_tests();
let mut state: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
let rpc = RpcImpl::new(
"Zebra version test".to_string(),
@ -80,7 +81,7 @@ async fn rpc_getblock_error() {
zebra_test::init();
let mut mempool: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
let mut state = MockService::build().for_unit_tests();
let mut state: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
// Init RPC
let rpc = RpcImpl {

View File

@ -3,6 +3,9 @@
//! This endpoint is compatible with clients that incorrectly send
//! `"jsonrpc" = 1.0` fields in JSON-RPC 1.0 requests,
//! such as `lightwalletd`.
//!
//! See the full list of
//! [Differences between JSON-RPC 1.0 and 2.0.](https://www.simple-is-better.org/rpc/#differences-between-1-0-and-2-0)
use jsonrpc_core;
use jsonrpc_http_server::ServerBuilder;
@ -30,7 +33,7 @@ impl RpcServer {
config: Config,
app_version: String,
mempool: Buffer<Mempool, mempool::Request>,
state: Buffer<State, zebra_state::Request>,
state: State,
) -> tokio::task::JoinHandle<()>
where
Mempool: tower::Service<mempool::Request, Response = mempool::Response, Error = BoxError>
@ -40,7 +43,10 @@ impl RpcServer {
zebra_state::Request,
Response = zebra_state::Response,
Error = zebra_state::BoxError,
> + 'static,
> + Clone
+ Send
+ Sync
+ 'static,
State::Future: Send,
{
if let Some(listen_addr) = config.listen_addr {

View File

@ -1,5 +1,17 @@
//! [`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::TryInto,
future::Future,
pin::Pin,
sync::Arc,
@ -8,7 +20,7 @@ use std::{
};
use futures::future::FutureExt;
use tokio::sync::oneshot;
use tokio::sync::{oneshot, watch};
use tower::{util::BoxService, Service};
use tracing::instrument;
@ -24,16 +36,15 @@ use zebra_chain::{
};
use crate::{
request::HashOrHeight, service::chain_tip::ChainTipBlock, BoxError, CloneError,
CommitBlockError, Config, FinalizedBlock, PreparedBlock, Request, Response,
ValidateContextError,
};
use self::{
chain_tip::{ChainTipChange, ChainTipSender, LatestChainTip},
finalized_state::FinalizedState,
non_finalized_state::{NonFinalizedState, QueuedBlocks},
pending_utxos::PendingUtxos,
request::HashOrHeight,
service::{
chain_tip::{ChainTipBlock, ChainTipChange, ChainTipSender, LatestChainTip},
finalized_state::{DiskDb, FinalizedState},
non_finalized_state::{Chain, NonFinalizedState, QueuedBlocks},
pending_utxos::PendingUtxos,
},
BoxError, CloneError, CommitBlockError, Config, FinalizedBlock, PreparedBlock, Request,
Response, ValidateContextError,
};
pub mod block_iter;
@ -69,27 +80,85 @@ pub type QueuedFinalized = (
/// - 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.
///
/// Requests to this service are processed in series,
/// so read requests wait for all queued write requests to complete,
/// then return their answers.
///
/// This behaviour is implicitly used by Zebra's syncer,
/// to delay the next ObtainTips until all queued blocks have been commited.
///
/// But most state users can ignore any queued blocks, and get faster read responses
/// using the [`ReadOnlyStateService`].
#[derive(Debug)]
pub(crate) struct StateService {
/// Holds data relating to finalized chain state.
/// The finalized chain state, including its on-disk database.
pub(crate) disk: FinalizedState,
/// Holds data relating to non-finalized chain state.
/// The non-finalized chain state, including its in-memory chain forks.
mem: NonFinalizedState,
/// The configured Zcash network.
network: Network,
/// Blocks awaiting their parent blocks for contextual verification.
queued_blocks: QueuedBlocks,
/// The set of outpoints with pending requests for their associated transparent::Output
/// The set of outpoints with pending requests for their associated transparent::Output.
pending_utxos: PendingUtxos,
/// The configured Zcash network
network: Network,
/// Instant tracking the last time `pending_utxos` was pruned
/// Instant tracking the last time `pending_utxos` was pruned.
last_prune: Instant,
/// The current best chain tip height.
/// A sender channel for the current best chain tip.
chain_tip_sender: ChainTipSender,
/// A sender channel for the current best non-finalized chain.
best_chain_sender: watch::Sender<Option<Arc<Chain>>>,
}
/// 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.
#[allow(dead_code)]
#[derive(Clone, Debug)]
pub struct ReadStateService {
/// The shared inner on-disk database for the finalized state.
///
/// RocksDB allows reads and writes via a shared reference.
/// TODO: prevent write access via this type.
///
/// This chain is updated concurrently with requests,
/// so it might include some block data that is also in `best_mem`.
disk: DiskDb,
/// 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_mem: watch::Receiver<Option<Arc<Chain>>>,
/// The configured Zcash network.
network: Network,
}
impl StateService {
const PRUNE_INTERVAL: Duration = Duration::from_secs(30);
pub fn new(config: Config, network: Network) -> (Self, LatestChainTip, ChainTipChange) {
/// 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 disk = FinalizedState::new(&config, network);
let initial_tip = disk
.tip_block()
@ -99,6 +168,9 @@ impl StateService {
ChainTipSender::new(initial_tip, network);
let mem = NonFinalizedState::new(network);
let (read_only_service, best_chain_sender) = ReadStateService::new(&disk);
let queued_blocks = QueuedBlocks::default();
let pending_utxos = PendingUtxos::default();
@ -110,6 +182,7 @@ impl StateService {
network,
last_prune: Instant::now(),
chain_tip_sender,
best_chain_sender,
};
tracing::info!("starting legacy chain check");
@ -136,7 +209,7 @@ impl StateService {
}
tracing::info!("no legacy chain found");
(state, latest_chain_tip, chain_tip_change)
(state, read_only_service, latest_chain_tip, chain_tip_change)
}
/// Queue a finalized block for verification and storage in the finalized state.
@ -214,7 +287,11 @@ impl StateService {
);
self.queued_blocks.prune_by_height(finalized_tip_height);
let tip_block = self.mem.best_tip_block().map(ChainTipBlock::from);
let best_chain = self.mem.best_chain();
let tip_block = best_chain
.and_then(|chain| chain.tip_block())
.cloned()
.map(ChainTipBlock::from);
// update metrics using the best non-finalized tip
if let Some(tip_block) = tip_block.as_ref() {
@ -229,6 +306,13 @@ impl StateService {
metrics::gauge!("zcash.chain.verified.block.height", tip_block.height.0 as _);
}
// update the chain watch channels
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);
tracing::trace!("finished processing queued block");
@ -581,6 +665,26 @@ impl StateService {
}
}
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_only_service = Self {
disk: disk.db().clone(),
best_mem: best_chain_receiver,
network: disk.network(),
};
tracing::info!("created new read-only state service");
(read_only_service, best_chain_sender)
}
}
impl Service<Request> for StateService {
type Response = Response;
type Error = BoxError;
@ -727,13 +831,97 @@ impl Service<Request> for StateService {
}
}
impl Service<Request> for ReadStateService {
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>> {
Poll::Ready(Ok(()))
}
#[instrument(name = "read_state", skip(self, req))]
fn call(&mut self, req: Request) -> Self::Future {
match req {
// TODO: implement for lightwalletd before using this state in RPC methods
// Used by get_block RPC.
Request::Block(_hash_or_height) => unimplemented!("ReadStateService doesn't Block yet"),
// Used by get_best_block_hash & get_blockchain_info (#3143) RPCs.
//
// These RPC methods could use the ChainTip struct instead,
// if that's easier or more consistent.
Request::Tip => unimplemented!("ReadStateService doesn't Tip yet"),
// TODO: implement for lightwalletd as part of these tickets
// get_raw_transaction (#3145)
Request::Transaction(_hash) => {
unimplemented!("ReadStateService doesn't Transaction yet")
}
// TODO: split the Request enum, then implement these new ReadRequests for lightwalletd
// as part of these tickets
// z_get_tree_state (#3156)
// depends on transparent address indexes (#3150)
// get_address_tx_ids (#3147)
// get_address_balance (#3157)
// get_address_utxos (#3158)
// Out of Scope
// TODO: delete when splitting the Request enum
// These requests don't need better performance at the moment.
Request::FindBlockHashes {
known_blocks: _,
stop: _,
} => {
unreachable!("ReadStateService doesn't need to FindBlockHashes")
}
Request::FindBlockHeaders {
known_blocks: _,
stop: _,
} => {
unreachable!("ReadStateService doesn't need to FindBlockHeaders")
}
// Some callers of this request need to wait for queued blocks.
Request::Depth(_hash) => unreachable!("ReadStateService could change depth behaviour"),
// This request needs to wait for queued blocks.
Request::BlockLocator => {
unreachable!("ReadStateService should not be used for block locators")
}
// Impossible Requests
// The read-only service doesn't have the shared internal state
// needed to await UTXOs.
Request::AwaitUtxo(_outpoint) => unreachable!("ReadStateService can't await UTXOs"),
// The read-only service can't write.
Request::CommitBlock(_prepared) => unreachable!("ReadStateService can't commit blocks"),
Request::CommitFinalizedBlock(_finalized) => {
unreachable!("ReadStateService can't commit blocks")
}
}
}
}
/// Initialize a state service from the provided [`Config`].
/// Returns a boxed state service, and receivers for state chain tip updates.
/// 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`. It's
/// possible to construct multiple state services in the same application (as
/// 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.
pub fn init(
@ -741,13 +929,16 @@ pub fn init(
network: Network,
) -> (
BoxService<Request, Response, BoxError>,
ReadStateService,
LatestChainTip,
ChainTipChange,
) {
let (state_service, latest_chain_tip, chain_tip_change) = StateService::new(config, network);
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,
)
@ -758,7 +949,7 @@ pub fn init(
/// This can be used to create a state service for testing. See also [`init`].
#[cfg(any(test, feature = "proptest-impl"))]
pub fn init_test(network: Network) -> Buffer<BoxService<Request, Response, BoxError>, Request> {
let (state_service, _, _) = StateService::new(Config::ephemeral(), network);
let (state_service, _, _, _) = StateService::new(Config::ephemeral(), network);
Buffer::new(BoxService::new(state_service), 1)
}

View File

@ -18,14 +18,9 @@ use std::{
collections::HashMap,
io::{stderr, stdout, Write},
path::Path,
sync::Arc,
};
use zebra_chain::{
block::{self, Block},
history_tree::HistoryTree,
parameters::{Network, GENESIS_PREVIOUS_BLOCK_HASH},
};
use zebra_chain::{block, history_tree::HistoryTree, parameters::Network};
use crate::{
service::{check, QueuedFinalized},
@ -118,29 +113,19 @@ impl FinalizedState {
new_state
}
/// Returns the configured network for this database.
pub fn network(&self) -> Network {
self.network
}
/// Returns the `Path` where the files used by this database are located.
#[allow(dead_code)]
pub fn path(&self) -> &Path {
self.db.path()
}
/// Returns the hash of the current finalized tip block.
pub fn finalized_tip_hash(&self) -> block::Hash {
self.tip()
.map(|(_, hash)| hash)
// if the state is empty, return the genesis previous block hash
.unwrap_or(GENESIS_PREVIOUS_BLOCK_HASH)
}
/// Returns the height of the current finalized tip block.
pub fn finalized_tip_height(&self) -> Option<block::Height> {
self.tip().map(|(height, _)| height)
}
/// Returns the tip block, if there is one.
pub fn tip_block(&self) -> Option<Arc<Block>> {
let (height, _hash) = self.tip()?;
self.block(height.into())
/// Returns a reference to the inner database instance.
pub(crate) fn db(&self) -> &DiskDb {
&self.db
}
/// Queue a finalized block to be committed to the state.

View File

@ -78,6 +78,27 @@ impl FinalizedState {
self.db.zs_get(block_by_height, &height)
}
// Read tip block methods
/// Returns the hash of the current finalized tip block.
pub fn finalized_tip_hash(&self) -> block::Hash {
self.tip()
.map(|(_, hash)| hash)
// if the state is empty, return the genesis previous block hash
.unwrap_or(GENESIS_PREVIOUS_BLOCK_HASH)
}
/// Returns the height of the current finalized tip block.
pub fn finalized_tip_height(&self) -> Option<block::Height> {
self.tip().map(|(height, _)| height)
}
/// Returns the tip block, if there is one.
pub fn tip_block(&self) -> Option<Arc<Block>> {
let (height, _hash) = self.tip()?;
self.block(height.into())
}
// Read transaction methods
/// Returns the given transaction if it exists.

View File

@ -10,7 +10,7 @@ mod tests;
pub use queued_blocks::QueuedBlocks;
use std::{collections::BTreeSet, mem, ops::Deref, sync::Arc};
use std::{collections::BTreeSet, mem, sync::Arc};
use zebra_chain::{
block::{self, Block},
@ -332,9 +332,11 @@ impl NonFinalizedState {
}
/// Returns the block at the tip of the best chain.
pub fn best_tip_block(&self) -> Option<ContextuallyValidBlock> {
let (height, _hash) = self.best_tip()?;
self.best_block(height.into())
#[allow(dead_code)]
pub fn best_tip_block(&self) -> Option<&ContextuallyValidBlock> {
let best_chain = self.best_chain()?;
best_chain.tip_block()
}
/// Returns the height of `hash` in the best chain.
@ -388,12 +390,9 @@ impl NonFinalizedState {
.unwrap_or(false)
}
/// Return the non-finalized portion of the current best chain
pub(crate) fn best_chain(&self) -> Option<&Chain> {
self.chain_set
.iter()
.next_back()
.map(|box_chain| box_chain.deref())
/// Return the non-finalized portion of the current best chain.
pub(crate) fn best_chain(&self) -> Option<&Arc<Chain>> {
self.chain_set.iter().next_back()
}
/// Return the chain whose tip block hash is `parent_hash`.

View File

@ -369,6 +369,12 @@ impl Chain {
self.blocks.keys().next_back().cloned()
}
/// Return the non-finalized tip block for this chain,
/// or `None` if `self.blocks` is empty.
pub fn tip_block(&self) -> Option<&ContextuallyValidBlock> {
self.blocks.values().next_back()
}
/// Returns true if the non-finalized part of this chain is empty.
pub fn is_empty(&self) -> bool {
self.blocks.is_empty()

View File

@ -390,7 +390,7 @@ proptest! {
) {
zebra_test::init();
let (mut state_service, latest_chain_tip, mut chain_tip_change) = StateService::new(Config::ephemeral(), network);
let (mut state_service, _read_only_state_service, latest_chain_tip, mut chain_tip_change) = StateService::new(Config::ephemeral(), network);
prop_assert_eq!(latest_chain_tip.best_tip_height(), None);
prop_assert_eq!(chain_tip_change.last_tip_change(), None);
@ -443,7 +443,7 @@ proptest! {
) {
zebra_test::init();
let (mut state_service, _, _) = StateService::new(Config::ephemeral(), network);
let (mut state_service, _, _, _) = StateService::new(Config::ephemeral(), network);
prop_assert_eq!(state_service.disk.finalized_value_pool(), ValueBalance::zero());
prop_assert_eq!(

View File

@ -84,7 +84,7 @@ pub(crate) fn new_state_with_mainnet_genesis() -> (StateService, FinalizedBlock)
.zcash_deserialize_into::<Arc<Block>>()
.expect("block should deserialize");
let (mut state, _, _) = StateService::new(Config::ephemeral(), Mainnet);
let (mut state, _, _, _) = StateService::new(Config::ephemeral(), Mainnet);
assert_eq!(None, state.best_tip());

View File

@ -69,7 +69,7 @@ async fn check_transcripts(network: Network) -> Result<(), Report> {
Network::Mainnet => mainnet_transcript,
Network::Testnet => testnet_transcript,
} {
let (service, _, _) = zebra_state::init(Config::ephemeral(), network);
let (service, _, _, _) = zebra_state::init(Config::ephemeral(), network);
let transcript = Transcript::from(transcript_data.iter().cloned());
/// SPANDOC: check the on disk service against the transcript
transcript.check(service).await?;

View File

@ -111,8 +111,13 @@ impl CopyStateCmd {
);
let source_start_time = Instant::now();
let (mut source_state, _source_latest_chain_tip, _source_chain_tip_change) =
old_zs::init(source_config.clone(), network);
// TODO: use ReadStateService for the source?
let (
mut source_state,
_source_read_only_state_service,
_source_latest_chain_tip,
_source_chain_tip_change,
) = old_zs::init(source_config.clone(), network);
let elapsed = source_start_time.elapsed();
info!(?elapsed, "finished initializing source state service");
@ -123,8 +128,15 @@ impl CopyStateCmd {
);
let target_start_time = Instant::now();
let (mut target_state, _target_latest_chain_tip, _target_chain_tip_change) =
new_zs::init(target_config.clone(), network);
// TODO: call Options::PrepareForBulkLoad()
// See "What's the fastest way to load data into RocksDB?" in
// https://github.com/facebook/rocksdb/wiki/RocksDB-FAQ
let (
mut target_state,
_target_read_only_state_service,
_target_latest_chain_tip,
_target_chain_tip_change,
) = new_zs::init(target_config.clone(), network);
let elapsed = target_start_time.elapsed();
info!(?elapsed, "finished initializing target state service");

View File

@ -53,6 +53,11 @@
//! * Transaction Gossip Task
//! * runs in the background and gossips newly added mempool transactions
//! to peers
//!
//! Remote Procedure Calls:
//! * JSON-RPC Service
//! * answers RPC client requests using the State Service and Mempool Service
//! * submits client transactions to the node's mempool
use std::{cmp::max, ops::Add, time::Duration};
@ -101,7 +106,7 @@ impl StartCmd {
info!(?config);
info!("initializing node state");
let (state_service, latest_chain_tip, chain_tip_change) =
let (state_service, _read_only_state_service, latest_chain_tip, chain_tip_change) =
zebra_state::init(config.state.clone(), config.network.network);
let state = ServiceBuilder::new()
.buffer(Self::state_buffer_bound())

View File

@ -698,7 +698,7 @@ async fn setup(
let address_book = AddressBook::new(SocketAddr::from_str("0.0.0.0:0").unwrap(), Span::none());
let address_book = Arc::new(std::sync::Mutex::new(address_book));
let (sync_status, mut recent_syncs) = SyncStatus::new();
let (state, latest_chain_tip, chain_tip_change) =
let (state, _read_only_state_service, latest_chain_tip, chain_tip_change) =
zebra_state::init(state_config.clone(), network);
let mut state_service = ServiceBuilder::new().buffer(1).service(state);

View File

@ -645,7 +645,7 @@ async fn setup(
// State
let state_config = StateConfig::ephemeral();
let (state_service, latest_chain_tip, chain_tip_change) =
let (state_service, _read_only_state_service, latest_chain_tip, chain_tip_change) =
zebra_state::init(state_config, network);
let state_service = ServiceBuilder::new().buffer(10).service(state_service);

View File

@ -704,7 +704,8 @@ async fn setup(
let peer_set = MockService::build().for_unit_tests();
let state_config = StateConfig::ephemeral();
let (state, latest_chain_tip, chain_tip_change) = zebra_state::init(state_config, network);
let (state, _read_only_state_service, latest_chain_tip, chain_tip_change) =
zebra_state::init(state_config, network);
let state_service = ServiceBuilder::new().buffer(1).service(state);
let tx_verifier = MockService::build().for_unit_tests();