From 4306a00f3cb66210c7320168dc66bcad58b91d6c Mon Sep 17 00:00:00 2001 From: teor Date: Mon, 4 Dec 2023 07:58:48 +1000 Subject: [PATCH] Scan blocks with sapling keys and write the results to the database (#8040) * Fix availability of tokio::time in scanner * Create a function that parses a key into a list of keys * Pass a ChainTipChange to the scanner function * Convert a scanned block to a sapling result * Make it easier to pass keys and blocks * Increase scanner wait times * Parse keys once at the start of the scan * Get a block from the state instead of the tip * Don't log secret keys, only log every 100,000 blocks * Scan each block and add the results to storage * Move blocking tasks into spawn_blocking() * Update the acceptance test * Use a dummy sapling tree size * Use a larger dummy size --- Cargo.lock | 1 + zebra-scan/Cargo.toml | 3 +- zebra-scan/src/config.rs | 4 +- zebra-scan/src/init.rs | 15 +- zebra-scan/src/scan.rs | 225 ++++++++++++++++++++++----- zebra-scan/src/storage.rs | 4 +- zebra-scan/src/tests.rs | 5 +- zebra-state/src/service/chain_tip.rs | 5 + zebrad/src/commands/start.rs | 10 +- zebrad/tests/acceptance.rs | 9 +- 10 files changed, 227 insertions(+), 54 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index e52eaaf9a..6fdbefe34 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5802,6 +5802,7 @@ dependencies = [ "ff", "group", "indexmap 2.1.0", + "itertools 0.12.0", "jubjub", "rand 0.8.5", "semver 1.0.20", diff --git a/zebra-scan/Cargo.toml b/zebra-scan/Cargo.toml index bd828ee72..c4cc20f40 100644 --- a/zebra-scan/Cargo.toml +++ b/zebra-scan/Cargo.toml @@ -22,9 +22,10 @@ categories = ["cryptography::cryptocurrencies"] color-eyre = "0.6.2" indexmap = { version = "2.0.1", features = ["serde"] } +itertools = "0.12.0" semver = "1.0.20" serde = { version = "1.0.193", features = ["serde_derive"] } -tokio = "1.34.0" +tokio = { version = "1.34.0", features = ["time"] } tower = "0.4.13" tracing = "0.1.39" diff --git a/zebra-scan/src/config.rs b/zebra-scan/src/config.rs index cda7e4910..df217e906 100644 --- a/zebra-scan/src/config.rs +++ b/zebra-scan/src/config.rs @@ -12,13 +12,15 @@ use crate::storage::SaplingScanningKey; /// Configuration for scanning. pub struct Config { /// The sapling keys to scan for and the birthday height of each of them. + /// + /// Currently only supports Extended Full Viewing Keys in ZIP-32 format. // // TODO: allow keys without birthdays pub sapling_keys_to_scan: IndexMap, /// The scanner results database config. // - // TODO: Remove fields that are only used by the state to create a common database config. + // TODO: Remove fields that are only used by the state, and create a common database config. #[serde(flatten)] db_config: DbConfig, } diff --git a/zebra-scan/src/init.rs b/zebra-scan/src/init.rs index bb6193ec8..a94676ead 100644 --- a/zebra-scan/src/init.rs +++ b/zebra-scan/src/init.rs @@ -5,6 +5,7 @@ use tokio::task::JoinHandle; use tracing::Instrument; use zebra_chain::{diagnostic::task::WaitForPanics, parameters::Network}; +use zebra_state::ChainTipChange; use crate::{scan, storage::Storage, Config}; @@ -15,19 +16,27 @@ pub fn spawn_init( config: &Config, network: Network, state: scan::State, + chain_tip_change: ChainTipChange, ) -> JoinHandle> { let config = config.clone(); - tokio::spawn(init(config, network, state).in_current_span()) + + // TODO: spawn an entirely new executor here, to avoid timing attacks. + tokio::spawn(init(config, network, state, chain_tip_change).in_current_span()) } /// Initialize the scanner based on its config. /// /// TODO: add a test for this function. -pub async fn init(config: Config, network: Network, state: scan::State) -> Result<(), Report> { +pub async fn init( + config: Config, + network: Network, + state: scan::State, + chain_tip_change: ChainTipChange, +) -> Result<(), Report> { let storage = tokio::task::spawn_blocking(move || Storage::new(&config, network)) .wait_for_panics() .await; // TODO: add more tasks here? - scan::start(state, storage).await + scan::start(state, chain_tip_change, storage).await } diff --git a/zebra-scan/src/scan.rs b/zebra-scan/src/scan.rs index 111a8fa2e..907036933 100644 --- a/zebra-scan/src/scan.rs +++ b/zebra-scan/src/scan.rs @@ -1,26 +1,37 @@ //! The scanner task and scanning APIs. -use std::{sync::Arc, time::Duration}; +use std::{collections::HashMap, sync::Arc, time::Duration}; use color_eyre::{eyre::eyre, Report}; +use itertools::Itertools; use tower::{buffer::Buffer, util::BoxService, Service, ServiceExt}; use tracing::info; use zcash_client_backend::{ data_api::ScannedBlock, + encoding::decode_extended_full_viewing_key, proto::compact_formats::{ ChainMetadata, CompactBlock, CompactSaplingOutput, CompactSaplingSpend, CompactTx, }, scanning::{ScanError, ScanningKey}, }; -use zcash_primitives::zip32::AccountId; - -use zebra_chain::{ - block::Block, diagnostic::task::WaitForPanics, parameters::Network, - serialization::ZcashSerialize, transaction::Transaction, +use zcash_primitives::{ + constants::*, + sapling::SaplingIvk, + zip32::{AccountId, DiversifiableFullViewingKey, Scope}, }; -use crate::storage::Storage; +use zebra_chain::{ + block::Block, + chain_tip::ChainTip, + diagnostic::task::WaitForPanics, + parameters::Network, + serialization::ZcashSerialize, + transaction::{self, Transaction}, +}; +use zebra_state::{ChainTipChange, SaplingScannedResult}; + +use crate::storage::{SaplingScanningKey, Storage}; /// The generic state type used by the scanner. pub type State = Buffer< @@ -28,54 +39,146 @@ pub type State = Buffer< zebra_state::Request, >; -/// Wait a few seconds at startup so tip height is always `Some`. -const INITIAL_WAIT: Duration = Duration::from_secs(10); +/// Wait a few seconds at startup for some blocks to get verified. +/// +/// But sometimes the state might be empty if the network is slow. +const INITIAL_WAIT: Duration = Duration::from_secs(15); -/// The amount of time between checking and starting new scans. +/// The amount of time between checking for new blocks and starting new scans. +/// +/// This is just under half the target block interval. const CHECK_INTERVAL: Duration = Duration::from_secs(30); -/// Start the scan task given state and storage. -/// -/// - This function is dummy at the moment. It just makes sure we can read the storage and the state. -/// - Modifications here might have an impact in the `scan_task_starts` test. -/// - Real scanning code functionality will be added in the future here. -pub async fn start(mut state: State, storage: Storage) -> Result<(), Report> { - // We want to make sure the state has a tip height available before we start scanning. +/// We log an info log with progress after this many blocks. +const INFO_LOG_INTERVAL: u32 = 100_000; + +/// Start a scan task that reads blocks from `state`, scans them with the configured keys in +/// `storage`, and then writes the results to `storage`. +pub async fn start( + mut state: State, + chain_tip_change: ChainTipChange, + storage: Storage, +) -> Result<(), Report> { + let network = storage.network(); + let mut height = storage.min_sapling_birthday_height(); + + // Read keys from the storage on disk, which can block async execution. + let key_storage = storage.clone(); + let key_birthdays = tokio::task::spawn_blocking(move || key_storage.sapling_keys()) + .wait_for_panics() + .await; + + // Parse and convert keys once, then use them to scan all blocks. + // There is some cryptography here, but it should be fast even with thousands of keys. + let parsed_keys: HashMap< + SaplingScanningKey, + (Vec, Vec), + > = key_birthdays + .keys() + .map(|key| { + let parsed_keys = sapling_key_to_scan_block_keys(key, network)?; + Ok::<_, Report>((key.clone(), parsed_keys)) + }) + .try_collect()?; + + // Give empty states time to verify some blocks before we start scanning. tokio::time::sleep(INITIAL_WAIT).await; loop { - // Make sure we can query the state - let request = state + // Get a block from the state. + // We can't use ServiceExt::oneshot() here, because it causes lifetime errors in init(). + let block = state .ready() .await .map_err(|e| eyre!(e))? - .call(zebra_state::Request::Tip) + .call(zebra_state::Request::Block(height.into())) .await - .map_err(|e| eyre!(e)); + .map_err(|e| eyre!(e))?; - let tip = match request? { - zebra_state::Response::Tip(tip) => tip, + let block = match block { + zebra_state::Response::Block(Some(block)) => block, + zebra_state::Response::Block(None) => { + // If we've reached the tip, sleep for a while then try and get the same block. + tokio::time::sleep(CHECK_INTERVAL).await; + continue; + } _ => unreachable!("unmatched response to a state::Tip request"), }; - // Read keys from the storage on disk, which can block. - let key_storage = storage.clone(); - let available_keys = tokio::task::spawn_blocking(move || key_storage.sapling_keys()) - .wait_for_panics() - .await; + // Only log at info level every 100,000 blocks + let is_info_log = + height == storage.min_sapling_birthday_height() || height.0 % INFO_LOG_INTERVAL == 0; - for key in available_keys { + // TODO: add debug logs? + if is_info_log { info!( - "Scanning the blockchain for key {} from block {:?} to {:?}", - key.0, key.1, tip, + "Scanning the blockchain: now at block {:?}, current tip {:?}", + height, + chain_tip_change + .latest_chain_tip() + .best_tip_height_and_hash(), ); } - tokio::time::sleep(CHECK_INTERVAL).await; + for (key_num, (sapling_key, birthday_height)) in key_birthdays.iter().enumerate() { + // # Security + // + // We can't log `sapling_key` here because it is a private viewing key. Anyone who reads + // the logs could use the key to view those transactions. + if is_info_log { + info!( + "Scanning the blockchain for key {}, started at block {:?}", + key_num, birthday_height, + ); + } + + // Get the pre-parsed keys for this configured key. + let (dfvks, ivks) = parsed_keys.get(sapling_key).cloned().unwrap_or_default(); + + // Scan the block, which blocks async execution until the scan is complete. + // + // TODO: skip scanning before birthday height (#8022) + // TODO: scan each key in parallel (after MVP?) + let sapling_key = sapling_key.clone(); + let block = block.clone(); + let mut storage = storage.clone(); + + // We use a dummy size of the Sapling note commitment tree. + // + // We can't set the size to zero, because the underlying scanning function would return + // `zcash_client_backeng::scanning::ScanError::TreeSizeUnknown`. + // + // And we can't set them close to 0, because the scanner subtracts the number of notes + // in the block, and panics with "attempt to subtract with overflow". The number of + // notes in a block must be less than this value, this is a consensus rule. + let sapling_tree_size = 1 << 16; + + tokio::task::spawn_blocking(move || { + let dfvk_res = + scan_block(network, &block, sapling_tree_size, &dfvks).map_err(|e| eyre!(e))?; + let ivk_res = + scan_block(network, &block, sapling_tree_size, &ivks).map_err(|e| eyre!(e))?; + + let dfvk_res = scanned_block_to_db_result(dfvk_res); + let ivk_res = scanned_block_to_db_result(ivk_res); + + storage.add_sapling_result(sapling_key.clone(), height, dfvk_res); + storage.add_sapling_result(sapling_key, height, ivk_res); + + Ok::<_, Report>(()) + }) + .wait_for_panics() + .await?; + } + + height = height + .next() + .expect("a valid blockchain never reaches the max height"); } } -/// Returns transactions belonging to the given `ScanningKey`. +/// Returns transactions belonging to the given `ScanningKey`. This list of keys should come from +/// a single configured `SaplingScanningKey`. /// /// # Performance / Hangs /// @@ -88,9 +191,9 @@ pub async fn start(mut state: State, storage: Storage) -> Result<(), Report> { /// - Add prior block metadata once we have access to Zebra's state. pub fn scan_block( network: Network, - block: Arc, + block: &Arc, sapling_tree_size: u32, - scanning_key: &K, + scanning_keys: &[K], ) -> Result, ScanError> { // TODO: Implement a check that returns early when the block height is below the Sapling // activation height. @@ -105,14 +208,15 @@ pub fn scan_block( // Use a dummy `AccountId` as we don't use accounts yet. let dummy_account = AccountId::from(0); - - // We only support scanning one key and one block per function call for now. - let scanning_keys = vec![(&dummy_account, scanning_key)]; + let scanning_keys: Vec<_> = scanning_keys + .iter() + .map(|key| (&dummy_account, key)) + .collect(); zcash_client_backend::scanning::scan_block( &network, block_to_compact(block, chain_metadata), - &scanning_keys, + scanning_keys.as_slice(), // Ignore whether notes are change from a viewer's own spends for now. &[], // Ignore previous blocks for now. @@ -120,14 +224,45 @@ pub fn scan_block( ) } +/// Converts a Zebra-format scanning key into some `scan_block()` keys. +/// +/// Currently only accepts extended full viewing keys, and returns both their diversifiable full +/// viewing key and their individual viewing key, for testing purposes. +/// +/// TODO: work out what string format is used for SaplingIvk, if any, and support it here +/// performance: stop returning both the dfvk and ivk for the same key +pub fn sapling_key_to_scan_block_keys( + sapling_key: &SaplingScanningKey, + network: Network, +) -> Result<(Vec, Vec), Report> { + let hrp = if network.is_a_test_network() { + // Assume custom testnets have the same HRP + // + // TODO: add the regtest HRP here + testnet::HRP_SAPLING_EXTENDED_FULL_VIEWING_KEY + } else { + mainnet::HRP_SAPLING_EXTENDED_FULL_VIEWING_KEY + }; + + let efvk = decode_extended_full_viewing_key(hrp, sapling_key).map_err(|e| eyre!(e))?; + + // Just return all the keys for now, so we can be sure our code supports them. + let dfvk = efvk.to_diversifiable_full_viewing_key(); + let eivk = dfvk.to_ivk(Scope::External); + let iivk = dfvk.to_ivk(Scope::Internal); + + Ok((vec![dfvk], vec![eivk, iivk])) +} + /// Converts a zebra block and meta data into a compact block. -pub fn block_to_compact(block: Arc, chain_metadata: ChainMetadata) -> CompactBlock { +pub fn block_to_compact(block: &Arc, chain_metadata: ChainMetadata) -> CompactBlock { CompactBlock { height: block .coinbase_height() .expect("verified block should have a valid height") .0 .into(), + // TODO: performance: look up the block hash from the state rather than recalculating it hash: block.hash().bytes_in_display_order().to_vec(), prev_hash: block .header @@ -164,6 +299,7 @@ fn transaction_to_compact((index, tx): (usize, Arc)) -> CompactTx { index: index .try_into() .expect("tx index in block should fit in u64"), + // TODO: performance: look up the tx hash from the state rather than recalculating it hash: tx.hash().bytes_in_display_order().to_vec(), // `fee` is not checked by the `scan_block` function. It is allowed to be unset. @@ -202,3 +338,12 @@ fn transaction_to_compact((index, tx): (usize, Arc)) -> CompactTx { actions: vec![], } } + +/// Convert a scanned block to a list of scanner database results. +fn scanned_block_to_db_result(scanned_block: ScannedBlock) -> Vec { + scanned_block + .transactions() + .iter() + .map(|tx| transaction::Hash::from_bytes_in_display_order(tx.txid.as_ref())) + .collect() +} diff --git a/zebra-scan/src/storage.rs b/zebra-scan/src/storage.rs index 1b4eba8e9..bb3b0fe04 100644 --- a/zebra-scan/src/storage.rs +++ b/zebra-scan/src/storage.rs @@ -101,7 +101,7 @@ impl Storage { &mut self, sapling_key: SaplingScanningKey, height: Height, - result: Vec, + sapling_result: Vec, ) { // It's ok to write some results and not others during shutdown, so each result can get its // own batch. (They will be re-scanned on startup anyway.) @@ -114,7 +114,7 @@ impl Storage { let entry = SaplingScannedDatabaseEntry { index, - value: result, + value: sapling_result, }; batch.insert_sapling_result(self, entry); diff --git a/zebra-scan/src/tests.rs b/zebra-scan/src/tests.rs index dd4a7fdf8..d11b6a102 100644 --- a/zebra-scan/src/tests.rs +++ b/zebra-scan/src/tests.rs @@ -109,6 +109,7 @@ async fn scanning_zecpages_from_populated_zebra_state() -> Result<()> { // Build a vector of viewing keys `vks` to scan for. let fvk = efvk.fvk; let ivk = fvk.vk.ivk(); + let ivks = vec![ivk]; let network = zebra_chain::parameters::Network::Mainnet; @@ -143,9 +144,9 @@ async fn scanning_zecpages_from_populated_zebra_state() -> Result<()> { orchard_commitment_tree_size, }; - let compact_block = block_to_compact(block.clone(), chain_metadata); + let compact_block = block_to_compact(&block, chain_metadata); - let res = scan_block(network, block, sapling_commitment_tree_size, &ivk) + let res = scan_block(network, &block, sapling_commitment_tree_size, &ivks) .expect("scanning block for the ZECpages viewing key should work"); transactions_found += res.transactions().len(); diff --git a/zebra-state/src/service/chain_tip.rs b/zebra-state/src/service/chain_tip.rs index 76f57bfab..04d2af863 100644 --- a/zebra-state/src/service/chain_tip.rs +++ b/zebra-state/src/service/chain_tip.rs @@ -628,6 +628,11 @@ impl ChainTipChange { } } } + + /// Returns the inner `LatestChainTip`. + pub fn latest_chain_tip(&self) -> LatestChainTip { + self.latest_chain_tip.clone() + } } impl Clone for ChainTipChange { diff --git a/zebrad/src/commands/start.rs b/zebrad/src/commands/start.rs index e40197111..99f2ef7f3 100644 --- a/zebrad/src/commands/start.rs +++ b/zebrad/src/commands/start.rs @@ -282,7 +282,7 @@ impl StartCmd { peer_set, mempool.clone(), sync_status, - chain_tip_change, + chain_tip_change.clone(), ); info!("spawning syncer task"); @@ -291,8 +291,14 @@ impl StartCmd { #[cfg(feature = "shielded-scan")] // Spawn never ending scan task. let scan_task_handle = { + // TODO: log the number of keys and update the scan_task_starts() test info!("spawning shielded scanner with configured viewing keys"); - zebra_scan::spawn_init(&config.shielded_scan, config.network.network, state) + zebra_scan::spawn_init( + &config.shielded_scan, + config.network.network, + state, + chain_tip_change, + ) }; #[cfg(not(feature = "shielded-scan"))] diff --git a/zebrad/tests/acceptance.rs b/zebrad/tests/acceptance.rs index afab36317..0b91ff8e9 100644 --- a/zebrad/tests/acceptance.rs +++ b/zebrad/tests/acceptance.rs @@ -2805,8 +2805,8 @@ async fn fully_synced_rpc_z_getsubtreesbyindex_snapshot_test() -> Result<()> { } /// Test that the scanner task gets started when the node starts. -#[cfg(feature = "shielded-scan")] #[test] +#[cfg(feature = "shielded-scan")] fn scan_task_starts() -> Result<()> { use indexmap::IndexMap; @@ -2832,13 +2832,16 @@ fn scan_task_starts() -> Result<()> { let output = child.wait_with_output()?; output.stdout_line_contains("spawning shielded scanner with configured viewing keys")?; + /* + TODO: these lines only happen when we reach sapling activation height + output.stdout_line_contains("Scanning the blockchain: now at block")?; output.stdout_line_contains( format!( - "Scanning the blockchain for key {} from block", - ZECPAGES_VIEWING_KEY + "Scanning the blockchain for key 0, started at block", ) .as_str(), )?; + */ // Make sure the command was killed output.assert_was_killed()?;