From 0039c20d3651d0d726d5fba2dd4a51e3f17df9e8 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 28 Feb 2023 17:25:18 +0100 Subject: [PATCH 01/88] feat: Dump state to S3 --- chain/chain/src/store.rs | 21 +- core/chain-configs/src/client_config.rs | 19 +- core/primitives/src/syncing.rs | 20 +- nearcore/src/config.rs | 149 ++++------ nearcore/src/lib.rs | 89 ++---- nearcore/src/metrics.rs | 66 ++--- nearcore/src/state_sync.rs | 370 +++++++++--------------- 7 files changed, 255 insertions(+), 479 deletions(-) diff --git a/chain/chain/src/store.rs b/chain/chain/src/store.rs index 22ec2b5d0ea..a59793cbd82 100644 --- a/chain/chain/src/store.rs +++ b/chain/chain/src/store.rs @@ -47,6 +47,7 @@ use crate::chunks_store::ReadOnlyChunksStore; use crate::types::{Block, BlockHeader, LatestKnown}; use crate::{byzantine_assert, RuntimeWithEpochManagerAdapter}; use near_store::db::StoreStatistics; +use near_store::flat_state::{BlockInfo, ChainAccessForFlatStorage}; use std::sync::Arc; /// lru cache size @@ -841,7 +842,6 @@ impl ChainStore { /// Constructs key 'STATE_SYNC_DUMP:', /// for example 'STATE_SYNC_DUMP:2' for shard_id=2. - /// Doesn't contain epoch_id, because only one dump process per shard is allowed. fn state_sync_dump_progress_key(shard_id: ShardId) -> Vec { let mut key = b"STATE_SYNC_DUMP:".to_vec(); key.extend(shard_id.to_le_bytes()); @@ -1165,6 +1165,20 @@ impl ChainStoreAccess for ChainStore { } } +impl ChainAccessForFlatStorage for ChainStore { + fn get_block_info(&self, block_hash: &CryptoHash) -> BlockInfo { + let header = self.get_block_header(block_hash).unwrap(); + BlockInfo { hash: *block_hash, height: header.height(), prev_hash: *header.prev_hash() } + } + + fn get_block_hashes_at_height(&self, height: BlockHeight) -> HashSet { + match self.get_all_block_hashes_by_height(height) { + Ok(hashes) => hashes.values().flatten().copied().collect::>(), + Err(_) => Default::default(), + } + } +} + /// Cache update for ChainStore #[derive(Default)] struct ChainStoreCacheUpdate { @@ -2437,10 +2451,7 @@ impl<'a> ChainStoreUpdate<'a> { unreachable!(); } #[cfg(feature = "protocol_feature_flat_state")] - DBCol::FlatState - | DBCol::FlatStateChanges - | DBCol::FlatStateDeltaMetadata - | DBCol::FlatStorageStatus => { + DBCol::FlatState | DBCol::FlatStateDeltas | DBCol::FlatStateMisc => { unreachable!(); } } diff --git a/core/chain-configs/src/client_config.rs b/core/chain-configs/src/client_config.rs index 3d7ee29b827..202363eb6a6 100644 --- a/core/chain-configs/src/client_config.rs +++ b/core/chain-configs/src/client_config.rs @@ -166,18 +166,10 @@ pub struct ClientConfig { pub client_background_migration_threads: usize, /// Duration to perform background flat storage creation step. pub flat_storage_creation_period: Duration, - /// If enabled, will dump state of every epoch to external storage. - pub state_sync_dump_enabled: bool, /// S3 bucket for storing state dumps. - pub state_sync_s3_bucket: String, + pub state_sync_s3_bucket: Option, /// S3 region for storing state dumps. - pub state_sync_s3_region: String, - /// Restart dumping state of selected shards. - /// Use for troubleshooting of the state dumping process. - pub state_sync_restart_dump_for_shards: Vec, - /// Whether to use the State Sync mechanism. - /// If disabled, the node will do Block Sync instead of State Sync. - pub state_sync_enabled: bool, + pub state_sync_s3_region: Option, } impl ClientConfig { @@ -247,11 +239,8 @@ impl ClientConfig { enable_statistics_export: true, client_background_migration_threads: 1, flat_storage_creation_period: Duration::from_secs(1), - state_sync_dump_enabled: false, - state_sync_s3_bucket: String::new(), - state_sync_s3_region: String::new(), - state_sync_restart_dump_for_shards: vec![], - state_sync_enabled: true, + state_sync_s3_bucket: None, + state_sync_s3_region: None, } } } diff --git a/core/primitives/src/syncing.rs b/core/primitives/src/syncing.rs index a1450fceb8d..c948fd7a895 100644 --- a/core/primitives/src/syncing.rs +++ b/core/primitives/src/syncing.rs @@ -229,27 +229,15 @@ pub fn get_num_state_parts(memory_usage: u64) -> u64 { } #[derive(BorshSerialize, BorshDeserialize, Debug, Clone)] -/// Represents the progress of dumps state of a shard. +/// Represents the state of the state machine that dumps state. pub enum StateSyncDumpProgress { - /// Represents two cases: - /// * An epoch dump is complete - /// * The node is running its first epoch and there is nothing to dump. - AllDumped { - /// The dumped state corresponds to the state at the beginning of the specified epoch. - epoch_id: EpochId, - epoch_height: EpochHeight, - // Missing in case of a node running the first epoch. - num_parts: Option, - }, - /// Represents the case of an epoch being partially dumped. + AllDumped(EpochId), InProgress { - /// The dumped state corresponds to the state at the beginning of the specified epoch. epoch_id: EpochId, epoch_height: EpochHeight, - /// Block hash of the first block of the epoch. - /// The dumped state corresponds to the state before applying this block. sync_hash: CryptoHash, - /// Progress made. + state_root: StateRoot, parts_dumped: u64, + num_parts: u64, }, } diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index 2ca1f01e359..456ec1cee17 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -1,10 +1,26 @@ -use crate::download_file::{run_download_file, FileDownloadError}; use anyhow::{anyhow, bail, Context}; +use near_primitives::static_clock::StaticClock; +use near_primitives::test_utils::create_test_signer; +use num_rational::Rational32; +use std::fs; +use std::fs::File; +use std::io::{Read, Write}; +use std::path::{Path, PathBuf}; +use std::str::FromStr; +use std::sync::Arc; +use std::time::Duration; + +use near_config_utils::{ValidationError, ValidationErrors}; + +#[cfg(test)] +use tempfile::tempdir; +use tracing::{info, warn}; + +use crate::download_file::{run_download_file, FileDownloadError}; use near_chain_configs::{ get_initial_supply, ClientConfig, GCConfig, Genesis, GenesisConfig, GenesisValidationMode, LogSummaryStyle, MutableConfigValue, }; -use near_config_utils::{ValidationError, ValidationErrors}; use near_crypto::{InMemorySigner, KeyFile, KeyType, PublicKey, Signer}; #[cfg(feature = "json_rpc")] use near_jsonrpc::RpcConfig; @@ -16,11 +32,9 @@ use near_primitives::hash::CryptoHash; use near_primitives::shard_layout::account_id_to_shard_id; use near_primitives::shard_layout::ShardLayout; use near_primitives::state_record::StateRecord; -use near_primitives::static_clock::StaticClock; -use near_primitives::test_utils::create_test_signer; use near_primitives::types::{ - AccountId, AccountInfo, Balance, BlockHeight, BlockHeightDelta, Gas, NumBlocks, NumSeats, - NumShards, ShardId, + AccountId, AccountInfo, Balance, BlockHeight, BlockHeightDelta, EpochHeight, Gas, NumBlocks, + NumSeats, NumShards, ShardId, }; use near_primitives::utils::{generate_random_string, get_num_seats_per_shard}; use near_primitives::validator_signer::{InMemoryValidatorSigner, ValidatorSigner}; @@ -28,17 +42,6 @@ use near_primitives::version::PROTOCOL_VERSION; #[cfg(feature = "rosetta_rpc")] use near_rosetta_rpc::RosettaRpcConfig; use near_telemetry::TelemetryConfig; -use num_rational::Rational32; -use std::fs; -use std::fs::File; -use std::io::{Read, Write}; -use std::path::Path; -use std::str::FromStr; -use std::sync::Arc; -use std::time::Duration; -#[cfg(test)] -use tempfile::tempdir; -use tracing::{info, warn}; /// Initial balance used in tests. pub const TESTING_INIT_BALANCE: Balance = 1_000_000_000 * NEAR_BASE; @@ -64,6 +67,9 @@ pub const MAX_BLOCK_PRODUCTION_DELAY: u64 = 2_000; /// Maximum time until skipping the previous block is ms. pub const MAX_BLOCK_WAIT_DELAY: u64 = 6_000; +/// Reduce wait time for every missing block in ms. +const REDUCE_DELAY_FOR_MISSING_BLOCKS: u64 = 100; + /// Horizon at which instead of fetching block, fetch full state. const BLOCK_FETCH_HORIZON: BlockHeightDelta = 50; @@ -117,6 +123,9 @@ pub const NUM_BLOCK_PRODUCER_SEATS: NumSeats = 50; /// The minimum stake required for staking is last seat price divided by this number. pub const MINIMUM_STAKE_DIVISOR: u64 = 10; +/// Number of epochs before protocol upgrade. +pub const PROTOCOL_UPGRADE_NUM_EPOCHS: EpochHeight = 2; + pub const CONFIG_FILENAME: &str = "config.json"; pub const GENESIS_CONFIG_FILENAME: &str = "genesis.json"; pub const NODE_KEY_FILE: &str = "node_key.json"; @@ -143,6 +152,11 @@ pub const MAX_INFLATION_RATE: Rational32 = Rational32::new_raw(1, 20); /// Protocol upgrade stake threshold. pub const PROTOCOL_UPGRADE_STAKE_THRESHOLD: Rational32 = Rational32::new_raw(4, 5); +/// Serde default only supports functions without parameters. +fn default_reduce_wait_for_missing_block() -> Duration { + Duration::from_millis(REDUCE_DELAY_FOR_MISSING_BLOCKS) +} + fn default_header_sync_initial_timeout() -> Duration { Duration::from_secs(10) } @@ -203,6 +217,9 @@ pub struct Consensus { pub max_block_production_delay: Duration, /// Maximum duration before skipping given height. pub max_block_wait_delay: Duration, + /// Duration to reduce the wait for each missed block by validator. + #[serde(default = "default_reduce_wait_for_missing_block")] + pub reduce_wait_for_missing_block: Duration, /// Produce empty blocks, use `false` for testing. pub produce_empty_blocks: bool, /// Horizon at which instead of fetching block, fetch full state. @@ -251,6 +268,7 @@ impl Default for Consensus { min_block_production_delay: Duration::from_millis(MIN_BLOCK_PRODUCTION_DELAY), max_block_production_delay: Duration::from_millis(MAX_BLOCK_PRODUCTION_DELAY), max_block_wait_delay: Duration::from_millis(MAX_BLOCK_WAIT_DELAY), + reduce_wait_for_missing_block: default_reduce_wait_for_missing_block(), produce_empty_blocks: true, block_fetch_horizon: BLOCK_FETCH_HORIZON, state_fetch_horizon: STATE_FETCH_HORIZON, @@ -289,16 +307,14 @@ pub struct Config { pub consensus: Consensus, pub tracked_accounts: Vec, pub tracked_shards: Vec, - #[serde(skip_serializing_if = "Option::is_none")] - pub tracked_shard_schedule: Option>>, #[serde(skip_serializing_if = "is_false")] pub archive: bool, /// If save_trie_changes is not set it will get inferred from the `archive` field as follows: /// save_trie_changes = !archive /// save_trie_changes should be set to true iff /// - archive if false - non-archival nodes need trie changes to perform garbage collection - /// - archive is true and cold_store is configured - node working in split storage mode - /// needs trie changes in order to do garbage collection on hot and populate cold State column. + /// - archive is true, cold_store is configured and migration to split_storage is finished - node + /// working in split storage mode needs trie changes in order to do garbage collection on hot. #[serde(skip_serializing_if = "Option::is_none")] pub save_trie_changes: Option, pub log_summary_style: LogSummaryStyle, @@ -321,19 +337,22 @@ pub struct Config { /// This feature is under development, do not use in production. #[serde(default, skip_serializing_if = "Option::is_none")] pub cold_store: Option, - /// Configuration for the + + // TODO(mina86): Remove those two altogether at some point. We need to be + // somewhat careful though and make sure that we don’t start silently + // ignoring this option without users setting corresponding store option. + // For the time being, we’re failing inside of create_db_checkpoint if this + // option is set. + /// Deprecated; use `store.migration_snapshot` instead. #[serde(default, skip_serializing_if = "Option::is_none")] - pub split_storage: Option, - /// The node will stop after the head exceeds this height. - /// The node usually stops within several seconds after reaching the target height. + pub use_db_migration_snapshot: Option, + /// Deprecated; use `store.migration_snapshot` instead. + #[serde(skip_serializing_if = "Option::is_none")] + pub db_migration_snapshot_path: Option, #[serde(default, skip_serializing_if = "Option::is_none")] pub expected_shutdown: Option, /// Options for dumping state of every epoch to S3. - #[serde(skip_serializing_if = "Option::is_none")] pub state_sync: Option, - /// Whether to use state sync (unreliable and corrupts the DB if fails) or do a block sync instead. - #[serde(skip_serializing_if = "is_false")] - pub state_sync_enabled: bool, } fn is_false(value: &bool) -> bool { @@ -355,7 +374,6 @@ impl Default for Config { consensus: Consensus::default(), tracked_accounts: vec![], tracked_shards: vec![], - tracked_shard_schedule: None, archive: false, save_trie_changes: None, log_summary_style: LogSummaryStyle::Colored, @@ -365,55 +383,12 @@ impl Default for Config { view_client_throttle_period: default_view_client_throttle_period(), trie_viewer_state_size_limit: default_trie_viewer_state_size_limit(), max_gas_burnt_view: None, + db_migration_snapshot_path: None, + use_db_migration_snapshot: None, store: near_store::StoreConfig::default(), cold_store: None, - split_storage: None, expected_shutdown: None, state_sync: None, - state_sync_enabled: false, - } - } -} - -fn default_enable_split_storage_view_client() -> bool { - false -} - -fn default_cold_store_initial_migration_batch_size() -> usize { - 500_000_000 -} - -fn default_cold_store_initial_migration_loop_sleep_duration() -> Duration { - Duration::from_secs(30) -} - -fn default_cold_store_loop_sleep_duration() -> Duration { - Duration::from_secs(1) -} - -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] -pub struct SplitStorageConfig { - #[serde(default = "default_enable_split_storage_view_client")] - pub enable_split_storage_view_client: bool, - - #[serde(default = "default_cold_store_initial_migration_batch_size")] - pub cold_store_initial_migration_batch_size: usize, - #[serde(default = "default_cold_store_initial_migration_loop_sleep_duration")] - pub cold_store_initial_migration_loop_sleep_duration: Duration, - - #[serde(default = "default_cold_store_loop_sleep_duration")] - pub cold_store_loop_sleep_duration: Duration, -} - -impl Default for SplitStorageConfig { - fn default() -> Self { - SplitStorageConfig { - enable_split_storage_view_client: default_enable_split_storage_view_client(), - cold_store_initial_migration_batch_size: - default_cold_store_initial_migration_batch_size(), - cold_store_initial_migration_loop_sleep_duration: - default_cold_store_initial_migration_loop_sleep_duration(), - cold_store_loop_sleep_duration: default_cold_store_loop_sleep_duration(), } } } @@ -545,6 +520,7 @@ impl Genesis { avg_hidden_validator_seats_per_shard: vec![0; num_validator_seats_per_shard.len()], dynamic_resharding: false, protocol_upgrade_stake_threshold: PROTOCOL_UPGRADE_STAKE_THRESHOLD, + protocol_upgrade_num_epochs: PROTOCOL_UPGRADE_NUM_EPOCHS, epoch_length: FAST_EPOCH_LENGTH, gas_limit: INITIAL_GAS_LIMIT, gas_price_adjustment_rate: GAS_PRICE_ADJUSTMENT_RATE, @@ -638,6 +614,7 @@ impl NearConfig { min_block_production_delay: config.consensus.min_block_production_delay, max_block_production_delay: config.consensus.max_block_production_delay, max_block_wait_delay: config.consensus.max_block_wait_delay, + reduce_wait_for_missing_block: config.consensus.reduce_wait_for_missing_block, skip_sync_wait: config.network.skip_sync_wait, sync_check_period: config.consensus.sync_check_period, sync_step_period: config.consensus.sync_step_period, @@ -654,6 +631,7 @@ impl NearConfig { produce_empty_blocks: config.consensus.produce_empty_blocks, epoch_length: genesis.config.epoch_length, num_block_producer_seats: genesis.config.num_block_producer_seats, + announce_account_horizon: genesis.config.epoch_length / 2, ttl_account_id_router: config.network.ttl_account_id_router, // TODO(1047): this should be adjusted depending on the speed of sync of state. block_fetch_horizon: config.consensus.block_fetch_horizon, @@ -664,7 +642,6 @@ impl NearConfig { doosmslug_step_period: config.consensus.doomslug_step_period, tracked_accounts: config.tracked_accounts, tracked_shards: config.tracked_shards, - tracked_shard_schedule: config.tracked_shard_schedule.unwrap_or(vec![]), archive: config.archive, save_trie_changes: config.save_trie_changes.unwrap_or(!config.archive), log_summary_style: config.log_summary_style, @@ -677,23 +654,14 @@ impl NearConfig { enable_statistics_export: config.store.enable_statistics_export, client_background_migration_threads: config.store.background_migration_threads, flat_storage_creation_period: config.store.flat_storage_creation_period, - state_sync_dump_enabled: config - .state_sync - .as_ref() - .map_or(false, |x| x.dump_enabled.unwrap_or(false)), state_sync_s3_bucket: config .state_sync .as_ref() - .map_or(String::new(), |x| x.s3_bucket.clone()), + .map_or(None, |x| Some(x.s3_bucket.clone())), state_sync_s3_region: config .state_sync .as_ref() - .map_or(String::new(), |x| x.s3_region.clone()), - state_sync_restart_dump_for_shards: config - .state_sync - .as_ref() - .map_or(vec![], |x| x.drop_state_of_dump.clone().unwrap_or(vec![])), - state_sync_enabled: config.state_sync_enabled, + .map_or(None, |x| Some(x.s3_region.clone())), }, network_config: NetworkConfig::new( config.network, @@ -1132,6 +1100,7 @@ pub fn init_configs( avg_hidden_validator_seats_per_shard: (0..num_shards).map(|_| 0).collect(), dynamic_resharding: false, protocol_upgrade_stake_threshold: PROTOCOL_UPGRADE_STAKE_THRESHOLD, + protocol_upgrade_num_epochs: PROTOCOL_UPGRADE_NUM_EPOCHS, epoch_length: if fast { FAST_EPOCH_LENGTH } else { EXPECTED_EPOCH_LENGTH }, gas_limit: INITIAL_GAS_LIMIT, gas_price_adjustment_rate: GAS_PRICE_ADJUSTMENT_RATE, @@ -1516,15 +1485,11 @@ pub fn load_test_config(seed: &str, addr: tcp::ListenerAddr, genesis: Genesis) - NearConfig::new(config, genesis, signer.into(), validator_signer).unwrap() } -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default)] +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] /// Options for dumping state to S3. pub struct StateSyncConfig { pub s3_bucket: String, pub s3_region: String, - #[serde(skip_serializing_if = "Option::is_none")] - pub dump_enabled: Option, - #[serde(skip_serializing_if = "Option::is_none")] - pub drop_state_of_dump: Option>, } #[test] diff --git a/nearcore/src/lib.rs b/nearcore/src/lib.rs index c5276578b3c..7ea9b4f5eed 100644 --- a/nearcore/src/lib.rs +++ b/nearcore/src/lib.rs @@ -1,7 +1,7 @@ +use crate::cold_storage::spawn_cold_store_loop; pub use crate::config::{init_configs, load_config, load_test_config, NearConfig, NEAR_BASE}; pub use crate::runtime::NightshadeRuntime; - -use crate::cold_storage::spawn_cold_store_loop; +pub use crate::shard_tracker::TrackedConfig; use crate::state_sync::{spawn_state_sync_dump, StateSyncDumpHandle}; use actix::{Actor, Addr}; use actix_rt::ArbiterHandle; @@ -16,9 +16,7 @@ use near_client::{start_client, start_view_client, ClientActor, ConfigUpdater, V use near_network::PeerManagerActor; use near_primitives::block::GenesisId; use near_primitives::time; -use near_store::metadata::DbKind; -use near_store::metrics::spawn_db_metrics_loop; -use near_store::{DBCol, Mode, NodeStorage, Store, StoreOpenerError}; +use near_store::{DBCol, Mode, NodeStorage, StoreOpenerError, Temperature}; use near_telemetry::TelemetryActor; use std::path::{Path, PathBuf}; use std::sync::Arc; @@ -34,6 +32,7 @@ pub mod dyn_config; mod metrics; pub mod migrations; mod runtime; +mod shard_tracker; mod state_sync; pub fn get_default_home() -> PathBuf { @@ -140,10 +139,10 @@ fn open_storage(home_dir: &Path, near_config: &mut NearConfig) -> anyhow::Result {latest_release} release" )) }, - Err(StoreOpenerError::DbVersionTooNew { got, want }) => { + Err(StoreOpenerError::DbVersionTooNew { got, .. }) => { Err(anyhow::anyhow!( - "Database version {got} is higher than the expected version {want}. \ - It was likely created by newer version of neard. Please upgrade your neard." + "Database version {got} is created by a newer version of \ + neard, please update neard" )) }, Err(StoreOpenerError::MigrationError(err)) => { @@ -155,33 +154,6 @@ fn open_storage(home_dir: &Path, near_config: &mut NearConfig) -> anyhow::Result Ok(storage) } -// Safely get the split store while checking that all conditions to use it are met. -fn get_split_store(config: &NearConfig, storage: &NodeStorage) -> anyhow::Result> { - // SplitStore should only be used on archival nodes. - if !config.config.archive { - return Ok(None); - } - - // SplitStore should only be used if cold store is configured. - if config.config.cold_store.is_none() { - return Ok(None); - } - - // SplitStore should only be used in the view client if it is enabled. - if !config.config.split_storage.as_ref().map_or(false, |c| c.enable_split_storage_view_client) { - return Ok(None); - } - - // SplitStore should only be used if the migration is finished. The - // migration to cold store is finished when the db kind of the hot store is - // changed from Archive to Hot. - if storage.get_hot_store().get_db_kind()? != Some(DbKind::Hot) { - return Ok(None); - } - - Ok(storage.get_split_store()) -} - pub struct NearNode { pub client: Addr, pub view_client: Addr, @@ -206,27 +178,15 @@ pub fn start_with_config_and_synchronization( shutdown_signal: Option>, config_updater: Option, ) -> anyhow::Result { - let storage = open_storage(home_dir, &mut config)?; - let db_metrics_arbiter = if config.client_config.enable_statistics_export { - let period = config.client_config.log_summary_period; - let db_metrics_arbiter_handle = spawn_db_metrics_loop(&storage, period)?; - Some(db_metrics_arbiter_handle) - } else { - None - }; + let store = open_storage(home_dir, &mut config)?; - let runtime = NightshadeRuntime::from_config(home_dir, storage.get_hot_store(), &config); - - // Get the split store. If split store is some then create a new runtime for - // the view client. Otherwise just re-use the existing runtime. - let split_store = get_split_store(&config, &storage)?; - let view_runtime = if let Some(split_store) = split_store { - NightshadeRuntime::from_config(home_dir, split_store, &config) - } else { - runtime.clone() - }; + let runtime = Arc::new(NightshadeRuntime::from_config( + home_dir, + store.get_store(Temperature::Hot), + &config, + )); - let cold_store_loop_handle = spawn_cold_store_loop(&config, &storage, runtime.clone())?; + let cold_store_loop_handle = spawn_cold_store_loop(&config, &store, runtime.clone())?; let telemetry = TelemetryActor::new(config.telemetry_config.clone()).start(); let chain_genesis = ChainGenesis::new(&config.genesis); @@ -245,7 +205,7 @@ pub fn start_with_config_and_synchronization( let view_client = start_view_client( config.validator_signer.as_ref().map(|signer| signer.validator_id().clone()), chain_genesis.clone(), - view_runtime, + runtime.clone(), network_adapter.clone().into(), config.client_config.clone(), adv.clone(), @@ -269,15 +229,15 @@ pub fn start_with_config_and_synchronization( network_adapter.as_sender(), client_adapter_for_shards_manager.as_sender(), config.validator_signer.as_ref().map(|signer| signer.validator_id().clone()), - storage.get_hot_store(), + store.get_store(Temperature::Hot), config.client_config.chunk_request_retry_period, ); shards_manager_adapter.bind(shards_manager_actor); let state_sync_dump_handle = spawn_state_sync_dump( &config, - chain_genesis, - runtime, + &chain_genesis.clone(), + runtime.clone(), config.network_config.node_id().public_key(), )?; @@ -285,7 +245,7 @@ pub fn start_with_config_and_synchronization( let mut rpc_servers = Vec::new(); let network_actor = PeerManagerActor::spawn( time::Clock::real(), - storage.into_inner(near_store::Temperature::Hot), + store.into_inner(near_store::Temperature::Hot), config.network_config, Arc::new(near_client::adapter::Adapter::new(client_actor.clone(), view_client.clone())), shards_manager_adapter.as_sender(), @@ -323,16 +283,11 @@ pub fn start_with_config_and_synchronization( tracing::trace!(target: "diagnostic", key = "log", "Starting NEAR node with diagnostic activated"); - let mut arbiters = vec![client_arbiter_handle, shards_manager_arbiter_handle]; - if let Some(db_metrics_arbiter) = db_metrics_arbiter { - arbiters.push(db_metrics_arbiter); - } - Ok(NearNode { client: client_actor, view_client, rpc_servers, - arbiters, + arbiters: vec![client_arbiter_handle, shards_manager_arbiter_handle], cold_store_loop_handle, state_sync_dump_handle, }) @@ -379,7 +334,7 @@ pub fn recompress_storage(home_dir: &Path, opts: RecompressOpts) -> anyhow::Resu "Recompressing database"); info!("Opening database at {}", src_path.display()); - let src_store = src_opener.open_in_mode(Mode::ReadOnly)?.get_hot_store(); + let src_store = src_opener.open_in_mode(Mode::ReadOnly)?.get_store(Temperature::Hot); let final_head_height = if skip_columns.contains(&DBCol::PartialChunks) { let tip: Option = @@ -396,7 +351,7 @@ pub fn recompress_storage(home_dir: &Path, opts: RecompressOpts) -> anyhow::Resu }; info!("Creating database at {}", dst_path.display()); - let dst_store = dst_opener.open_in_mode(Mode::Create)?.get_hot_store(); + let dst_store = dst_opener.open_in_mode(Mode::Create)?.get_store(Temperature::Hot); const BATCH_SIZE_BYTES: u64 = 150_000_000; diff --git a/nearcore/src/metrics.rs b/nearcore/src/metrics.rs index 19605ceeca1..2983683b16e 100644 --- a/nearcore/src/metrics.rs +++ b/nearcore/src/metrics.rs @@ -1,7 +1,6 @@ use near_o11y::metrics::{ exponential_buckets, linear_buckets, try_create_histogram_vec, try_create_int_counter_vec, - try_create_int_gauge, try_create_int_gauge_vec, HistogramVec, IntCounterVec, IntGauge, - IntGaugeVec, + try_create_int_gauge, HistogramVec, IntCounterVec, IntGauge, }; use once_cell::sync::Lazy; @@ -15,6 +14,20 @@ pub static APPLY_CHUNK_DELAY: Lazy = Lazy::new(|| { .unwrap() }); +pub static SECONDS_PER_PETAGAS: Lazy = Lazy::new(|| { + try_create_histogram_vec( + "near_execution_seconds_per_petagas_ratio", + "Execution time per unit of gas, measured in seconds per petagas. Ignore label 'label'.", + &["shard_id"], + // Non-linear buckets with higher resolution around 1.0. + Some(vec![ + 0.0, 0.1, 0.2, 0.5, 0.7, 0.8, 0.9, 0.95, 0.97, 0.99, 1.0, 1.01, 1.03, 1.05, 1.1, 1.2, + 1.3, 1.5, 2.0, 5.0, 10.0, + ]), + ) + .unwrap() +}); + pub(crate) static CONFIG_CORRECT: Lazy = Lazy::new(|| { try_create_int_gauge( "near_config_correct", @@ -50,53 +63,12 @@ pub(crate) static STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED: Lazy = Lazy: ) .unwrap() }); -pub(crate) static STATE_SYNC_DUMP_NUM_PARTS_TOTAL: Lazy = Lazy::new(|| { - try_create_int_gauge_vec( - "near_state_sync_dump_num_parts_total", - "Total number of parts in the epoch that being dumped", - &["shard_id"], - ) - .unwrap() -}); -pub(crate) static STATE_SYNC_DUMP_NUM_PARTS_DUMPED: Lazy = Lazy::new(|| { - try_create_int_gauge_vec( - "near_state_sync_dump_num_parts_dumped", - "Number of parts dumped in the epoch that is being dumped", - &["shard_id"], - ) - .unwrap() -}); -pub(crate) static STATE_SYNC_DUMP_SIZE_TOTAL: Lazy = Lazy::new(|| { - try_create_int_counter_vec( - "near_state_sync_dump_size_total", - "Total size of parts written to S3", - &["shard_id"], - ) - .unwrap() -}); -pub(crate) static STATE_SYNC_DUMP_EPOCH_HEIGHT: Lazy = Lazy::new(|| { - try_create_int_gauge_vec( - "near_state_sync_dump_epoch_height", - "Epoch Height of an epoch being dumped", - &["shard_id"], - ) - .unwrap() -}); -pub static STATE_SYNC_APPLY_PART_DELAY: Lazy = Lazy::new(|| { +pub(crate) static STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED: Lazy = Lazy::new(|| { try_create_histogram_vec( - "near_state_sync_apply_part_delay_sec", - "Latency of applying a state part", + "near_state_sync_dump_obtain_part_elapsed_sec", + "Time needed to obtain a part", &["shard_id"], - Some(exponential_buckets(0.001, 2.0, 20).unwrap()), - ) - .unwrap() -}); -pub static STATE_SYNC_OBTAIN_PART_DELAY: Lazy = Lazy::new(|| { - try_create_histogram_vec( - "near_state_sync_obtain_part_delay_sec", - "Latency of applying a state part", - &["shard_id"], - Some(exponential_buckets(0.001, 2.0, 20).unwrap()), + Some(exponential_buckets(0.001, 1.6, 25).unwrap()), ) .unwrap() }); diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 0df51e42802..44483f28c5c 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -1,61 +1,52 @@ use crate::{metrics, NearConfig, NightshadeRuntime}; -use borsh::BorshSerialize; use near_chain::types::RuntimeAdapter; -use near_chain::{Chain, ChainGenesis, ChainStoreAccess, DoomslugThresholdMode, Error}; +use near_chain::{Chain, ChainGenesis, DoomslugThresholdMode, Error}; use near_chain_configs::ClientConfig; use near_client::sync::state::StateSync; use near_crypto::PublicKey; use near_epoch_manager::EpochManagerAdapter; use near_primitives::hash::CryptoHash; use near_primitives::state_part::PartId; -use near_primitives::syncing::{get_num_state_parts, StatePartKey, StateSyncDumpProgress}; -use near_primitives::types::{EpochHeight, EpochId, ShardId, StateRoot}; -use near_store::DBCol; +use near_primitives::syncing::{get_num_state_parts, StateSyncDumpProgress}; +use near_primitives::types::{EpochId, ShardId}; use std::sync::Arc; -/// Starts one a thread per tracked shard. -/// Each started thread will be dumping state parts of a single epoch to external storage. pub fn spawn_state_sync_dump( config: &NearConfig, - chain_genesis: ChainGenesis, + chain_genesis: &ChainGenesis, runtime: Arc, node_key: &PublicKey, ) -> anyhow::Result> { - if !config.client_config.state_sync_dump_enabled { - return Ok(None); - } - if config.client_config.state_sync_s3_bucket.is_empty() - || config.client_config.state_sync_s3_region.is_empty() + if config.client_config.state_sync_s3_bucket.is_none() + || config.client_config.state_sync_s3_region.is_none() { - panic!("Enabled dumps of state to external storage. Please specify state_sync.s3_bucket and state_sync.s3_region"); + return Ok(None); } tracing::info!(target: "state_sync_dump", "Spawning the state sync dump loop"); // Create a connection to S3. - let s3_bucket = config.client_config.state_sync_s3_bucket.clone(); - let s3_region = config.client_config.state_sync_s3_region.clone(); - - // Credentials to establish a connection are taken from environment variables: AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY. + let s3_bucket = config.client_config.state_sync_s3_bucket.as_ref().unwrap(); + let s3_region = config.client_config.state_sync_s3_region.as_ref().unwrap(); let bucket = s3::Bucket::new( &s3_bucket, s3_region .parse::() .map_err(|err| >::into(err))?, s3::creds::Credentials::default().map_err(|err| { - tracing::error!(target: "state_sync_dump", "Failed to create a connection to S3. Did you provide environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY?"); >::into(err) })?, - ).map_err(|err| >::into(err))?; + ) + .map_err(|err| >::into(err))?; // Determine how many threads to start. - // TODO: Handle the case of changing the shard layout. + // Doesn't handle the case of changing the shard layout. let num_shards = { // Sadly, `Chain` is not `Send` and each thread needs to create its own `Chain` instance. let chain = Chain::new_for_view_client( runtime.clone(), - &chain_genesis, + chain_genesis, DoomslugThresholdMode::TwoThirds, - false, + config.client_config.save_trie_changes, )?; let epoch_id = chain.head()?.epoch_id; runtime.num_shards(&epoch_id) @@ -66,12 +57,13 @@ pub fn spawn_state_sync_dump( .map(|shard_id| { let client_config = config.client_config.clone(); let runtime = runtime.clone(); + let save_trie_changes = client_config.save_trie_changes; let chain_genesis = chain_genesis.clone(); let chain = Chain::new_for_view_client( runtime.clone(), &chain_genesis, DoomslugThresholdMode::TwoThirds, - false, + save_trie_changes, ) .unwrap(); let arbiter_handle = actix_rt::Arbiter::new().handle(); @@ -102,7 +94,7 @@ impl Drop for StateSyncDumpHandle { } impl StateSyncDumpHandle { - pub fn stop(&self) { + pub fn stop(&mut self) { let _: Vec = self.handles.iter().map(|handle| handle.stop()).collect(); } } @@ -116,16 +108,11 @@ async fn state_sync_dump( runtime: Arc, config: ClientConfig, bucket: s3::Bucket, - _node_key: PublicKey, + node_key: PublicKey, ) { tracing::info!(target: "state_sync_dump", shard_id, "Running StateSyncDump loop"); let mut interval = actix_rt::time::interval(std::time::Duration::from_secs(10)); - if config.state_sync_restart_dump_for_shards.contains(&shard_id) { - tracing::debug!(target: "state_sync_dump", shard_id, "Dropped existing progress"); - chain.store().set_state_sync_dump_progress(shard_id, None).unwrap(); - } - loop { // Avoid a busy-loop when there is nothing to do. interval.tick().await; @@ -134,26 +121,19 @@ async fn state_sync_dump( tracing::debug!(target: "state_sync_dump", shard_id, ?progress, "Running StateSyncDump loop iteration"); // The `match` returns the next state of the state machine. let next_state = match progress { - Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts })) => { + Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) => { // The latest epoch was dumped. Check if a newer epoch is available. - check_new_epoch( - Some(epoch_id), - Some(epoch_height), - num_parts, - shard_id, - &chain, - &runtime, - ) + check_new_epoch(Some(epoch_id), shard_id, &chain, &runtime, &config) } Err(Error::DBNotFoundErr(_)) | Ok(None) => { - // First invocation of this state-machine. See if at least one epoch is available for dumping. - check_new_epoch(None, None, None, shard_id, &chain, &runtime) + // First invokation of this state-machine. See if at least one epoch is available for dumping. + check_new_epoch(None, shard_id, &chain, &runtime, &config) } Err(err) => { // Something went wrong, let's retry. - tracing::warn!(target: "state_sync_dump", shard_id, ?err, "Failed to read the progress, will now delete and retry"); + tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to read the progress, delete and retry"); if let Err(err) = chain.store().set_state_sync_dump_progress(shard_id, None) { - tracing::warn!(target: "state_sync_dump", shard_id, ?err, "and failed to delete the progress. Will later retry."); + tracing::debug!(target: "state_sync_dump", shard_id, ?err, "And failed to delete it too :("); } Ok(None) } @@ -161,75 +141,99 @@ async fn state_sync_dump( epoch_id, epoch_height, sync_hash, + state_root, parts_dumped, + num_parts, })) => { - let state_header = chain.get_state_response_header(shard_id, sync_hash); - match state_header { - Ok(state_header) => { - let state_root = state_header.chunk_prev_state_root(); - let num_parts = - get_num_state_parts(state_header.state_root_node().memory_usage); + // The actual dumping of state to S3. + tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, ?sync_hash, ?state_root, parts_dumped, num_parts, "Creating parts and dumping them"); + let mut res = None; + for part_id in parts_dumped..num_parts { + // Dump parts sequentially synchronously. + // TODO: How to make it possible to dump state more effectively using multiple nodes? + let _timer = metrics::STATE_SYNC_DUMP_ITERATION_ELAPSED + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + let state_part = { + let _timer = metrics::STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + runtime.obtain_state_part( + shard_id, + &sync_hash, + &state_root, + PartId::new(part_id, num_parts), + ) + }; + let state_part = match state_part { + Ok(state_part) => state_part, + Err(err) => { + res = Some(err); + break; + } + }; + let location = s3_location(&config.chain_id, epoch_height, shard_id, part_id); - let mut res = None; - // The actual dumping of state to S3. - tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, %sync_hash, parts_dumped, "Creating parts and dumping them"); - for part_id in parts_dumped..num_parts { - // Dump parts sequentially synchronously. - // TODO: How to make it possible to dump state more effectively using multiple nodes? - let _timer = metrics::STATE_SYNC_DUMP_ITERATION_ELAPSED - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); + { + let _timer = metrics::STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + let put = bucket + .put_object(&location, &state_part) + .await + .map_err(|err| Error::Other(err.to_string())); + if let Err(err) = put { + res = Some(err); + break; + } - let state_part = match obtain_and_store_state_part( - &runtime, - &shard_id, - &sync_hash, - &state_root, - part_id, - num_parts, - &chain, - ) { - Ok(state_part) => state_part, - Err(err) => { - res = Some(err); - break; - } - }; - let location = s3_location( - &config.chain_id, - epoch_height, - shard_id, - part_id, - num_parts, - ); - if let Err(err) = - put_state_part(&location, &state_part, &shard_id, &bucket).await - { - res = Some(err); - break; - } - update_progress( - &shard_id, - &epoch_id, - epoch_height, - &sync_hash, - part_id, - num_parts, - state_part.len(), - &chain, - ); + // Optional, we probably don't need this. + let put = bucket + .put_object_tagging( + &location, + &[ + ("chain_id", &config.chain_id), + ("epoch_id", &format!("{:?}", epoch_id.0)), + ("epoch_height", &epoch_height.to_string()), + ("state_root", &format!("{:?}", state_root)), + ("sync_hash", &format!("{:?}", sync_hash)), + ("node_key", &format!("{:?}", node_key)), + ], + ) + .await + .map_err(|err| Error::Other(err.to_string())); + if let Err(err) = put { + res = Some(err); + break; } - if let Some(err) = res { - Err(err) - } else { - Ok(Some(StateSyncDumpProgress::AllDumped { - epoch_id, - epoch_height, - num_parts: Some(num_parts), - })) + } + + // Record that a part was obtained and dumped. + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, ?sync_hash, ?state_root, part_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); + let next_progress = StateSyncDumpProgress::InProgress { + epoch_id: epoch_id.clone(), + epoch_height, + sync_hash, + state_root, + parts_dumped: part_id + 1, + num_parts, + }; + match chain + .store() + .set_state_sync_dump_progress(shard_id, Some(next_progress.clone())) + { + Ok(_) => { + tracing::debug!(target: "state_sync_dump", shard_id, ?next_progress, "Updated dump progress"); + } + Err(err) => { + tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to update dump progress, continue"); } } - Err(err) => Err(err), + } + if let Some(err) = res { + Err(err) + } else { + Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) } } }; @@ -258,109 +262,6 @@ async fn state_sync_dump( } } -async fn put_state_part( - location: &str, - state_part: &[u8], - shard_id: &ShardId, - bucket: &s3::Bucket, -) -> Result { - let _timer = metrics::STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - let put = bucket - .put_object(&location, &state_part) - .await - .map_err(|err| Error::Other(err.to_string())); - tracing::debug!(target: "state_sync_dump", shard_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); - put -} - -fn update_progress( - shard_id: &ShardId, - epoch_id: &EpochId, - epoch_height: EpochHeight, - sync_hash: &CryptoHash, - part_id: u64, - num_parts: u64, - part_len: usize, - chain: &Chain, -) { - // Record that a part was obtained and dumped. - metrics::STATE_SYNC_DUMP_SIZE_TOTAL - .with_label_values(&[&shard_id.to_string()]) - .inc_by(part_len as u64); - let next_progress = StateSyncDumpProgress::InProgress { - epoch_id: epoch_id.clone(), - epoch_height, - sync_hash: *sync_hash, - parts_dumped: part_id + 1, - }; - match chain.store().set_state_sync_dump_progress(*shard_id, Some(next_progress.clone())) { - Ok(_) => { - tracing::debug!(target: "state_sync_dump", shard_id, ?next_progress, "Updated dump progress"); - } - Err(err) => { - tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to update dump progress, continue"); - } - } - set_metrics(shard_id, Some(part_id + 1), Some(num_parts), Some(epoch_height)); -} - -fn set_metrics( - shard_id: &ShardId, - parts_dumped: Option, - num_parts: Option, - epoch_height: Option, -) { - if let Some(parts_dumped) = parts_dumped { - metrics::STATE_SYNC_DUMP_NUM_PARTS_DUMPED - .with_label_values(&[&shard_id.to_string()]) - .set(parts_dumped as i64); - } - if let Some(num_parts) = num_parts { - metrics::STATE_SYNC_DUMP_NUM_PARTS_TOTAL - .with_label_values(&[&shard_id.to_string()]) - .set(num_parts as i64); - } - if let Some(epoch_height) = epoch_height { - assert!( - epoch_height < 10000, - "Impossible: {:?} {:?} {:?} {:?}", - shard_id, - parts_dumped, - num_parts, - epoch_height - ); - metrics::STATE_SYNC_DUMP_EPOCH_HEIGHT - .with_label_values(&[&shard_id.to_string()]) - .set(epoch_height as i64); - } -} - -/// Obtains and then saves the part data. -fn obtain_and_store_state_part( - runtime: &Arc, - shard_id: &ShardId, - sync_hash: &CryptoHash, - state_root: &StateRoot, - part_id: u64, - num_parts: u64, - chain: &Chain, -) -> Result, Error> { - let state_part = runtime.obtain_state_part( - *shard_id, - &sync_hash, - &state_root, - PartId::new(part_id, num_parts), - )?; - - let key = StatePartKey(*sync_hash, *shard_id, part_id).try_to_vec()?; - let mut store_update = chain.store().store().store_update(); - store_update.set(DBCol::StateParts, &key, &state_part); - store_update.commit()?; - Ok(state_part) -} - /// Gets basic information about the epoch to be dumped. fn start_dumping( epoch_id: EpochId, @@ -371,25 +272,27 @@ fn start_dumping( ) -> Result, Error> { let epoch_info = runtime.get_epoch_info(&epoch_id)?; let epoch_height = epoch_info.epoch_height(); - let sync_prev_header = chain.get_block_header(&sync_hash)?; - let sync_prev_hash = sync_prev_header.hash(); - - let state_header = chain.get_state_response_header(shard_id, sync_hash)?; - let num_parts = get_num_state_parts(state_header.state_root_node().memory_usage); - if runtime.cares_about_shard(None, sync_prev_hash, shard_id, false) { - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_prev_hash, %sync_hash, "Initialize dumping state of Epoch"); + let num_shards = runtime.num_shards(&epoch_id)?; + let sync_hash_block = chain.get_block(&sync_hash)?; + if runtime.cares_about_shard(None, sync_hash_block.header().prev_hash(), shard_id, false) { + assert_eq!(num_shards, sync_hash_block.chunks().len() as u64); + let state_root = sync_hash_block.chunks()[shard_id as usize].prev_state_root(); + let state_root_node = runtime.get_state_root_node(shard_id, &sync_hash, &state_root)?; + let num_parts = get_num_state_parts(state_root_node.memory_usage); + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, ?state_root, num_parts, "Initialize dumping state of Epoch"); // Note that first the state of the state machines gets changes to // `InProgress` and it starts dumping state after a short interval. - set_metrics(&shard_id, Some(0), Some(num_parts), Some(epoch_height)); Ok(Some(StateSyncDumpProgress::InProgress { epoch_id, epoch_height, sync_hash, + state_root, parts_dumped: 0, + num_parts, })) } else { - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_prev_hash, %sync_hash, "Shard is not tracked, skip the epoch"); - Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts: Some(0) })) + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, "Shard is not tracked, skip the epoch"); + Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) } } @@ -397,42 +300,35 @@ fn start_dumping( /// `epoch_id` represents the last fully dumped epoch. fn check_new_epoch( epoch_id: Option, - epoch_height: Option, - num_parts: Option, shard_id: ShardId, chain: &Chain, runtime: &Arc, + config: &ClientConfig, ) -> Result, Error> { let head = chain.head()?; if Some(&head.epoch_id) == epoch_id.as_ref() { - set_metrics(&shard_id, num_parts, num_parts, epoch_height); Ok(None) } else { - // Check if the final block is now in the next epoch. tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, "Check if a new complete epoch is available"); - let hash = head.last_block_hash; - let header = chain.get_block_header(&hash)?; - let final_hash = header.last_final_block(); - let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &final_hash)?; + let mut sync_hash = head.prev_block_hash; + // Step back a few blocks to avoid dealing with forks. + for _ in 0..config.state_fetch_horizon { + sync_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); + } + let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash)?; let header = chain.get_block_header(&sync_hash)?; if Some(header.epoch_id()) == epoch_id.as_ref() { // Still in the latest dumped epoch. Do nothing. Ok(None) } else { - start_dumping(head.epoch_id, sync_hash, shard_id, &chain, runtime) + start_dumping(head.epoch_id.clone(), sync_hash, shard_id, &chain, runtime) } } } -fn s3_location( - chain_id: &str, - epoch_height: u64, - shard_id: u64, - part_id: u64, - num_parts: u64, -) -> String { +fn s3_location(chain_id: &str, epoch_height: u64, shard_id: u64, part_id: u64) -> String { format!( - "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}_of_{:06}", - chain_id, epoch_height, shard_id, part_id, num_parts + "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}", + chain_id, epoch_height, shard_id, part_id ) } From e9cb32c4daa3ec88794ce34f8a29516b0e7c071e Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 28 Feb 2023 21:52:57 +0100 Subject: [PATCH 02/88] fix --- nearcore/src/state_sync.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 44483f28c5c..c5cd53c163f 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -94,7 +94,7 @@ impl Drop for StateSyncDumpHandle { } impl StateSyncDumpHandle { - pub fn stop(&mut self) { + pub fn stop(&self) { let _: Vec = self.handles.iter().map(|handle| handle.stop()).collect(); } } From 95ff6dc5a90ca04c16f12c12eac506b702a87ae4 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 1 Mar 2023 11:38:48 +0100 Subject: [PATCH 03/88] Metrics --- core/primitives/src/syncing.rs | 5 ++++- nearcore/src/metrics.rs | 19 +++++++++++++++- nearcore/src/state_sync.rs | 40 +++++++++++++++++++++++++++++----- 3 files changed, 56 insertions(+), 8 deletions(-) diff --git a/core/primitives/src/syncing.rs b/core/primitives/src/syncing.rs index c948fd7a895..b43bcda5925 100644 --- a/core/primitives/src/syncing.rs +++ b/core/primitives/src/syncing.rs @@ -231,7 +231,10 @@ pub fn get_num_state_parts(memory_usage: u64) -> u64 { #[derive(BorshSerialize, BorshDeserialize, Debug, Clone)] /// Represents the state of the state machine that dumps state. pub enum StateSyncDumpProgress { - AllDumped(EpochId), + AllDumped { + epoch_id: EpochId, + num_parts: Option, + }, InProgress { epoch_id: EpochId, epoch_height: EpochHeight, diff --git a/nearcore/src/metrics.rs b/nearcore/src/metrics.rs index 2983683b16e..b6dd1e2c630 100644 --- a/nearcore/src/metrics.rs +++ b/nearcore/src/metrics.rs @@ -1,6 +1,7 @@ use near_o11y::metrics::{ exponential_buckets, linear_buckets, try_create_histogram_vec, try_create_int_counter_vec, - try_create_int_gauge, HistogramVec, IntCounterVec, IntGauge, + try_create_int_gauge, try_create_int_gauge_vec, HistogramVec, IntCounterVec, IntGauge, + IntGaugeVec, }; use once_cell::sync::Lazy; @@ -72,3 +73,19 @@ pub(crate) static STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED: Lazy = Lazy ) .unwrap() }); +pub(crate) static STATE_SYNC_DUMP_NUM_PARTS_TOTAL: Lazy = Lazy::new(|| { + try_create_int_gauge_vec( + "near_state_sync_dump_num_parts_total", + "Total number of parts in the epoch that being dumped", + &["shard_id"], + ) + .unwrap() +}); +pub(crate) static STATE_SYNC_DUMP_NUM_PARTS_DUMPED: Lazy = Lazy::new(|| { + try_create_int_gauge_vec( + "near_state_sync_dump_num_parts_dumped", + "Number of parts dumped in the epoch that is being dumped", + &["shard_id"], + ) + .unwrap() +}); diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index c5cd53c163f..5abc492a515 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -121,13 +121,13 @@ async fn state_sync_dump( tracing::debug!(target: "state_sync_dump", shard_id, ?progress, "Running StateSyncDump loop iteration"); // The `match` returns the next state of the state machine. let next_state = match progress { - Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) => { + Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, num_parts })) => { // The latest epoch was dumped. Check if a newer epoch is available. - check_new_epoch(Some(epoch_id), shard_id, &chain, &runtime, &config) + check_new_epoch(Some(epoch_id), num_parts, shard_id, &chain, &runtime, &config) } Err(Error::DBNotFoundErr(_)) | Ok(None) => { - // First invokation of this state-machine. See if at least one epoch is available for dumping. - check_new_epoch(None, shard_id, &chain, &runtime, &config) + // First invocation of this state-machine. See if at least one epoch is available for dumping. + check_new_epoch(None, None, shard_id, &chain, &runtime, &config) } Err(err) => { // Something went wrong, let's retry. @@ -145,6 +145,9 @@ async fn state_sync_dump( parts_dumped, num_parts, })) => { + // TODO: Metric for num_parts per shard + // TODO: Metric for num_dumped per shard + // The actual dumping of state to S3. tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, ?sync_hash, ?state_root, parts_dumped, num_parts, "Creating parts and dumping them"); let mut res = None; @@ -198,6 +201,7 @@ async fn state_sync_dump( ("state_root", &format!("{:?}", state_root)), ("sync_hash", &format!("{:?}", sync_hash)), ("node_key", &format!("{:?}", node_key)), + ("num_parts", &format!("{}", num_parts)), ], ) .await @@ -229,11 +233,20 @@ async fn state_sync_dump( tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to update dump progress, continue"); } } + metrics::STATE_SYNC_DUMP_NUM_PARTS_DUMPED + .with_label_values(&[&shard_id.to_string()]) + .set(part_id as i64 + 1); + metrics::STATE_SYNC_DUMP_NUM_PARTS_TOTAL + .with_label_values(&[&shard_id.to_string()]) + .set(num_parts as i64); } if let Some(err) = res { Err(err) } else { - Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) + Ok(Some(StateSyncDumpProgress::AllDumped { + epoch_id, + num_parts: Some(num_parts), + })) } } }; @@ -282,6 +295,12 @@ fn start_dumping( tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, ?state_root, num_parts, "Initialize dumping state of Epoch"); // Note that first the state of the state machines gets changes to // `InProgress` and it starts dumping state after a short interval. + metrics::STATE_SYNC_DUMP_NUM_PARTS_DUMPED + .with_label_values(&[&shard_id.to_string()]) + .set(0); + metrics::STATE_SYNC_DUMP_NUM_PARTS_TOTAL + .with_label_values(&[&shard_id.to_string()]) + .set(num_parts as i64); Ok(Some(StateSyncDumpProgress::InProgress { epoch_id, epoch_height, @@ -292,7 +311,7 @@ fn start_dumping( })) } else { tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, "Shard is not tracked, skip the epoch"); - Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) + Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, num_parts: Some(0) })) } } @@ -300,6 +319,7 @@ fn start_dumping( /// `epoch_id` represents the last fully dumped epoch. fn check_new_epoch( epoch_id: Option, + num_parts: Option, shard_id: ShardId, chain: &Chain, runtime: &Arc, @@ -307,6 +327,14 @@ fn check_new_epoch( ) -> Result, Error> { let head = chain.head()?; if Some(&head.epoch_id) == epoch_id.as_ref() { + if let Some(num_parts) = num_parts { + metrics::STATE_SYNC_DUMP_NUM_PARTS_DUMPED + .with_label_values(&[&shard_id.to_string()]) + .set(num_parts as i64); + metrics::STATE_SYNC_DUMP_NUM_PARTS_TOTAL + .with_label_values(&[&shard_id.to_string()]) + .set(num_parts as i64); + } Ok(None) } else { tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, "Check if a new complete epoch is available"); From c3a0112dd4002c034e390d5d75687c44e690b4d5 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 1 Mar 2023 12:40:38 +0100 Subject: [PATCH 04/88] Metrics --- nearcore/src/metrics.rs | 8 ++++++++ nearcore/src/state_sync.rs | 3 +++ 2 files changed, 11 insertions(+) diff --git a/nearcore/src/metrics.rs b/nearcore/src/metrics.rs index b6dd1e2c630..b4f05bb3078 100644 --- a/nearcore/src/metrics.rs +++ b/nearcore/src/metrics.rs @@ -89,3 +89,11 @@ pub(crate) static STATE_SYNC_DUMP_NUM_PARTS_DUMPED: Lazy = Lazy::ne ) .unwrap() }); +pub(crate) static STATE_SYNC_DUMP_SIZE_TOTAL: Lazy = Lazy::new(|| { + try_create_int_counter_vec( + "near_state_sync_dump_size_total", + "Total size of parts written to S3", + &["shard_id"], + ) + .unwrap() +}); diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 5abc492a515..8590bcec7bb 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -214,6 +214,9 @@ async fn state_sync_dump( // Record that a part was obtained and dumped. tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, ?sync_hash, ?state_root, part_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); + metrics::STATE_SYNC_DUMP_SIZE_TOTAL + .with_label_values(&[&shard_id.to_string()]) + .inc_by(state_part.len() as u64); let next_progress = StateSyncDumpProgress::InProgress { epoch_id: epoch_id.clone(), epoch_height, From 85a8ded33a5560146f4e9c8718b268837731b356 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 1 Mar 2023 13:40:13 +0100 Subject: [PATCH 05/88] Metrics --- core/primitives/src/syncing.rs | 1 + nearcore/src/metrics.rs | 8 ++++ nearcore/src/state_sync.rs | 69 ++++++++++++++++++++-------------- 3 files changed, 50 insertions(+), 28 deletions(-) diff --git a/core/primitives/src/syncing.rs b/core/primitives/src/syncing.rs index b43bcda5925..f64e2760432 100644 --- a/core/primitives/src/syncing.rs +++ b/core/primitives/src/syncing.rs @@ -233,6 +233,7 @@ pub fn get_num_state_parts(memory_usage: u64) -> u64 { pub enum StateSyncDumpProgress { AllDumped { epoch_id: EpochId, + epoch_height: EpochHeight, num_parts: Option, }, InProgress { diff --git a/nearcore/src/metrics.rs b/nearcore/src/metrics.rs index b4f05bb3078..ee53c9c2422 100644 --- a/nearcore/src/metrics.rs +++ b/nearcore/src/metrics.rs @@ -97,3 +97,11 @@ pub(crate) static STATE_SYNC_DUMP_SIZE_TOTAL: Lazy = Lazy::new(|| ) .unwrap() }); +pub(crate) static STATE_SYNC_DUMP_EPOCH_HEIGHT: Lazy = Lazy::new(|| { + try_create_int_gauge_vec( + "near_state_sync_dump_epoch_height", + "Epoch Height of an epoch being dumped", + &["shard_id"], + ) + .unwrap() +}); diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 8590bcec7bb..6825b1992d7 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -8,7 +8,7 @@ use near_epoch_manager::EpochManagerAdapter; use near_primitives::hash::CryptoHash; use near_primitives::state_part::PartId; use near_primitives::syncing::{get_num_state_parts, StateSyncDumpProgress}; -use near_primitives::types::{EpochId, ShardId}; +use near_primitives::types::{EpochHeight, EpochId, ShardId}; use std::sync::Arc; pub fn spawn_state_sync_dump( @@ -121,13 +121,21 @@ async fn state_sync_dump( tracing::debug!(target: "state_sync_dump", shard_id, ?progress, "Running StateSyncDump loop iteration"); // The `match` returns the next state of the state machine. let next_state = match progress { - Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, num_parts })) => { + Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts })) => { // The latest epoch was dumped. Check if a newer epoch is available. - check_new_epoch(Some(epoch_id), num_parts, shard_id, &chain, &runtime, &config) + check_new_epoch( + Some(epoch_id), + Some(epoch_height), + num_parts, + shard_id, + &chain, + &runtime, + &config, + ) } Err(Error::DBNotFoundErr(_)) | Ok(None) => { // First invocation of this state-machine. See if at least one epoch is available for dumping. - check_new_epoch(None, None, shard_id, &chain, &runtime, &config) + check_new_epoch(None, None, None, shard_id, &chain, &runtime, &config) } Err(err) => { // Something went wrong, let's retry. @@ -145,9 +153,6 @@ async fn state_sync_dump( parts_dumped, num_parts, })) => { - // TODO: Metric for num_parts per shard - // TODO: Metric for num_dumped per shard - // The actual dumping of state to S3. tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, ?sync_hash, ?state_root, parts_dumped, num_parts, "Creating parts and dumping them"); let mut res = None; @@ -236,18 +241,14 @@ async fn state_sync_dump( tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to update dump progress, continue"); } } - metrics::STATE_SYNC_DUMP_NUM_PARTS_DUMPED - .with_label_values(&[&shard_id.to_string()]) - .set(part_id as i64 + 1); - metrics::STATE_SYNC_DUMP_NUM_PARTS_TOTAL - .with_label_values(&[&shard_id.to_string()]) - .set(num_parts as i64); + set_metrics(shard_id, Some(part_id + 1), Some(num_parts), Some(epoch_height)); } if let Some(err) = res { Err(err) } else { Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, + epoch_height, num_parts: Some(num_parts), })) } @@ -278,6 +279,29 @@ async fn state_sync_dump( } } +fn set_metrics( + shard_id: ShardId, + parts_dumped: Option, + num_parts: Option, + epoch_height: Option, +) { + if let Some(parts_dumped) = parts_dumped { + metrics::STATE_SYNC_DUMP_NUM_PARTS_DUMPED + .with_label_values(&[&shard_id.to_string()]) + .set(parts_dumped as i64); + } + if let Some(num_parts) = num_parts { + metrics::STATE_SYNC_DUMP_NUM_PARTS_TOTAL + .with_label_values(&[&shard_id.to_string()]) + .set(num_parts as i64); + } + if let Some(epoch_height) = epoch_height { + metrics::STATE_SYNC_DUMP_EPOCH_HEIGHT + .with_label_values(&[&shard_id.to_string()]) + .set(epoch_height as i64); + } +} + /// Gets basic information about the epoch to be dumped. fn start_dumping( epoch_id: EpochId, @@ -298,12 +322,7 @@ fn start_dumping( tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, ?state_root, num_parts, "Initialize dumping state of Epoch"); // Note that first the state of the state machines gets changes to // `InProgress` and it starts dumping state after a short interval. - metrics::STATE_SYNC_DUMP_NUM_PARTS_DUMPED - .with_label_values(&[&shard_id.to_string()]) - .set(0); - metrics::STATE_SYNC_DUMP_NUM_PARTS_TOTAL - .with_label_values(&[&shard_id.to_string()]) - .set(num_parts as i64); + set_metrics(shard_id, Some(0), Some(num_parts), Some(epoch_height)); Ok(Some(StateSyncDumpProgress::InProgress { epoch_id, epoch_height, @@ -314,7 +333,7 @@ fn start_dumping( })) } else { tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, "Shard is not tracked, skip the epoch"); - Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, num_parts: Some(0) })) + Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts: Some(0) })) } } @@ -322,6 +341,7 @@ fn start_dumping( /// `epoch_id` represents the last fully dumped epoch. fn check_new_epoch( epoch_id: Option, + epoch_height: Option, num_parts: Option, shard_id: ShardId, chain: &Chain, @@ -330,14 +350,7 @@ fn check_new_epoch( ) -> Result, Error> { let head = chain.head()?; if Some(&head.epoch_id) == epoch_id.as_ref() { - if let Some(num_parts) = num_parts { - metrics::STATE_SYNC_DUMP_NUM_PARTS_DUMPED - .with_label_values(&[&shard_id.to_string()]) - .set(num_parts as i64); - metrics::STATE_SYNC_DUMP_NUM_PARTS_TOTAL - .with_label_values(&[&shard_id.to_string()]) - .set(num_parts as i64); - } + set_metrics(shard_id, num_parts, num_parts, epoch_height); Ok(None) } else { tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, "Check if a new complete epoch is available"); From 4f0d69c221f81545a4ebc66d5775a3e094db5cb4 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 1 Mar 2023 13:47:59 +0100 Subject: [PATCH 06/88] Metrics --- nearcore/src/state_sync.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 6825b1992d7..16a1cbbccf1 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -201,12 +201,13 @@ async fn state_sync_dump( &location, &[ ("chain_id", &config.chain_id), - ("epoch_id", &format!("{:?}", epoch_id.0)), ("epoch_height", &epoch_height.to_string()), - ("state_root", &format!("{:?}", state_root)), - ("sync_hash", &format!("{:?}", sync_hash)), + ("epoch_id", &format!("{:?}", epoch_id.0)), ("node_key", &format!("{:?}", node_key)), ("num_parts", &format!("{}", num_parts)), + ("part_id", &format!("{}", part_id)), + ("state_root", &format!("{:?}", state_root)), + ("sync_hash", &format!("{:?}", sync_hash)), ], ) .await From 556248f2383393fb72cb4fabb6863552ad8d3906 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 1 Mar 2023 15:14:01 +0100 Subject: [PATCH 07/88] Add config options, change state part file naming - include num_parts. --- core/chain-configs/src/client_config.rs | 14 +++++++---- nearcore/src/config.rs | 17 +++++++++++--- nearcore/src/state_sync.rs | 31 ++++++++++++++++++------- 3 files changed, 47 insertions(+), 15 deletions(-) diff --git a/core/chain-configs/src/client_config.rs b/core/chain-configs/src/client_config.rs index 202363eb6a6..a5fdddfd4eb 100644 --- a/core/chain-configs/src/client_config.rs +++ b/core/chain-configs/src/client_config.rs @@ -166,10 +166,14 @@ pub struct ClientConfig { pub client_background_migration_threads: usize, /// Duration to perform background flat storage creation step. pub flat_storage_creation_period: Duration, + /// Whether to enable dumping state of every epoch to S3. + pub state_sync_dump_enabled: bool, /// S3 bucket for storing state dumps. - pub state_sync_s3_bucket: Option, + pub state_sync_s3_bucket: String, /// S3 region for storing state dumps. - pub state_sync_s3_region: Option, + pub state_sync_s3_region: String, + /// Discard the existing progress of dumping an epoch state to S3. + pub state_sync_dump_drop_state: Vec, } impl ClientConfig { @@ -239,8 +243,10 @@ impl ClientConfig { enable_statistics_export: true, client_background_migration_threads: 1, flat_storage_creation_period: Duration::from_secs(1), - state_sync_s3_bucket: None, - state_sync_s3_region: None, + state_sync_dump_enabled: false, + state_sync_s3_bucket: String::new(), + state_sync_s3_region: String::new(), + state_sync_dump_drop_state: vec![], } } } diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index 456ec1cee17..06785bcc8a4 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -352,6 +352,7 @@ pub struct Config { #[serde(default, skip_serializing_if = "Option::is_none")] pub expected_shutdown: Option, /// Options for dumping state of every epoch to S3. + #[serde(skip_serializing_if = "Option::is_none")] pub state_sync: Option, } @@ -654,14 +655,22 @@ impl NearConfig { enable_statistics_export: config.store.enable_statistics_export, client_background_migration_threads: config.store.background_migration_threads, flat_storage_creation_period: config.store.flat_storage_creation_period, + state_sync_dump_enabled: config + .state_sync + .as_ref() + .map_or(false, |x| x.dump_enabled), state_sync_s3_bucket: config .state_sync .as_ref() - .map_or(None, |x| Some(x.s3_bucket.clone())), + .map_or(String::new(), |x| x.s3_bucket.clone()), state_sync_s3_region: config .state_sync .as_ref() - .map_or(None, |x| Some(x.s3_region.clone())), + .map_or(String::new(), |x| x.s3_region.clone()), + state_sync_dump_drop_state: config + .state_sync + .as_ref() + .map_or(vec![], |x| x.drop_state_of_dump.clone()), }, network_config: NetworkConfig::new( config.network, @@ -1485,11 +1494,13 @@ pub fn load_test_config(seed: &str, addr: tcp::ListenerAddr, genesis: Genesis) - NearConfig::new(config, genesis, signer.into(), validator_signer).unwrap() } -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default)] /// Options for dumping state to S3. pub struct StateSyncConfig { pub s3_bucket: String, pub s3_region: String, + pub dump_enabled: bool, + pub drop_state_of_dump: Vec, } #[test] diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 16a1cbbccf1..8399eb19924 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -17,16 +17,17 @@ pub fn spawn_state_sync_dump( runtime: Arc, node_key: &PublicKey, ) -> anyhow::Result> { - if config.client_config.state_sync_s3_bucket.is_none() - || config.client_config.state_sync_s3_region.is_none() + if !config.client_config.state_sync_dump_enabled + || config.client_config.state_sync_s3_bucket.is_empty() + || config.client_config.state_sync_s3_region.is_empty() { return Ok(None); } tracing::info!(target: "state_sync_dump", "Spawning the state sync dump loop"); // Create a connection to S3. - let s3_bucket = config.client_config.state_sync_s3_bucket.as_ref().unwrap(); - let s3_region = config.client_config.state_sync_s3_region.as_ref().unwrap(); + let s3_bucket = config.client_config.state_sync_s3_bucket.clone(); + let s3_region = config.client_config.state_sync_s3_region.clone(); let bucket = s3::Bucket::new( &s3_bucket, s3_region @@ -113,6 +114,11 @@ async fn state_sync_dump( tracing::info!(target: "state_sync_dump", shard_id, "Running StateSyncDump loop"); let mut interval = actix_rt::time::interval(std::time::Duration::from_secs(10)); + if config.state_sync_dump_drop_state.contains(&shard_id) { + tracing::debug!(target: "state_sync_dump", shard_id, "Dropped existing progress"); + chain.store().set_state_sync_dump_progress(shard_id, None).unwrap(); + } + loop { // Avoid a busy-loop when there is nothing to do. interval.tick().await; @@ -180,7 +186,8 @@ async fn state_sync_dump( break; } }; - let location = s3_location(&config.chain_id, epoch_height, shard_id, part_id); + let location = + s3_location(&config.chain_id, epoch_height, shard_id, part_id, num_parts); { let _timer = metrics::STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED @@ -195,6 +202,7 @@ async fn state_sync_dump( break; } + /* // Optional, we probably don't need this. let put = bucket .put_object_tagging( @@ -216,6 +224,7 @@ async fn state_sync_dump( res = Some(err); break; } + */ } // Record that a part was obtained and dumped. @@ -371,9 +380,15 @@ fn check_new_epoch( } } -fn s3_location(chain_id: &str, epoch_height: u64, shard_id: u64, part_id: u64) -> String { +fn s3_location( + chain_id: &str, + epoch_height: u64, + shard_id: u64, + part_id: u64, + num_parts: u64, +) -> String { format!( - "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}", - chain_id, epoch_height, shard_id, part_id + "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}_of_{:06}", + chain_id, epoch_height, shard_id, part_id, num_parts ) } From b15dd003726b0169bc15637263621e4393537edc Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 1 Mar 2023 15:14:48 +0100 Subject: [PATCH 08/88] Add config options, change state part file naming - include num_parts. --- nearcore/src/state_sync.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 8399eb19924..91a3822943f 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -109,7 +109,7 @@ async fn state_sync_dump( runtime: Arc, config: ClientConfig, bucket: s3::Bucket, - node_key: PublicKey, + _node_key: PublicKey, ) { tracing::info!(target: "state_sync_dump", shard_id, "Running StateSyncDump loop"); let mut interval = actix_rt::time::interval(std::time::Duration::from_secs(10)); From 3e17aba0b9cc81a0cc751e49e243a4fc64094534 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 1 Mar 2023 15:19:59 +0100 Subject: [PATCH 09/88] Add config options, change state part file naming - include num_parts. --- nearcore/src/state_sync.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 91a3822943f..82a354506a6 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -306,6 +306,14 @@ fn set_metrics( .set(num_parts as i64); } if let Some(epoch_height) = epoch_height { + assert!( + epoch_height < 10000, + "Impossible: {:?} {:?} {:?} {:?}", + shard_id, + parts_dumped, + num_parts, + epoch_height + ); metrics::STATE_SYNC_DUMP_EPOCH_HEIGHT .with_label_values(&[&shard_id.to_string()]) .set(epoch_height as i64); From 81050050b189f594fd4b725ad9d410f84111ff85 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 2 Mar 2023 15:45:34 +0100 Subject: [PATCH 10/88] Refactor state syncing to avoid very long functions. --- core/chain-configs/src/client_config.rs | 8 ++++++-- nearcore/src/config.rs | 10 ++++++---- nearcore/src/state_sync.rs | 2 +- 3 files changed, 13 insertions(+), 7 deletions(-) diff --git a/core/chain-configs/src/client_config.rs b/core/chain-configs/src/client_config.rs index a5fdddfd4eb..775871a8730 100644 --- a/core/chain-configs/src/client_config.rs +++ b/core/chain-configs/src/client_config.rs @@ -167,13 +167,16 @@ pub struct ClientConfig { /// Duration to perform background flat storage creation step. pub flat_storage_creation_period: Duration, /// Whether to enable dumping state of every epoch to S3. - pub state_sync_dump_enabled: bool, + pub state_dump_enabled: bool, /// S3 bucket for storing state dumps. pub state_sync_s3_bucket: String, /// S3 region for storing state dumps. pub state_sync_s3_region: String, /// Discard the existing progress of dumping an epoch state to S3. pub state_sync_dump_drop_state: Vec, + /// Whether to enable state sync from S3. + /// If disabled will perform state sync from the peers. + pub state_sync_from_s3_enabled: bool, } impl ClientConfig { @@ -243,7 +246,8 @@ impl ClientConfig { enable_statistics_export: true, client_background_migration_threads: 1, flat_storage_creation_period: Duration::from_secs(1), - state_sync_dump_enabled: false, + state_dump_enabled: false, + state_sync_from_s3_enabled: false, state_sync_s3_bucket: String::new(), state_sync_s3_region: String::new(), state_sync_dump_drop_state: vec![], diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index 06785bcc8a4..ad6b87fabad 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -655,10 +655,7 @@ impl NearConfig { enable_statistics_export: config.store.enable_statistics_export, client_background_migration_threads: config.store.background_migration_threads, flat_storage_creation_period: config.store.flat_storage_creation_period, - state_sync_dump_enabled: config - .state_sync - .as_ref() - .map_or(false, |x| x.dump_enabled), + state_dump_enabled: config.state_sync.as_ref().map_or(false, |x| x.dump_enabled), state_sync_s3_bucket: config .state_sync .as_ref() @@ -671,6 +668,10 @@ impl NearConfig { .state_sync .as_ref() .map_or(vec![], |x| x.drop_state_of_dump.clone()), + state_sync_from_s3_enabled: config + .state_sync + .as_ref() + .map_or(false, |x| x.sync_from_s3_enabled), }, network_config: NetworkConfig::new( config.network, @@ -1501,6 +1502,7 @@ pub struct StateSyncConfig { pub s3_region: String, pub dump_enabled: bool, pub drop_state_of_dump: Vec, + pub sync_from_s3_enabled: bool, } #[test] diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 82a354506a6..1f3220057cc 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -17,7 +17,7 @@ pub fn spawn_state_sync_dump( runtime: Arc, node_key: &PublicKey, ) -> anyhow::Result> { - if !config.client_config.state_sync_dump_enabled + if !config.client_config.state_dump_enabled || config.client_config.state_sync_s3_bucket.is_empty() || config.client_config.state_sync_s3_region.is_empty() { From cb2731e9f42141a7ad34eb9879ee27876746eba8 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 8 Mar 2023 15:54:14 +0100 Subject: [PATCH 11/88] Syncing works --- Cargo.lock | 1 + chain/client-primitives/src/types.rs | 35 ++++--- chain/client/Cargo.toml | 1 + chain/client/src/client.rs | 98 ++++++++++--------- chain/client/src/sync/state.rs | 71 ++++++++++++-- chain/network/src/network_protocol/mod.rs | 1 + .../src/peer_manager/peer_manager_actor.rs | 1 + nearcore/src/config.rs | 5 + nearcore/src/state_sync.rs | 15 +-- tools/state-viewer/src/cli.rs | 67 +++++++------ 10 files changed, 182 insertions(+), 113 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f951859d5d6..2b36539eb3c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3255,6 +3255,7 @@ dependencies = [ "once_cell", "rand 0.8.5", "reed-solomon-erasure", + "rust-s3", "serde_json", "strum", "sysinfo", diff --git a/chain/client-primitives/src/types.rs b/chain/client-primitives/src/types.rs index 5436c1041a9..99d6af97207 100644 --- a/chain/client-primitives/src/types.rs +++ b/chain/client-primitives/src/types.rs @@ -22,7 +22,7 @@ pub use near_primitives::views::{StatusResponse, StatusSyncInfo}; use once_cell::sync::OnceCell; use std::collections::HashMap; use std::sync::atomic::{AtomicBool, AtomicU64, Ordering}; -use std::sync::Arc; +use std::sync::{Arc, Mutex}; /// Combines errors coming from chain, tx pool and block producer. #[derive(Debug, thiserror::Error)] @@ -50,6 +50,7 @@ pub enum AccountOrPeerIdOrHash { AccountId(AccountId), PeerId(PeerId), Hash(CryptoHash), + ExternalStorage(), } #[derive(Debug, serde::Serialize)] @@ -61,6 +62,23 @@ pub struct DownloadStatus { pub done: bool, pub state_requests_count: u64, pub last_target: Option, + #[serde(skip_serializing, skip_deserializing)] + pub response: Arc), String>>>>, +} + +impl DownloadStatus { + pub fn new(now: DateTime) -> Self { + Self { + start_time: now, + prev_update_time: now, + run_me: Arc::new(AtomicBool::new(true)), + error: false, + done: false, + state_requests_count: 0, + last_target: None, + response: Arc::new(Mutex::new(None)), + } + } } impl Clone for DownloadStatus { @@ -73,6 +91,8 @@ impl Clone for DownloadStatus { done: self.done, state_requests_count: self.state_requests_count, last_target: self.last_target.clone(), + // Copy the contents of `response`, but make it an independent object. + response: Arc::new(Mutex::new(self.response.lock().unwrap().clone().into())), } } } @@ -167,18 +187,7 @@ impl ShardSyncDownload { /// Creates a instance of self which includes initial statuses for shard sync and download at the given time. pub fn new(now: DateTime) -> Self { Self { - downloads: vec![ - DownloadStatus { - start_time: now, - prev_update_time: now, - run_me: Arc::new(AtomicBool::new(true)), - error: false, - done: false, - state_requests_count: 0, - last_target: None, - }; - 1 - ], + downloads: vec![DownloadStatus::new(now); 1], status: ShardSyncStatus::StateDownloadHeader, } } diff --git a/chain/client/Cargo.toml b/chain/client/Cargo.toml index a07766d503e..7add2bb5916 100644 --- a/chain/client/Cargo.toml +++ b/chain/client/Cargo.toml @@ -20,6 +20,7 @@ num-rational.workspace = true once_cell.workspace = true rand.workspace = true reed-solomon-erasure.workspace = true +rust-s3.workspace = true serde_json.workspace = true strum.workspace = true sysinfo.workspace = true diff --git a/chain/client/src/client.rs b/chain/client/src/client.rs index 57594fcca26..33fdf0b3b17 100644 --- a/chain/client/src/client.rs +++ b/chain/client/src/client.rs @@ -1,16 +1,23 @@ //! Client is responsible for tracking the chain, chunks, and producing them when needed. //! This client works completely synchronously and must be operated by some async actor outside. -use crate::adapter::ProcessTxResponse; -use crate::debug::BlockProductionTracker; -use crate::debug::PRODUCTION_TIMES_CACHE_SIZE; -use crate::sync::block::BlockSync; -use crate::sync::epoch::EpochSync; -use crate::sync::header::HeaderSync; -use crate::sync::state::{StateSync, StateSyncResult}; -use crate::{metrics, SyncStatus}; +use std::cmp::max; +use std::collections::{HashMap, HashSet}; +use std::sync::Arc; +use std::time::{Duration, Instant}; + use lru::LruCache; use near_async::messaging::{CanSend, Sender}; +use near_chunks::adapter::ShardsManagerRequestFromClient; +use near_chunks::client::ShardedTransactionPool; +use near_chunks::logic::{ + cares_about_shard_this_or_next_epoch, decode_encoded_chunk, persist_chunk, +}; +use near_client_primitives::debug::ChunkProduction; +use near_primitives::static_clock::StaticClock; +use near_store::metadata::DbKind; +use tracing::{debug, error, info, trace, warn}; + use near_chain::chain::{ ApplyStatePartsRequest, BlockCatchUpRequest, BlockMissingChunks, BlocksCatchUpState, OrphanMissingChunks, StateSplitRequest, TX_ROUTING_HEIGHT_HORIZON, @@ -23,49 +30,43 @@ use near_chain::{ DoneApplyChunkCallback, Doomslug, DoomslugThresholdMode, Provenance, RuntimeWithEpochManagerAdapter, }; -use near_chain_configs::{ClientConfig, LogSummaryStyle, UpdateableClientConfig}; -use near_chunks::adapter::ShardsManagerRequestFromClient; -use near_chunks::client::ShardedTransactionPool; -use near_chunks::logic::{ - cares_about_shard_this_or_next_epoch, decode_encoded_chunk, persist_chunk, -}; +use near_chain_configs::{ClientConfig, UpdateableClientConfig}; use near_chunks::ShardsManager; -use near_client_primitives::debug::ChunkProduction; -use near_client_primitives::types::{Error, ShardSyncDownload, ShardSyncStatus}; -use near_epoch_manager::shard_tracker::ShardTracker; -use near_network::types::{AccountKeys, ChainInfo, PeerManagerMessageRequest, SetChainInfo}; use near_network::types::{ HighestHeightPeerInfo, NetworkRequests, PeerManagerAdapter, ReasonForBan, }; -use near_o11y::log_assert; use near_primitives::block::{Approval, ApprovalInner, ApprovalMessage, Block, BlockHeader, Tip}; -use near_primitives::block_header::ApprovalType; use near_primitives::challenge::{Challenge, ChallengeBody}; -use near_primitives::epoch_manager::RngSeed; -use near_primitives::errors::EpochError; use near_primitives::hash::CryptoHash; use near_primitives::merkle::{merklize, MerklePath, PartialMerkleTree}; -use near_primitives::network::PeerId; use near_primitives::receipt::Receipt; use near_primitives::sharding::{ ChunkHash, EncodedShardChunk, PartialEncodedChunk, ReedSolomonWrapper, ShardChunk, ShardChunkHeader, ShardInfo, }; -use near_primitives::static_clock::StaticClock; use near_primitives::transaction::SignedTransaction; use near_primitives::types::chunk_extra::ChunkExtra; use near_primitives::types::{AccountId, ApprovalStake, BlockHeight, EpochId, NumBlocks, ShardId}; use near_primitives::unwrap_or_return; use near_primitives::utils::MaybeValidated; use near_primitives::validator_signer::ValidatorSigner; + +use crate::adapter::ProcessTxResponse; +use crate::debug::BlockProductionTracker; +use crate::debug::PRODUCTION_TIMES_CACHE_SIZE; +use crate::sync::block::BlockSync; +use crate::sync::epoch::EpochSync; +use crate::sync::header::HeaderSync; +use crate::sync::state::{StateSync, StateSyncResult}; +use crate::{metrics, SyncStatus}; +use near_client_primitives::types::{Error, ShardSyncDownload, ShardSyncStatus}; +use near_network::types::{AccountKeys, ChainInfo, PeerManagerMessageRequest, SetChainInfo}; +use near_o11y::log_assert; +use near_primitives::block_header::ApprovalType; +use near_primitives::epoch_manager::RngSeed; +use near_primitives::network::PeerId; use near_primitives::version::PROTOCOL_VERSION; use near_primitives::views::{CatchupStatusView, DroppedReason}; -use near_store::metadata::DbKind; -use std::cmp::max; -use std::collections::{HashMap, HashSet}; -use std::sync::Arc; -use std::time::{Duration, Instant}; -use tracing::{debug, error, info, trace, warn}; const NUM_REBROADCAST_BLOCKS: usize = 30; const CHUNK_HEADERS_FOR_INCLUSION_CACHE_SIZE: usize = 2048; @@ -103,7 +104,6 @@ pub struct Client { pub chain: Chain, pub doomslug: Doomslug, pub runtime_adapter: Arc, - pub shard_tracker: ShardTracker, pub shards_manager_adapter: Sender, pub sharded_tx_pool: ShardedTransactionPool, prev_block_to_chunk_headers_ready_for_inclusion: LruCache< @@ -242,13 +242,16 @@ impl Client { config.header_sync_stall_ban_timeout, config.header_sync_expected_height_per_second, ); - let block_sync = BlockSync::new( + let block_sync = + BlockSync::new(network_adapter.clone(), config.block_fetch_horizon, config.archive); + let state_sync = StateSync::new( network_adapter.clone(), - config.block_fetch_horizon, - config.archive, - config.state_sync_enabled, + config.state_sync_timeout, + &config.chain_id, + config.state_sync_from_s3_enabled, + &config.state_sync_s3_bucket, + &config.state_sync_s3_region, ); - let state_sync = StateSync::new(network_adapter.clone(), config.state_sync_timeout); let num_block_producer_seats = config.num_block_producer_seats as usize; let data_parts = runtime_adapter.num_data_parts(); let parity_parts = runtime_adapter.num_total_parts() - data_parts; @@ -277,7 +280,6 @@ impl Client { sync_status, chain, doomslug, - shard_tracker: runtime_adapter.shard_tracker(), runtime_adapter, shards_manager_adapter, sharded_tx_pool, @@ -330,7 +332,7 @@ impl Client { block.header().prev_hash(), shard_id, true, - &self.shard_tracker, + self.runtime_adapter.as_ref(), ) { self.sharded_tx_pool.remove_transactions( shard_id, @@ -354,7 +356,7 @@ impl Client { block.header().prev_hash(), shard_id, false, - &self.shard_tracker, + self.runtime_adapter.as_ref(), ) { self.sharded_tx_pool.reintroduce_transactions( shard_id, @@ -1591,8 +1593,7 @@ impl Client { &encoded_chunk, merkle_paths.clone(), Some(&validator_id), - self.runtime_adapter.epoch_manager_adapter(), - &self.shard_tracker, + self.runtime_adapter.as_ref(), )?; persist_chunk(partial_chunk.clone(), Some(shard_chunk), self.chain.mut_store())?; self.on_chunk_header_ready_for_inclusion(encoded_chunk.cloned_header(), validator_id); @@ -1763,7 +1764,7 @@ impl Client { let next_block_epoch_id = match self.runtime_adapter.get_epoch_id_from_prev_block(&parent_hash) { Err(e) => { - self.handle_process_approval_error(approval, approval_type, true, e.into()); + self.handle_process_approval_error(approval, approval_type, true, e); return; } Ok(next_epoch_id) => next_epoch_id, @@ -1784,7 +1785,7 @@ impl Client { account_id, ) { Ok(_) => next_block_epoch_id.clone(), - Err(EpochError::NotAValidator(_, _)) => { + Err(near_chain::Error::NotAValidator) => { match self.runtime_adapter.get_next_epoch_id_from_prev_block(&parent_hash) { Ok(next_block_next_epoch_id) => next_block_next_epoch_id, Err(_) => return, @@ -2121,7 +2122,14 @@ impl Client { let (state_sync, new_shard_sync, blocks_catch_up_state) = self.catchup_state_syncs.entry(sync_hash).or_insert_with(|| { ( - StateSync::new(network_adapter1, state_sync_timeout), + StateSync::new( + network_adapter1, + state_sync_timeout, + &self.config.chain_id, + self.config.state_sync_from_s3_enabled, + &self.config.state_sync_s3_bucket, + &self.config.state_sync_s3_region, + ), new_shard_sync, BlocksCatchUpState::new(sync_hash, epoch_id), ) @@ -2132,7 +2140,6 @@ impl Client { "Catchup me: {:?}: sync_hash: {:?}, sync_info: {:?}", me, sync_hash, new_shard_sync ); - let use_colour = matches!(self.config.log_summary_style, LogSummaryStyle::Colored); match state_sync.run( me, sync_hash, @@ -2143,7 +2150,6 @@ impl Client { state_sync_info.shards.iter().map(|tuple| tuple.0).collect(), state_parts_task_scheduler, state_split_scheduler, - use_colour, )? { StateSyncResult::Unchanged => {} StateSyncResult::Changed(fetch_block) => { diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 378be79df37..d9b3b3f4689 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -91,18 +91,43 @@ fn make_account_or_peer_id_or_hash( From::AccountId(a) => To::AccountId(a), From::PeerId(p) => To::PeerId(p), From::Hash(h) => To::Hash(h), + From::ExternalStorage() => To::ExternalStorage(), } } +/// How to retrieve the state data. +pub enum StateSyncMode { + /// Request both the state header and state parts from the peers. + Peers, + /// Requests the state header from peers but gets the state parts from an + /// external storage. + HeaderFromPeersAndPartsFromExternal { + /// Chain ID. + chain_id: String, + /// Connection to the external storage. + bucket: Arc, + }, +} + +/// Information about which parts were requested from which peer and when. +pub struct PartsRequestState { + last_part_id_requested: HashMap<(AccountOrPeerIdOrHash, ShardId), PendingRequestStatus>, + /// Map from which part we requested to whom. + requested_target: lru::LruCache<(u64, CryptoHash), AccountOrPeerIdOrHash>, +} + /// Helper to track state sync. pub struct StateSync { + /// How to retrieve the state data. + mode: StateSyncMode, + + /// Is used for communication with the peers. network_adapter: PeerManagerAdapter, last_time_block_requested: Option>, - last_part_id_requested: HashMap<(AccountOrPeerIdOrHash, ShardId), PendingRequestStatus>, - /// Map from which part we requested to whom. - requested_target: lru::LruCache<(u64, CryptoHash), AccountOrPeerIdOrHash>, + /// Information about which parts were requested from which peer and when. + parts_request_state: Option, /// Timeout (set in config - by default to 60 seconds) is used to figure out how long we should wait /// for the answer from the other node before giving up. @@ -116,12 +141,46 @@ pub struct StateSync { } impl StateSync { - pub fn new(network_adapter: PeerManagerAdapter, timeout: TimeDuration) -> Self { + pub fn new( + network_adapter: PeerManagerAdapter, + timeout: TimeDuration, + chain_id: &str, + state_sync_from_s3_enabled: bool, + s3_bucket: &str, + s3_region: &str, + ) -> Self { + let (mode, parts_request_state) = if state_sync_from_s3_enabled { + tracing::debug!(target: "sync", s3_bucket, s3_region, "Initializing S3 bucket connection."); + assert!(!s3_bucket.is_empty() && !s3_region.is_empty(), "State sync from S3 is enabled. This requires that both `s3_bucket and `s3_region` and specified and non-empty"); + let bucket = Arc::new( + s3::Bucket::new( + s3_bucket, + s3_region.parse::().unwrap(), + s3::creds::Credentials::default().unwrap(), + ) + .unwrap(), + ); + ( + StateSyncMode::HeaderFromPeersAndPartsFromExternal { + chain_id: chain_id.to_string(), + bucket, + }, + None, + ) + } else { + ( + StateSyncMode::Peers, + Some(PartsRequestState { + last_part_id_requested: Default::default(), + requested_target: lru::LruCache::new(MAX_PENDING_PART as usize), + }), + ) + }; StateSync { + mode, network_adapter, last_time_block_requested: None, - last_part_id_requested: Default::default(), - requested_target: lru::LruCache::new(MAX_PENDING_PART as usize), + parts_request_state, timeout: Duration::from_std(timeout).unwrap(), state_parts_apply_results: HashMap::new(), split_state_roots: HashMap::new(), diff --git a/chain/network/src/network_protocol/mod.rs b/chain/network/src/network_protocol/mod.rs index 93fb8fa1ba8..ea231c1174c 100644 --- a/chain/network/src/network_protocol/mod.rs +++ b/chain/network/src/network_protocol/mod.rs @@ -795,6 +795,7 @@ pub enum AccountOrPeerIdOrHash { AccountId(AccountId), PeerId(PeerId), Hash(CryptoHash), + ExternalStorage(), } pub(crate) struct RawRoutedMessage { diff --git a/chain/network/src/peer_manager/peer_manager_actor.rs b/chain/network/src/peer_manager/peer_manager_actor.rs index ed68932f308..22d85bc553d 100644 --- a/chain/network/src/peer_manager/peer_manager_actor.rs +++ b/chain/network/src/peer_manager/peer_manager_actor.rs @@ -658,6 +658,7 @@ impl PeerManagerActor { } AccountOrPeerIdOrHash::PeerId(it) => PeerIdOrHash::PeerId(it.clone()), AccountOrPeerIdOrHash::Hash(it) => PeerIdOrHash::Hash(*it), + AccountOrPeerIdOrHash::ExternalStorage() => unreachable!(), }; self.state.send_message_to_peer( diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index ad6b87fabad..f58182c0985 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -1498,10 +1498,15 @@ pub fn load_test_config(seed: &str, addr: tcp::ListenerAddr, genesis: Genesis) - #[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default)] /// Options for dumping state to S3. pub struct StateSyncConfig { + /// Location of state dumps on S3. pub s3_bucket: String, + /// Region is very important on S3. pub s3_region: String, + /// Whether a node should dump state of each epoch to the external storage. pub dump_enabled: bool, + /// Use carefully in case a node that dumps state to the external storage gets in trouble. pub drop_state_of_dump: Vec, + /// If enabled, will download state parts from external storage and not from the peers. pub sync_from_s3_enabled: bool, } diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 1f3220057cc..79e185c13e9 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -2,7 +2,7 @@ use crate::{metrics, NearConfig, NightshadeRuntime}; use near_chain::types::RuntimeAdapter; use near_chain::{Chain, ChainGenesis, DoomslugThresholdMode, Error}; use near_chain_configs::ClientConfig; -use near_client::sync::state::StateSync; +use near_client::sync::state::{s3_location, StateSync}; use near_crypto::PublicKey; use near_epoch_manager::EpochManagerAdapter; use near_primitives::hash::CryptoHash; @@ -387,16 +387,3 @@ fn check_new_epoch( } } } - -fn s3_location( - chain_id: &str, - epoch_height: u64, - shard_id: u64, - part_id: u64, - num_parts: u64, -) -> String { - format!( - "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}_of_{:06}", - chain_id, epoch_height, shard_id, part_id, num_parts - ) -} diff --git a/tools/state-viewer/src/cli.rs b/tools/state-viewer/src/cli.rs index a3218119f07..baa55daa44a 100644 --- a/tools/state-viewer/src/cli.rs +++ b/tools/state-viewer/src/cli.rs @@ -5,7 +5,7 @@ use near_chain_configs::{GenesisChangeConfig, GenesisValidationMode}; use near_primitives::account::id::AccountId; use near_primitives::hash::CryptoHash; use near_primitives::sharding::ChunkHash; -use near_primitives::types::{BlockHeight, ShardId}; +use near_primitives::types::{BlockHeight, ShardId, StateRoot}; use near_store::{Mode, NodeStorage, Store, Temperature}; use nearcore::{load_config, NearConfig}; use std::path::{Path, PathBuf}; @@ -493,6 +493,38 @@ impl StateChangesCmd { } } +#[derive(clap::Parser)] +pub struct StatePartsCmd { + /// Shard id. + #[clap(long)] + shard_id: ShardId, + /// Location of serialized state parts. + #[clap(long)] + root_dir: Option, + /// Store state parts in an S3 bucket. + #[clap(long)] + s3_bucket: Option, + /// Store state parts in an S3 bucket. + #[clap(long)] + s3_region: Option, + /// Dump or Apply state parts. + #[clap(subcommand)] + command: crate::state_parts::StatePartsSubCommand, +} + +impl StatePartsCmd { + pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { + self.command.run( + self.shard_id, + self.root_dir, + self.s3_bucket, + self.s3_region, + home_dir, + near_config, + store, + ); + } +} #[derive(clap::Parser)] pub struct ViewChainCmd { #[clap(long)] @@ -572,36 +604,3 @@ impl ViewTrieCmd { } } } - -#[derive(clap::Parser)] -pub struct StatePartsCmd { - /// Shard id. - #[clap(long)] - shard_id: ShardId, - /// Location of serialized state parts. - #[clap(long)] - root_dir: Option, - /// Store state parts in an S3 bucket. - #[clap(long)] - s3_bucket: Option, - /// Store state parts in an S3 bucket. - #[clap(long)] - s3_region: Option, - /// Dump or Apply state parts. - #[clap(subcommand)] - command: crate::state_parts::StatePartsSubCommand, -} - -impl StatePartsCmd { - pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { - self.command.run( - self.shard_id, - self.root_dir, - self.s3_bucket, - self.s3_region, - home_dir, - near_config, - store, - ); - } -} From 98cd7cf609e7c9efaf1c7148a4ef9520be0dd71e Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 9 Mar 2023 12:00:53 +0100 Subject: [PATCH 12/88] Metrics --- chain/client-primitives/src/types.rs | 15 ++ chain/client/src/metrics.rs | 87 +++++++++++ chain/client/src/sync/state.rs | 218 ++++++++++++++++++++++++--- 3 files changed, 295 insertions(+), 25 deletions(-) diff --git a/chain/client-primitives/src/types.rs b/chain/client-primitives/src/types.rs index 99d6af97207..51c872f87bb 100644 --- a/chain/client-primitives/src/types.rs +++ b/chain/client-primitives/src/types.rs @@ -110,6 +110,21 @@ pub enum ShardSyncStatus { StateSyncDone, } +impl ShardSyncStatus { + pub fn repr(&self) -> u8 { + match self { + ShardSyncStatus::StateDownloadHeader => 0, + ShardSyncStatus::StateDownloadParts => 1, + ShardSyncStatus::StateDownloadScheduling => 2, + ShardSyncStatus::StateDownloadApplying => 3, + ShardSyncStatus::StateDownloadComplete => 4, + ShardSyncStatus::StateSplitScheduling => 5, + ShardSyncStatus::StateSplitApplying(_) => 6, + ShardSyncStatus::StateSyncDone => 7, + } + } +} + /// Manually implement compare for ShardSyncStatus to compare only based on variant name impl PartialEq for ShardSyncStatus { fn eq(&self, other: &Self) -> bool { diff --git a/chain/client/src/metrics.rs b/chain/client/src/metrics.rs index 4c3d202c06a..4df5ffb17ca 100644 --- a/chain/client/src/metrics.rs +++ b/chain/client/src/metrics.rs @@ -354,3 +354,90 @@ pub(crate) fn export_version(neard_version: &near_primitives::version::Version) ]) .inc(); } + +pub static STATE_SYNC_STAGE: Lazy = Lazy::new(|| { + try_create_int_gauge_vec( + "near_state_sync_stage", + "Stage of state sync per shard", + &["shard_id"], + ) + .unwrap() +}); + +pub static STATE_SYNC_RETRY_PART: Lazy = Lazy::new(|| { + try_create_int_counter_vec( + "near_state_sync_retry_part_total", + "Number of part requests retried", + &["shard_id"], + ) + .unwrap() +}); + +pub static STATE_SYNC_PARTS_DONE: Lazy = Lazy::new(|| { + try_create_int_gauge_vec( + "near_state_sync_parts_done", + "Number of parts downloaded", + &["shard_id"], + ) + .unwrap() +}); + +pub static STATE_SYNC_PARTS_TOTAL: Lazy = Lazy::new(|| { + try_create_int_gauge_vec( + "near_state_sync_parts_per_shard", + "Number of parts that need to be downloaded for the shard", + &["shard_id"], + ) + .unwrap() +}); + +pub static STATE_SYNC_DISCARD_PARTS: Lazy = Lazy::new(|| { + try_create_int_counter_vec( + "near_state_sync_discard_parts_total", + "Number of times all downloaded parts so far were discarded to try again", + &["shard_id"], + ) + .unwrap() +}); + +pub static STATE_SYNC_EXTERNAL_PARTS_DONE: Lazy = + Lazy::new(|| { + try_create_int_counter_vec( + "near_state_sync_external_parts_done_total", + "Number of parts successfully retrieved from an external storage", + &["shard_id"], + ) + .unwrap() + }); + +pub static STATE_SYNC_EXTERNAL_PARTS_FAILED: Lazy = + Lazy::new(|| { + try_create_int_counter_vec( + "near_state_sync_external_parts_failed_total", + "Number of parts failed attempts to retrieve parts from an external storage", + &["shard_id"], + ) + .unwrap() + }); + +pub static STATE_SYNC_EXTERNAL_PARTS_SCHEDULING_DELAY: Lazy = + Lazy::new(|| { + try_create_histogram_vec( + "near_state_sync_external_parts_scheduling_delay_sec", + "Delay for a request for parts from an external storage", + &["shard_id"], + Some(exponential_buckets(0.001, 2.0, 20).unwrap()), + ) + .unwrap() + }); + +pub static STATE_SYNC_EXTERNAL_PARTS_REQUEST_DELAY: Lazy = + Lazy::new(|| { + try_create_histogram_vec( + "near_state_sync_external_parts_request_delay_sec", + "Latency of state part requests to external storage", + &["shard_id"], + Some(exponential_buckets(0.001, 2.0, 20).unwrap()), + ) + .unwrap() + }); diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index d9b3b3f4689..6e78e52f04a 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -21,7 +21,6 @@ //! use ansi_term::Color::{Purple, Yellow}; -use ansi_term::Style; use chrono::{DateTime, Duration, Utc}; use futures::{future, FutureExt}; use near_async::messaging::CanSendAsync; @@ -236,7 +235,6 @@ impl StateSync { now: DateTime, state_parts_task_scheduler: &dyn Fn(ApplyStatePartsRequest), state_split_scheduler: &dyn Fn(StateSplitRequest), - use_colour: bool, ) -> Result<(bool, bool), near_chain::Error> { let mut all_done = true; let mut update_sync_status = false; @@ -337,12 +335,30 @@ impl StateSync { %shard_id, timeout_sec = self.timeout.num_seconds(), "State sync didn't download the state, sending StateRequest again"); - tracing::debug!( - target: "sync", + tracing::info!(target: "sync", %shard_id, %sync_hash, ?me, - phase = format_shard_sync_phase(&shard_sync_download, use_colour), + phase = ?match shard_sync_download.status { + ShardSyncStatus::StateDownloadHeader => format!("{} requests sent {}, last target {:?}", + Purple.bold().paint("HEADER".to_string()), + shard_sync_download.downloads[0].state_requests_count, + shard_sync_download.downloads[0].last_target), + ShardSyncStatus::StateDownloadParts => { let mut text = "".to_string(); + for (i, download) in shard_sync_download.downloads.iter().enumerate() { + text.push_str(&format!("[{}: {}, {}, {:?}] ", + Yellow.bold().paint(i.to_string()), + download.done, + download.state_requests_count, + download.last_target)); + } + format!("{} [{}: is_done, requests sent, last target] {}", + Purple.bold().paint("PARTS"), + Yellow.bold().paint("part_id"), + text) + } + _ => unreachable!("timeout cannot happen when all state is downloaded"), + }, "State sync status"); } @@ -612,11 +628,88 @@ impl StateSync { .filter(|(_, download)| download.run_me.load(Ordering::SeqCst)) .zip(possible_targets_sampler) { - self.sent_request_part(target.clone(), part_id as u64, shard_id, sync_hash); - download.run_me.store(false, Ordering::SeqCst); - download.state_requests_count += 1; - download.last_target = Some(make_account_or_peer_id_or_hash(target.clone())); - let run_me = download.run_me.clone(); + match &self.mode { + StateSyncMode::Peers => { + // For every part that needs to be requested it is selected one + // peer (target) randomly to request the part from. + // IMPORTANT: here we use 'zip' with possible_target_sampler - + // which is limited. So at any moment we'll not request more + // than possible_targets.len() * MAX_STATE_PART_REQUEST parts. + let target = possible_targets_sampler.next().unwrap(); + self.request_part_from_peers( + part_id as u64, + target, + download, + shard_id, + sync_hash, + ); + } + StateSyncMode::HeaderFromPeersAndPartsFromExternal { chain_id, bucket } => { + self.request_part_from_external_storage( + part_id as u64, + download, + shard_id, + epoch_height, + state_num_parts, + &chain_id, + bucket.clone(), + ); + } + } + } + } + + /// Starts an asynchronous network request to external storage to obtain the given state part. + fn request_part_from_external_storage( + &self, + part_id: u64, + download: &mut DownloadStatus, + shard_id: ShardId, + epoch_height: EpochHeight, + num_parts: u64, + chain_id: &str, + bucket: Arc, + ) { + download.run_me.store(false, Ordering::SeqCst); + download.state_requests_count += 1; + download.last_target = + Some(make_account_or_peer_id_or_hash(AccountOrPeerIdOrHash::ExternalStorage())); + + let location = s3_location(chain_id, epoch_height, shard_id, part_id, num_parts); + let download_response = download.response.clone(); + near_performance_metrics::actix::spawn(std::any::type_name::(), { + async move { + tracing::info!(target: "sync", %shard_id, part_id, location, "Getting an object from the external storage"); + match bucket.get_object(location.clone()).await { + Ok(response) => { + tracing::info!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), "S3 request finished"); + let mut lock = download_response.lock().unwrap(); + *lock = Some(Ok((response.status_code(), response.bytes().to_vec()))); + } + Err(err) => { + tracing::info!(target: "sync", %shard_id, part_id, location, ?err, "S3 request failed"); + let mut lock = download_response.lock().unwrap(); + *lock = Some(Err(err.to_string())); + } + } + } + }); + } + + /// Asynchronously requests a state part from a suitable peer. + fn request_part_from_peers( + &mut self, + part_id: u64, + target: AccountOrPeerIdOrHash, + download: &mut DownloadStatus, + shard_id: ShardId, + sync_hash: CryptoHash, + ) { + self.sent_request_part(target.clone(), part_id, shard_id, sync_hash); + download.run_me.store(false, Ordering::SeqCst); + download.state_requests_count += 1; + download.last_target = Some(make_account_or_peer_id_or_hash(target.clone())); + let run_me = download.run_me.clone(); near_performance_metrics::actix::spawn( std::any::type_name::(), @@ -661,7 +754,6 @@ impl StateSync { tracking_shards: Vec, state_parts_task_scheduler: &dyn Fn(ApplyStatePartsRequest), state_split_scheduler: &dyn Fn(StateSplitRequest), - use_colour: bool, ) -> Result { let _span = tracing::debug_span!(target: "sync", "run", sync = "StateSync").entered(); tracing::debug!(target: "sync", %sync_hash, ?tracking_shards, "syncing state"); @@ -695,7 +787,6 @@ impl StateSync { now, state_parts_task_scheduler, state_split_scheduler, - use_colour, )?; if have_block && all_done { @@ -845,14 +936,26 @@ impl StateSync { let mut run_shard_state_download = false; let mut parts_done = true; - for part_download in shard_sync_download.downloads.iter_mut() { + let num_parts = shard_sync_download.downloads.len(); + for (part_id, part_download) in shard_sync_download.downloads.iter_mut().enumerate() { + if !part_download.done { + // Check if a download from an external storage is finished. + check_external_storage_part_response( + part_id as u64, + num_parts as u64, + shard_id, + sync_hash, + part_download, + chain, + ); + } if !part_download.done { parts_done = false; let prev = part_download.prev_update_time; let error = part_download.error; let part_timeout = now - prev > self.timeout; // Retry parts that failed. - if part_timeout || error { + if part_timeout || part_download.error { download_timeout |= part_timeout; part_download.run_me.store(true, Ordering::SeqCst); part_download.error = false; @@ -1018,20 +1121,86 @@ impl StateSync { } } -fn paint(s: &str, colour: Style, use_colour: bool) -> String { - if use_colour { - colour.paint(s).to_string() - } else { - s.to_string() +/// Works around how data requests to external storage are done. +/// The response is stored on the DownloadStatus object. +/// This function investigates if the response is available and updates `done` and `error` appropriately. +/// If the response is successful, then also writes the state part to the DB. +fn check_external_storage_part_response( + part_id: u64, + num_parts: u64, + shard_id: ShardId, + sync_hash: CryptoHash, + part_download: &mut DownloadStatus, + chain: &mut Chain, +) { + let external_storage_response = { + let mut lock = part_download.response.lock().unwrap(); + if let Some(response) = lock.clone() { + // Remove the response from DownloadStatus, because + // we're going to write positive responses to the DB + // and retry negative responses. + *lock = None; + response + } else { + return; + } + }; + + let mut err_to_retry = None; + match external_storage_response { + // HTTP status code 200 means success. + Ok((200, data)) => { + match chain.set_state_part( + shard_id, + sync_hash, + PartId::new(part_id as u64, num_parts as u64), + &data, + ) { + Ok(_) => { + part_download.done = true; + } + Err(err) => { + tracing::warn!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to save a state part"); + err_to_retry = Some(Error::Other("Failed to save a state part".to_string())); + } + } + } + // Other HTTP status codes are considered errors. + Ok((status_code, _)) => { + err_to_retry = Some(Error::Other(format!("status_code: {}", status_code).to_string())); + } + // The request failed without reaching the external storage. + Err(err) => { + err_to_retry = Some(Error::Other(err.to_string())); + } + }; + + if let Some(err) = err_to_retry { + tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to get a part from external storage, will retry"); + part_download.error = true; } } +/// Construct a location on the external storage. +pub fn s3_location( + chain_id: &str, + epoch_height: u64, + shard_id: u64, + part_id: u64, + num_parts: u64, +) -> String { + format!( + "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}_of_{:06}", + chain_id, epoch_height, shard_id, part_id, num_parts + ) +} + /// Formats the given ShardSyncDownload for logging. -fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload, use_colour: bool) -> String { +fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload) -> String { match shard_sync_download.status { ShardSyncStatus::StateDownloadHeader => format!( "{} requests sent {}, last target {:?}", - paint("HEADER", Purple.bold(), use_colour), + Purple.bold().paint("HEADER".to_string()), shard_sync_download.downloads[0].state_requests_count, shard_sync_download.downloads[0].last_target ), @@ -1040,7 +1209,7 @@ fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload, use_colour: for (i, download) in shard_sync_download.downloads.iter().enumerate() { text.push_str(&format!( "[{}: {}, {}, {:?}] ", - paint(&i.to_string(), Yellow.bold(), use_colour), + Yellow.bold().paint(i.to_string()), download.done, download.state_requests_count, download.last_target @@ -1048,8 +1217,8 @@ fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload, use_colour: } format!( "{} [{}: is_done, requests sent, last target] {}", - paint("PARTS", Purple.bold(), use_colour), - paint("part_id", Yellow.bold(), use_colour), + Purple.bold().paint("PARTS"), + Yellow.bold().paint("part_id"), text ) } @@ -1193,7 +1362,6 @@ mod test { vec![0], &apply_parts_fn, &state_split_fn, - false, ) .unwrap(); From 712ca557bf57c6b3435eb0d0a070b1a1fa3861ca Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 9 Mar 2023 13:19:36 +0100 Subject: [PATCH 13/88] Fix --- chain/client/src/sync/state.rs | 29 ++++++++++++++++++++--------- 1 file changed, 20 insertions(+), 9 deletions(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 6e78e52f04a..47c7d36f060 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -476,13 +476,16 @@ impl StateSync { highest_height_peers: &[HighestHeightPeerInfo], ) -> Result, near_chain::Error> { // Remove candidates from pending list if request expired due to timeout - self.last_part_id_requested.retain(|_, request| !request.expired()); + self.parts_request_state.as_mut().map(|parts_request_state| { + parts_request_state.last_part_id_requested.retain(|_, request| !request.expired()) + }); let prev_block_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); let epoch_hash = runtime_adapter.get_epoch_id_from_prev_block(&prev_block_hash)?; - Ok(runtime_adapter - .get_epoch_block_producers_ordered(&epoch_hash, &sync_hash)? + let block_producers = + runtime_adapter.get_epoch_block_producers_ordered(&epoch_hash, &sync_hash)?; + let peers = block_producers .iter() .filter_map(|(validator_stake, _slashed)| { let account_id = validator_stake.account_id(); @@ -512,12 +515,20 @@ impl StateSync { } else { None } - })) - .filter(|candidate| { - // If we still have a pending request from this node - don't add another one. - !self.last_part_id_requested.contains_key(&(candidate.clone(), shard_id)) - }) - .collect::>()) + })); + let result = if let Some(parts_request_state) = &self.parts_request_state { + peers + .filter(|candidate| { + // If we still have a pending request from this node - don't add another one. + !parts_request_state + .last_part_id_requested + .contains_key(&(candidate.clone(), shard_id)) + }) + .collect::>() + } else { + peers.collect::>() + }; + Ok(result) } /// Returns new ShardSyncDownload if successful, otherwise returns given shard_sync_download From bc953ea8ce4954514e63c53e7ed048a42f8f74e8 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 9 Mar 2023 14:24:35 +0100 Subject: [PATCH 14/88] Fields are not optional --- nearcore/src/config.rs | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index f58182c0985..c74370593f5 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -655,7 +655,10 @@ impl NearConfig { enable_statistics_export: config.store.enable_statistics_export, client_background_migration_threads: config.store.background_migration_threads, flat_storage_creation_period: config.store.flat_storage_creation_period, - state_dump_enabled: config.state_sync.as_ref().map_or(false, |x| x.dump_enabled), + state_dump_enabled: config + .state_sync + .as_ref() + .map_or(false, |x| x.dump_enabled.unwrap_or(false)), state_sync_s3_bucket: config .state_sync .as_ref() @@ -667,11 +670,11 @@ impl NearConfig { state_sync_dump_drop_state: config .state_sync .as_ref() - .map_or(vec![], |x| x.drop_state_of_dump.clone()), + .map_or(vec![], |x| x.drop_state_of_dump.clone().unwrap_or(vec![])), state_sync_from_s3_enabled: config .state_sync .as_ref() - .map_or(false, |x| x.sync_from_s3_enabled), + .map_or(false, |x| x.sync_from_s3_enabled.unwrap_or(false)), }, network_config: NetworkConfig::new( config.network, @@ -1503,11 +1506,14 @@ pub struct StateSyncConfig { /// Region is very important on S3. pub s3_region: String, /// Whether a node should dump state of each epoch to the external storage. - pub dump_enabled: bool, + #[serde(skip_serializing_if = "Option::is_none")] + pub dump_enabled: Option, /// Use carefully in case a node that dumps state to the external storage gets in trouble. - pub drop_state_of_dump: Vec, + #[serde(skip_serializing_if = "Option::is_none")] + pub drop_state_of_dump: Option>, /// If enabled, will download state parts from external storage and not from the peers. - pub sync_from_s3_enabled: bool, + #[serde(skip_serializing_if = "Option::is_none")] + pub sync_from_s3_enabled: Option, } #[test] From 61671f5c7956f488d887d0a943a475b192401a0b Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 13 Mar 2023 15:40:18 +0100 Subject: [PATCH 15/88] . --- chain/client/src/client.rs | 2 + chain/client/src/sync/state.rs | 86 +++++++++++++++++++++---- core/chain-configs/src/client_config.rs | 3 + nearcore/src/config.rs | 26 ++++++-- 4 files changed, 98 insertions(+), 19 deletions(-) diff --git a/chain/client/src/client.rs b/chain/client/src/client.rs index 33fdf0b3b17..6b01941f63f 100644 --- a/chain/client/src/client.rs +++ b/chain/client/src/client.rs @@ -251,6 +251,7 @@ impl Client { config.state_sync_from_s3_enabled, &config.state_sync_s3_bucket, &config.state_sync_s3_region, + config.state_sync_num_s3_requests_per_shard, ); let num_block_producer_seats = config.num_block_producer_seats as usize; let data_parts = runtime_adapter.num_data_parts(); @@ -2129,6 +2130,7 @@ impl Client { self.config.state_sync_from_s3_enabled, &self.config.state_sync_s3_bucket, &self.config.state_sync_s3_region, + self.config.state_sync_num_s3_requests_per_shard, ), new_shard_sync, BlocksCatchUpState::new(sync_hash, epoch_id), diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 47c7d36f060..e40c3c445bb 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -45,7 +45,7 @@ use rand::seq::SliceRandom; use rand::{thread_rng, Rng}; use std::collections::HashMap; use std::ops::Add; -use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::atomic::Ordering; use std::sync::Arc; use std::time::Duration as TimeDuration; @@ -105,6 +105,8 @@ pub enum StateSyncMode { chain_id: String, /// Connection to the external storage. bucket: Arc, + /// Number state part requests allowed to be in-flight in parallel per shard. + num_s3_requests_per_shard: u64, }, } @@ -137,6 +139,10 @@ pub struct StateSync { /// Maps shard_id to result of splitting state for resharding split_state_roots: HashMap, near_chain::Error>>, + + /// The number of requests for state parts from external storage that are + /// allowed to be started for this shard. + requests_remaining: Arc, } impl StateSync { @@ -147,22 +153,25 @@ impl StateSync { state_sync_from_s3_enabled: bool, s3_bucket: &str, s3_region: &str, + num_s3_requests_per_shard: u64, ) -> Self { let (mode, parts_request_state) = if state_sync_from_s3_enabled { tracing::debug!(target: "sync", s3_bucket, s3_region, "Initializing S3 bucket connection."); assert!(!s3_bucket.is_empty() && !s3_region.is_empty(), "State sync from S3 is enabled. This requires that both `s3_bucket and `s3_region` and specified and non-empty"); - let bucket = Arc::new( - s3::Bucket::new( - s3_bucket, - s3_region.parse::().unwrap(), - s3::creds::Credentials::default().unwrap(), - ) - .unwrap(), - ); + let mut bucket = s3::Bucket::new( + s3_bucket, + s3_region.parse::().unwrap(), + s3::creds::Credentials::default().unwrap(), + ) + .unwrap(); + // Ensure requests finish in finite amount of time. + bucket.set_request_timeout(Some(timeout)); + let bucket = Arc::new(bucket); ( StateSyncMode::HeaderFromPeersAndPartsFromExternal { chain_id: chain_id.to_string(), bucket, + num_s3_requests_per_shard, }, None, ) @@ -175,14 +184,16 @@ impl StateSync { }), ) }; + let timeout = Duration::from_std(timeout).unwrap(); StateSync { mode, network_adapter, last_time_block_requested: None, parts_request_state, - timeout: Duration::from_std(timeout).unwrap(), + timeout, state_parts_apply_results: HashMap::new(), split_state_roots: HashMap::new(), + requests_remaining: Arc::new(AtomicI64::new(num_s3_requests_per_shard as i64)), } } @@ -655,7 +666,11 @@ impl StateSync { sync_hash, ); } - StateSyncMode::HeaderFromPeersAndPartsFromExternal { chain_id, bucket } => { + StateSyncMode::HeaderFromPeersAndPartsFromExternal { + chain_id, + bucket, + num_s3_requests_per_shard: _, + } => { self.request_part_from_external_storage( part_id as u64, download, @@ -681,6 +696,10 @@ impl StateSync { chain_id: &str, bucket: Arc, ) { + let requests_remaining = self.requests_remaining.clone(); + if !allow_request(&requests_remaining) { + return; + } download.run_me.store(false, Ordering::SeqCst); download.state_requests_count += 1; download.last_target = @@ -691,7 +710,25 @@ impl StateSync { near_performance_metrics::actix::spawn(std::any::type_name::(), { async move { tracing::info!(target: "sync", %shard_id, part_id, location, "Getting an object from the external storage"); - match bucket.get_object(location.clone()).await { + let started = StaticClock::utc(); + metrics::STATE_SYNC_EXTERNAL_PARTS_SCHEDULING_DELAY + .with_label_values(&[&shard_id.to_string()]) + .observe( + started.signed_duration_since(scheduled).num_nanoseconds().unwrap_or(0) + as f64 + / 1e9, + ); + let result = bucket.get_object(location.clone()).await; + let completed = StaticClock::utc(); + finished_request(&requests_remaining); + metrics::STATE_SYNC_EXTERNAL_PARTS_REQUEST_DELAY + .with_label_values(&[&shard_id.to_string()]) + .observe( + completed.signed_duration_since(started).num_nanoseconds().unwrap_or(0) + as f64 + / 1e9, + ); + match result { Ok(response) => { tracing::info!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), "S3 request finished"); let mut lock = download_response.lock().unwrap(); @@ -1132,6 +1169,20 @@ impl StateSync { } } +fn allow_request(requests_remaining: &AtomicI64) -> bool { + let remaining = requests_remaining.fetch_sub(1, Ordering::SeqCst); + if remaining <= 0 { + requests_remaining.fetch_add(1, Ordering::SeqCst); + false + } else { + true + } +} + +fn finished_request(requests_remaining: &AtomicI64) { + requests_remaining.fetch_add(1, Ordering::SeqCst); +} + /// Works around how data requests to external storage are done. /// The response is stored on the DownloadStatus object. /// This function investigates if the response is available and updates `done` and `error` appropriately. @@ -1322,8 +1373,15 @@ mod test { // Start a new state sync - and check that it asks for a header. fn test_ask_for_header() { let mock_peer_manager = Arc::new(MockPeerManagerAdapter::default()); - let mut state_sync = - StateSync::new(mock_peer_manager.clone().into(), TimeDuration::from_secs(1)); + let mut state_sync = StateSync::new( + mock_peer_manager.clone().into(), + TimeDuration::from_secs(1), + "chain_id".to_string(), + false, + "".to_string(), + "".to_string(), + 100, + ); let mut new_shard_sync = HashMap::new(); let (mut chain, kv, signer) = test_utils::setup(); diff --git a/core/chain-configs/src/client_config.rs b/core/chain-configs/src/client_config.rs index 775871a8730..6d1d6336be6 100644 --- a/core/chain-configs/src/client_config.rs +++ b/core/chain-configs/src/client_config.rs @@ -177,6 +177,8 @@ pub struct ClientConfig { /// Whether to enable state sync from S3. /// If disabled will perform state sync from the peers. pub state_sync_from_s3_enabled: bool, + /// Number of parallel in-flight requests allowed per shard. + pub state_sync_num_s3_requests_per_shard: u64, } impl ClientConfig { @@ -251,6 +253,7 @@ impl ClientConfig { state_sync_s3_bucket: String::new(), state_sync_s3_region: String::new(), state_sync_dump_drop_state: vec![], + state_sync_num_s3_requests_per_shard: 100, } } } diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index c74370593f5..56be887c4cf 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -658,23 +658,37 @@ impl NearConfig { state_dump_enabled: config .state_sync .as_ref() - .map_or(false, |x| x.dump_enabled.unwrap_or(false)), + .map(|x| x.dump_enabled) + .flatten() + .unwrap_or(false), state_sync_s3_bucket: config .state_sync .as_ref() - .map_or(String::new(), |x| x.s3_bucket.clone()), + .map(|x| x.s3_bucket.clone()) + .unwrap_or(String::new()), state_sync_s3_region: config .state_sync .as_ref() - .map_or(String::new(), |x| x.s3_region.clone()), + .map(|x| x.s3_region.clone()) + .unwrap_or(String::new()), state_sync_dump_drop_state: config .state_sync .as_ref() - .map_or(vec![], |x| x.drop_state_of_dump.clone().unwrap_or(vec![])), + .map(|x| x.drop_state_of_dump.clone()) + .flatten() + .unwrap_or(vec![]), state_sync_from_s3_enabled: config .state_sync .as_ref() - .map_or(false, |x| x.sync_from_s3_enabled.unwrap_or(false)), + .map(|x| x.sync_from_s3_enabled) + .flatten() + .unwrap_or(false), + state_sync_num_s3_requests_per_shard: config + .state_sync + .as_ref() + .map(|x| x.num_s3_requests_per_shard) + .flatten() + .unwrap_or(100), }, network_config: NetworkConfig::new( config.network, @@ -1514,6 +1528,8 @@ pub struct StateSyncConfig { /// If enabled, will download state parts from external storage and not from the peers. #[serde(skip_serializing_if = "Option::is_none")] pub sync_from_s3_enabled: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub num_s3_requests_per_shard: Option, } #[test] From 0667d8bf692b9492e7acfc3e2ab2574635f32853 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 13 Mar 2023 21:57:58 +0100 Subject: [PATCH 16/88] Throttling of requests now works! :pray: --- chain/client-primitives/src/types.rs | 14 ++++- chain/client/src/sync/state.rs | 89 +++++++++++++++++----------- 2 files changed, 65 insertions(+), 38 deletions(-) diff --git a/chain/client-primitives/src/types.rs b/chain/client-primitives/src/types.rs index 51c872f87bb..dee1bd94bc0 100644 --- a/chain/client-primitives/src/types.rs +++ b/chain/client-primitives/src/types.rs @@ -92,7 +92,7 @@ impl Clone for DownloadStatus { state_requests_count: self.state_requests_count, last_target: self.last_target.clone(), // Copy the contents of `response`, but make it an independent object. - response: Arc::new(Mutex::new(self.response.lock().unwrap().clone().into())), + response: self.response.clone(), } } } @@ -200,13 +200,23 @@ pub struct ShardSyncDownload { impl ShardSyncDownload { /// Creates a instance of self which includes initial statuses for shard sync and download at the given time. - pub fn new(now: DateTime) -> Self { + pub fn new_download_state_header(now: DateTime) -> Self { Self { downloads: vec![DownloadStatus::new(now); 1], status: ShardSyncStatus::StateDownloadHeader, } } + + pub fn new_download_state_parts(now: DateTime, num_parts: u64) -> Self { + // Avoid using `vec![x; num_parts]`, because cloning DownloadStatus::response is a terrible idea. + let mut downloads = Vec::with_capacity(num_parts as usize); + for _ in 0..num_parts { + downloads.push(DownloadStatus::new(now)); + } + Self { downloads, status: ShardSyncStatus::StateDownloadParts } + } } + /// Various status sync can be in, whether it's fast sync or archival. #[derive(Clone, Debug, strum::AsRefStr)] pub enum SyncStatus { diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index e40c3c445bb..a2a8831415a 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -266,7 +266,7 @@ impl StateSync { let shard_sync_download = new_shard_sync.entry(shard_id).or_insert_with(|| { run_shard_state_download = true; update_sync_status = true; - ShardSyncDownload::new(now) + ShardSyncDownload::new_download_state_header(now) }); let old_status = shard_sync_download.status.clone(); @@ -376,13 +376,13 @@ impl StateSync { // Execute syncing for shard `shard_id` if run_shard_state_download { update_sync_status = true; - *shard_sync_download = self.request_shard( + self.request_shard( me, shard_id, chain, runtime_adapter, sync_hash, - shard_sync_download.clone(), + shard_sync_download, highest_height_peers, )?; } @@ -550,9 +550,9 @@ impl StateSync { chain: &Chain, runtime_adapter: &Arc, sync_hash: CryptoHash, - shard_sync_download: ShardSyncDownload, + shard_sync_download: &mut ShardSyncDownload, highest_height_peers: &[HighestHeightPeerInfo], - ) -> Result { + ) -> Result<(), near_chain::Error> { let possible_targets = self.possible_targets( me, shard_id, @@ -564,19 +564,17 @@ impl StateSync { if possible_targets.is_empty() { // In most cases it means that all the targets are currently busy (that we have a pending request with them). - return Ok(shard_sync_download); + return Ok(()); } // Downloading strategy starts here - let mut new_shard_sync_download = shard_sync_download.clone(); - match shard_sync_download.status { ShardSyncStatus::StateDownloadHeader => { self.request_shard_header( shard_id, sync_hash, &possible_targets, - &mut new_shard_sync_download, + shard_sync_download, ); } ShardSyncStatus::StateDownloadParts => { @@ -585,12 +583,13 @@ impl StateSync { sync_hash, possible_targets, &mut new_shard_sync_download, + chain, ); } _ => {} } - Ok(new_shard_sync_download) + Ok(()) } fn request_shard_header( @@ -698,9 +697,15 @@ impl StateSync { ) { let requests_remaining = self.requests_remaining.clone(); if !allow_request(&requests_remaining) { + tracing::info!(target: "sync", %shard_id, part_id, "Request throttled"); return; + } else { + tracing::info!(target: "sync", %shard_id, part_id, "Request allowed"); + if !download.run_me.swap(false, Ordering::SeqCst) { + tracing::info!(target: "sync", %shard_id, part_id, "run_me is already false"); + return; + } } - download.run_me.store(false, Ordering::SeqCst); download.state_requests_count += 1; download.last_target = Some(make_account_or_peer_id_or_hash(AccountOrPeerIdOrHash::ExternalStorage())); @@ -730,14 +735,20 @@ impl StateSync { ); match result { Ok(response) => { - tracing::info!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), "S3 request finished"); + tracing::error!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), "S3 request finished"); let mut lock = download_response.lock().unwrap(); - *lock = Some(Ok((response.status_code(), response.bytes().to_vec()))); + let v: &mut Option> = &mut lock; + tracing::error!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), current_value=?v, "S3 request finished"); + *v = Some(Ok((response.status_code(), response.bytes().to_vec()))); + tracing::error!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), new_value=?v, "S3 request finished"); } Err(err) => { - tracing::info!(target: "sync", %shard_id, part_id, location, ?err, "S3 request failed"); + tracing::error!(target: "sync", %shard_id, part_id, location, ?err, "S3 request failed"); let mut lock = download_response.lock().unwrap(); - *lock = Some(Err(err.to_string())); + let v: &mut Option> = &mut lock; + tracing::error!(target: "sync", %shard_id, part_id, location, ?err, current_value=?v, "S3 request failed"); + *v = Some(Err(err.to_string())); + tracing::error!(target: "sync", %shard_id, part_id, location, ?err, new_value=?v, "S3 request failed"); } } } @@ -936,22 +947,9 @@ impl StateSync { get_num_state_parts(shard_state_header.state_root_node().memory_usage); // If the header was downloaded successfully - move to phase 2 (downloading parts). // Create the vector with entry for each part. - *shard_sync_download = ShardSyncDownload { - downloads: vec![ - DownloadStatus { - start_time: now, - prev_update_time: now, - run_me: Arc::new(AtomicBool::new(true)), - error: false, - done: false, - state_requests_count: 0, - last_target: None, - }; - state_num_parts as usize - ], - status: ShardSyncStatus::StateDownloadParts, - }; - run_shard_state_download = true; + *shard_sync_download = + ShardSyncDownload::new_download_state_parts(now, state_num_parts); + need_shard = true; } else { let prev = shard_sync_download.downloads[0].prev_update_time; let error = shard_sync_download.downloads[0].error; @@ -986,6 +984,7 @@ impl StateSync { let mut parts_done = true; let num_parts = shard_sync_download.downloads.len(); for (part_id, part_download) in shard_sync_download.downloads.iter_mut().enumerate() { + tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, part_download.done, part_download.error, ?part_download); if !part_download.done { // Check if a download from an external storage is finished. check_external_storage_part_response( @@ -997,12 +996,12 @@ impl StateSync { chain, ); } + tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, part_download.done, part_download.error); if !part_download.done { parts_done = false; let prev = part_download.prev_update_time; - let error = part_download.error; - let part_timeout = now - prev > self.timeout; - // Retry parts that failed. + let part_timeout = false; // now - prev > self.timeout; + // Retry parts that failed. if part_timeout || part_download.error { download_timeout |= part_timeout; part_download.run_me.store(true, Ordering::SeqCst); @@ -1014,6 +1013,13 @@ impl StateSync { } } } + tracing::debug!(target: "sync", %shard_id, %sync_hash, num_parts_done, parts_done); + metrics::STATE_SYNC_PARTS_DONE + .with_label_values(&[&shard_id.to_string()]) + .set(num_parts_done); + metrics::STATE_SYNC_PARTS_TOTAL + .with_label_values(&[&shard_id.to_string()]) + .set(num_parts as i64); // If all parts are done - we can move towards scheduling. if parts_done { *shard_sync_download = ShardSyncDownload { @@ -1055,7 +1061,7 @@ impl StateSync { // Cannot finalize the downloaded state. // The reasonable behavior here is to start from the very beginning. tracing::error!(target: "sync", %shard_id, %sync_hash, ?err, "State sync finalizing error"); - *shard_sync_download = ShardSyncDownload::new(now); + *shard_sync_download = ShardSyncDownload::new_download_state_header(now); chain.clear_downloaded_parts(shard_id, sync_hash, state_num_parts)?; } } @@ -1085,7 +1091,7 @@ impl StateSync { // Cannot finalize the downloaded state. // The reasonable behavior here is to start from the very beginning. tracing::error!(target: "sync", %shard_id, %sync_hash, ?err, "State sync finalizing error"); - *shard_sync_download = ShardSyncDownload::new(now); + *shard_sync_download = ShardSyncDownload::new_download_state_header(now); let shard_state_header = chain.get_state_header(shard_id, sync_hash)?; let state_num_parts = get_num_state_parts(shard_state_header.state_root_node().memory_usage); @@ -1198,6 +1204,7 @@ fn check_external_storage_part_response( let external_storage_response = { let mut lock = part_download.response.lock().unwrap(); if let Some(response) = lock.clone() { + tracing::debug!(target: "sync", %shard_id, part_id, "Got response from external storage"); // Remove the response from DownloadStatus, because // we're going to write positive responses to the DB // and retry negative responses. @@ -1207,11 +1214,17 @@ fn check_external_storage_part_response( return; } }; + { + let lock = part_download.response.lock().unwrap(); + assert!(lock.is_none()); + tracing::debug!(target: "sync", %shard_id, part_id, "And `response` is now clear"); + } let mut err_to_retry = None; match external_storage_response { // HTTP status code 200 means success. Ok((200, data)) => { + tracing::debug!(target: "sync", %shard_id, part_id, "Got 200 response from external storage"); match chain.set_state_part( shard_id, sync_hash, @@ -1220,6 +1233,8 @@ fn check_external_storage_part_response( ) { Ok(_) => { part_download.done = true; + tracing::debug!(target: "sync", %shard_id, part_id, ?part_download, "Got 200 response from external storage and saved state part"); + // TODO } Err(err) => { tracing::warn!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to save a state part"); @@ -1240,6 +1255,8 @@ fn check_external_storage_part_response( if let Some(err) = err_to_retry { tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to get a part from external storage, will retry"); part_download.error = true; + } else { + tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, "Checked, ok"); } } From 0063fa889891c9368b3a11cf08db4f1c876f7926 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 14 Mar 2023 07:27:39 +0100 Subject: [PATCH 17/88] Too much output --- chain/client/src/sync/state.rs | 24 ++++-------------------- 1 file changed, 4 insertions(+), 20 deletions(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index a2a8831415a..7e7670b0256 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -697,10 +697,8 @@ impl StateSync { ) { let requests_remaining = self.requests_remaining.clone(); if !allow_request(&requests_remaining) { - tracing::info!(target: "sync", %shard_id, part_id, "Request throttled"); return; } else { - tracing::info!(target: "sync", %shard_id, part_id, "Request allowed"); if !download.run_me.swap(false, Ordering::SeqCst) { tracing::info!(target: "sync", %shard_id, part_id, "run_me is already false"); return; @@ -735,20 +733,14 @@ impl StateSync { ); match result { Ok(response) => { - tracing::error!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), "S3 request finished"); + tracing::info!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), "S3 request finished"); let mut lock = download_response.lock().unwrap(); - let v: &mut Option> = &mut lock; - tracing::error!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), current_value=?v, "S3 request finished"); - *v = Some(Ok((response.status_code(), response.bytes().to_vec()))); - tracing::error!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), new_value=?v, "S3 request finished"); + *lock = Some(Ok((response.status_code(), response.bytes().to_vec()))); } Err(err) => { - tracing::error!(target: "sync", %shard_id, part_id, location, ?err, "S3 request failed"); + tracing::info!(target: "sync", %shard_id, part_id, location, ?err, "S3 request failed"); let mut lock = download_response.lock().unwrap(); - let v: &mut Option> = &mut lock; - tracing::error!(target: "sync", %shard_id, part_id, location, ?err, current_value=?v, "S3 request failed"); - *v = Some(Err(err.to_string())); - tracing::error!(target: "sync", %shard_id, part_id, location, ?err, new_value=?v, "S3 request failed"); + *lock = Some(Err(err.to_string())); } } } @@ -1214,11 +1206,6 @@ fn check_external_storage_part_response( return; } }; - { - let lock = part_download.response.lock().unwrap(); - assert!(lock.is_none()); - tracing::debug!(target: "sync", %shard_id, part_id, "And `response` is now clear"); - } let mut err_to_retry = None; match external_storage_response { @@ -1233,8 +1220,6 @@ fn check_external_storage_part_response( ) { Ok(_) => { part_download.done = true; - tracing::debug!(target: "sync", %shard_id, part_id, ?part_download, "Got 200 response from external storage and saved state part"); - // TODO } Err(err) => { tracing::warn!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to save a state part"); @@ -1256,7 +1241,6 @@ fn check_external_storage_part_response( tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to get a part from external storage, will retry"); part_download.error = true; } else { - tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, "Checked, ok"); } } From f9b7f24f252af006fe91d368cf3f88eb52137ff9 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 14 Mar 2023 07:28:02 +0100 Subject: [PATCH 18/88] Timeouts --- chain/client/src/sync/state.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 7e7670b0256..4e860c150f0 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -992,8 +992,7 @@ impl StateSync { if !part_download.done { parts_done = false; let prev = part_download.prev_update_time; - let part_timeout = false; // now - prev > self.timeout; - // Retry parts that failed. + let part_timeout = now - prev > self.timeout; // Retry parts that failed. if part_timeout || part_download.error { download_timeout |= part_timeout; part_download.run_me.store(true, Ordering::SeqCst); From 2195391f3522f7f99ea46eb7637d9291e7c8290f Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 14 Mar 2023 14:27:15 +0100 Subject: [PATCH 19/88] Use colour for state sync phase display --- chain/client/src/client.rs | 61 ++++++++++++++++---------------- chain/client/src/client_actor.rs | 14 +++----- chain/client/src/sync/state.rs | 43 ++++++++++------------ 3 files changed, 52 insertions(+), 66 deletions(-) diff --git a/chain/client/src/client.rs b/chain/client/src/client.rs index 6b01941f63f..115090d8f6e 100644 --- a/chain/client/src/client.rs +++ b/chain/client/src/client.rs @@ -1,23 +1,16 @@ //! Client is responsible for tracking the chain, chunks, and producing them when needed. //! This client works completely synchronously and must be operated by some async actor outside. -use std::cmp::max; -use std::collections::{HashMap, HashSet}; -use std::sync::Arc; -use std::time::{Duration, Instant}; - +use crate::adapter::ProcessTxResponse; +use crate::debug::BlockProductionTracker; +use crate::debug::PRODUCTION_TIMES_CACHE_SIZE; +use crate::sync::block::BlockSync; +use crate::sync::epoch::EpochSync; +use crate::sync::header::HeaderSync; +use crate::sync::state::{StateSync, StateSyncResult}; +use crate::{metrics, SyncStatus}; use lru::LruCache; use near_async::messaging::{CanSend, Sender}; -use near_chunks::adapter::ShardsManagerRequestFromClient; -use near_chunks::client::ShardedTransactionPool; -use near_chunks::logic::{ - cares_about_shard_this_or_next_epoch, decode_encoded_chunk, persist_chunk, -}; -use near_client_primitives::debug::ChunkProduction; -use near_primitives::static_clock::StaticClock; -use near_store::metadata::DbKind; -use tracing::{debug, error, info, trace, warn}; - use near_chain::chain::{ ApplyStatePartsRequest, BlockCatchUpRequest, BlockMissingChunks, BlocksCatchUpState, OrphanMissingChunks, StateSplitRequest, TX_ROUTING_HEIGHT_HORIZON, @@ -30,43 +23,47 @@ use near_chain::{ DoneApplyChunkCallback, Doomslug, DoomslugThresholdMode, Provenance, RuntimeWithEpochManagerAdapter, }; -use near_chain_configs::{ClientConfig, UpdateableClientConfig}; +use near_chain_configs::{ClientConfig, LogSummaryStyle, UpdateableClientConfig}; +use near_chunks::adapter::ShardsManagerRequestFromClient; +use near_chunks::client::ShardedTransactionPool; +use near_chunks::logic::{ + cares_about_shard_this_or_next_epoch, decode_encoded_chunk, persist_chunk, +}; use near_chunks::ShardsManager; +use near_client_primitives::debug::ChunkProduction; +use near_client_primitives::types::{Error, ShardSyncDownload, ShardSyncStatus}; +use near_network::types::{AccountKeys, ChainInfo, PeerManagerMessageRequest, SetChainInfo}; use near_network::types::{ HighestHeightPeerInfo, NetworkRequests, PeerManagerAdapter, ReasonForBan, }; +use near_o11y::log_assert; use near_primitives::block::{Approval, ApprovalInner, ApprovalMessage, Block, BlockHeader, Tip}; +use near_primitives::block_header::ApprovalType; use near_primitives::challenge::{Challenge, ChallengeBody}; +use near_primitives::epoch_manager::RngSeed; use near_primitives::hash::CryptoHash; use near_primitives::merkle::{merklize, MerklePath, PartialMerkleTree}; +use near_primitives::network::PeerId; use near_primitives::receipt::Receipt; use near_primitives::sharding::{ ChunkHash, EncodedShardChunk, PartialEncodedChunk, ReedSolomonWrapper, ShardChunk, ShardChunkHeader, ShardInfo, }; +use near_primitives::static_clock::StaticClock; use near_primitives::transaction::SignedTransaction; use near_primitives::types::chunk_extra::ChunkExtra; use near_primitives::types::{AccountId, ApprovalStake, BlockHeight, EpochId, NumBlocks, ShardId}; use near_primitives::unwrap_or_return; use near_primitives::utils::MaybeValidated; use near_primitives::validator_signer::ValidatorSigner; - -use crate::adapter::ProcessTxResponse; -use crate::debug::BlockProductionTracker; -use crate::debug::PRODUCTION_TIMES_CACHE_SIZE; -use crate::sync::block::BlockSync; -use crate::sync::epoch::EpochSync; -use crate::sync::header::HeaderSync; -use crate::sync::state::{StateSync, StateSyncResult}; -use crate::{metrics, SyncStatus}; -use near_client_primitives::types::{Error, ShardSyncDownload, ShardSyncStatus}; -use near_network::types::{AccountKeys, ChainInfo, PeerManagerMessageRequest, SetChainInfo}; -use near_o11y::log_assert; -use near_primitives::block_header::ApprovalType; -use near_primitives::epoch_manager::RngSeed; -use near_primitives::network::PeerId; use near_primitives::version::PROTOCOL_VERSION; use near_primitives::views::{CatchupStatusView, DroppedReason}; +use near_store::metadata::DbKind; +use std::cmp::max; +use std::collections::{HashMap, HashSet}; +use std::sync::Arc; +use std::time::{Duration, Instant}; +use tracing::{debug, error, info, trace, warn}; const NUM_REBROADCAST_BLOCKS: usize = 30; const CHUNK_HEADERS_FOR_INCLUSION_CACHE_SIZE: usize = 2048; @@ -2142,6 +2139,7 @@ impl Client { "Catchup me: {:?}: sync_hash: {:?}, sync_info: {:?}", me, sync_hash, new_shard_sync ); + let use_colour = matches!(self.config.log_summary_style, LogSummaryStyle::Colored); match state_sync.run( me, sync_hash, @@ -2152,6 +2150,7 @@ impl Client { state_sync_info.shards.iter().map(|tuple| tuple.0).collect(), state_parts_task_scheduler, state_split_scheduler, + use_colour, )? { StateSyncResult::Unchanged => {} StateSyncResult::Changed(fetch_block) => { diff --git a/chain/client/src/client_actor.rs b/chain/client/src/client_actor.rs index 6240c621c6e..f69264dae55 100644 --- a/chain/client/src/client_actor.rs +++ b/chain/client/src/client_actor.rs @@ -33,7 +33,6 @@ use near_chain::{ ChainGenesis, DoneApplyChunkCallback, Provenance, RuntimeWithEpochManagerAdapter, }; use near_chain_configs::{ClientConfig, LogSummaryStyle}; -use near_chain_primitives::error::EpochErrorResultToChainError; use near_chunks::adapter::ShardsManagerRequestFromClient; use near_chunks::client::ShardsManagerResponse; use near_chunks::logic::cares_about_shard_this_or_next_epoch; @@ -655,8 +654,7 @@ impl Handler> for ClientActor { let validators: Vec = self .client .runtime_adapter - .get_epoch_block_producers_ordered(&head.epoch_id, &head.last_block_hash) - .into_chain_error()? + .get_epoch_block_producers_ordered(&head.epoch_id, &head.last_block_hash)? .into_iter() .map(|(validator_stake, is_slashed)| ValidatorInfo { account_id: validator_stake.take_account_id(), @@ -667,11 +665,8 @@ impl Handler> for ClientActor { let epoch_start_height = self.client.runtime_adapter.get_epoch_start_height(&head.last_block_hash).ok(); - let protocol_version = self - .client - .runtime_adapter - .get_epoch_protocol_version(&head.epoch_id) - .into_chain_error()?; + let protocol_version = + self.client.runtime_adapter.get_epoch_protocol_version(&head.epoch_id)?; let node_public_key = self.node_id.public_key().clone(); let (validator_account_id, validator_public_key) = match &self.client.validator_signer { @@ -1326,8 +1321,7 @@ impl ClientActor { .runtime_adapter .get_epoch_start_height(&last_final_hash) .unwrap_or(last_final_block_height); - let last_final_block_height_in_epoch = - last_final_block_height.checked_sub(epoch_start_height); + let last_final_block_height_in_epoch = last_final_block_height - epoch_start_height; self.info_helper.block_processed( gas_used, diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 4e860c150f0..df4cdeb6d47 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -21,6 +21,7 @@ //! use ansi_term::Color::{Purple, Yellow}; +use ansi_term::Style; use chrono::{DateTime, Duration, Utc}; use futures::{future, FutureExt}; use near_async::messaging::CanSendAsync; @@ -246,6 +247,7 @@ impl StateSync { now: DateTime, state_parts_task_scheduler: &dyn Fn(ApplyStatePartsRequest), state_split_scheduler: &dyn Fn(StateSplitRequest), + use_colour: bool, ) -> Result<(bool, bool), near_chain::Error> { let mut all_done = true; let mut update_sync_status = false; @@ -350,26 +352,7 @@ impl StateSync { %shard_id, %sync_hash, ?me, - phase = ?match shard_sync_download.status { - ShardSyncStatus::StateDownloadHeader => format!("{} requests sent {}, last target {:?}", - Purple.bold().paint("HEADER".to_string()), - shard_sync_download.downloads[0].state_requests_count, - shard_sync_download.downloads[0].last_target), - ShardSyncStatus::StateDownloadParts => { let mut text = "".to_string(); - for (i, download) in shard_sync_download.downloads.iter().enumerate() { - text.push_str(&format!("[{}: {}, {}, {:?}] ", - Yellow.bold().paint(i.to_string()), - download.done, - download.state_requests_count, - download.last_target)); - } - format!("{} [{}: is_done, requests sent, last target] {}", - Purple.bold().paint("PARTS"), - Yellow.bold().paint("part_id"), - text) - } - _ => unreachable!("timeout cannot happen when all state is downloaded"), - }, + phase = format_shard_sync_phase(&shard_sync_download, use_colour), "State sync status"); } @@ -805,6 +788,7 @@ impl StateSync { tracking_shards: Vec, state_parts_task_scheduler: &dyn Fn(ApplyStatePartsRequest), state_split_scheduler: &dyn Fn(StateSplitRequest), + use_colour: bool, ) -> Result { let _span = tracing::debug_span!(target: "sync", "run", sync = "StateSync").entered(); tracing::debug!(target: "sync", %sync_hash, ?tracking_shards, "syncing state"); @@ -838,6 +822,7 @@ impl StateSync { now, state_parts_task_scheduler, state_split_scheduler, + use_colour, )?; if have_block && all_done { @@ -1257,12 +1242,20 @@ pub fn s3_location( ) } +fn paint(s: &str, colour: Style, use_colour: bool) -> String { + if use_colour { + colour.paint(s).to_string() + } else { + s.to_string() + } +} + /// Formats the given ShardSyncDownload for logging. -fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload) -> String { +fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload, use_colour: bool) -> String { match shard_sync_download.status { ShardSyncStatus::StateDownloadHeader => format!( "{} requests sent {}, last target {:?}", - Purple.bold().paint("HEADER".to_string()), + paint("HEADER", Purple.bold(), use_colour), shard_sync_download.downloads[0].state_requests_count, shard_sync_download.downloads[0].last_target ), @@ -1271,7 +1264,7 @@ fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload) -> String { for (i, download) in shard_sync_download.downloads.iter().enumerate() { text.push_str(&format!( "[{}: {}, {}, {:?}] ", - Yellow.bold().paint(i.to_string()), + paint(&i.to_string(), Yellow.bold(), use_colour), download.done, download.state_requests_count, download.last_target @@ -1279,8 +1272,8 @@ fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload) -> String { } format!( "{} [{}: is_done, requests sent, last target] {}", - Purple.bold().paint("PARTS"), - Yellow.bold().paint("part_id"), + paint("PARTS", Purple.bold(), use_colour), + paint("part_id", Yellow.bold(), use_colour), text ) } From 8b1f311e31ae3a4c60c389fd3935a0b611d8a30e Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 15 Mar 2023 14:45:08 +0100 Subject: [PATCH 20/88] Fix DB corruption --- chain/client/src/client_actor.rs | 372 ++++++++++++++----------------- 1 file changed, 163 insertions(+), 209 deletions(-) diff --git a/chain/client/src/client_actor.rs b/chain/client/src/client_actor.rs index f69264dae55..91d60fe908e 100644 --- a/chain/client/src/client_actor.rs +++ b/chain/client/src/client_actor.rs @@ -47,7 +47,6 @@ use near_network::types::{ use near_o11y::{handler_debug_span, OpenTelemetrySpanExt, WithSpanContext, WithSpanContextExt}; use near_performance_metrics; use near_performance_metrics_macros::perf; -use near_primitives::block::Tip; use near_primitives::block_header::ApprovalType; use near_primitives::epoch_manager::RngSeed; use near_primitives::hash::CryptoHash; @@ -66,12 +65,11 @@ use near_telemetry::TelemetryActor; use rand::seq::SliceRandom; use rand::{thread_rng, Rng}; use std::collections::HashMap; -use std::fmt; use std::sync::Arc; use std::thread; use std::time::{Duration, Instant}; use tokio::sync::broadcast; -use tracing::{debug, error, info, trace, warn}; +use tracing::{error, info, warn}; /// Multiplier on `max_block_time` to wait until deciding that chain stalled. const STATUS_WAIT_TIME_MULTIPLIER: u64 = 10; @@ -258,7 +256,7 @@ impl Actor for ClientActor { self.catchup(ctx); if let Err(err) = self.client.send_network_chain_info() { - error!(target: "client", ?err, "Failed to update network chain info"); + tracing::error!(target: "client", ?err, "Failed to update network chain info"); } } } @@ -502,7 +500,7 @@ impl Handler> for ClientActor { fn handle(&mut self, msg: WithSpanContext, ctx: &mut Context) { self.wrap(msg, ctx, "BlockApproval", |this, msg| { let BlockApproval(approval, peer_id) = msg; - debug!(target: "client", "Receive approval {:?} from peer {:?}", approval, peer_id); + tracing::debug!(target: "client", "Receive approval {:?} from peer {:?}", approval, peer_id); this.client.collect_block_approval(&approval, ApprovalType::PeerApproval(peer_id)); }) } @@ -520,7 +518,7 @@ impl Handler> for ClientActor { let hash = state_response_info.sync_hash(); let state_response = state_response_info.take_state_response(); - trace!(target: "sync", "Received state response shard_id: {} sync_hash: {:?} part(id/size): {:?}", + tracing::trace!(target: "sync", "Received state response shard_id: {} sync_hash: {:?} part(id/size): {:?}", shard_id, hash, state_response.part().as_ref().map(|(part_id, data)| (part_id, data.len())) @@ -811,54 +809,13 @@ impl Handler> for ClientActor { } } -#[derive(Debug)] -enum SyncRequirement { - SyncNeeded { peer_id: PeerId, highest_height: BlockHeight, head: Tip }, - AlreadyCaughtUp { peer_id: PeerId, highest_height: BlockHeight, head: Tip }, - NoPeers, - AdvHeaderSyncDisabled, -} - -impl SyncRequirement { - fn sync_needed(&self) -> bool { - matches!(self, Self::SyncNeeded { .. }) - } -} - -impl fmt::Display for SyncRequirement { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match self { - Self::SyncNeeded { peer_id, highest_height, head: my_head } => write!( - f, - "sync needed at #{} [{}]. highest height peer: {} at #{}", - my_head.height, - format_hash(my_head.last_block_hash), - peer_id, - highest_height - ), - Self::AlreadyCaughtUp { peer_id, highest_height, head: my_head } => write!( - f, - "synced at #{} [{}]. highest height peer: {} at #{}", - my_head.height, - format_hash(my_head.last_block_hash), - peer_id, - highest_height - ), - Self::NoPeers => write!(f, "no available peers"), - Self::AdvHeaderSyncDisabled => { - write!(f, "syncing disabled via adv_disable_header_sync") - } - } - } -} - impl ClientActor { /// Check if client Account Id should be sent and send it. /// Account Id is sent when is not current a validator but are becoming a validator soon. fn check_send_announce_account(&mut self, prev_block_hash: CryptoHash) { // If no peers, there is no one to announce to. if self.network_info.num_connected_peers == 0 { - debug!(target: "client", "No peers: skip account announce"); + tracing::debug!(target: "client", "No peers: skip account announce"); return; } @@ -878,7 +835,7 @@ impl ClientActor { } } - debug!(target: "client", "Check announce account for {}, last announce time {:?}", validator_signer.validator_id(), self.last_validator_announce_time); + tracing::debug!(target: "client", "Check announce account for {}, last announce time {:?}", validator_signer.validator_id(), self.last_validator_announce_time); // Announce AccountId if client is becoming a validator soon. let next_epoch_id = unwrap_or_return!(self @@ -888,7 +845,7 @@ impl ClientActor { // Check client is part of the futures validators if self.client.is_validator(&next_epoch_id, &prev_block_hash) { - debug!(target: "client", "Sending announce account for {}", validator_signer.validator_id()); + tracing::debug!(target: "client", "Sending announce account for {}", validator_signer.validator_id()); self.last_validator_announce_time = Some(now); let signature = validator_signer.sign_account_announce( @@ -991,7 +948,7 @@ impl ClientActor { let _span = tracing::debug_span!(target: "client", "handle_block_production").entered(); // If syncing, don't try to produce blocks. if self.client.sync_status.is_syncing() { - debug!(target:"client", sync_status=?self.client.sync_status, "Syncing - block production disabled"); + tracing::debug!(target:"client", sync_status=?self.client.sync_status, "Syncing - block production disabled"); return Ok(()); } @@ -1022,9 +979,9 @@ impl ClientActor { // We try to produce block for multiple heights (up to the highest height for which we've seen 2/3 of approvals). if latest_known.height + 1 <= self.client.doomslug.get_largest_height_crossing_threshold() { - debug!(target: "client", "Considering blocks for production between {} and {} ", latest_known.height + 1, self.client.doomslug.get_largest_height_crossing_threshold()); + tracing::debug!(target: "client", "Considering blocks for production between {} and {} ", latest_known.height + 1, self.client.doomslug.get_largest_height_crossing_threshold()); } else { - debug!(target: "client", "Cannot produce any block: not enough approvals beyond {}", latest_known.height); + tracing::debug!(target: "client", "Cannot produce any block: not enough approvals beyond {}", latest_known.height); } let me = if let Some(me) = &self.client.validator_signer { @@ -1373,7 +1330,7 @@ impl ClientActor { error!(target: "client", "Error processing sync blocks: {}", err); false } else { - debug!(target: "client", "Block headers refused by chain: {}", err); + tracing::debug!(target: "client", "Block headers refused by chain: {}", err); true } } @@ -1382,13 +1339,9 @@ impl ClientActor { /// Check whether need to (continue) sync. /// Also return higher height with known peers at that height. - fn syncing_info(&self) -> Result { - if self.adv.disable_header_sync() { - return Ok(SyncRequirement::AdvHeaderSyncDisabled); - } - + fn syncing_info(&self) -> Result<(bool, u64), near_chain::Error> { let head = self.client.chain.head()?; - let is_syncing = self.client.sync_status.is_syncing(); + let mut is_syncing = self.client.sync_status.is_syncing(); // Only consider peers whose latest block is not invalid blocks let eligible_peers: Vec<_> = self @@ -1402,25 +1355,39 @@ impl ClientActor { let peer_info = if let Some(peer_info) = eligible_peers.choose(&mut thread_rng()) { peer_info } else { - return Ok(SyncRequirement::NoPeers); + if !self.client.config.skip_sync_wait { + warn!(target: "client", "Sync: no peers available, disabling sync"); + } + return Ok((false, 0)); }; - let peer_id = peer_info.peer_info.id.clone(); - let highest_height = peer_info.highest_block_height; - if is_syncing { - if highest_height <= head.height { - Ok(SyncRequirement::AlreadyCaughtUp { peer_id, highest_height, head }) - } else { - Ok(SyncRequirement::SyncNeeded { peer_id, highest_height, head }) + if peer_info.highest_block_height <= head.height { + info!(target: "client", "Sync: synced at {} [{}], {}, highest height peer: {}", + head.height, format_hash(head.last_block_hash), + peer_info.peer_info.id, peer_info.highest_block_height, + ); + is_syncing = false; } } else { - if highest_height > head.height + self.client.config.sync_height_threshold { - Ok(SyncRequirement::SyncNeeded { peer_id, highest_height, head }) - } else { - Ok(SyncRequirement::AlreadyCaughtUp { peer_id, highest_height, head }) + if peer_info.highest_block_height + > head.height + self.client.config.sync_height_threshold + { + info!( + target: "client", + "Sync: height: {}, peer id/height: {}/{}, enabling sync", + head.height, + peer_info.peer_info.id, + peer_info.highest_block_height, + ); + is_syncing = true; } } + Ok((is_syncing, peer_info.highest_block_height)) + } + + fn needs_syncing(&self, needs_syncing: bool) -> bool { + !self.adv.disable_header_sync() && needs_syncing } fn start_flat_storage_creation(&mut self, ctx: &mut Context) { @@ -1545,8 +1512,8 @@ impl ClientActor { } fn sync_wait_period(&self) -> Duration { - if let Ok(sync) = self.syncing_info() { - if !sync.sync_needed() { + if let Ok((needs_syncing, _)) = self.syncing_info() { + if !self.needs_syncing(needs_syncing) { // If we don't need syncing - retry the sync call rarely. self.client.config.sync_check_period } else { @@ -1574,149 +1541,136 @@ impl ClientActor { })); let currently_syncing = self.client.sync_status.is_syncing(); - let sync = unwrap_and_report!(self.syncing_info()); - - match sync { - SyncRequirement::AlreadyCaughtUp { .. } - | SyncRequirement::NoPeers - | SyncRequirement::AdvHeaderSyncDisabled => { - if currently_syncing { - info!(target: "client", "disabling sync: {}", &sync); - self.client.sync_status = SyncStatus::NoSync; - - // Initial transition out of "syncing" state. - // Announce this client's account id if their epoch is coming up. - let head = unwrap_and_report!(self.client.chain.head()); - self.check_send_announce_account(head.prev_block_hash); - } - } + let (needs_syncing, highest_height) = unwrap_and_report!(self.syncing_info()); + + if !self.needs_syncing(needs_syncing) { + if currently_syncing { + tracing::debug!( + target: "client", + "{:?} transitions to no sync", + self.client.validator_signer.as_ref().map(|vs| vs.validator_id()), + ); + self.client.sync_status = SyncStatus::NoSync; - SyncRequirement::SyncNeeded { highest_height, .. } => { - if !currently_syncing { - info!( - target: "client", - "enabling sync: {}", &sync, - ); + // Initial transition out of "syncing" state. + // Announce this client's account id if their epoch is coming up. + let head = unwrap_and_report!(self.client.chain.head()); + self.check_send_announce_account(head.prev_block_hash); + } + } else { + // Run each step of syncing separately. + unwrap_and_report!(self.client.header_sync.run( + &mut self.client.sync_status, + &mut self.client.chain, + highest_height, + &self.network_info.highest_height_peers + )); + // Only body / state sync if header height is close to the latest. + let header_head = unwrap_and_report!(self.client.chain.header_head()); + + // Sync state if already running sync state or if block sync is too far. + let sync_state = match self.client.sync_status { + SyncStatus::StateSync(_, _) => true, + _ if header_head.height + >= highest_height + .saturating_sub(self.client.config.block_header_fetch_horizon) => + { + unwrap_and_report!(self.client.block_sync.run( + &mut self.client.sync_status, + &self.client.chain, + highest_height, + &self.network_info.highest_height_peers + )) + } + _ => false, + }; + if sync_state { + let (sync_hash, mut new_shard_sync, just_enter_state_sync) = + match &self.client.sync_status { + SyncStatus::StateSync(sync_hash, shard_sync) => { + (*sync_hash, shard_sync.clone(), false) + } + _ => { + let sync_hash = unwrap_and_report!(self.find_sync_hash()); + (sync_hash, HashMap::default(), true) + } + }; + + let me = self.client.validator_signer.as_ref().map(|x| x.validator_id().clone()); + let block_header = + unwrap_and_report!(self.client.chain.get_block_header(&sync_hash)); + let prev_hash = *block_header.prev_hash(); + let epoch_id = + self.client.chain.get_block_header(&sync_hash).unwrap().epoch_id().clone(); + let shards_to_sync = + (0..self.client.runtime_adapter.num_shards(&epoch_id).unwrap()) + .filter(|x| { + cares_about_shard_this_or_next_epoch( + me.as_ref(), + &prev_hash, + *x, + true, + self.client.runtime_adapter.as_ref(), + ) + }) + .collect(); + + if !self.client.config.archive && just_enter_state_sync { + unwrap_and_report!(self.client.chain.reset_data_pre_state_sync(sync_hash)); } - // Run each step of syncing separately. - unwrap_and_report!(self.client.header_sync.run( - &mut self.client.sync_status, - &mut self.client.chain, - highest_height, - &self.network_info.highest_height_peers - )); - // Only body / state sync if header height is close to the latest. - let header_head = unwrap_and_report!(self.client.chain.header_head()); - - // Sync state if already running sync state or if block sync is too far. - let sync_state = match self.client.sync_status { - SyncStatus::StateSync(_, _) => true, - _ if header_head.height - >= highest_height - .saturating_sub(self.client.config.block_header_fetch_horizon) => - { - unwrap_and_report!(self.client.block_sync.run( - &mut self.client.sync_status, - &self.client.chain, - highest_height, - &self.network_info.highest_height_peers - )) - } - _ => false, - }; - if sync_state { - let (sync_hash, mut new_shard_sync, just_enter_state_sync) = - match &self.client.sync_status { - SyncStatus::StateSync(sync_hash, shard_sync) => { - (*sync_hash, shard_sync.clone(), false) - } - _ => { - let sync_hash = unwrap_and_report!(self.find_sync_hash()); - (sync_hash, HashMap::default(), true) - } - }; - - let me = - self.client.validator_signer.as_ref().map(|x| x.validator_id().clone()); - let block_header = - unwrap_and_report!(self.client.chain.get_block_header(&sync_hash)); - let prev_hash = *block_header.prev_hash(); - let epoch_id = - self.client.chain.get_block_header(&sync_hash).unwrap().epoch_id().clone(); - let shards_to_sync = - (0..self.client.runtime_adapter.num_shards(&epoch_id).unwrap()) - .filter(|x| { - cares_about_shard_this_or_next_epoch( - me.as_ref(), - &prev_hash, - *x, - true, - &self.client.shard_tracker, - ) - }) - .collect(); - - if !self.client.config.archive && just_enter_state_sync { - unwrap_and_report!(self.client.chain.reset_data_pre_state_sync(sync_hash)); - } - let use_colour = - matches!(self.client.config.log_summary_style, LogSummaryStyle::Colored); - match unwrap_and_report!(self.client.state_sync.run( - &me, - sync_hash, - &mut new_shard_sync, - &mut self.client.chain, - &self.client.runtime_adapter, - &self.network_info.highest_height_peers, - shards_to_sync, - &self.state_parts_task_scheduler, - &self.state_split_scheduler, - use_colour, - )) { - StateSyncResult::Unchanged => (), - StateSyncResult::Changed(fetch_block) => { - self.client.sync_status = - SyncStatus::StateSync(sync_hash, new_shard_sync); - if fetch_block { - if let Some(peer_info) = - self.network_info.highest_height_peers.choose(&mut thread_rng()) - { - let id = peer_info.peer_info.id.clone(); - - if let Ok(header) = - self.client.chain.get_block_header(&sync_hash) + let use_colour = + matches!(self.client.config.log_summary_style, LogSummaryStyle::Colored); + match unwrap_and_report!(self.client.state_sync.run( + &me, + sync_hash, + &mut new_shard_sync, + &mut self.client.chain, + &self.client.runtime_adapter, + &self.network_info.highest_height_peers, + shards_to_sync, + &self.state_parts_task_scheduler, + &self.state_split_scheduler, + use_colour, + )) { + StateSyncResult::Unchanged => (), + StateSyncResult::Changed(fetch_block) => { + self.client.sync_status = SyncStatus::StateSync(sync_hash, new_shard_sync); + if fetch_block { + if let Some(peer_info) = + self.network_info.highest_height_peers.choose(&mut thread_rng()) + { + let id = peer_info.peer_info.id.clone(); + + if let Ok(header) = self.client.chain.get_block_header(&sync_hash) { + for hash in + vec![*header.prev_hash(), *header.hash()].into_iter() { - for hash in - vec![*header.prev_hash(), *header.hash()].into_iter() - { - self.client.request_block(hash, id.clone()); - } + self.client.request_block(hash, id.clone()); } } } } - StateSyncResult::Completed => { - info!(target: "sync", "State sync: all shards are done"); - - let mut block_processing_artifacts = BlockProcessingArtifact::default(); - - unwrap_and_report!(self.client.chain.reset_heads_post_state_sync( - &me, - sync_hash, - &mut block_processing_artifacts, - self.get_apply_chunks_done_callback(), - )); - - self.client - .process_block_processing_artifact(block_processing_artifacts); - - self.client.sync_status = SyncStatus::BodySync { - start_height: 0, - current_height: 0, - highest_height: 0, - }; - } + } + StateSyncResult::Completed => { + info!(target: "sync", "State sync: all shards are done"); + + let mut block_processing_artifacts = BlockProcessingArtifact::default(); + + unwrap_and_report!(self.client.chain.reset_heads_post_state_sync( + &me, + sync_hash, + &mut block_processing_artifacts, + self.get_apply_chunks_done_callback(), + )); + + self.client.process_block_processing_artifact(block_processing_artifacts); + + self.client.sync_status = SyncStatus::BodySync { + start_height: 0, + current_height: 0, + highest_height: 0, + }; } } } From b224412b9d69ece71ecd9ad484ca20606931f243 Mon Sep 17 00:00:00 2001 From: wacban Date: Thu, 2 Mar 2023 12:11:48 +0000 Subject: [PATCH 21/88] feat: use split store in the view client (#8656) - use split store in the view client - when configured - removed the get_store(Temperature) method and replaced usages with safer get_cold_store and get_hot_store - added SplitStorageConfig and used it to configure a few things --- nearcore/src/config.rs | 48 +++++- nearcore/src/lib.rs | 78 ++++++---- nearcore/src/runtime/mod.rs | 131 +++++++++++------ tools/cold-store/src/cli.rs | 268 ++-------------------------------- tools/state-viewer/src/cli.rs | 5 +- 5 files changed, 192 insertions(+), 338 deletions(-) diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index 56be887c4cf..f8bc90dc195 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -337,7 +337,9 @@ pub struct Config { /// This feature is under development, do not use in production. #[serde(default, skip_serializing_if = "Option::is_none")] pub cold_store: Option, - + /// Configuration for the + #[serde(default, skip_serializing_if = "Option::is_none")] + pub split_storage: Option, // TODO(mina86): Remove those two altogether at some point. We need to be // somewhat careful though and make sure that we don’t start silently // ignoring this option without users setting corresponding store option. @@ -388,12 +390,56 @@ impl Default for Config { use_db_migration_snapshot: None, store: near_store::StoreConfig::default(), cold_store: None, + split_storage: None, expected_shutdown: None, state_sync: None, } } } +fn default_enable_split_storage_view_client() -> bool { + false +} + +fn default_cold_store_initial_migration_batch_size() -> usize { + 500_000_000 +} + +fn default_cold_store_initial_migration_loop_sleep_duration() -> Duration { + Duration::from_secs(30) +} + +fn default_cold_store_loop_sleep_duration() -> Duration { + Duration::from_secs(1) +} + +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] +pub struct SplitStorageConfig { + #[serde(default = "default_enable_split_storage_view_client")] + pub enable_split_storage_view_client: bool, + + #[serde(default = "default_cold_store_initial_migration_batch_size")] + pub cold_store_initial_migration_batch_size: usize, + #[serde(default = "default_cold_store_initial_migration_loop_sleep_duration")] + pub cold_store_initial_migration_loop_sleep_duration: Duration, + + #[serde(default = "default_cold_store_loop_sleep_duration")] + pub cold_store_loop_sleep_duration: Duration, +} + +impl Default for SplitStorageConfig { + fn default() -> Self { + SplitStorageConfig { + enable_split_storage_view_client: default_enable_split_storage_view_client(), + cold_store_initial_migration_batch_size: + default_cold_store_initial_migration_batch_size(), + cold_store_initial_migration_loop_sleep_duration: + default_cold_store_initial_migration_loop_sleep_duration(), + cold_store_loop_sleep_duration: default_cold_store_loop_sleep_duration(), + } + } +} + impl Config { /// load Config from config.json without panic. Do semantic validation on field values. /// If config file issues occur, a ValidationError::ConfigFileError will be returned; diff --git a/nearcore/src/lib.rs b/nearcore/src/lib.rs index 7ea9b4f5eed..7498a01830d 100644 --- a/nearcore/src/lib.rs +++ b/nearcore/src/lib.rs @@ -2,7 +2,6 @@ use crate::cold_storage::spawn_cold_store_loop; pub use crate::config::{init_configs, load_config, load_test_config, NearConfig, NEAR_BASE}; pub use crate::runtime::NightshadeRuntime; pub use crate::shard_tracker::TrackedConfig; -use crate::state_sync::{spawn_state_sync_dump, StateSyncDumpHandle}; use actix::{Actor, Addr}; use actix_rt::ArbiterHandle; use actix_web; @@ -13,16 +12,17 @@ use near_async::messaging::{IntoSender, LateBoundSender}; use near_chain::{Chain, ChainGenesis}; use near_chunks::shards_manager_actor::start_shards_manager; use near_client::{start_client, start_view_client, ClientActor, ConfigUpdater, ViewClientActor}; +use near_primitives::time; + use near_network::PeerManagerActor; use near_primitives::block::GenesisId; -use near_primitives::time; -use near_store::{DBCol, Mode, NodeStorage, StoreOpenerError, Temperature}; +use near_store::metadata::DbKind; +use near_store::{DBCol, Mode, NodeStorage, Store, StoreOpenerError}; use near_telemetry::TelemetryActor; use std::path::{Path, PathBuf}; use std::sync::Arc; use tokio::sync::broadcast; -use tracing::info; - +use tracing::{info, trace}; pub mod append_only_map; mod cold_storage; pub mod config; @@ -33,7 +33,6 @@ mod metrics; pub mod migrations; mod runtime; mod shard_tracker; -mod state_sync; pub fn get_default_home() -> PathBuf { if let Ok(near_home) = std::env::var("NEAR_HOME") { @@ -154,6 +153,33 @@ fn open_storage(home_dir: &Path, near_config: &mut NearConfig) -> anyhow::Result Ok(storage) } +// Safely get the split store while checking that all conditions to use it are met. +fn get_split_store(config: &NearConfig, storage: &NodeStorage) -> anyhow::Result> { + // SplitStore should only be used on archival nodes. + if !config.config.archive { + return Ok(None); + } + + // SplitStore should only be used if cold store is configured. + if config.config.cold_store.is_none() { + return Ok(None); + } + + // SplitStore should only be used in the view client if it is enabled. + if !config.config.split_storage.as_ref().map_or(false, |c| c.enable_split_storage_view_client) { + return Ok(None); + } + + // SplitStore should only be used if the migration is finished. The + // migration to cold store is finished when the db kind of the hot store is + // changed from Archive to Hot. + if storage.get_hot_store().get_db_kind()? != Some(DbKind::Hot) { + return Ok(None); + } + + Ok(storage.get_split_store()) +} + pub struct NearNode { pub client: Addr, pub view_client: Addr, @@ -162,8 +188,6 @@ pub struct NearNode { /// The cold_store_loop_handle will only be set if the cold store is configured. /// It's a handle to a background thread that copies data from the hot store to the cold store. pub cold_store_loop_handle: Option, - /// Contains handles to background threads that may be dumping state to S3. - pub state_sync_dump_handle: Option, } pub fn start_with_config(home_dir: &Path, config: NearConfig) -> anyhow::Result { @@ -180,11 +204,17 @@ pub fn start_with_config_and_synchronization( ) -> anyhow::Result { let store = open_storage(home_dir, &mut config)?; - let runtime = Arc::new(NightshadeRuntime::from_config( - home_dir, - store.get_store(Temperature::Hot), - &config, - )); + let runtime = + Arc::new(NightshadeRuntime::from_config(home_dir, store.get_hot_store(), &config)); + + // Get the split store. If split store is some then create a new runtime for + // the view client. Otherwise just re-use the existing runtime. + let split_store = get_split_store(&config, &store)?; + let view_runtime = if let Some(split_store) = split_store { + Arc::new(NightshadeRuntime::from_config(home_dir, split_store, &config)) + } else { + runtime.clone() + }; let cold_store_loop_handle = spawn_cold_store_loop(&config, &store, runtime.clone())?; @@ -205,14 +235,14 @@ pub fn start_with_config_and_synchronization( let view_client = start_view_client( config.validator_signer.as_ref().map(|signer| signer.validator_id().clone()), chain_genesis.clone(), - runtime.clone(), + view_runtime, network_adapter.clone().into(), config.client_config.clone(), adv.clone(), ); let (client_actor, client_arbiter_handle) = start_client( config.client_config.clone(), - chain_genesis.clone(), + chain_genesis, runtime.clone(), node_id, network_adapter.clone().into(), @@ -225,22 +255,15 @@ pub fn start_with_config_and_synchronization( ); client_adapter_for_shards_manager.bind(client_actor.clone().with_auto_span_context()); let (shards_manager_actor, shards_manager_arbiter_handle) = start_shards_manager( - runtime.clone(), + runtime, network_adapter.as_sender(), client_adapter_for_shards_manager.as_sender(), config.validator_signer.as_ref().map(|signer| signer.validator_id().clone()), - store.get_store(Temperature::Hot), + store.get_hot_store(), config.client_config.chunk_request_retry_period, ); shards_manager_adapter.bind(shards_manager_actor); - let state_sync_dump_handle = spawn_state_sync_dump( - &config, - &chain_genesis.clone(), - runtime.clone(), - config.network_config.node_id().public_key(), - )?; - #[allow(unused_mut)] let mut rpc_servers = Vec::new(); let network_actor = PeerManagerActor::spawn( @@ -281,7 +304,7 @@ pub fn start_with_config_and_synchronization( rpc_servers.shrink_to_fit(); - tracing::trace!(target: "diagnostic", key = "log", "Starting NEAR node with diagnostic activated"); + trace!(target: "diagnostic", key="log", "Starting NEAR node with diagnostic activated"); Ok(NearNode { client: client_actor, @@ -289,7 +312,6 @@ pub fn start_with_config_and_synchronization( rpc_servers, arbiters: vec![client_arbiter_handle, shards_manager_arbiter_handle], cold_store_loop_handle, - state_sync_dump_handle, }) } @@ -334,7 +356,7 @@ pub fn recompress_storage(home_dir: &Path, opts: RecompressOpts) -> anyhow::Resu "Recompressing database"); info!("Opening database at {}", src_path.display()); - let src_store = src_opener.open_in_mode(Mode::ReadOnly)?.get_store(Temperature::Hot); + let src_store = src_opener.open_in_mode(Mode::ReadOnly)?.get_hot_store(); let final_head_height = if skip_columns.contains(&DBCol::PartialChunks) { let tip: Option = @@ -351,7 +373,7 @@ pub fn recompress_storage(home_dir: &Path, opts: RecompressOpts) -> anyhow::Resu }; info!("Creating database at {}", dst_path.display()); - let dst_store = dst_opener.open_in_mode(Mode::Create)?.get_store(Temperature::Hot); + let dst_store = dst_opener.open_in_mode(Mode::Create)?.get_hot_store(); const BATCH_SIZE_BYTES: u64 = 150_000_000; diff --git a/nearcore/src/runtime/mod.rs b/nearcore/src/runtime/mod.rs index 358387ae547..32595e631d1 100644 --- a/nearcore/src/runtime/mod.rs +++ b/nearcore/src/runtime/mod.rs @@ -48,7 +48,10 @@ use near_primitives::views::{ AccessKeyInfoView, CallResult, QueryRequest, QueryResponse, QueryResponseKind, ViewApplyState, ViewStateResult, }; -use near_store::flat::{store_helper, FlatStorage, FlatStorageManager, FlatStorageStatus}; +use near_store::flat_state::ChainAccessForFlatStorage; +use near_store::flat_state::{ + store_helper, FlatStateFactory, FlatStorageCreationStatus, FlatStorageState, +}; use near_store::metadata::DbKind; use near_store::split_state::get_delayed_receipts; use near_store::{ @@ -86,7 +89,7 @@ pub struct NightshadeRuntime { store: Store, tries: ShardTries, trie_viewer: TrieViewer, - flat_storage_manager: FlatStorageManager, + flat_state_factory: FlatStateFactory, pub runtime: Runtime, epoch_manager: EpochManagerHandle, shard_tracker: ShardTracker, @@ -142,12 +145,12 @@ impl NightshadeRuntime { ); let state_roots = Self::initialize_genesis_state_if_needed(store.clone(), home_dir, genesis); - let flat_storage_manager = FlatStorageManager::new(store.clone()); + let flat_state_factory = FlatStateFactory::new(store.clone()); let tries = ShardTries::new( store.clone(), trie_config, &genesis_config.shard_layout.get_shard_uids(), - flat_storage_manager.clone(), + flat_state_factory.clone(), ); let epoch_manager = EpochManager::new_from_genesis_config(store.clone().into(), &genesis_config) @@ -163,7 +166,7 @@ impl NightshadeRuntime { trie_viewer, epoch_manager, shard_tracker, - flat_storage_manager, + flat_state_factory, genesis_state_roots: state_roots, migration_data: Arc::new(load_migration_data(&genesis.config.chain_id)), gc_num_epochs_to_keep: gc_num_epochs_to_keep.max(MIN_GC_NUM_EPOCHS_TO_KEEP), @@ -270,7 +273,7 @@ impl NightshadeRuntime { store.clone(), TrieConfig::default(), &genesis.config.shard_layout.get_shard_uids(), - FlatStorageManager::new(store), + FlatStateFactory::new(store), ); let runtime = Runtime::new(); let runtime_config_store = @@ -553,6 +556,11 @@ impl NightshadeRuntime { metrics::APPLY_CHUNK_DELAY .with_label_values(&[&format_total_gas_burnt(total_gas_burnt)]) .observe(elapsed.as_secs_f64()); + if total_gas_burnt > 0 { + metrics::SECONDS_PER_PETAGAS + .with_label_values(&[&shard_id.to_string()]) + .observe(elapsed.as_secs_f64() * 1e15 / total_gas_burnt as f64); + } let total_balance_burnt = apply_result .stats .tx_burnt_amount @@ -746,46 +754,43 @@ impl RuntimeAdapter for NightshadeRuntime { Ok(self.tries.get_view_trie_for_shard(shard_uid, state_root)) } - fn get_flat_storage_for_shard(&self, shard_uid: ShardUId) -> Option { - self.flat_storage_manager.get_flat_storage_for_shard(shard_uid) + fn get_flat_storage_for_shard(&self, shard_id: ShardId) -> Option { + self.flat_storage_manager.get_flat_storage_for_shard(shard_id) } - fn get_flat_storage_status(&self, shard_uid: ShardUId) -> FlatStorageStatus { - store_helper::get_flat_storage_status(&self.store, shard_uid) + fn get_flat_storage_creation_status(&self, shard_id: ShardId) -> FlatStorageCreationStatus { + store_helper::get_flat_storage_creation_status(&self.store, shard_id) } // TODO (#7327): consider passing flat storage errors here to handle them gracefully fn create_flat_storage_for_shard(&self, shard_uid: ShardUId) { let flat_storage = FlatStorage::new(self.store.clone(), shard_uid); - self.flat_storage_manager.add_flat_storage_for_shard(shard_uid, flat_storage); + self.flat_storage_manager.add_flat_storage_for_shard(shard_uid.shard_id(), flat_storage); } - fn remove_flat_storage_for_shard( + fn remove_flat_storage_state_for_shard( &self, shard_uid: ShardUId, epoch_id: &EpochId, ) -> Result<(), Error> { let shard_layout = self.get_shard_layout(epoch_id)?; self.flat_storage_manager - .remove_flat_storage_for_shard(shard_uid, shard_layout) + .remove_flat_storage_for_shard(shard_id, shard_layout) .map_err(|e| Error::StorageError(e))?; Ok(()) } - fn set_flat_storage_for_genesis( + fn set_flat_storage_state_for_genesis( &self, genesis_block: &CryptoHash, - genesis_block_height: BlockHeight, genesis_epoch_id: &EpochId, ) -> Result { let mut store_update = self.store.store_update(); for shard_id in 0..self.num_shards(genesis_epoch_id)? { - let shard_uid = self.shard_id_to_uid(shard_id, genesis_epoch_id)?; - self.flat_storage_manager.set_flat_storage_for_genesis( + self.flat_state_factory.set_flat_storage_state_for_genesis( &mut store_update, - shard_uid, + shard_id, genesis_block, - genesis_block_height, ); } Ok(store_update) @@ -1393,10 +1398,8 @@ impl RuntimeAdapter for NightshadeRuntime { let shard_uid = self.get_shard_uid_from_epoch_id(shard_id, epoch_id)?; let mut store_update = tries.store_update(); tries.apply_all(&trie_changes, shard_uid, &mut store_update); - if cfg!(feature = "protocol_feature_flat_state") { - debug!(target: "chain", %shard_id, "Inserting {} values to flat storage", flat_state_delta.len()); - flat_state_delta.apply_to_flat_state(&mut store_update, shard_uid); - } + debug!(target: "chain", %shard_id, "Inserting {} values to flat storage", flat_state_delta.len()); + flat_state_delta.apply_to_flat_state(&mut store_update); self.precompile_contracts(epoch_id, contract_codes)?; Ok(store_update.commit()?) } @@ -1574,7 +1577,6 @@ mod test { use near_epoch_manager::shard_tracker::TrackedConfig; use near_primitives::test_utils::create_test_signer; use near_primitives::types::validator_stake::ValidatorStake; - use near_store::flat::{FlatStateChanges, FlatStateDelta, FlatStateDeltaMetadata}; use num_rational::Ratio; use crate::config::{GenesisExt, TESTING_INIT_BALANCE, TESTING_INIT_STAKE}; @@ -1593,7 +1595,7 @@ mod test { AccountView, CurrentEpochValidatorInfo, EpochValidatorInfo, NextEpochValidatorInfo, ValidatorKickoutView, }; - use near_store::NodeStorage; + use near_store::{flat_state, FlatStateDelta, NodeStorage}; use super::*; @@ -1655,13 +1657,12 @@ mod test { ) .unwrap(); let mut store_update = self.store.store_update(); - let flat_state_changes = - FlatStateChanges::from_state_changes(&result.trie_changes.state_changes()); + let flat_state_delta = + FlatStateDelta::from_state_changes(&result.trie_changes.state_changes()); result.trie_changes.insertions_into(&mut store_update); result.trie_changes.state_changes_into(&mut store_update); - let shard_uid = self.shard_id_to_uid(shard_id, &EpochId::default()).unwrap(); - match self.get_flat_storage_for_shard(shard_uid) { + match self.get_flat_storage_for_shard(shard_id) { Some(flat_storage) => { let delta = FlatStateDelta { changes: flat_state_changes, @@ -1673,7 +1674,9 @@ mod test { }, }, }; - let new_store_update = flat_storage.add_delta(delta).unwrap(); + let new_store_update = flat_storage_state + .add_block(&block_hash, flat_state_delta, block_info) + .unwrap(); store_update.merge(new_store_update); } None => {} @@ -1684,6 +1687,43 @@ mod test { } } + /// Stores chain data for genesis block to initialize flat storage in test environment. + struct MockChainForFlatStorage { + height_to_hashes: HashMap, + blocks: HashMap, + } + + impl ChainAccessForFlatStorage for MockChainForFlatStorage { + fn get_block_info(&self, block_hash: &CryptoHash) -> flat_state::BlockInfo { + self.blocks.get(block_hash).unwrap().clone() + } + + fn get_block_hashes_at_height(&self, block_height: BlockHeight) -> HashSet { + HashSet::from([self.get_block_hash(block_height)]) + } + } + + impl MockChainForFlatStorage { + /// Creates mock chain containing only genesis block data. + pub fn new(genesis_height: BlockHeight, genesis_hash: CryptoHash) -> Self { + Self { + height_to_hashes: HashMap::from([(genesis_height, genesis_hash)]), + blocks: HashMap::from([( + genesis_hash, + flat_state::BlockInfo { + hash: genesis_hash, + height: genesis_height, + prev_hash: CryptoHash::default(), + }, + )]), + } + } + + fn get_block_hash(&self, height: BlockHeight) -> CryptoHash { + *self.height_to_hashes.get(&height).unwrap() + } + } + /// Environment to test runtime behaviour separate from Chain. /// Runtime operates in a mock chain where i-th block is attached to (i-1)-th one, has height `i` and hash /// `hash([i])`. @@ -1787,18 +1827,18 @@ mod test { // Create flat storage. Naturally it happens on Chain creation, but here we test only Runtime behaviour // and use a mock chain, so we need to initialize flat storage manually. - if cfg!(feature = "protocol_feature_flat_state") { - let store_update = runtime - .set_flat_storage_for_genesis(&genesis_hash, 0, &EpochId::default()) - .unwrap(); - store_update.commit().unwrap(); - for shard_id in 0..runtime.num_shards(&EpochId::default()).unwrap() { - let shard_uid = runtime.shard_id_to_uid(shard_id, &EpochId::default()).unwrap(); - assert!(matches!( - runtime.get_flat_storage_status(shard_uid), - FlatStorageStatus::Ready(_) - )); - runtime.create_flat_storage_for_shard(shard_uid); + let store_update = runtime + .set_flat_storage_state_for_genesis(&genesis_hash, &EpochId::default()) + .unwrap(); + store_update.commit().unwrap(); + let mock_chain = MockChainForFlatStorage::new(0, genesis_hash); + for shard_id in 0..runtime.num_shards(&EpochId::default()).unwrap() { + let status = runtime.get_flat_storage_creation_status(shard_id); + if cfg!(feature = "protocol_feature_flat_state") { + assert_eq!(status, FlatStorageCreationStatus::Ready); + runtime.create_flat_storage_state_for_shard(shard_id, 0, &mock_chain); + } else { + assert_eq!(status, FlatStorageCreationStatus::DontCreate); } } @@ -3066,16 +3106,13 @@ mod test { .runtime .get_trie_for_shard(0, &env.head.prev_block_hash, Trie::EMPTY_ROOT, true) .unwrap(); - assert_eq!( - trie.flat_storage_chunk_view.is_some(), - cfg!(feature = "protocol_feature_flat_state") - ); + assert_eq!(trie.flat_state.is_some(), cfg!(feature = "protocol_feature_flat_state")); let trie = env .runtime .get_view_trie_for_shard(0, &env.head.prev_block_hash, Trie::EMPTY_ROOT) .unwrap(); - assert!(trie.flat_storage_chunk_view.is_none()); + assert!(trie.flat_state.is_none()); } /// Check that querying trie and flat state gives the same result. diff --git a/tools/cold-store/src/cli.rs b/tools/cold-store/src/cli.rs index a6a2f976f33..a5f11780d03 100644 --- a/tools/cold-store/src/cli.rs +++ b/tools/cold-store/src/cli.rs @@ -1,16 +1,13 @@ -use crate::cli::SubCommand::CheckStateRoot; use anyhow; -use anyhow::Context; use clap; use near_epoch_manager::EpochManagerAdapter; use near_primitives::block::Tip; use near_primitives::hash::CryptoHash; use near_store::cold_storage::{copy_all_data_to_cold, update_cold_db, update_cold_head}; use near_store::metadata::DbKind; -use near_store::{DBCol, NodeStorage, Store, StoreOpener}; +use near_store::{DBCol, NodeStorage, Store}; use near_store::{COLD_HEAD_KEY, FINAL_HEAD_KEY, HEAD_KEY, TAIL_KEY}; use nearcore::{NearConfig, NightshadeRuntime}; -use rand::seq::SliceRandom; use std::io::Result; use std::path::Path; use std::sync::Arc; @@ -18,10 +15,6 @@ use strum::IntoEnumIterator; #[derive(clap::Parser)] pub struct ColdStoreCommand { - /// By default state viewer opens rocks DB in the read only mode, which allows it to run - /// multiple instances in parallel and be sure that no unintended changes get written to the DB. - #[clap(long, short = 'w')] - readwrite: bool, #[clap(subcommand)] subcmd: SubCommand, } @@ -46,28 +39,23 @@ enum SubCommand { /// - config.cold_store.path points to an existing database with kind Cold /// - store_relative_path points to an existing database with kind Rpc PrepareHot(PrepareHotCmd), - /// Traverse trie and check that every node is in cold db. - /// Can start from given state_root or compute previous roots for every chunk in provided block - /// and use them as starting point. - /// You can provide maximum depth and/or maximum number of vertices to traverse for each root. - /// Trie is traversed using DFS with randomly shuffled kids for every node. - CheckStateRoot(CheckStateRootCmd), } impl ColdStoreCommand { pub fn run(self, home_dir: &Path) -> anyhow::Result<()> { - let mode = - if self.readwrite { near_store::Mode::ReadWrite } else { near_store::Mode::ReadOnly }; - let mut near_config = nearcore::config::load_config( + let near_config = nearcore::config::load_config( &home_dir, near_chain_configs::GenesisValidationMode::Full, ) .unwrap_or_else(|e| panic!("Error loading config: {:#}", e)); - let opener = self.get_opener(home_dir, &mut near_config); - - let storage = - opener.open_in_mode(mode).unwrap_or_else(|e| panic!("Error opening storage: {:#}", e)); + let opener = NodeStorage::opener( + home_dir, + true, + &near_config.config.store, + near_config.config.cold_store.as_ref(), + ); + let storage = opener.open().unwrap_or_else(|e| panic!("Error opening storage: {:#}", e)); let hot_runtime = NightshadeRuntime::from_config(home_dir, storage.get_hot_store(), &near_config); @@ -85,53 +73,8 @@ impl ColdStoreCommand { Ok(()) } SubCommand::PrepareHot(cmd) => cmd.run(&storage, &home_dir, &near_config), - SubCommand::CheckStateRoot(cmd) => cmd.run(&storage), } } - - /// Returns opener suitable for subcommand. - /// If subcommand is CheckStateRoot, creates checkpoint for cold db - /// and modifies `near_config.config.cold_store.path` to path to that checkpoint. - /// Then returns opener for dbs at `store.path` and `cold_store.path`. - pub fn get_opener<'a>( - &'a self, - home_dir: &Path, - near_config: &'a mut NearConfig, - ) -> StoreOpener<'a> { - if !near_config.config.archive { - tracing::warn!("Expected archive option in config to be set to true."); - } - - let opener = NodeStorage::opener( - home_dir, - near_config.config.archive, - &near_config.config.store, - near_config.config.cold_store.as_ref(), - ); - - match self.subcmd { - CheckStateRoot(_) => { - let (hot_snapshot, cold_snapshot) = opener - .create_snapshots(near_store::Mode::ReadOnly) - .expect("Failed to create snapshots"); - if let Some(_) = &hot_snapshot.0 { - hot_snapshot.remove().expect("Failed to remove unnecessary hot snapshot"); - } - if let Some(cold_store_config) = near_config.config.cold_store.as_mut() { - cold_store_config.path = - Some(cold_snapshot.0.clone().expect("cold_snapshot should be Some")); - } - } - _ => {} - } - - NodeStorage::opener( - home_dir, - near_config.config.archive, - &near_config.config.store, - near_config.config.cold_store.as_ref(), - ) - } } #[derive(clap::Parser)] @@ -438,196 +381,3 @@ impl PrepareHotCmd { Ok(()) } } - -/// The StateRootSelector is a subcommand that allows the user to select the state root either by block height or by the state root hash. -#[derive(clap::Subcommand)] -enum StateRootSelector { - Height { height: near_primitives::types::BlockHeight }, - Hash { hash: CryptoHash }, -} - -impl StateRootSelector { - pub fn get_hashes( - &self, - storage: &NodeStorage, - cold_store: &Store, - ) -> anyhow::Result> { - match self { - // If height is provided, calculate previous state roots for this block's chunks. - StateRootSelector::Height { height } => { - let hash_key = { - let height_key = height.to_le_bytes(); - storage - .get_hot_store() - .get(DBCol::BlockHeight, &height_key)? - .ok_or(anyhow::anyhow!( - "Failed to find block hash for height {:?}", - height - ))? - .as_slice() - .to_vec() - }; - let block = cold_store - .get_ser::(DBCol::Block, &hash_key)? - .ok_or(anyhow::anyhow!("Failed to find Block: {:?}", hash_key))?; - let mut hashes = vec![]; - for chunk in block.chunks().iter() { - hashes.push( - cold_store - .get_ser::( - DBCol::Chunks, - chunk.chunk_hash().as_bytes(), - )? - .ok_or(anyhow::anyhow!( - "Failed to find Chunk: {:?}", - chunk.chunk_hash() - ))? - .take_header() - .prev_state_root(), - ); - } - Ok(hashes) - } - // If state root is provided, then just use it. - StateRootSelector::Hash { hash } => Ok(vec![*hash]), - } - } -} - -/// Struct that holds all conditions for node in Trie -/// to be checked by CheckStateRootCmd::check_trie. -#[derive(Debug)] -struct PruneCondition { - /// Maximum depth (measured in number of nodes, not trie key length). - max_depth: Option, - /// Maximum number of nodes checked for each state_root. - max_count: Option, -} - -/// Struct that holds data related to pruning of node in CheckStateRootCmd::check_trie. -#[derive(Debug)] -struct PruneState { - /// Depth of node in trie (measured in number of nodes, not trie key length). - depth: u64, - /// Number of already checked nodes. - count: u64, -} - -impl PruneState { - pub fn new() -> Self { - Self { depth: 0, count: 0 } - } - - /// Return `true` if node should be pruned. - pub fn should_prune(&self, condition: &PruneCondition) -> bool { - if let Some(md) = condition.max_depth { - if self.depth > md { - return true; - } - } - if let Some(mc) = condition.max_count { - if self.count > mc { - return true; - } - } - false - } - - /// Modify self to reflect going down a tree. - /// We increment node count, because we are visiting a new node. - pub fn down(&mut self) { - self.count += 1; - self.depth += 1; - } - - /// Modify self to reflect going up a tree. - /// We do not change node count, because we already visited parent node before. - pub fn up(&mut self) { - self.depth -= 1; - } -} - -#[derive(clap::Args)] -struct CheckStateRootCmd { - /// Maximum depth (measured in number of nodes, not trie key length) for checking trie. - #[clap(long)] - max_depth: Option, - /// Maximum number of nodes checked for each state_root. - #[clap(long)] - max_count: Option, - #[clap(subcommand)] - state_root_selector: StateRootSelector, -} - -impl CheckStateRootCmd { - pub fn run(self, storage: &NodeStorage) -> anyhow::Result<()> { - let cold_store = - storage.get_cold_store().ok_or(anyhow::anyhow!("Cold storage is not configured"))?; - - let hashes = self.state_root_selector.get_hashes(storage, &cold_store)?; - for hash in hashes.iter() { - Self::check_trie( - &cold_store, - &hash, - &mut PruneState::new(), - &PruneCondition { max_depth: self.max_depth, max_count: self.max_count }, - )?; - } - - Ok(()) - } - - /// Check that trie subtree of `hash` is fully present in `store`. - fn check_trie( - store: &Store, - hash: &CryptoHash, - prune_state: &mut PruneState, - prune_condition: &PruneCondition, - ) -> anyhow::Result<()> { - tracing::debug!(target: "check_trie", "Checking {:?} at {:?}", hash, prune_state); - if prune_state.should_prune(prune_condition) { - tracing::debug!(target: "check_trie", "Reached prune condition: {:?}", prune_condition); - return Ok(()); - } - - let bytes = Self::read_state(store, hash.as_ref()) - .with_context(|| format!("Failed to read raw bytes for hash {:?}", hash))? - .with_context(|| format!("Failed to find raw bytes for hash {:?}", hash))?; - let node = near_store::RawTrieNodeWithSize::decode(&bytes)?; - match node.node { - near_store::RawTrieNode::Leaf(..) => { - tracing::debug!(target: "check_trie", "Reached leaf node"); - return Ok(()); - } - near_store::RawTrieNode::Branch(mut children, _) => { - children.0.shuffle(&mut rand::thread_rng()); - for (_, child) in children.iter() { - // Record in prune state that we are visiting a child node - prune_state.down(); - // Visit a child node - Self::check_trie(store, child, prune_state, prune_condition)?; - // Record in prune state that we are returning from a child node - prune_state.up(); - } - } - near_store::RawTrieNode::Extension(_, child) => { - // Record in prune state that we are visiting a child node - prune_state.down(); - // Visit a child node - Self::check_trie(store, &child, prune_state, prune_condition)?; - // Record in prune state that we are returning from a child node - prune_state.up(); - } - } - Ok(()) - } - - fn read_state<'a>( - store: &'a Store, - trie_key: &'a [u8], - ) -> std::io::Result>> { - // As cold db strips shard_uid at the beginning of State key, we can add any 8 u8s as prefix. - let cold_state_key = [&[1; 8], trie_key.as_ref()].concat(); - store.get(DBCol::State, &cold_state_key) - } -} diff --git a/tools/state-viewer/src/cli.rs b/tools/state-viewer/src/cli.rs index baa55daa44a..1b620f84e89 100644 --- a/tools/state-viewer/src/cli.rs +++ b/tools/state-viewer/src/cli.rs @@ -105,8 +105,7 @@ impl StateViewerSubCommand { let storage = store_opener.open_in_mode(mode).unwrap(); let store = match temperature { Temperature::Hot => storage.get_hot_store(), - // Cold store on it's own is useless in majority of subcommands - Temperature::Cold => storage.get_split_store().unwrap(), + Temperature::Cold => storage.get_cold_store().unwrap(), }; match self { @@ -410,7 +409,7 @@ impl DumpTxCmd { #[derive(clap::Args)] pub struct EpochInfoCmd { #[clap(subcommand)] - epoch_selection: crate::epoch_info::EpochSelection, + epoch_selection: epoch_info::EpochSelection, /// Displays kickouts of the given validator and expected and missed blocks and chunks produced. #[clap(long)] validator_account_id: Option, From 5846e1a37aa30a3ce4ded79445a2e008b21d4b0d Mon Sep 17 00:00:00 2001 From: nikurt <86772482+nikurt@users.noreply.github.com> Date: Fri, 10 Mar 2023 10:39:14 +0100 Subject: [PATCH 22/88] feat: Dump state of every epoch to S3 (#8661) * Start a thread per shard to do the dumping * AWS credentials are provided as environment variables: `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` * In `config.json` specify both `config.state_sync.s3_bucket` and `config.state_sync.s3_region` to enable the new behavior. * No changes to the behavior of the node if those options are not enabled in `config.json`. * State is persisted to RocksDB such that restarts of the node are well handled. * Some useful metrics are exported. * The node assumes it's the only node in the this and all alternative universes that does the dumping. * * Unclear how to use multiple nodes to complete the dump faster * TODO: Speed this up by doing things in parallel: obtain parts, upload parts, set tags * * Do we even need tags? --- chain/chain/src/store.rs | 1 + core/chain-configs/src/client_config.rs | 20 +- core/primitives/src/syncing.rs | 13 +- nearcore/src/config.rs | 39 +--- nearcore/src/lib.rs | 24 ++- nearcore/src/state_sync.rs | 260 ++++++++++++++---------- 6 files changed, 198 insertions(+), 159 deletions(-) diff --git a/chain/chain/src/store.rs b/chain/chain/src/store.rs index a59793cbd82..2cbeb18eefb 100644 --- a/chain/chain/src/store.rs +++ b/chain/chain/src/store.rs @@ -842,6 +842,7 @@ impl ChainStore { /// Constructs key 'STATE_SYNC_DUMP:', /// for example 'STATE_SYNC_DUMP:2' for shard_id=2. + /// Doesn't contain epoch_id, because only one dump process per shard is allowed. fn state_sync_dump_progress_key(shard_id: ShardId) -> Vec { let mut key = b"STATE_SYNC_DUMP:".to_vec(); key.extend(shard_id.to_le_bytes()); diff --git a/core/chain-configs/src/client_config.rs b/core/chain-configs/src/client_config.rs index 6d1d6336be6..5e3b8ea0fa6 100644 --- a/core/chain-configs/src/client_config.rs +++ b/core/chain-configs/src/client_config.rs @@ -166,19 +166,15 @@ pub struct ClientConfig { pub client_background_migration_threads: usize, /// Duration to perform background flat storage creation step. pub flat_storage_creation_period: Duration, - /// Whether to enable dumping state of every epoch to S3. - pub state_dump_enabled: bool, + /// If enabled, will dump state of every epoch to external storage. + pub state_sync_dump_enabled: bool, /// S3 bucket for storing state dumps. pub state_sync_s3_bucket: String, /// S3 region for storing state dumps. pub state_sync_s3_region: String, - /// Discard the existing progress of dumping an epoch state to S3. - pub state_sync_dump_drop_state: Vec, - /// Whether to enable state sync from S3. - /// If disabled will perform state sync from the peers. - pub state_sync_from_s3_enabled: bool, - /// Number of parallel in-flight requests allowed per shard. - pub state_sync_num_s3_requests_per_shard: u64, + /// Restart dumping state of selected shards. + /// Use for troubleshooting of the state dumping process. + pub state_sync_restart_dump_for_shards: Vec, } impl ClientConfig { @@ -248,12 +244,10 @@ impl ClientConfig { enable_statistics_export: true, client_background_migration_threads: 1, flat_storage_creation_period: Duration::from_secs(1), - state_dump_enabled: false, - state_sync_from_s3_enabled: false, + state_sync_dump_enabled: false, state_sync_s3_bucket: String::new(), state_sync_s3_region: String::new(), - state_sync_dump_drop_state: vec![], - state_sync_num_s3_requests_per_shard: 100, + state_sync_restart_dump_for_shards: vec![], } } } diff --git a/core/primitives/src/syncing.rs b/core/primitives/src/syncing.rs index f64e2760432..4dc68f3b7a6 100644 --- a/core/primitives/src/syncing.rs +++ b/core/primitives/src/syncing.rs @@ -229,18 +229,29 @@ pub fn get_num_state_parts(memory_usage: u64) -> u64 { } #[derive(BorshSerialize, BorshDeserialize, Debug, Clone)] -/// Represents the state of the state machine that dumps state. +/// Represents the progress of dumps state of a shard. pub enum StateSyncDumpProgress { + /// Represents two cases: + /// * An epoch dump is complete + /// * The node is running its first epoch and there is nothing to dump. AllDumped { + /// The dumped state corresponds to the state at the beginning of the specified epoch. epoch_id: EpochId, epoch_height: EpochHeight, + // Missing in case of a node running the first epoch. num_parts: Option, }, + /// Represents the case of an epoch being partially dumped. InProgress { + /// The dumped state corresponds to the state at the beginning of the specified epoch. epoch_id: EpochId, epoch_height: EpochHeight, + /// Block hash of the first block of the epoch. + /// The dumped state corresponds to the state before applying this block. sync_hash: CryptoHash, + /// Root of the state being dumped. state_root: StateRoot, + /// Progress made. parts_dumped: u64, num_parts: u64, }, diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index f8bc90dc195..a1533c77de1 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -701,40 +701,22 @@ impl NearConfig { enable_statistics_export: config.store.enable_statistics_export, client_background_migration_threads: config.store.background_migration_threads, flat_storage_creation_period: config.store.flat_storage_creation_period, - state_dump_enabled: config + state_sync_dump_enabled: config .state_sync .as_ref() - .map(|x| x.dump_enabled) - .flatten() - .unwrap_or(false), + .map_or(false, |x| x.dump_enabled.unwrap_or(false)), state_sync_s3_bucket: config .state_sync .as_ref() - .map(|x| x.s3_bucket.clone()) - .unwrap_or(String::new()), + .map_or(String::new(), |x| x.s3_bucket.clone()), state_sync_s3_region: config .state_sync .as_ref() - .map(|x| x.s3_region.clone()) - .unwrap_or(String::new()), - state_sync_dump_drop_state: config + .map_or(String::new(), |x| x.s3_region.clone()), + state_sync_restart_dump_for_shards: config .state_sync .as_ref() - .map(|x| x.drop_state_of_dump.clone()) - .flatten() - .unwrap_or(vec![]), - state_sync_from_s3_enabled: config - .state_sync - .as_ref() - .map(|x| x.sync_from_s3_enabled) - .flatten() - .unwrap_or(false), - state_sync_num_s3_requests_per_shard: config - .state_sync - .as_ref() - .map(|x| x.num_s3_requests_per_shard) - .flatten() - .unwrap_or(100), + .map_or(vec![], |x| x.drop_state_of_dump.clone().unwrap_or(vec![])), }, network_config: NetworkConfig::new( config.network, @@ -1561,21 +1543,12 @@ pub fn load_test_config(seed: &str, addr: tcp::ListenerAddr, genesis: Genesis) - #[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default)] /// Options for dumping state to S3. pub struct StateSyncConfig { - /// Location of state dumps on S3. pub s3_bucket: String, - /// Region is very important on S3. pub s3_region: String, - /// Whether a node should dump state of each epoch to the external storage. #[serde(skip_serializing_if = "Option::is_none")] pub dump_enabled: Option, - /// Use carefully in case a node that dumps state to the external storage gets in trouble. #[serde(skip_serializing_if = "Option::is_none")] pub drop_state_of_dump: Option>, - /// If enabled, will download state parts from external storage and not from the peers. - #[serde(skip_serializing_if = "Option::is_none")] - pub sync_from_s3_enabled: Option, - #[serde(skip_serializing_if = "Option::is_none")] - pub num_s3_requests_per_shard: Option, } #[test] diff --git a/nearcore/src/lib.rs b/nearcore/src/lib.rs index 7498a01830d..3da5e6a2766 100644 --- a/nearcore/src/lib.rs +++ b/nearcore/src/lib.rs @@ -2,6 +2,7 @@ use crate::cold_storage::spawn_cold_store_loop; pub use crate::config::{init_configs, load_config, load_test_config, NearConfig, NEAR_BASE}; pub use crate::runtime::NightshadeRuntime; pub use crate::shard_tracker::TrackedConfig; +use crate::state_sync::{spawn_state_sync_dump, StateSyncDumpHandle}; use actix::{Actor, Addr}; use actix_rt::ArbiterHandle; use actix_web; @@ -12,17 +13,17 @@ use near_async::messaging::{IntoSender, LateBoundSender}; use near_chain::{Chain, ChainGenesis}; use near_chunks::shards_manager_actor::start_shards_manager; use near_client::{start_client, start_view_client, ClientActor, ConfigUpdater, ViewClientActor}; -use near_primitives::time; - use near_network::PeerManagerActor; use near_primitives::block::GenesisId; +use near_primitives::time; use near_store::metadata::DbKind; use near_store::{DBCol, Mode, NodeStorage, Store, StoreOpenerError}; use near_telemetry::TelemetryActor; use std::path::{Path, PathBuf}; use std::sync::Arc; use tokio::sync::broadcast; -use tracing::{info, trace}; +use tracing::info; + pub mod append_only_map; mod cold_storage; pub mod config; @@ -33,6 +34,7 @@ mod metrics; pub mod migrations; mod runtime; mod shard_tracker; +mod state_sync; pub fn get_default_home() -> PathBuf { if let Ok(near_home) = std::env::var("NEAR_HOME") { @@ -188,6 +190,8 @@ pub struct NearNode { /// The cold_store_loop_handle will only be set if the cold store is configured. /// It's a handle to a background thread that copies data from the hot store to the cold store. pub cold_store_loop_handle: Option, + /// Contains handles to background threads that may be dumping state to S3. + pub state_sync_dump_handle: Option, } pub fn start_with_config(home_dir: &Path, config: NearConfig) -> anyhow::Result { @@ -242,7 +246,7 @@ pub fn start_with_config_and_synchronization( ); let (client_actor, client_arbiter_handle) = start_client( config.client_config.clone(), - chain_genesis, + chain_genesis.clone(), runtime.clone(), node_id, network_adapter.clone().into(), @@ -255,7 +259,7 @@ pub fn start_with_config_and_synchronization( ); client_adapter_for_shards_manager.bind(client_actor.clone().with_auto_span_context()); let (shards_manager_actor, shards_manager_arbiter_handle) = start_shards_manager( - runtime, + runtime.clone(), network_adapter.as_sender(), client_adapter_for_shards_manager.as_sender(), config.validator_signer.as_ref().map(|signer| signer.validator_id().clone()), @@ -264,6 +268,13 @@ pub fn start_with_config_and_synchronization( ); shards_manager_adapter.bind(shards_manager_actor); + let state_sync_dump_handle = spawn_state_sync_dump( + &config, + chain_genesis, + runtime, + config.network_config.node_id().public_key(), + )?; + #[allow(unused_mut)] let mut rpc_servers = Vec::new(); let network_actor = PeerManagerActor::spawn( @@ -304,7 +315,7 @@ pub fn start_with_config_and_synchronization( rpc_servers.shrink_to_fit(); - trace!(target: "diagnostic", key="log", "Starting NEAR node with diagnostic activated"); + tracing::trace!(target: "diagnostic", key = "log", "Starting NEAR node with diagnostic activated"); Ok(NearNode { client: client_actor, @@ -312,6 +323,7 @@ pub fn start_with_config_and_synchronization( rpc_servers, arbiters: vec![client_arbiter_handle, shards_manager_arbiter_handle], cold_store_loop_handle, + state_sync_dump_handle, }) } diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 79e185c13e9..458bff4e27e 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -1,53 +1,61 @@ use crate::{metrics, NearConfig, NightshadeRuntime}; +use borsh::BorshSerialize; use near_chain::types::RuntimeAdapter; -use near_chain::{Chain, ChainGenesis, DoomslugThresholdMode, Error}; +use near_chain::{Chain, ChainGenesis, ChainStoreAccess, DoomslugThresholdMode, Error}; use near_chain_configs::ClientConfig; -use near_client::sync::state::{s3_location, StateSync}; +use near_client::sync::state::StateSync; use near_crypto::PublicKey; use near_epoch_manager::EpochManagerAdapter; use near_primitives::hash::CryptoHash; use near_primitives::state_part::PartId; -use near_primitives::syncing::{get_num_state_parts, StateSyncDumpProgress}; -use near_primitives::types::{EpochHeight, EpochId, ShardId}; +use near_primitives::syncing::{get_num_state_parts, StatePartKey, StateSyncDumpProgress}; +use near_primitives::types::{EpochHeight, EpochId, ShardId, StateRoot}; +use near_store::DBCol; use std::sync::Arc; +/// Starts one a thread per tracked shard. +/// Each started thread will be dumping state parts of a single epoch to external storage. pub fn spawn_state_sync_dump( config: &NearConfig, - chain_genesis: &ChainGenesis, + chain_genesis: ChainGenesis, runtime: Arc, node_key: &PublicKey, ) -> anyhow::Result> { - if !config.client_config.state_dump_enabled - || config.client_config.state_sync_s3_bucket.is_empty() + if !config.client_config.state_sync_dump_enabled { + return Ok(None); + } + if config.client_config.state_sync_s3_bucket.is_empty() || config.client_config.state_sync_s3_region.is_empty() { - return Ok(None); + panic!("Enabled dumps of state to external storage. Please specify state_sync.s3_bucket and state_sync.s3_region"); } tracing::info!(target: "state_sync_dump", "Spawning the state sync dump loop"); // Create a connection to S3. let s3_bucket = config.client_config.state_sync_s3_bucket.clone(); let s3_region = config.client_config.state_sync_s3_region.clone(); + + // Credentials to establish a connection are taken from environment variables: AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY. let bucket = s3::Bucket::new( &s3_bucket, s3_region .parse::() .map_err(|err| >::into(err))?, s3::creds::Credentials::default().map_err(|err| { + tracing::error!(target: "state_sync_dump", "Failed to create a connection to S3. Did you provide environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY?"); >::into(err) })?, - ) - .map_err(|err| >::into(err))?; + ).map_err(|err| >::into(err))?; // Determine how many threads to start. - // Doesn't handle the case of changing the shard layout. + // TODO: Handle the case of changing the shard layout. let num_shards = { // Sadly, `Chain` is not `Send` and each thread needs to create its own `Chain` instance. let chain = Chain::new_for_view_client( runtime.clone(), - chain_genesis, + &chain_genesis, DoomslugThresholdMode::TwoThirds, - config.client_config.save_trie_changes, + false, )?; let epoch_id = chain.head()?.epoch_id; runtime.num_shards(&epoch_id) @@ -58,13 +66,12 @@ pub fn spawn_state_sync_dump( .map(|shard_id| { let client_config = config.client_config.clone(); let runtime = runtime.clone(); - let save_trie_changes = client_config.save_trie_changes; let chain_genesis = chain_genesis.clone(); let chain = Chain::new_for_view_client( runtime.clone(), &chain_genesis, DoomslugThresholdMode::TwoThirds, - save_trie_changes, + false, ) .unwrap(); let arbiter_handle = actix_rt::Arbiter::new().handle(); @@ -114,7 +121,7 @@ async fn state_sync_dump( tracing::info!(target: "state_sync_dump", shard_id, "Running StateSyncDump loop"); let mut interval = actix_rt::time::interval(std::time::Duration::from_secs(10)); - if config.state_sync_dump_drop_state.contains(&shard_id) { + if config.state_sync_restart_dump_for_shards.contains(&shard_id) { tracing::debug!(target: "state_sync_dump", shard_id, "Dropped existing progress"); chain.store().set_state_sync_dump_progress(shard_id, None).unwrap(); } @@ -136,18 +143,17 @@ async fn state_sync_dump( shard_id, &chain, &runtime, - &config, ) } Err(Error::DBNotFoundErr(_)) | Ok(None) => { // First invocation of this state-machine. See if at least one epoch is available for dumping. - check_new_epoch(None, None, None, shard_id, &chain, &runtime, &config) + check_new_epoch(None, None, None, shard_id, &chain, &runtime) } Err(err) => { // Something went wrong, let's retry. - tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to read the progress, delete and retry"); + tracing::warn!(target: "state_sync_dump", shard_id, ?err, "Failed to read the progress, will now delete and retry"); if let Err(err) = chain.store().set_state_sync_dump_progress(shard_id, None) { - tracing::debug!(target: "state_sync_dump", shard_id, ?err, "And failed to delete it too :("); + tracing::warn!(target: "state_sync_dump", shard_id, ?err, "and failed to delete the progress. Will later retry."); } Ok(None) } @@ -160,7 +166,7 @@ async fn state_sync_dump( num_parts, })) => { // The actual dumping of state to S3. - tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, ?sync_hash, ?state_root, parts_dumped, num_parts, "Creating parts and dumping them"); + tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, %sync_hash, %state_root, parts_dumped, num_parts, "Creating parts and dumping them"); let mut res = None; for part_id in parts_dumped..num_parts { // Dump parts sequentially synchronously. @@ -168,18 +174,16 @@ async fn state_sync_dump( let _timer = metrics::STATE_SYNC_DUMP_ITERATION_ELAPSED .with_label_values(&[&shard_id.to_string()]) .start_timer(); - let state_part = { - let _timer = metrics::STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - runtime.obtain_state_part( - shard_id, - &sync_hash, - &state_root, - PartId::new(part_id, num_parts), - ) - }; - let state_part = match state_part { + + let state_part = match get_state_part( + &runtime, + &shard_id, + &sync_hash, + &state_root, + part_id, + num_parts, + &chain, + ) { Ok(state_part) => state_part, Err(err) => { res = Some(err); @@ -188,70 +192,23 @@ async fn state_sync_dump( }; let location = s3_location(&config.chain_id, epoch_height, shard_id, part_id, num_parts); - + if let Err(err) = + put_state_part(&location, &state_part, &shard_id, &bucket).await { - let _timer = metrics::STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - let put = bucket - .put_object(&location, &state_part) - .await - .map_err(|err| Error::Other(err.to_string())); - if let Err(err) = put { - res = Some(err); - break; - } - - /* - // Optional, we probably don't need this. - let put = bucket - .put_object_tagging( - &location, - &[ - ("chain_id", &config.chain_id), - ("epoch_height", &epoch_height.to_string()), - ("epoch_id", &format!("{:?}", epoch_id.0)), - ("node_key", &format!("{:?}", node_key)), - ("num_parts", &format!("{}", num_parts)), - ("part_id", &format!("{}", part_id)), - ("state_root", &format!("{:?}", state_root)), - ("sync_hash", &format!("{:?}", sync_hash)), - ], - ) - .await - .map_err(|err| Error::Other(err.to_string())); - if let Err(err) = put { - res = Some(err); - break; - } - */ + res = Some(err); + break; } - - // Record that a part was obtained and dumped. - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, ?sync_hash, ?state_root, part_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); - metrics::STATE_SYNC_DUMP_SIZE_TOTAL - .with_label_values(&[&shard_id.to_string()]) - .inc_by(state_part.len() as u64); - let next_progress = StateSyncDumpProgress::InProgress { - epoch_id: epoch_id.clone(), + update_progress( + &shard_id, + &epoch_id, epoch_height, - sync_hash, - state_root, - parts_dumped: part_id + 1, + &sync_hash, + &state_root, + part_id, num_parts, - }; - match chain - .store() - .set_state_sync_dump_progress(shard_id, Some(next_progress.clone())) - { - Ok(_) => { - tracing::debug!(target: "state_sync_dump", shard_id, ?next_progress, "Updated dump progress"); - } - Err(err) => { - tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to update dump progress, continue"); - } - } - set_metrics(shard_id, Some(part_id + 1), Some(num_parts), Some(epoch_height)); + state_part.len(), + &chain, + ); } if let Some(err) = res { Err(err) @@ -289,8 +246,59 @@ async fn state_sync_dump( } } +async fn put_state_part( + location: &str, + state_part: &[u8], + shard_id: &ShardId, + bucket: &s3::Bucket, +) -> Result { + let _timer = metrics::STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + let put = bucket + .put_object(&location, &state_part) + .await + .map_err(|err| Error::Other(err.to_string())); + tracing::debug!(target: "state_sync_dump", shard_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); + put +} + +fn update_progress( + shard_id: &ShardId, + epoch_id: &EpochId, + epoch_height: EpochHeight, + sync_hash: &CryptoHash, + state_root: &StateRoot, + part_id: u64, + num_parts: u64, + part_len: usize, + chain: &Chain, +) { + // Record that a part was obtained and dumped. + metrics::STATE_SYNC_DUMP_SIZE_TOTAL + .with_label_values(&[&shard_id.to_string()]) + .inc_by(part_len as u64); + let next_progress = StateSyncDumpProgress::InProgress { + epoch_id: epoch_id.clone(), + epoch_height, + sync_hash: *sync_hash, + state_root: *state_root, + parts_dumped: part_id + 1, + num_parts, + }; + match chain.store().set_state_sync_dump_progress(*shard_id, Some(next_progress.clone())) { + Ok(_) => { + tracing::debug!(target: "state_sync_dump", shard_id, ?next_progress, "Updated dump progress"); + } + Err(err) => { + tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to update dump progress, continue"); + } + } + set_metrics(shard_id, Some(part_id + 1), Some(num_parts), Some(epoch_height)); +} + fn set_metrics( - shard_id: ShardId, + shard_id: &ShardId, parts_dumped: Option, num_parts: Option, epoch_height: Option, @@ -320,6 +328,35 @@ fn set_metrics( } } +fn get_state_part( + runtime: &Arc, + shard_id: &ShardId, + sync_hash: &CryptoHash, + state_root: &StateRoot, + part_id: u64, + num_parts: u64, + chain: &Chain, +) -> Result, Error> { + let state_part = { + let _timer = metrics::STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + runtime.obtain_state_part( + *shard_id, + &sync_hash, + &state_root, + PartId::new(part_id, num_parts), + )? + }; + + // Save the part data. + let key = StatePartKey(*sync_hash, *shard_id, part_id).try_to_vec()?; + let mut store_update = chain.store().store().store_update(); + store_update.set(DBCol::StateParts, &key, &state_part); + store_update.commit()?; + Ok(state_part) +} + /// Gets basic information about the epoch to be dumped. fn start_dumping( epoch_id: EpochId, @@ -337,10 +374,10 @@ fn start_dumping( let state_root = sync_hash_block.chunks()[shard_id as usize].prev_state_root(); let state_root_node = runtime.get_state_root_node(shard_id, &sync_hash, &state_root)?; let num_parts = get_num_state_parts(state_root_node.memory_usage); - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, ?state_root, num_parts, "Initialize dumping state of Epoch"); + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, %state_root, num_parts, "Initialize dumping state of Epoch"); // Note that first the state of the state machines gets changes to // `InProgress` and it starts dumping state after a short interval. - set_metrics(shard_id, Some(0), Some(num_parts), Some(epoch_height)); + set_metrics(&shard_id, Some(0), Some(num_parts), Some(epoch_height)); Ok(Some(StateSyncDumpProgress::InProgress { epoch_id, epoch_height, @@ -350,7 +387,7 @@ fn start_dumping( num_parts, })) } else { - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, "Shard is not tracked, skip the epoch"); + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, "Shard is not tracked, skip the epoch"); Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts: Some(0) })) } } @@ -364,26 +401,37 @@ fn check_new_epoch( shard_id: ShardId, chain: &Chain, runtime: &Arc, - config: &ClientConfig, ) -> Result, Error> { let head = chain.head()?; if Some(&head.epoch_id) == epoch_id.as_ref() { - set_metrics(shard_id, num_parts, num_parts, epoch_height); + set_metrics(&shard_id, num_parts, num_parts, epoch_height); Ok(None) } else { + // Check if the final block is now in the next epoch. tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, "Check if a new complete epoch is available"); - let mut sync_hash = head.prev_block_hash; - // Step back a few blocks to avoid dealing with forks. - for _ in 0..config.state_fetch_horizon { - sync_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); - } - let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash)?; + let hash = head.last_block_hash; + let header = chain.get_block_header(&hash)?; + let final_hash = header.last_final_block(); + let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &final_hash)?; let header = chain.get_block_header(&sync_hash)?; if Some(header.epoch_id()) == epoch_id.as_ref() { // Still in the latest dumped epoch. Do nothing. Ok(None) } else { - start_dumping(head.epoch_id.clone(), sync_hash, shard_id, &chain, runtime) + start_dumping(head.epoch_id, sync_hash, shard_id, &chain, runtime) } } } + +fn s3_location( + chain_id: &str, + epoch_height: u64, + shard_id: u64, + part_id: u64, + num_parts: u64, +) -> String { + format!( + "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}_of_{:06}", + chain_id, epoch_height, shard_id, part_id, num_parts + ) +} From 4cb5139537cc054e827576d808654cdce82a2e51 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 28 Feb 2023 17:25:18 +0100 Subject: [PATCH 23/88] feat: Dump state to S3 --- chain/chain/src/store.rs | 1 - core/chain-configs/src/client_config.rs | 15 +- core/primitives/src/syncing.rs | 19 +- nearcore/src/config.rs | 67 +---- nearcore/src/lib.rs | 58 +---- nearcore/src/metrics.rs | 35 +-- nearcore/src/state_sync.rs | 313 ++++++++---------------- 7 files changed, 128 insertions(+), 380 deletions(-) diff --git a/chain/chain/src/store.rs b/chain/chain/src/store.rs index 2cbeb18eefb..a59793cbd82 100644 --- a/chain/chain/src/store.rs +++ b/chain/chain/src/store.rs @@ -842,7 +842,6 @@ impl ChainStore { /// Constructs key 'STATE_SYNC_DUMP:', /// for example 'STATE_SYNC_DUMP:2' for shard_id=2. - /// Doesn't contain epoch_id, because only one dump process per shard is allowed. fn state_sync_dump_progress_key(shard_id: ShardId) -> Vec { let mut key = b"STATE_SYNC_DUMP:".to_vec(); key.extend(shard_id.to_le_bytes()); diff --git a/core/chain-configs/src/client_config.rs b/core/chain-configs/src/client_config.rs index 5e3b8ea0fa6..202363eb6a6 100644 --- a/core/chain-configs/src/client_config.rs +++ b/core/chain-configs/src/client_config.rs @@ -166,15 +166,10 @@ pub struct ClientConfig { pub client_background_migration_threads: usize, /// Duration to perform background flat storage creation step. pub flat_storage_creation_period: Duration, - /// If enabled, will dump state of every epoch to external storage. - pub state_sync_dump_enabled: bool, /// S3 bucket for storing state dumps. - pub state_sync_s3_bucket: String, + pub state_sync_s3_bucket: Option, /// S3 region for storing state dumps. - pub state_sync_s3_region: String, - /// Restart dumping state of selected shards. - /// Use for troubleshooting of the state dumping process. - pub state_sync_restart_dump_for_shards: Vec, + pub state_sync_s3_region: Option, } impl ClientConfig { @@ -244,10 +239,8 @@ impl ClientConfig { enable_statistics_export: true, client_background_migration_threads: 1, flat_storage_creation_period: Duration::from_secs(1), - state_sync_dump_enabled: false, - state_sync_s3_bucket: String::new(), - state_sync_s3_region: String::new(), - state_sync_restart_dump_for_shards: vec![], + state_sync_s3_bucket: None, + state_sync_s3_region: None, } } } diff --git a/core/primitives/src/syncing.rs b/core/primitives/src/syncing.rs index 4dc68f3b7a6..c948fd7a895 100644 --- a/core/primitives/src/syncing.rs +++ b/core/primitives/src/syncing.rs @@ -229,29 +229,14 @@ pub fn get_num_state_parts(memory_usage: u64) -> u64 { } #[derive(BorshSerialize, BorshDeserialize, Debug, Clone)] -/// Represents the progress of dumps state of a shard. +/// Represents the state of the state machine that dumps state. pub enum StateSyncDumpProgress { - /// Represents two cases: - /// * An epoch dump is complete - /// * The node is running its first epoch and there is nothing to dump. - AllDumped { - /// The dumped state corresponds to the state at the beginning of the specified epoch. - epoch_id: EpochId, - epoch_height: EpochHeight, - // Missing in case of a node running the first epoch. - num_parts: Option, - }, - /// Represents the case of an epoch being partially dumped. + AllDumped(EpochId), InProgress { - /// The dumped state corresponds to the state at the beginning of the specified epoch. epoch_id: EpochId, epoch_height: EpochHeight, - /// Block hash of the first block of the epoch. - /// The dumped state corresponds to the state before applying this block. sync_hash: CryptoHash, - /// Root of the state being dumped. state_root: StateRoot, - /// Progress made. parts_dumped: u64, num_parts: u64, }, diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index a1533c77de1..456ec1cee17 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -337,9 +337,7 @@ pub struct Config { /// This feature is under development, do not use in production. #[serde(default, skip_serializing_if = "Option::is_none")] pub cold_store: Option, - /// Configuration for the - #[serde(default, skip_serializing_if = "Option::is_none")] - pub split_storage: Option, + // TODO(mina86): Remove those two altogether at some point. We need to be // somewhat careful though and make sure that we don’t start silently // ignoring this option without users setting corresponding store option. @@ -354,7 +352,6 @@ pub struct Config { #[serde(default, skip_serializing_if = "Option::is_none")] pub expected_shutdown: Option, /// Options for dumping state of every epoch to S3. - #[serde(skip_serializing_if = "Option::is_none")] pub state_sync: Option, } @@ -390,56 +387,12 @@ impl Default for Config { use_db_migration_snapshot: None, store: near_store::StoreConfig::default(), cold_store: None, - split_storage: None, expected_shutdown: None, state_sync: None, } } } -fn default_enable_split_storage_view_client() -> bool { - false -} - -fn default_cold_store_initial_migration_batch_size() -> usize { - 500_000_000 -} - -fn default_cold_store_initial_migration_loop_sleep_duration() -> Duration { - Duration::from_secs(30) -} - -fn default_cold_store_loop_sleep_duration() -> Duration { - Duration::from_secs(1) -} - -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] -pub struct SplitStorageConfig { - #[serde(default = "default_enable_split_storage_view_client")] - pub enable_split_storage_view_client: bool, - - #[serde(default = "default_cold_store_initial_migration_batch_size")] - pub cold_store_initial_migration_batch_size: usize, - #[serde(default = "default_cold_store_initial_migration_loop_sleep_duration")] - pub cold_store_initial_migration_loop_sleep_duration: Duration, - - #[serde(default = "default_cold_store_loop_sleep_duration")] - pub cold_store_loop_sleep_duration: Duration, -} - -impl Default for SplitStorageConfig { - fn default() -> Self { - SplitStorageConfig { - enable_split_storage_view_client: default_enable_split_storage_view_client(), - cold_store_initial_migration_batch_size: - default_cold_store_initial_migration_batch_size(), - cold_store_initial_migration_loop_sleep_duration: - default_cold_store_initial_migration_loop_sleep_duration(), - cold_store_loop_sleep_duration: default_cold_store_loop_sleep_duration(), - } - } -} - impl Config { /// load Config from config.json without panic. Do semantic validation on field values. /// If config file issues occur, a ValidationError::ConfigFileError will be returned; @@ -701,22 +654,14 @@ impl NearConfig { enable_statistics_export: config.store.enable_statistics_export, client_background_migration_threads: config.store.background_migration_threads, flat_storage_creation_period: config.store.flat_storage_creation_period, - state_sync_dump_enabled: config - .state_sync - .as_ref() - .map_or(false, |x| x.dump_enabled.unwrap_or(false)), state_sync_s3_bucket: config .state_sync .as_ref() - .map_or(String::new(), |x| x.s3_bucket.clone()), + .map_or(None, |x| Some(x.s3_bucket.clone())), state_sync_s3_region: config .state_sync .as_ref() - .map_or(String::new(), |x| x.s3_region.clone()), - state_sync_restart_dump_for_shards: config - .state_sync - .as_ref() - .map_or(vec![], |x| x.drop_state_of_dump.clone().unwrap_or(vec![])), + .map_or(None, |x| Some(x.s3_region.clone())), }, network_config: NetworkConfig::new( config.network, @@ -1540,15 +1485,11 @@ pub fn load_test_config(seed: &str, addr: tcp::ListenerAddr, genesis: Genesis) - NearConfig::new(config, genesis, signer.into(), validator_signer).unwrap() } -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default)] +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] /// Options for dumping state to S3. pub struct StateSyncConfig { pub s3_bucket: String, pub s3_region: String, - #[serde(skip_serializing_if = "Option::is_none")] - pub dump_enabled: Option, - #[serde(skip_serializing_if = "Option::is_none")] - pub drop_state_of_dump: Option>, } #[test] diff --git a/nearcore/src/lib.rs b/nearcore/src/lib.rs index 3da5e6a2766..7ea9b4f5eed 100644 --- a/nearcore/src/lib.rs +++ b/nearcore/src/lib.rs @@ -16,8 +16,7 @@ use near_client::{start_client, start_view_client, ClientActor, ConfigUpdater, V use near_network::PeerManagerActor; use near_primitives::block::GenesisId; use near_primitives::time; -use near_store::metadata::DbKind; -use near_store::{DBCol, Mode, NodeStorage, Store, StoreOpenerError}; +use near_store::{DBCol, Mode, NodeStorage, StoreOpenerError, Temperature}; use near_telemetry::TelemetryActor; use std::path::{Path, PathBuf}; use std::sync::Arc; @@ -155,33 +154,6 @@ fn open_storage(home_dir: &Path, near_config: &mut NearConfig) -> anyhow::Result Ok(storage) } -// Safely get the split store while checking that all conditions to use it are met. -fn get_split_store(config: &NearConfig, storage: &NodeStorage) -> anyhow::Result> { - // SplitStore should only be used on archival nodes. - if !config.config.archive { - return Ok(None); - } - - // SplitStore should only be used if cold store is configured. - if config.config.cold_store.is_none() { - return Ok(None); - } - - // SplitStore should only be used in the view client if it is enabled. - if !config.config.split_storage.as_ref().map_or(false, |c| c.enable_split_storage_view_client) { - return Ok(None); - } - - // SplitStore should only be used if the migration is finished. The - // migration to cold store is finished when the db kind of the hot store is - // changed from Archive to Hot. - if storage.get_hot_store().get_db_kind()? != Some(DbKind::Hot) { - return Ok(None); - } - - Ok(storage.get_split_store()) -} - pub struct NearNode { pub client: Addr, pub view_client: Addr, @@ -208,17 +180,11 @@ pub fn start_with_config_and_synchronization( ) -> anyhow::Result { let store = open_storage(home_dir, &mut config)?; - let runtime = - Arc::new(NightshadeRuntime::from_config(home_dir, store.get_hot_store(), &config)); - - // Get the split store. If split store is some then create a new runtime for - // the view client. Otherwise just re-use the existing runtime. - let split_store = get_split_store(&config, &store)?; - let view_runtime = if let Some(split_store) = split_store { - Arc::new(NightshadeRuntime::from_config(home_dir, split_store, &config)) - } else { - runtime.clone() - }; + let runtime = Arc::new(NightshadeRuntime::from_config( + home_dir, + store.get_store(Temperature::Hot), + &config, + )); let cold_store_loop_handle = spawn_cold_store_loop(&config, &store, runtime.clone())?; @@ -239,7 +205,7 @@ pub fn start_with_config_and_synchronization( let view_client = start_view_client( config.validator_signer.as_ref().map(|signer| signer.validator_id().clone()), chain_genesis.clone(), - view_runtime, + runtime.clone(), network_adapter.clone().into(), config.client_config.clone(), adv.clone(), @@ -263,15 +229,15 @@ pub fn start_with_config_and_synchronization( network_adapter.as_sender(), client_adapter_for_shards_manager.as_sender(), config.validator_signer.as_ref().map(|signer| signer.validator_id().clone()), - store.get_hot_store(), + store.get_store(Temperature::Hot), config.client_config.chunk_request_retry_period, ); shards_manager_adapter.bind(shards_manager_actor); let state_sync_dump_handle = spawn_state_sync_dump( &config, - chain_genesis, - runtime, + &chain_genesis.clone(), + runtime.clone(), config.network_config.node_id().public_key(), )?; @@ -368,7 +334,7 @@ pub fn recompress_storage(home_dir: &Path, opts: RecompressOpts) -> anyhow::Resu "Recompressing database"); info!("Opening database at {}", src_path.display()); - let src_store = src_opener.open_in_mode(Mode::ReadOnly)?.get_hot_store(); + let src_store = src_opener.open_in_mode(Mode::ReadOnly)?.get_store(Temperature::Hot); let final_head_height = if skip_columns.contains(&DBCol::PartialChunks) { let tip: Option = @@ -385,7 +351,7 @@ pub fn recompress_storage(home_dir: &Path, opts: RecompressOpts) -> anyhow::Resu }; info!("Creating database at {}", dst_path.display()); - let dst_store = dst_opener.open_in_mode(Mode::Create)?.get_hot_store(); + let dst_store = dst_opener.open_in_mode(Mode::Create)?.get_store(Temperature::Hot); const BATCH_SIZE_BYTES: u64 = 150_000_000; diff --git a/nearcore/src/metrics.rs b/nearcore/src/metrics.rs index ee53c9c2422..2983683b16e 100644 --- a/nearcore/src/metrics.rs +++ b/nearcore/src/metrics.rs @@ -1,7 +1,6 @@ use near_o11y::metrics::{ exponential_buckets, linear_buckets, try_create_histogram_vec, try_create_int_counter_vec, - try_create_int_gauge, try_create_int_gauge_vec, HistogramVec, IntCounterVec, IntGauge, - IntGaugeVec, + try_create_int_gauge, HistogramVec, IntCounterVec, IntGauge, }; use once_cell::sync::Lazy; @@ -73,35 +72,3 @@ pub(crate) static STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED: Lazy = Lazy ) .unwrap() }); -pub(crate) static STATE_SYNC_DUMP_NUM_PARTS_TOTAL: Lazy = Lazy::new(|| { - try_create_int_gauge_vec( - "near_state_sync_dump_num_parts_total", - "Total number of parts in the epoch that being dumped", - &["shard_id"], - ) - .unwrap() -}); -pub(crate) static STATE_SYNC_DUMP_NUM_PARTS_DUMPED: Lazy = Lazy::new(|| { - try_create_int_gauge_vec( - "near_state_sync_dump_num_parts_dumped", - "Number of parts dumped in the epoch that is being dumped", - &["shard_id"], - ) - .unwrap() -}); -pub(crate) static STATE_SYNC_DUMP_SIZE_TOTAL: Lazy = Lazy::new(|| { - try_create_int_counter_vec( - "near_state_sync_dump_size_total", - "Total size of parts written to S3", - &["shard_id"], - ) - .unwrap() -}); -pub(crate) static STATE_SYNC_DUMP_EPOCH_HEIGHT: Lazy = Lazy::new(|| { - try_create_int_gauge_vec( - "near_state_sync_dump_epoch_height", - "Epoch Height of an epoch being dumped", - &["shard_id"], - ) - .unwrap() -}); diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 458bff4e27e..44483f28c5c 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -1,61 +1,52 @@ use crate::{metrics, NearConfig, NightshadeRuntime}; -use borsh::BorshSerialize; use near_chain::types::RuntimeAdapter; -use near_chain::{Chain, ChainGenesis, ChainStoreAccess, DoomslugThresholdMode, Error}; +use near_chain::{Chain, ChainGenesis, DoomslugThresholdMode, Error}; use near_chain_configs::ClientConfig; use near_client::sync::state::StateSync; use near_crypto::PublicKey; use near_epoch_manager::EpochManagerAdapter; use near_primitives::hash::CryptoHash; use near_primitives::state_part::PartId; -use near_primitives::syncing::{get_num_state_parts, StatePartKey, StateSyncDumpProgress}; -use near_primitives::types::{EpochHeight, EpochId, ShardId, StateRoot}; -use near_store::DBCol; +use near_primitives::syncing::{get_num_state_parts, StateSyncDumpProgress}; +use near_primitives::types::{EpochId, ShardId}; use std::sync::Arc; -/// Starts one a thread per tracked shard. -/// Each started thread will be dumping state parts of a single epoch to external storage. pub fn spawn_state_sync_dump( config: &NearConfig, - chain_genesis: ChainGenesis, + chain_genesis: &ChainGenesis, runtime: Arc, node_key: &PublicKey, ) -> anyhow::Result> { - if !config.client_config.state_sync_dump_enabled { - return Ok(None); - } - if config.client_config.state_sync_s3_bucket.is_empty() - || config.client_config.state_sync_s3_region.is_empty() + if config.client_config.state_sync_s3_bucket.is_none() + || config.client_config.state_sync_s3_region.is_none() { - panic!("Enabled dumps of state to external storage. Please specify state_sync.s3_bucket and state_sync.s3_region"); + return Ok(None); } tracing::info!(target: "state_sync_dump", "Spawning the state sync dump loop"); // Create a connection to S3. - let s3_bucket = config.client_config.state_sync_s3_bucket.clone(); - let s3_region = config.client_config.state_sync_s3_region.clone(); - - // Credentials to establish a connection are taken from environment variables: AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY. + let s3_bucket = config.client_config.state_sync_s3_bucket.as_ref().unwrap(); + let s3_region = config.client_config.state_sync_s3_region.as_ref().unwrap(); let bucket = s3::Bucket::new( &s3_bucket, s3_region .parse::() .map_err(|err| >::into(err))?, s3::creds::Credentials::default().map_err(|err| { - tracing::error!(target: "state_sync_dump", "Failed to create a connection to S3. Did you provide environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY?"); >::into(err) })?, - ).map_err(|err| >::into(err))?; + ) + .map_err(|err| >::into(err))?; // Determine how many threads to start. - // TODO: Handle the case of changing the shard layout. + // Doesn't handle the case of changing the shard layout. let num_shards = { // Sadly, `Chain` is not `Send` and each thread needs to create its own `Chain` instance. let chain = Chain::new_for_view_client( runtime.clone(), - &chain_genesis, + chain_genesis, DoomslugThresholdMode::TwoThirds, - false, + config.client_config.save_trie_changes, )?; let epoch_id = chain.head()?.epoch_id; runtime.num_shards(&epoch_id) @@ -66,12 +57,13 @@ pub fn spawn_state_sync_dump( .map(|shard_id| { let client_config = config.client_config.clone(); let runtime = runtime.clone(); + let save_trie_changes = client_config.save_trie_changes; let chain_genesis = chain_genesis.clone(); let chain = Chain::new_for_view_client( runtime.clone(), &chain_genesis, DoomslugThresholdMode::TwoThirds, - false, + save_trie_changes, ) .unwrap(); let arbiter_handle = actix_rt::Arbiter::new().handle(); @@ -102,7 +94,7 @@ impl Drop for StateSyncDumpHandle { } impl StateSyncDumpHandle { - pub fn stop(&self) { + pub fn stop(&mut self) { let _: Vec = self.handles.iter().map(|handle| handle.stop()).collect(); } } @@ -116,16 +108,11 @@ async fn state_sync_dump( runtime: Arc, config: ClientConfig, bucket: s3::Bucket, - _node_key: PublicKey, + node_key: PublicKey, ) { tracing::info!(target: "state_sync_dump", shard_id, "Running StateSyncDump loop"); let mut interval = actix_rt::time::interval(std::time::Duration::from_secs(10)); - if config.state_sync_restart_dump_for_shards.contains(&shard_id) { - tracing::debug!(target: "state_sync_dump", shard_id, "Dropped existing progress"); - chain.store().set_state_sync_dump_progress(shard_id, None).unwrap(); - } - loop { // Avoid a busy-loop when there is nothing to do. interval.tick().await; @@ -134,26 +121,19 @@ async fn state_sync_dump( tracing::debug!(target: "state_sync_dump", shard_id, ?progress, "Running StateSyncDump loop iteration"); // The `match` returns the next state of the state machine. let next_state = match progress { - Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts })) => { + Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) => { // The latest epoch was dumped. Check if a newer epoch is available. - check_new_epoch( - Some(epoch_id), - Some(epoch_height), - num_parts, - shard_id, - &chain, - &runtime, - ) + check_new_epoch(Some(epoch_id), shard_id, &chain, &runtime, &config) } Err(Error::DBNotFoundErr(_)) | Ok(None) => { - // First invocation of this state-machine. See if at least one epoch is available for dumping. - check_new_epoch(None, None, None, shard_id, &chain, &runtime) + // First invokation of this state-machine. See if at least one epoch is available for dumping. + check_new_epoch(None, shard_id, &chain, &runtime, &config) } Err(err) => { // Something went wrong, let's retry. - tracing::warn!(target: "state_sync_dump", shard_id, ?err, "Failed to read the progress, will now delete and retry"); + tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to read the progress, delete and retry"); if let Err(err) = chain.store().set_state_sync_dump_progress(shard_id, None) { - tracing::warn!(target: "state_sync_dump", shard_id, ?err, "and failed to delete the progress. Will later retry."); + tracing::debug!(target: "state_sync_dump", shard_id, ?err, "And failed to delete it too :("); } Ok(None) } @@ -166,7 +146,7 @@ async fn state_sync_dump( num_parts, })) => { // The actual dumping of state to S3. - tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, %sync_hash, %state_root, parts_dumped, num_parts, "Creating parts and dumping them"); + tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, ?sync_hash, ?state_root, parts_dumped, num_parts, "Creating parts and dumping them"); let mut res = None; for part_id in parts_dumped..num_parts { // Dump parts sequentially synchronously. @@ -174,50 +154,86 @@ async fn state_sync_dump( let _timer = metrics::STATE_SYNC_DUMP_ITERATION_ELAPSED .with_label_values(&[&shard_id.to_string()]) .start_timer(); - - let state_part = match get_state_part( - &runtime, - &shard_id, - &sync_hash, - &state_root, - part_id, - num_parts, - &chain, - ) { + let state_part = { + let _timer = metrics::STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + runtime.obtain_state_part( + shard_id, + &sync_hash, + &state_root, + PartId::new(part_id, num_parts), + ) + }; + let state_part = match state_part { Ok(state_part) => state_part, Err(err) => { res = Some(err); break; } }; - let location = - s3_location(&config.chain_id, epoch_height, shard_id, part_id, num_parts); - if let Err(err) = - put_state_part(&location, &state_part, &shard_id, &bucket).await + let location = s3_location(&config.chain_id, epoch_height, shard_id, part_id); + { - res = Some(err); - break; + let _timer = metrics::STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + let put = bucket + .put_object(&location, &state_part) + .await + .map_err(|err| Error::Other(err.to_string())); + if let Err(err) = put { + res = Some(err); + break; + } + + // Optional, we probably don't need this. + let put = bucket + .put_object_tagging( + &location, + &[ + ("chain_id", &config.chain_id), + ("epoch_id", &format!("{:?}", epoch_id.0)), + ("epoch_height", &epoch_height.to_string()), + ("state_root", &format!("{:?}", state_root)), + ("sync_hash", &format!("{:?}", sync_hash)), + ("node_key", &format!("{:?}", node_key)), + ], + ) + .await + .map_err(|err| Error::Other(err.to_string())); + if let Err(err) = put { + res = Some(err); + break; + } } - update_progress( - &shard_id, - &epoch_id, + + // Record that a part was obtained and dumped. + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, ?sync_hash, ?state_root, part_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); + let next_progress = StateSyncDumpProgress::InProgress { + epoch_id: epoch_id.clone(), epoch_height, - &sync_hash, - &state_root, - part_id, + sync_hash, + state_root, + parts_dumped: part_id + 1, num_parts, - state_part.len(), - &chain, - ); + }; + match chain + .store() + .set_state_sync_dump_progress(shard_id, Some(next_progress.clone())) + { + Ok(_) => { + tracing::debug!(target: "state_sync_dump", shard_id, ?next_progress, "Updated dump progress"); + } + Err(err) => { + tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to update dump progress, continue"); + } + } } if let Some(err) = res { Err(err) } else { - Ok(Some(StateSyncDumpProgress::AllDumped { - epoch_id, - epoch_height, - num_parts: Some(num_parts), - })) + Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) } } }; @@ -246,117 +262,6 @@ async fn state_sync_dump( } } -async fn put_state_part( - location: &str, - state_part: &[u8], - shard_id: &ShardId, - bucket: &s3::Bucket, -) -> Result { - let _timer = metrics::STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - let put = bucket - .put_object(&location, &state_part) - .await - .map_err(|err| Error::Other(err.to_string())); - tracing::debug!(target: "state_sync_dump", shard_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); - put -} - -fn update_progress( - shard_id: &ShardId, - epoch_id: &EpochId, - epoch_height: EpochHeight, - sync_hash: &CryptoHash, - state_root: &StateRoot, - part_id: u64, - num_parts: u64, - part_len: usize, - chain: &Chain, -) { - // Record that a part was obtained and dumped. - metrics::STATE_SYNC_DUMP_SIZE_TOTAL - .with_label_values(&[&shard_id.to_string()]) - .inc_by(part_len as u64); - let next_progress = StateSyncDumpProgress::InProgress { - epoch_id: epoch_id.clone(), - epoch_height, - sync_hash: *sync_hash, - state_root: *state_root, - parts_dumped: part_id + 1, - num_parts, - }; - match chain.store().set_state_sync_dump_progress(*shard_id, Some(next_progress.clone())) { - Ok(_) => { - tracing::debug!(target: "state_sync_dump", shard_id, ?next_progress, "Updated dump progress"); - } - Err(err) => { - tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to update dump progress, continue"); - } - } - set_metrics(shard_id, Some(part_id + 1), Some(num_parts), Some(epoch_height)); -} - -fn set_metrics( - shard_id: &ShardId, - parts_dumped: Option, - num_parts: Option, - epoch_height: Option, -) { - if let Some(parts_dumped) = parts_dumped { - metrics::STATE_SYNC_DUMP_NUM_PARTS_DUMPED - .with_label_values(&[&shard_id.to_string()]) - .set(parts_dumped as i64); - } - if let Some(num_parts) = num_parts { - metrics::STATE_SYNC_DUMP_NUM_PARTS_TOTAL - .with_label_values(&[&shard_id.to_string()]) - .set(num_parts as i64); - } - if let Some(epoch_height) = epoch_height { - assert!( - epoch_height < 10000, - "Impossible: {:?} {:?} {:?} {:?}", - shard_id, - parts_dumped, - num_parts, - epoch_height - ); - metrics::STATE_SYNC_DUMP_EPOCH_HEIGHT - .with_label_values(&[&shard_id.to_string()]) - .set(epoch_height as i64); - } -} - -fn get_state_part( - runtime: &Arc, - shard_id: &ShardId, - sync_hash: &CryptoHash, - state_root: &StateRoot, - part_id: u64, - num_parts: u64, - chain: &Chain, -) -> Result, Error> { - let state_part = { - let _timer = metrics::STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - runtime.obtain_state_part( - *shard_id, - &sync_hash, - &state_root, - PartId::new(part_id, num_parts), - )? - }; - - // Save the part data. - let key = StatePartKey(*sync_hash, *shard_id, part_id).try_to_vec()?; - let mut store_update = chain.store().store().store_update(); - store_update.set(DBCol::StateParts, &key, &state_part); - store_update.commit()?; - Ok(state_part) -} - /// Gets basic information about the epoch to be dumped. fn start_dumping( epoch_id: EpochId, @@ -374,10 +279,9 @@ fn start_dumping( let state_root = sync_hash_block.chunks()[shard_id as usize].prev_state_root(); let state_root_node = runtime.get_state_root_node(shard_id, &sync_hash, &state_root)?; let num_parts = get_num_state_parts(state_root_node.memory_usage); - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, %state_root, num_parts, "Initialize dumping state of Epoch"); + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, ?state_root, num_parts, "Initialize dumping state of Epoch"); // Note that first the state of the state machines gets changes to // `InProgress` and it starts dumping state after a short interval. - set_metrics(&shard_id, Some(0), Some(num_parts), Some(epoch_height)); Ok(Some(StateSyncDumpProgress::InProgress { epoch_id, epoch_height, @@ -387,8 +291,8 @@ fn start_dumping( num_parts, })) } else { - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, "Shard is not tracked, skip the epoch"); - Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts: Some(0) })) + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, "Shard is not tracked, skip the epoch"); + Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) } } @@ -396,42 +300,35 @@ fn start_dumping( /// `epoch_id` represents the last fully dumped epoch. fn check_new_epoch( epoch_id: Option, - epoch_height: Option, - num_parts: Option, shard_id: ShardId, chain: &Chain, runtime: &Arc, + config: &ClientConfig, ) -> Result, Error> { let head = chain.head()?; if Some(&head.epoch_id) == epoch_id.as_ref() { - set_metrics(&shard_id, num_parts, num_parts, epoch_height); Ok(None) } else { - // Check if the final block is now in the next epoch. tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, "Check if a new complete epoch is available"); - let hash = head.last_block_hash; - let header = chain.get_block_header(&hash)?; - let final_hash = header.last_final_block(); - let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &final_hash)?; + let mut sync_hash = head.prev_block_hash; + // Step back a few blocks to avoid dealing with forks. + for _ in 0..config.state_fetch_horizon { + sync_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); + } + let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash)?; let header = chain.get_block_header(&sync_hash)?; if Some(header.epoch_id()) == epoch_id.as_ref() { // Still in the latest dumped epoch. Do nothing. Ok(None) } else { - start_dumping(head.epoch_id, sync_hash, shard_id, &chain, runtime) + start_dumping(head.epoch_id.clone(), sync_hash, shard_id, &chain, runtime) } } } -fn s3_location( - chain_id: &str, - epoch_height: u64, - shard_id: u64, - part_id: u64, - num_parts: u64, -) -> String { +fn s3_location(chain_id: &str, epoch_height: u64, shard_id: u64, part_id: u64) -> String { format!( - "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}_of_{:06}", - chain_id, epoch_height, shard_id, part_id, num_parts + "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}", + chain_id, epoch_height, shard_id, part_id ) } From afcce1ba1ba79a80ceb5e92bfe81b3c80537540c Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 15 Mar 2023 16:35:29 +0100 Subject: [PATCH 24/88] Merge --- chain/chain/src/store.rs | 15 +- chain/client/src/sync/state.rs | 209 ++++++----- core/chain-configs/src/client_config.rs | 21 +- core/primitives/src/syncing.rs | 19 +- nearcore/src/config.rs | 93 ++++- nearcore/src/lib.rs | 58 +++- nearcore/src/metrics.rs | 14 - nearcore/src/state_sync.rs | 313 +++++++++++------ tools/state-viewer/src/cli.rs | 140 +++++--- tools/state-viewer/src/state_parts.rs | 439 ++++++++---------------- 10 files changed, 730 insertions(+), 591 deletions(-) diff --git a/chain/chain/src/store.rs b/chain/chain/src/store.rs index a59793cbd82..cbdc3a37ff2 100644 --- a/chain/chain/src/store.rs +++ b/chain/chain/src/store.rs @@ -842,6 +842,7 @@ impl ChainStore { /// Constructs key 'STATE_SYNC_DUMP:', /// for example 'STATE_SYNC_DUMP:2' for shard_id=2. + /// Doesn't contain epoch_id, because only one dump process per shard is allowed. fn state_sync_dump_progress_key(shard_id: ShardId) -> Vec { let mut key = b"STATE_SYNC_DUMP:".to_vec(); key.extend(shard_id.to_le_bytes()); @@ -1165,20 +1166,6 @@ impl ChainStoreAccess for ChainStore { } } -impl ChainAccessForFlatStorage for ChainStore { - fn get_block_info(&self, block_hash: &CryptoHash) -> BlockInfo { - let header = self.get_block_header(block_hash).unwrap(); - BlockInfo { hash: *block_hash, height: header.height(), prev_hash: *header.prev_hash() } - } - - fn get_block_hashes_at_height(&self, height: BlockHeight) -> HashSet { - match self.get_all_block_hashes_by_height(height) { - Ok(hashes) => hashes.values().flatten().copied().collect::>(), - Err(_) => Default::default(), - } - } -} - /// Cache update for ChainStore #[derive(Default)] struct ChainStoreCacheUpdate { diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index df4cdeb6d47..e93952671f3 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -20,6 +20,7 @@ //! here to depend more on local peers instead. //! +use crate::metrics; use ansi_term::Color::{Purple, Yellow}; use ansi_term::Style; use chrono::{DateTime, Duration, Utc}; @@ -41,12 +42,12 @@ use near_primitives::shard_layout::ShardUId; use near_primitives::state_part::PartId; use near_primitives::static_clock::StaticClock; use near_primitives::syncing::{get_num_state_parts, ShardStateSyncResponse}; -use near_primitives::types::{AccountId, ShardId, StateRoot}; +use near_primitives::types::{AccountId, EpochHeight, ShardId, StateRoot}; use rand::seq::SliceRandom; use rand::{thread_rng, Rng}; use std::collections::HashMap; use std::ops::Add; -use std::sync::atomic::Ordering; +use std::sync::atomic::{AtomicI64, Ordering}; use std::sync::Arc; use std::time::Duration as TimeDuration; @@ -213,7 +214,7 @@ impl StateSync { target: "sync", %prev_hash, timeout_sec = self.timeout.num_seconds(), - "State sync: block request timed out"); + "State sync: block request timed"); (true, false) } else { (false, false) @@ -264,29 +265,36 @@ impl StateSync { for shard_id in tracking_shards { let mut download_timeout = false; - let mut run_shard_state_download = false; + let mut need_shard = false; let shard_sync_download = new_shard_sync.entry(shard_id).or_insert_with(|| { - run_shard_state_download = true; + need_shard = true; update_sync_status = true; ShardSyncDownload::new_download_state_header(now) }); let old_status = shard_sync_download.status.clone(); - let mut shard_sync_done = false; + let mut this_done = false; + metrics::STATE_SYNC_STAGE + .with_label_values(&[&shard_id.to_string()]) + .set(shard_sync_download.status.repr() as i64); match &shard_sync_download.status { ShardSyncStatus::StateDownloadHeader => { - (download_timeout, run_shard_state_download) = self - .sync_shards_download_header_status( - shard_id, - shard_sync_download, - sync_hash, - chain, - now, - )?; + (download_timeout, need_shard) = self.sync_shards_download_header_status( + shard_id, + shard_sync_download, + sync_hash, + chain, + now, + )?; } ShardSyncStatus::StateDownloadParts => { - (download_timeout, run_shard_state_download) = - self.sync_shards_download_parts_status(shard_sync_download, now); + (download_timeout, need_shard) = self.sync_shards_download_parts_status( + shard_id, + shard_sync_download, + sync_hash, + chain, + now, + ); } ShardSyncStatus::StateDownloadScheduling => { self.sync_shards_download_scheduling_status( @@ -308,7 +316,7 @@ impl StateSync { )?; } ShardSyncStatus::StateDownloadComplete => { - shard_sync_done = self.sync_shards_download_complete_status( + this_done = self.sync_shards_download_complete_status( split_states, shard_id, shard_sync_download, @@ -329,7 +337,7 @@ impl StateSync { } ShardSyncStatus::StateSplitApplying(_status) => { debug_assert!(split_states); - shard_sync_done = self.sync_shards_state_split_applying_status( + this_done = self.sync_shards_state_split_applying_status( shard_id, shard_sync_download, sync_hash, @@ -337,10 +345,10 @@ impl StateSync { )?; } ShardSyncStatus::StateSyncDone => { - shard_sync_done = true; + this_done = true; } } - all_done &= shard_sync_done; + all_done &= this_done; if download_timeout { tracing::warn!( @@ -348,7 +356,8 @@ impl StateSync { %shard_id, timeout_sec = self.timeout.num_seconds(), "State sync didn't download the state, sending StateRequest again"); - tracing::info!(target: "sync", + tracing::info!( + target: "sync", %shard_id, %sync_hash, ?me, @@ -357,7 +366,7 @@ impl StateSync { } // Execute syncing for shard `shard_id` - if run_shard_state_download { + if need_shard { update_sync_status = true; self.request_shard( me, @@ -424,15 +433,15 @@ impl StateSync { sync_hash: CryptoHash, ) { // FIXME: something is wrong - the index should have a shard_id too. - self.requested_target.put((part_id, sync_hash), target.clone()); - - let timeout = self.timeout; - self.last_part_id_requested + let parts_request_state = self.parts_request_state.as_mut().unwrap(); + parts_request_state.requested_target.put((part_id, sync_hash), target.clone()); + parts_request_state + .last_part_id_requested .entry((target, shard_id)) .and_modify(|pending_request| { pending_request.missing_parts += 1; }) - .or_insert_with(|| PendingRequestStatus::new(timeout)); + .or_insert_with(|| PendingRequestStatus::new(self.timeout)); } // Function called when our node receives the network response with a part. @@ -444,15 +453,17 @@ impl StateSync { ) { let key = (part_id, sync_hash); // Check that it came from the target that we requested it from. - if let Some(target) = self.requested_target.get(&key) { - if self.last_part_id_requested.get_mut(&(target.clone(), shard_id)).map_or( - false, - |request| { + let parts_request_state = self.parts_request_state.as_mut().unwrap(); + if let Some(target) = parts_request_state.requested_target.get(&key) { + if parts_request_state + .last_part_id_requested + .get_mut(&(target.clone(), shard_id)) + .map_or(false, |request| { request.missing_parts = request.missing_parts.saturating_sub(1); request.missing_parts == 0 - }, - ) { - self.last_part_id_requested.remove(&(target.clone(), shard_id)); + }) + { + parts_request_state.last_part_id_requested.remove(&(target.clone(), shard_id)); } } } @@ -565,7 +576,7 @@ impl StateSync { shard_id, sync_hash, possible_targets, - &mut new_shard_sync_download, + shard_sync_download, chain, ); } @@ -575,6 +586,7 @@ impl StateSync { Ok(()) } + /// Makes a StateRequestHeader header to one of the peers. fn request_shard_header( &mut self, shard_id: ShardId, @@ -607,30 +619,36 @@ impl StateSync { ); } + /// Makes requests to download state parts for the given epoch of the given shard. fn request_shard_parts( &mut self, shard_id: ShardId, sync_hash: CryptoHash, possible_targets: Vec, new_shard_sync_download: &mut ShardSyncDownload, + chain: &Chain, ) { // We'll select all the 'highest' peers + validators as candidates (excluding those that gave us timeout in the past). // And for each one of them, we'll ask for up to 16 (MAX_STATE_PART_REQUEST) parts. - let possible_targets_sampler = + let mut possible_targets_sampler = SamplerLimited::new(possible_targets, MAX_STATE_PART_REQUEST); + let sync_block_header = chain.get_block_header(&sync_hash).unwrap(); + let epoch_id = sync_block_header.epoch_id(); + let epoch_info = chain.runtime_adapter.get_epoch_info(epoch_id).unwrap(); + let epoch_height = epoch_info.epoch_height(); + + let shard_state_header = chain.get_state_header(shard_id, sync_hash).unwrap(); + let state_num_parts = + get_num_state_parts(shard_state_header.state_root_node().memory_usage); + // Iterate over all parts that needs to be requested (i.e. download.run_me is true). // Parts are ordered such that its index match its part_id. - // Finally, for every part that needs to be requested it is selected one peer (target) randomly - // to request the part from. - // IMPORTANT: here we use 'zip' with possible_target_sampler - which is limited. So at any moment we'll not request more than - // possible_targets.len() * MAX_STATE_PART_REQUEST parts. - for ((part_id, download), target) in new_shard_sync_download + for (part_id, download) in new_shard_sync_download .downloads .iter_mut() .enumerate() .filter(|(_, download)| download.run_me.load(Ordering::SeqCst)) - .zip(possible_targets_sampler) { match &self.mode { StateSyncMode::Peers => { @@ -693,6 +711,7 @@ impl StateSync { let location = s3_location(chain_id, epoch_height, shard_id, part_id, num_parts); let download_response = download.response.clone(); + let scheduled = StaticClock::utc(); near_performance_metrics::actix::spawn(std::any::type_name::(), { async move { tracing::info!(target: "sync", %shard_id, part_id, location, "Getting an object from the external storage"); @@ -745,31 +764,25 @@ impl StateSync { download.last_target = Some(make_account_or_peer_id_or_hash(target.clone())); let run_me = download.run_me.clone(); - near_performance_metrics::actix::spawn( - std::any::type_name::(), - self.network_adapter - .send_async(PeerManagerMessageRequest::NetworkRequests( - NetworkRequests::StateRequestPart { - shard_id, - sync_hash, - part_id: part_id as u64, - target: target.clone(), - }, - )) - .then(move |result| { - // TODO: possible optimization - in the current code, even if one of the targets it not present in the network graph - // (so we keep getting RouteNotFound) - we'll still keep trying to assign parts to it. - // Fortunately only once every 60 seconds (timeout value). - if let Ok(NetworkResponses::RouteNotFound) = - result.map(|f| f.as_network_response()) - { - // Send a StateRequestPart on the next iteration - run_me.store(true, Ordering::SeqCst); - } - future::ready(()) - }), - ); - } + near_performance_metrics::actix::spawn( + std::any::type_name::(), + self.network_adapter + .send_async(PeerManagerMessageRequest::NetworkRequests( + NetworkRequests::StateRequestPart { shard_id, sync_hash, part_id, target }, + )) + .then(move |result| { + // TODO: possible optimization - in the current code, even if one of the targets it not present in the network graph + // (so we keep getting RouteNotFound) - we'll still keep trying to assign parts to it. + // Fortunately only once every 60 seconds (timeout value). + if let Ok(NetworkResponses::RouteNotFound) = + result.map(|f| f.as_network_response()) + { + // Send a StateRequestPart on the next iteration + run_me.store(true, Ordering::SeqCst); + } + future::ready(()) + }), + ); } /// The main 'step' function that should be called periodically to check and update the sync process. @@ -902,11 +915,8 @@ impl StateSync { } /// Checks if the header is downloaded. - /// If the download is complete, then moves forward to `StateDownloadParts`, - /// otherwise retries the header request. - /// Returns `(download_timeout, run_shard_state_download)` where: - /// * `download_timeout` means that the state header request timed out (and needs to be retried). - /// * `run_shard_state_download` means that header or part download requests need to run for this shard. + /// If the download is complete, then moves forward to StateDownloadParts, otherwise retries the header request. + /// Returns (download_timeout, need_shard). fn sync_shards_download_header_status( &mut self, shard_id: ShardId, @@ -916,10 +926,10 @@ impl StateSync { now: DateTime, ) -> Result<(bool, bool), near_chain::Error> { let mut download_timeout = false; - let mut run_shard_state_download = false; + let mut need_shard = false; // StateDownloadHeader is the first step. We want to fetch the basic information about the state (its size, hash etc). if shard_sync_download.downloads[0].done { - let shard_state_header = chain.get_state_header(shard_id, sync_hash)?; + let shard_state_header = chain.get_state_header(shard_id.clone(), sync_hash)?; let state_num_parts = get_num_state_parts(shard_state_header.state_root_node().memory_usage); // If the header was downloaded successfully - move to phase 2 (downloading parts). @@ -938,28 +948,31 @@ impl StateSync { shard_sync_download.downloads[0].prev_update_time = now; } if shard_sync_download.downloads[0].run_me.load(Ordering::SeqCst) { - run_shard_state_download = true; + need_shard = true; } } - Ok((download_timeout, run_shard_state_download)) + Ok((download_timeout, need_shard)) } /// Checks if the parts are downloaded. - /// If download of all parts is complete, then moves forward to `StateDownloadScheduling`. - /// Returns `(download_timeout, run_shard_state_download)` where: - /// * `download_timeout` means that the state header request timed out (and needs to be retried). - /// * `run_shard_state_download` means that header or part download requests need to run for this shard. + /// If download of all parts is complete, then moves forward to StateDownloadScheduling. + /// Otherwise, retries the failed part downloads. + /// Returns (download_timeout, need_shard). fn sync_shards_download_parts_status( &mut self, + shard_id: ShardId, shard_sync_download: &mut ShardSyncDownload, + sync_hash: CryptoHash, + chain: &mut Chain, now: DateTime, ) -> (bool, bool) { // Step 2 - download all the parts (each part is usually around 1MB). let mut download_timeout = false; - let mut run_shard_state_download = false; + let mut need_shard = false; let mut parts_done = true; let num_parts = shard_sync_download.downloads.len(); + let mut num_parts_done = 0; for (part_id, part_download) in shard_sync_download.downloads.iter_mut().enumerate() { tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, part_download.done, part_download.error, ?part_download); if !part_download.done { @@ -979,15 +992,21 @@ impl StateSync { let prev = part_download.prev_update_time; let part_timeout = now - prev > self.timeout; // Retry parts that failed. if part_timeout || part_download.error { + metrics::STATE_SYNC_RETRY_PART + .with_label_values(&[&shard_id.to_string()]) + .inc(); download_timeout |= part_timeout; part_download.run_me.store(true, Ordering::SeqCst); part_download.error = false; part_download.prev_update_time = now; } if part_download.run_me.load(Ordering::SeqCst) { - run_shard_state_download = true; + need_shard = true; } } + if part_download.done { + num_parts_done += 1; + } } tracing::debug!(target: "sync", %shard_id, %sync_hash, num_parts_done, parts_done); metrics::STATE_SYNC_PARTS_DONE @@ -1003,7 +1022,7 @@ impl StateSync { status: ShardSyncStatus::StateDownloadScheduling, }; } - (download_timeout, run_shard_state_download) + (download_timeout, need_shard) } fn sync_shards_download_scheduling_status( @@ -1036,6 +1055,7 @@ impl StateSync { Err(err) => { // Cannot finalize the downloaded state. // The reasonable behavior here is to start from the very beginning. + metrics::STATE_SYNC_DISCARD_PARTS.with_label_values(&[&shard_id.to_string()]).inc(); tracing::error!(target: "sync", %shard_id, %sync_hash, ?err, "State sync finalizing error"); *shard_sync_download = ShardSyncDownload::new_download_state_header(now); chain.clear_downloaded_parts(shard_id, sync_hash, state_num_parts)?; @@ -1066,6 +1086,9 @@ impl StateSync { Err(err) => { // Cannot finalize the downloaded state. // The reasonable behavior here is to start from the very beginning. + metrics::STATE_SYNC_DISCARD_PARTS + .with_label_values(&[&shard_id.to_string()]) + .inc(); tracing::error!(target: "sync", %shard_id, %sync_hash, ?err, "State sync finalizing error"); *shard_sync_download = ShardSyncDownload::new_download_state_header(now); let shard_state_header = chain.get_state_header(shard_id, sync_hash)?; @@ -1091,7 +1114,7 @@ impl StateSync { get_num_state_parts(shard_state_header.state_root_node().memory_usage); chain.clear_downloaded_parts(shard_id, sync_hash, state_num_parts)?; - let mut shard_sync_done = false; + let mut this_done = false; // If the shard layout is changing in this epoch - we have to apply it right now. if split_states { *shard_sync_download = ShardSyncDownload { @@ -1102,9 +1125,9 @@ impl StateSync { // If there is no layout change - we're done. *shard_sync_download = ShardSyncDownload { downloads: vec![], status: ShardSyncStatus::StateSyncDone }; - shard_sync_done = true; + this_done = true; } - Ok(shard_sync_done) + Ok(this_done) } fn sync_shards_state_split_scheduling_status( @@ -1131,7 +1154,7 @@ impl StateSync { Ok(()) } - /// Returns whether the State Sync for the given shard is complete. + /// Returns `this_done`. fn sync_shards_state_split_applying_status( &mut self, shard_id: ShardId, @@ -1140,14 +1163,14 @@ impl StateSync { chain: &mut Chain, ) -> Result { let result = self.split_state_roots.remove(&shard_id); - let mut shard_sync_done = false; + let mut this_done = false; if let Some(state_roots) = result { chain.build_state_for_split_shards_postprocessing(&sync_hash, state_roots)?; *shard_sync_download = ShardSyncDownload { downloads: vec![], status: ShardSyncStatus::StateSyncDone }; - shard_sync_done = true; + this_done = true; } - Ok(shard_sync_done) + Ok(this_done) } } @@ -1203,9 +1226,15 @@ fn check_external_storage_part_response( &data, ) { Ok(_) => { + metrics::STATE_SYNC_EXTERNAL_PARTS_DONE + .with_label_values(&[&shard_id.to_string()]) + .inc(); part_download.done = true; } Err(err) => { + metrics::STATE_SYNC_EXTERNAL_PARTS_FAILED + .with_label_values(&[&shard_id.to_string()]) + .inc(); tracing::warn!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to save a state part"); err_to_retry = Some(Error::Other("Failed to save a state part".to_string())); } diff --git a/core/chain-configs/src/client_config.rs b/core/chain-configs/src/client_config.rs index 202363eb6a6..c71ae93a895 100644 --- a/core/chain-configs/src/client_config.rs +++ b/core/chain-configs/src/client_config.rs @@ -166,10 +166,20 @@ pub struct ClientConfig { pub client_background_migration_threads: usize, /// Duration to perform background flat storage creation step. pub flat_storage_creation_period: Duration, + /// If enabled, will dump state of every epoch to external storage. + pub state_sync_dump_enabled: bool, /// S3 bucket for storing state dumps. - pub state_sync_s3_bucket: Option, + pub state_sync_s3_bucket: String, /// S3 region for storing state dumps. - pub state_sync_s3_region: Option, + pub state_sync_s3_region: String, + /// Restart dumping state of selected shards. + /// Use for troubleshooting of the state dumping process. + pub state_sync_restart_dump_for_shards: Vec, + /// Whether to enable state sync from S3. + /// If disabled will perform state sync from the peers. + pub state_sync_from_s3_enabled: bool, + /// Number of parallel in-flight requests allowed per shard. + pub state_sync_num_s3_requests_per_shard: u64, } impl ClientConfig { @@ -239,8 +249,11 @@ impl ClientConfig { enable_statistics_export: true, client_background_migration_threads: 1, flat_storage_creation_period: Duration::from_secs(1), - state_sync_s3_bucket: None, - state_sync_s3_region: None, + state_sync_dump_enabled: false, + state_sync_s3_bucket: String::new(), + state_sync_s3_region: String::new(), + state_sync_restart_dump_for_shards: vec![], + state_sync_num_s3_requests_per_shard: 10, } } } diff --git a/core/primitives/src/syncing.rs b/core/primitives/src/syncing.rs index c948fd7a895..4dc68f3b7a6 100644 --- a/core/primitives/src/syncing.rs +++ b/core/primitives/src/syncing.rs @@ -229,14 +229,29 @@ pub fn get_num_state_parts(memory_usage: u64) -> u64 { } #[derive(BorshSerialize, BorshDeserialize, Debug, Clone)] -/// Represents the state of the state machine that dumps state. +/// Represents the progress of dumps state of a shard. pub enum StateSyncDumpProgress { - AllDumped(EpochId), + /// Represents two cases: + /// * An epoch dump is complete + /// * The node is running its first epoch and there is nothing to dump. + AllDumped { + /// The dumped state corresponds to the state at the beginning of the specified epoch. + epoch_id: EpochId, + epoch_height: EpochHeight, + // Missing in case of a node running the first epoch. + num_parts: Option, + }, + /// Represents the case of an epoch being partially dumped. InProgress { + /// The dumped state corresponds to the state at the beginning of the specified epoch. epoch_id: EpochId, epoch_height: EpochHeight, + /// Block hash of the first block of the epoch. + /// The dumped state corresponds to the state before applying this block. sync_hash: CryptoHash, + /// Root of the state being dumped. state_root: StateRoot, + /// Progress made. parts_dumped: u64, num_parts: u64, }, diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index 456ec1cee17..2f61744bd48 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -337,7 +337,9 @@ pub struct Config { /// This feature is under development, do not use in production. #[serde(default, skip_serializing_if = "Option::is_none")] pub cold_store: Option, - + /// Configuration for the + #[serde(default, skip_serializing_if = "Option::is_none")] + pub split_storage: Option, // TODO(mina86): Remove those two altogether at some point. We need to be // somewhat careful though and make sure that we don’t start silently // ignoring this option without users setting corresponding store option. @@ -352,6 +354,7 @@ pub struct Config { #[serde(default, skip_serializing_if = "Option::is_none")] pub expected_shutdown: Option, /// Options for dumping state of every epoch to S3. + #[serde(skip_serializing_if = "Option::is_none")] pub state_sync: Option, } @@ -387,12 +390,56 @@ impl Default for Config { use_db_migration_snapshot: None, store: near_store::StoreConfig::default(), cold_store: None, + split_storage: None, expected_shutdown: None, state_sync: None, } } } +fn default_enable_split_storage_view_client() -> bool { + false +} + +fn default_cold_store_initial_migration_batch_size() -> usize { + 500_000_000 +} + +fn default_cold_store_initial_migration_loop_sleep_duration() -> Duration { + Duration::from_secs(30) +} + +fn default_cold_store_loop_sleep_duration() -> Duration { + Duration::from_secs(1) +} + +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] +pub struct SplitStorageConfig { + #[serde(default = "default_enable_split_storage_view_client")] + pub enable_split_storage_view_client: bool, + + #[serde(default = "default_cold_store_initial_migration_batch_size")] + pub cold_store_initial_migration_batch_size: usize, + #[serde(default = "default_cold_store_initial_migration_loop_sleep_duration")] + pub cold_store_initial_migration_loop_sleep_duration: Duration, + + #[serde(default = "default_cold_store_loop_sleep_duration")] + pub cold_store_loop_sleep_duration: Duration, +} + +impl Default for SplitStorageConfig { + fn default() -> Self { + SplitStorageConfig { + enable_split_storage_view_client: default_enable_split_storage_view_client(), + cold_store_initial_migration_batch_size: + default_cold_store_initial_migration_batch_size(), + cold_store_initial_migration_loop_sleep_duration: + default_cold_store_initial_migration_loop_sleep_duration(), + cold_store_loop_sleep_duration: default_cold_store_loop_sleep_duration(), + } + } +} + impl Config { /// load Config from config.json without panic. Do semantic validation on field values. /// If config file issues occur, a ValidationError::ConfigFileError will be returned; @@ -654,14 +701,39 @@ impl NearConfig { enable_statistics_export: config.store.enable_statistics_export, client_background_migration_threads: config.store.background_migration_threads, flat_storage_creation_period: config.store.flat_storage_creation_period, + state_sync_dump_enabled: config + .state_sync + .as_ref() + .map(|x| x.dump_enabled) + .unwrap_or(false), state_sync_s3_bucket: config .state_sync .as_ref() - .map_or(None, |x| Some(x.s3_bucket.clone())), + .map(|x| x.s3_bucket.clone()) + .unwrap_or(String::new()), state_sync_s3_region: config .state_sync .as_ref() - .map_or(None, |x| Some(x.s3_region.clone())), + .map(|x| x.s3_region.clone()) + .unwrap_or(String::new()), + state_sync_restart_dump_for_shards: config + .state_sync + .as_ref() + .map(|x| x.drop_state_of_dump.clone()) + .flatten() + .unwrap_or(vec![]), + state_sync_from_s3_enabled: config + .state_sync + .as_ref() + .map(|x| x.sync_from_s3_enabled) + .flatten() + .unwrap_or(false), + state_sync_num_s3_requests_per_shard: config + .state_sync + .as_ref() + .map(|x| x.num_s3_requests_per_shard) + .flatten() + .unwrap_or(100), }, network_config: NetworkConfig::new( config.network, @@ -1485,11 +1557,24 @@ pub fn load_test_config(seed: &str, addr: tcp::ListenerAddr, genesis: Genesis) - NearConfig::new(config, genesis, signer.into(), validator_signer).unwrap() } -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default)] /// Options for dumping state to S3. pub struct StateSyncConfig { + /// Location of state dumps on S3. pub s3_bucket: String, + /// Region is very important on S3. pub s3_region: String, + /// Whether a node should dump state of each epoch to the external storage. + #[serde(skip_serializing_if = "Option::is_none")] + pub dump_enabled: Option, + /// Use carefully in case a node that dumps state to the external storage gets in trouble. + #[serde(skip_serializing_if = "Option::is_none")] + pub drop_state_of_dump: Option>, + /// If enabled, will download state parts from external storage and not from the peers. + #[serde(skip_serializing_if = "Option::is_none")] + pub sync_from_s3_enabled: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub num_s3_requests_per_shard: Option, } #[test] diff --git a/nearcore/src/lib.rs b/nearcore/src/lib.rs index 7ea9b4f5eed..3da5e6a2766 100644 --- a/nearcore/src/lib.rs +++ b/nearcore/src/lib.rs @@ -16,7 +16,8 @@ use near_client::{start_client, start_view_client, ClientActor, ConfigUpdater, V use near_network::PeerManagerActor; use near_primitives::block::GenesisId; use near_primitives::time; -use near_store::{DBCol, Mode, NodeStorage, StoreOpenerError, Temperature}; +use near_store::metadata::DbKind; +use near_store::{DBCol, Mode, NodeStorage, Store, StoreOpenerError}; use near_telemetry::TelemetryActor; use std::path::{Path, PathBuf}; use std::sync::Arc; @@ -154,6 +155,33 @@ fn open_storage(home_dir: &Path, near_config: &mut NearConfig) -> anyhow::Result Ok(storage) } +// Safely get the split store while checking that all conditions to use it are met. +fn get_split_store(config: &NearConfig, storage: &NodeStorage) -> anyhow::Result> { + // SplitStore should only be used on archival nodes. + if !config.config.archive { + return Ok(None); + } + + // SplitStore should only be used if cold store is configured. + if config.config.cold_store.is_none() { + return Ok(None); + } + + // SplitStore should only be used in the view client if it is enabled. + if !config.config.split_storage.as_ref().map_or(false, |c| c.enable_split_storage_view_client) { + return Ok(None); + } + + // SplitStore should only be used if the migration is finished. The + // migration to cold store is finished when the db kind of the hot store is + // changed from Archive to Hot. + if storage.get_hot_store().get_db_kind()? != Some(DbKind::Hot) { + return Ok(None); + } + + Ok(storage.get_split_store()) +} + pub struct NearNode { pub client: Addr, pub view_client: Addr, @@ -180,11 +208,17 @@ pub fn start_with_config_and_synchronization( ) -> anyhow::Result { let store = open_storage(home_dir, &mut config)?; - let runtime = Arc::new(NightshadeRuntime::from_config( - home_dir, - store.get_store(Temperature::Hot), - &config, - )); + let runtime = + Arc::new(NightshadeRuntime::from_config(home_dir, store.get_hot_store(), &config)); + + // Get the split store. If split store is some then create a new runtime for + // the view client. Otherwise just re-use the existing runtime. + let split_store = get_split_store(&config, &store)?; + let view_runtime = if let Some(split_store) = split_store { + Arc::new(NightshadeRuntime::from_config(home_dir, split_store, &config)) + } else { + runtime.clone() + }; let cold_store_loop_handle = spawn_cold_store_loop(&config, &store, runtime.clone())?; @@ -205,7 +239,7 @@ pub fn start_with_config_and_synchronization( let view_client = start_view_client( config.validator_signer.as_ref().map(|signer| signer.validator_id().clone()), chain_genesis.clone(), - runtime.clone(), + view_runtime, network_adapter.clone().into(), config.client_config.clone(), adv.clone(), @@ -229,15 +263,15 @@ pub fn start_with_config_and_synchronization( network_adapter.as_sender(), client_adapter_for_shards_manager.as_sender(), config.validator_signer.as_ref().map(|signer| signer.validator_id().clone()), - store.get_store(Temperature::Hot), + store.get_hot_store(), config.client_config.chunk_request_retry_period, ); shards_manager_adapter.bind(shards_manager_actor); let state_sync_dump_handle = spawn_state_sync_dump( &config, - &chain_genesis.clone(), - runtime.clone(), + chain_genesis, + runtime, config.network_config.node_id().public_key(), )?; @@ -334,7 +368,7 @@ pub fn recompress_storage(home_dir: &Path, opts: RecompressOpts) -> anyhow::Resu "Recompressing database"); info!("Opening database at {}", src_path.display()); - let src_store = src_opener.open_in_mode(Mode::ReadOnly)?.get_store(Temperature::Hot); + let src_store = src_opener.open_in_mode(Mode::ReadOnly)?.get_hot_store(); let final_head_height = if skip_columns.contains(&DBCol::PartialChunks) { let tip: Option = @@ -351,7 +385,7 @@ pub fn recompress_storage(home_dir: &Path, opts: RecompressOpts) -> anyhow::Resu }; info!("Creating database at {}", dst_path.display()); - let dst_store = dst_opener.open_in_mode(Mode::Create)?.get_store(Temperature::Hot); + let dst_store = dst_opener.open_in_mode(Mode::Create)?.get_hot_store(); const BATCH_SIZE_BYTES: u64 = 150_000_000; diff --git a/nearcore/src/metrics.rs b/nearcore/src/metrics.rs index 2983683b16e..9b5d99bf01f 100644 --- a/nearcore/src/metrics.rs +++ b/nearcore/src/metrics.rs @@ -14,20 +14,6 @@ pub static APPLY_CHUNK_DELAY: Lazy = Lazy::new(|| { .unwrap() }); -pub static SECONDS_PER_PETAGAS: Lazy = Lazy::new(|| { - try_create_histogram_vec( - "near_execution_seconds_per_petagas_ratio", - "Execution time per unit of gas, measured in seconds per petagas. Ignore label 'label'.", - &["shard_id"], - // Non-linear buckets with higher resolution around 1.0. - Some(vec![ - 0.0, 0.1, 0.2, 0.5, 0.7, 0.8, 0.9, 0.95, 0.97, 0.99, 1.0, 1.01, 1.03, 1.05, 1.1, 1.2, - 1.3, 1.5, 2.0, 5.0, 10.0, - ]), - ) - .unwrap() -}); - pub(crate) static CONFIG_CORRECT: Lazy = Lazy::new(|| { try_create_int_gauge( "near_config_correct", diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 44483f28c5c..458bff4e27e 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -1,52 +1,61 @@ use crate::{metrics, NearConfig, NightshadeRuntime}; +use borsh::BorshSerialize; use near_chain::types::RuntimeAdapter; -use near_chain::{Chain, ChainGenesis, DoomslugThresholdMode, Error}; +use near_chain::{Chain, ChainGenesis, ChainStoreAccess, DoomslugThresholdMode, Error}; use near_chain_configs::ClientConfig; use near_client::sync::state::StateSync; use near_crypto::PublicKey; use near_epoch_manager::EpochManagerAdapter; use near_primitives::hash::CryptoHash; use near_primitives::state_part::PartId; -use near_primitives::syncing::{get_num_state_parts, StateSyncDumpProgress}; -use near_primitives::types::{EpochId, ShardId}; +use near_primitives::syncing::{get_num_state_parts, StatePartKey, StateSyncDumpProgress}; +use near_primitives::types::{EpochHeight, EpochId, ShardId, StateRoot}; +use near_store::DBCol; use std::sync::Arc; +/// Starts one a thread per tracked shard. +/// Each started thread will be dumping state parts of a single epoch to external storage. pub fn spawn_state_sync_dump( config: &NearConfig, - chain_genesis: &ChainGenesis, + chain_genesis: ChainGenesis, runtime: Arc, node_key: &PublicKey, ) -> anyhow::Result> { - if config.client_config.state_sync_s3_bucket.is_none() - || config.client_config.state_sync_s3_region.is_none() - { + if !config.client_config.state_sync_dump_enabled { return Ok(None); } + if config.client_config.state_sync_s3_bucket.is_empty() + || config.client_config.state_sync_s3_region.is_empty() + { + panic!("Enabled dumps of state to external storage. Please specify state_sync.s3_bucket and state_sync.s3_region"); + } tracing::info!(target: "state_sync_dump", "Spawning the state sync dump loop"); // Create a connection to S3. - let s3_bucket = config.client_config.state_sync_s3_bucket.as_ref().unwrap(); - let s3_region = config.client_config.state_sync_s3_region.as_ref().unwrap(); + let s3_bucket = config.client_config.state_sync_s3_bucket.clone(); + let s3_region = config.client_config.state_sync_s3_region.clone(); + + // Credentials to establish a connection are taken from environment variables: AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY. let bucket = s3::Bucket::new( &s3_bucket, s3_region .parse::() .map_err(|err| >::into(err))?, s3::creds::Credentials::default().map_err(|err| { + tracing::error!(target: "state_sync_dump", "Failed to create a connection to S3. Did you provide environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY?"); >::into(err) })?, - ) - .map_err(|err| >::into(err))?; + ).map_err(|err| >::into(err))?; // Determine how many threads to start. - // Doesn't handle the case of changing the shard layout. + // TODO: Handle the case of changing the shard layout. let num_shards = { // Sadly, `Chain` is not `Send` and each thread needs to create its own `Chain` instance. let chain = Chain::new_for_view_client( runtime.clone(), - chain_genesis, + &chain_genesis, DoomslugThresholdMode::TwoThirds, - config.client_config.save_trie_changes, + false, )?; let epoch_id = chain.head()?.epoch_id; runtime.num_shards(&epoch_id) @@ -57,13 +66,12 @@ pub fn spawn_state_sync_dump( .map(|shard_id| { let client_config = config.client_config.clone(); let runtime = runtime.clone(); - let save_trie_changes = client_config.save_trie_changes; let chain_genesis = chain_genesis.clone(); let chain = Chain::new_for_view_client( runtime.clone(), &chain_genesis, DoomslugThresholdMode::TwoThirds, - save_trie_changes, + false, ) .unwrap(); let arbiter_handle = actix_rt::Arbiter::new().handle(); @@ -94,7 +102,7 @@ impl Drop for StateSyncDumpHandle { } impl StateSyncDumpHandle { - pub fn stop(&mut self) { + pub fn stop(&self) { let _: Vec = self.handles.iter().map(|handle| handle.stop()).collect(); } } @@ -108,11 +116,16 @@ async fn state_sync_dump( runtime: Arc, config: ClientConfig, bucket: s3::Bucket, - node_key: PublicKey, + _node_key: PublicKey, ) { tracing::info!(target: "state_sync_dump", shard_id, "Running StateSyncDump loop"); let mut interval = actix_rt::time::interval(std::time::Duration::from_secs(10)); + if config.state_sync_restart_dump_for_shards.contains(&shard_id) { + tracing::debug!(target: "state_sync_dump", shard_id, "Dropped existing progress"); + chain.store().set_state_sync_dump_progress(shard_id, None).unwrap(); + } + loop { // Avoid a busy-loop when there is nothing to do. interval.tick().await; @@ -121,19 +134,26 @@ async fn state_sync_dump( tracing::debug!(target: "state_sync_dump", shard_id, ?progress, "Running StateSyncDump loop iteration"); // The `match` returns the next state of the state machine. let next_state = match progress { - Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) => { + Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts })) => { // The latest epoch was dumped. Check if a newer epoch is available. - check_new_epoch(Some(epoch_id), shard_id, &chain, &runtime, &config) + check_new_epoch( + Some(epoch_id), + Some(epoch_height), + num_parts, + shard_id, + &chain, + &runtime, + ) } Err(Error::DBNotFoundErr(_)) | Ok(None) => { - // First invokation of this state-machine. See if at least one epoch is available for dumping. - check_new_epoch(None, shard_id, &chain, &runtime, &config) + // First invocation of this state-machine. See if at least one epoch is available for dumping. + check_new_epoch(None, None, None, shard_id, &chain, &runtime) } Err(err) => { // Something went wrong, let's retry. - tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to read the progress, delete and retry"); + tracing::warn!(target: "state_sync_dump", shard_id, ?err, "Failed to read the progress, will now delete and retry"); if let Err(err) = chain.store().set_state_sync_dump_progress(shard_id, None) { - tracing::debug!(target: "state_sync_dump", shard_id, ?err, "And failed to delete it too :("); + tracing::warn!(target: "state_sync_dump", shard_id, ?err, "and failed to delete the progress. Will later retry."); } Ok(None) } @@ -146,7 +166,7 @@ async fn state_sync_dump( num_parts, })) => { // The actual dumping of state to S3. - tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, ?sync_hash, ?state_root, parts_dumped, num_parts, "Creating parts and dumping them"); + tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, %sync_hash, %state_root, parts_dumped, num_parts, "Creating parts and dumping them"); let mut res = None; for part_id in parts_dumped..num_parts { // Dump parts sequentially synchronously. @@ -154,86 +174,50 @@ async fn state_sync_dump( let _timer = metrics::STATE_SYNC_DUMP_ITERATION_ELAPSED .with_label_values(&[&shard_id.to_string()]) .start_timer(); - let state_part = { - let _timer = metrics::STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - runtime.obtain_state_part( - shard_id, - &sync_hash, - &state_root, - PartId::new(part_id, num_parts), - ) - }; - let state_part = match state_part { + + let state_part = match get_state_part( + &runtime, + &shard_id, + &sync_hash, + &state_root, + part_id, + num_parts, + &chain, + ) { Ok(state_part) => state_part, Err(err) => { res = Some(err); break; } }; - let location = s3_location(&config.chain_id, epoch_height, shard_id, part_id); - + let location = + s3_location(&config.chain_id, epoch_height, shard_id, part_id, num_parts); + if let Err(err) = + put_state_part(&location, &state_part, &shard_id, &bucket).await { - let _timer = metrics::STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - let put = bucket - .put_object(&location, &state_part) - .await - .map_err(|err| Error::Other(err.to_string())); - if let Err(err) = put { - res = Some(err); - break; - } - - // Optional, we probably don't need this. - let put = bucket - .put_object_tagging( - &location, - &[ - ("chain_id", &config.chain_id), - ("epoch_id", &format!("{:?}", epoch_id.0)), - ("epoch_height", &epoch_height.to_string()), - ("state_root", &format!("{:?}", state_root)), - ("sync_hash", &format!("{:?}", sync_hash)), - ("node_key", &format!("{:?}", node_key)), - ], - ) - .await - .map_err(|err| Error::Other(err.to_string())); - if let Err(err) = put { - res = Some(err); - break; - } + res = Some(err); + break; } - - // Record that a part was obtained and dumped. - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, ?sync_hash, ?state_root, part_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); - let next_progress = StateSyncDumpProgress::InProgress { - epoch_id: epoch_id.clone(), + update_progress( + &shard_id, + &epoch_id, epoch_height, - sync_hash, - state_root, - parts_dumped: part_id + 1, + &sync_hash, + &state_root, + part_id, num_parts, - }; - match chain - .store() - .set_state_sync_dump_progress(shard_id, Some(next_progress.clone())) - { - Ok(_) => { - tracing::debug!(target: "state_sync_dump", shard_id, ?next_progress, "Updated dump progress"); - } - Err(err) => { - tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to update dump progress, continue"); - } - } + state_part.len(), + &chain, + ); } if let Some(err) = res { Err(err) } else { - Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) + Ok(Some(StateSyncDumpProgress::AllDumped { + epoch_id, + epoch_height, + num_parts: Some(num_parts), + })) } } }; @@ -262,6 +246,117 @@ async fn state_sync_dump( } } +async fn put_state_part( + location: &str, + state_part: &[u8], + shard_id: &ShardId, + bucket: &s3::Bucket, +) -> Result { + let _timer = metrics::STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + let put = bucket + .put_object(&location, &state_part) + .await + .map_err(|err| Error::Other(err.to_string())); + tracing::debug!(target: "state_sync_dump", shard_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); + put +} + +fn update_progress( + shard_id: &ShardId, + epoch_id: &EpochId, + epoch_height: EpochHeight, + sync_hash: &CryptoHash, + state_root: &StateRoot, + part_id: u64, + num_parts: u64, + part_len: usize, + chain: &Chain, +) { + // Record that a part was obtained and dumped. + metrics::STATE_SYNC_DUMP_SIZE_TOTAL + .with_label_values(&[&shard_id.to_string()]) + .inc_by(part_len as u64); + let next_progress = StateSyncDumpProgress::InProgress { + epoch_id: epoch_id.clone(), + epoch_height, + sync_hash: *sync_hash, + state_root: *state_root, + parts_dumped: part_id + 1, + num_parts, + }; + match chain.store().set_state_sync_dump_progress(*shard_id, Some(next_progress.clone())) { + Ok(_) => { + tracing::debug!(target: "state_sync_dump", shard_id, ?next_progress, "Updated dump progress"); + } + Err(err) => { + tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to update dump progress, continue"); + } + } + set_metrics(shard_id, Some(part_id + 1), Some(num_parts), Some(epoch_height)); +} + +fn set_metrics( + shard_id: &ShardId, + parts_dumped: Option, + num_parts: Option, + epoch_height: Option, +) { + if let Some(parts_dumped) = parts_dumped { + metrics::STATE_SYNC_DUMP_NUM_PARTS_DUMPED + .with_label_values(&[&shard_id.to_string()]) + .set(parts_dumped as i64); + } + if let Some(num_parts) = num_parts { + metrics::STATE_SYNC_DUMP_NUM_PARTS_TOTAL + .with_label_values(&[&shard_id.to_string()]) + .set(num_parts as i64); + } + if let Some(epoch_height) = epoch_height { + assert!( + epoch_height < 10000, + "Impossible: {:?} {:?} {:?} {:?}", + shard_id, + parts_dumped, + num_parts, + epoch_height + ); + metrics::STATE_SYNC_DUMP_EPOCH_HEIGHT + .with_label_values(&[&shard_id.to_string()]) + .set(epoch_height as i64); + } +} + +fn get_state_part( + runtime: &Arc, + shard_id: &ShardId, + sync_hash: &CryptoHash, + state_root: &StateRoot, + part_id: u64, + num_parts: u64, + chain: &Chain, +) -> Result, Error> { + let state_part = { + let _timer = metrics::STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + runtime.obtain_state_part( + *shard_id, + &sync_hash, + &state_root, + PartId::new(part_id, num_parts), + )? + }; + + // Save the part data. + let key = StatePartKey(*sync_hash, *shard_id, part_id).try_to_vec()?; + let mut store_update = chain.store().store().store_update(); + store_update.set(DBCol::StateParts, &key, &state_part); + store_update.commit()?; + Ok(state_part) +} + /// Gets basic information about the epoch to be dumped. fn start_dumping( epoch_id: EpochId, @@ -279,9 +374,10 @@ fn start_dumping( let state_root = sync_hash_block.chunks()[shard_id as usize].prev_state_root(); let state_root_node = runtime.get_state_root_node(shard_id, &sync_hash, &state_root)?; let num_parts = get_num_state_parts(state_root_node.memory_usage); - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, ?state_root, num_parts, "Initialize dumping state of Epoch"); + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, %state_root, num_parts, "Initialize dumping state of Epoch"); // Note that first the state of the state machines gets changes to // `InProgress` and it starts dumping state after a short interval. + set_metrics(&shard_id, Some(0), Some(num_parts), Some(epoch_height)); Ok(Some(StateSyncDumpProgress::InProgress { epoch_id, epoch_height, @@ -291,8 +387,8 @@ fn start_dumping( num_parts, })) } else { - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, "Shard is not tracked, skip the epoch"); - Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, "Shard is not tracked, skip the epoch"); + Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts: Some(0) })) } } @@ -300,35 +396,42 @@ fn start_dumping( /// `epoch_id` represents the last fully dumped epoch. fn check_new_epoch( epoch_id: Option, + epoch_height: Option, + num_parts: Option, shard_id: ShardId, chain: &Chain, runtime: &Arc, - config: &ClientConfig, ) -> Result, Error> { let head = chain.head()?; if Some(&head.epoch_id) == epoch_id.as_ref() { + set_metrics(&shard_id, num_parts, num_parts, epoch_height); Ok(None) } else { + // Check if the final block is now in the next epoch. tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, "Check if a new complete epoch is available"); - let mut sync_hash = head.prev_block_hash; - // Step back a few blocks to avoid dealing with forks. - for _ in 0..config.state_fetch_horizon { - sync_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); - } - let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash)?; + let hash = head.last_block_hash; + let header = chain.get_block_header(&hash)?; + let final_hash = header.last_final_block(); + let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &final_hash)?; let header = chain.get_block_header(&sync_hash)?; if Some(header.epoch_id()) == epoch_id.as_ref() { // Still in the latest dumped epoch. Do nothing. Ok(None) } else { - start_dumping(head.epoch_id.clone(), sync_hash, shard_id, &chain, runtime) + start_dumping(head.epoch_id, sync_hash, shard_id, &chain, runtime) } } } -fn s3_location(chain_id: &str, epoch_height: u64, shard_id: u64, part_id: u64) -> String { +fn s3_location( + chain_id: &str, + epoch_height: u64, + shard_id: u64, + part_id: u64, + num_parts: u64, +) -> String { format!( - "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}", - chain_id, epoch_height, shard_id, part_id + "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}_of_{:06}", + chain_id, epoch_height, shard_id, part_id, num_parts ) } diff --git a/tools/state-viewer/src/cli.rs b/tools/state-viewer/src/cli.rs index 1b620f84e89..fa4cafa5441 100644 --- a/tools/state-viewer/src/cli.rs +++ b/tools/state-viewer/src/cli.rs @@ -1,11 +1,13 @@ use crate::commands::*; use crate::contract_accounts::ContractAccountFilter; use crate::rocksdb_stats::get_rocksdb_stats; +use crate::state_parts::{apply_state_parts, dump_state_parts}; +use crate::{epoch_info, state_parts}; use near_chain_configs::{GenesisChangeConfig, GenesisValidationMode}; use near_primitives::account::id::AccountId; use near_primitives::hash::CryptoHash; use near_primitives::sharding::ChunkHash; -use near_primitives::types::{BlockHeight, ShardId, StateRoot}; +use near_primitives::types::{BlockHeight, ShardId}; use near_store::{Mode, NodeStorage, Store, Temperature}; use nearcore::{load_config, NearConfig}; use std::path::{Path, PathBuf}; @@ -26,6 +28,8 @@ pub enum StateViewerSubCommand { /// even if it's not included in any block on disk #[clap(alias = "apply_receipt")] ApplyReceipt(ApplyReceiptCmd), + /// Apply all or a single state part of a shard. + ApplyStateParts(ApplyStatePartsCmd), /// Apply a transaction if it occurs in some chunk we know about, /// even if it's not included in any block on disk #[clap(alias = "apply_tx")] @@ -49,6 +53,8 @@ pub enum StateViewerSubCommand { /// Generate a genesis file from the current state of the DB. #[clap(alias = "dump_state")] DumpState(DumpStateCmd), + /// Dump all or a single state part of a shard. + DumpStateParts(DumpStatePartsCmd), /// Writes state to a remote redis server. #[clap(alias = "dump_state_redis")] DumpStateRedis(DumpStateRedisCmd), @@ -70,11 +76,6 @@ pub enum StateViewerSubCommand { RocksDBStats(RocksDBStatsCmd), /// Iterates over a trie and prints the StateRecords. State, - /// Dumps or applies StateChanges. - /// Experimental tool for shard shadowing development. - StateChanges(StateChangesCmd), - /// Dump or apply state parts. - StateParts(StatePartsCmd), /// View head of the storage. #[clap(alias = "view_chain")] ViewChain(ViewChainCmd), @@ -113,6 +114,7 @@ impl StateViewerSubCommand { StateViewerSubCommand::ApplyChunk(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ApplyRange(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ApplyReceipt(cmd) => cmd.run(home_dir, near_config, store), + StateViewerSubCommand::ApplyStateParts(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ApplyTx(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::Chain(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::CheckBlock => check_block_chunk_existence(near_config, store), @@ -121,6 +123,7 @@ impl StateViewerSubCommand { StateViewerSubCommand::DumpAccountStorage(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpCode(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpState(cmd) => cmd.run(home_dir, near_config, store), + StateViewerSubCommand::DumpStateParts(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpStateRedis(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpTx(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::EpochInfo(cmd) => cmd.run(home_dir, near_config, store), @@ -129,8 +132,6 @@ impl StateViewerSubCommand { StateViewerSubCommand::Replay(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::RocksDBStats(cmd) => cmd.run(store_opener.path()), StateViewerSubCommand::State => state(home_dir, near_config, store), - StateViewerSubCommand::StateChanges(cmd) => cmd.run(home_dir, near_config, store), - StateViewerSubCommand::StateParts(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ViewChain(cmd) => cmd.run(near_config, store), StateViewerSubCommand::ViewTrie(cmd) => cmd.run(store), } @@ -214,6 +215,42 @@ impl ApplyReceiptCmd { } } +#[derive(clap::Parser)] +pub struct ApplyStatePartsCmd { + /// Selects an epoch. The dump will be of the state at the beginning of this epoch. + #[clap(subcommand)] + epoch_selection: state_parts::EpochSelection, + /// Shard id. + #[clap(long)] + shard_id: ShardId, + /// State part id. Leave empty to go through every part in the shard. + #[clap(long)] + part_id: Option, + /// Where to write the state parts to. + #[clap(long)] + root_dir: Option, + /// Store state parts in an S3 bucket. + #[clap(long)] + s3_bucket: Option, + /// Store state parts in an S3 bucket. + #[clap(long)] + s3_region: Option, +} + +impl ApplyStatePartsCmd { + pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { + apply_state_parts( + self.epoch_selection, + self.shard_id, + self.part_id, + home_dir, + near_config, + store, + state_parts::Location::new(self.root_dir, (self.s3_bucket, self.s3_region)), + ); + } +} + #[derive(clap::Parser)] pub struct ApplyTxCmd { #[clap(long)] @@ -361,6 +398,49 @@ impl DumpStateCmd { } } +#[derive(clap::Parser)] +pub struct DumpStatePartsCmd { + /// Selects an epoch. The dump will be of the state at the beginning of this epoch. + #[clap(subcommand)] + epoch_selection: state_parts::EpochSelection, + /// Shard id. + #[clap(long)] + shard_id: ShardId, + /// Dump a single part id. + #[clap(long)] + part_id: Option, + /// Dump part ids starting from this part. + #[clap(long)] + part_from: Option, + /// Dump part ids up to this part (exclusive). + #[clap(long)] + part_to: Option, + /// Where to write the state parts to. + #[clap(long)] + root_dir: Option, + /// Store state parts in an S3 bucket. + #[clap(long)] + s3_bucket: Option, + /// S3 region to store state parts. + #[clap(long)] + s3_region: Option, +} + +impl DumpStatePartsCmd { + pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { + dump_state_parts( + self.epoch_selection, + self.shard_id, + self.part_from.or(self.part_id), + self.part_to.or(self.part_id.map(|x| x + 1)), + home_dir, + near_config, + store, + state_parts::Location::new(self.root_dir, (self.s3_bucket, self.s3_region)), + ); + } +} + #[derive(clap::Parser)] pub struct DumpStateRedisCmd { /// Optionally, can specify at which height to dump state. @@ -480,50 +560,6 @@ impl RocksDBStatsCmd { } } -#[derive(clap::Parser)] -pub struct StateChangesCmd { - #[clap(subcommand)] - command: crate::state_changes::StateChangesSubCommand, -} - -impl StateChangesCmd { - pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { - self.command.run(home_dir, near_config, store) - } -} - -#[derive(clap::Parser)] -pub struct StatePartsCmd { - /// Shard id. - #[clap(long)] - shard_id: ShardId, - /// Location of serialized state parts. - #[clap(long)] - root_dir: Option, - /// Store state parts in an S3 bucket. - #[clap(long)] - s3_bucket: Option, - /// Store state parts in an S3 bucket. - #[clap(long)] - s3_region: Option, - /// Dump or Apply state parts. - #[clap(subcommand)] - command: crate::state_parts::StatePartsSubCommand, -} - -impl StatePartsCmd { - pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { - self.command.run( - self.shard_id, - self.root_dir, - self.s3_bucket, - self.s3_region, - home_dir, - near_config, - store, - ); - } -} #[derive(clap::Parser)] pub struct ViewChainCmd { #[clap(long)] diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index fb5f0f5b0ac..945e8c6490e 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -1,117 +1,25 @@ use crate::epoch_info::iterate_and_filter; -use borsh::BorshDeserialize; -use near_chain::{Chain, ChainGenesis, ChainStoreAccess, DoomslugThresholdMode}; -use near_client::sync::state::StateSync; +use clap::Subcommand; +use near_chain::types::RuntimeAdapter; +use near_chain::{ChainStore, ChainStoreAccess}; +use near_epoch_manager::EpochManager; use near_primitives::epoch_manager::epoch_info::EpochInfo; use near_primitives::state_part::PartId; -use near_primitives::state_record::StateRecord; use near_primitives::syncing::get_num_state_parts; -use near_primitives::types::{EpochId, StateRoot}; +use near_primitives::types::EpochId; use near_primitives_core::hash::CryptoHash; use near_primitives_core::types::{BlockHeight, EpochHeight, ShardId}; -use near_store::{PartialStorage, Store, Trie}; +use near_store::Store; use nearcore::{NearConfig, NightshadeRuntime}; use s3::serde_types::ListBucketResult; use std::fs::DirEntry; use std::ops::Range; use std::path::{Path, PathBuf}; use std::str::FromStr; +use std::sync::Arc; use std::time::Instant; -#[derive(clap::Subcommand, Debug, Clone)] -pub(crate) enum StatePartsSubCommand { - /// Apply all or a single state part of a shard. - Apply { - /// If true, validate the state part but don't write it to the DB. - #[clap(long)] - dry_run: bool, - /// If provided, this value will be used instead of looking it up in the headers. - /// Use if those headers or blocks are not available. - #[clap(long)] - state_root: Option, - /// Choose a single part id. - /// If None - affects all state parts. - #[clap(long)] - part_id: Option, - /// Select an epoch to work on. - #[clap(subcommand)] - epoch_selection: EpochSelection, - }, - /// Dump all or a single state part of a shard. - Dump { - /// Dump part ids starting from this part. - #[clap(long)] - part_from: Option, - /// Dump part ids up to this part (exclusive). - #[clap(long)] - part_to: Option, - /// Select an epoch to work on. - #[clap(subcommand)] - epoch_selection: EpochSelection, - }, - /// Read State Header from the DB - ReadStateHeader { - /// Select an epoch to work on. - #[clap(subcommand)] - epoch_selection: EpochSelection, - }, -} - -impl StatePartsSubCommand { - pub(crate) fn run( - self, - shard_id: ShardId, - root_dir: Option, - s3_bucket: Option, - s3_region: Option, - home_dir: &Path, - near_config: NearConfig, - store: Store, - ) { - let runtime = NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); - let chain_genesis = ChainGenesis::new(&near_config.genesis); - let mut chain = Chain::new_for_view_client( - runtime, - &chain_genesis, - DoomslugThresholdMode::TwoThirds, - false, - ) - .unwrap(); - let chain_id = &near_config.genesis.config.chain_id; - match self { - StatePartsSubCommand::Apply { dry_run, state_root, part_id, epoch_selection } => { - apply_state_parts( - epoch_selection, - shard_id, - part_id, - dry_run, - state_root, - &mut chain, - chain_id, - store, - Location::new(root_dir, (s3_bucket, s3_region)), - ); - } - StatePartsSubCommand::Dump { part_from, part_to, epoch_selection } => { - dump_state_parts( - epoch_selection, - shard_id, - part_from, - part_to, - &chain, - chain_id, - store, - Location::new(root_dir, (s3_bucket, s3_region)), - ); - } - StatePartsSubCommand::ReadStateHeader { epoch_selection } => { - read_state_header(epoch_selection, shard_id, &chain, store) - } - } - } -} - -#[derive(clap::Subcommand, Debug, Clone)] +#[derive(Subcommand, Debug, Clone)] pub(crate) enum EpochSelection { /// Current epoch. Current, @@ -126,10 +34,15 @@ pub(crate) enum EpochSelection { } impl EpochSelection { - fn to_epoch_id(&self, store: Store, chain: &Chain) -> EpochId { + pub fn to_epoch_id( + &self, + store: Store, + chain_store: &ChainStore, + epoch_manager: &EpochManager, + ) -> EpochId { match self { EpochSelection::Current => { - chain.runtime_adapter.get_epoch_id(&chain.head().unwrap().last_block_hash).unwrap() + epoch_manager.get_epoch_id(&chain_store.head().unwrap().last_block_hash).unwrap() } EpochSelection::EpochId { epoch_id } => { EpochId(CryptoHash::from_str(&epoch_id).unwrap()) @@ -146,24 +59,24 @@ impl EpochSelection { } EpochSelection::BlockHash { block_hash } => { let block_hash = CryptoHash::from_str(&block_hash).unwrap(); - chain.runtime_adapter.get_epoch_id(&block_hash).unwrap() + epoch_manager.get_epoch_id(&block_hash).unwrap() } EpochSelection::BlockHeight { block_height } => { // Fetch an epoch containing the given block height. - let block_hash = chain.store().get_block_hash_by_height(*block_height).unwrap(); - chain.runtime_adapter.get_epoch_id(&block_hash).unwrap() + let block_hash = chain_store.get_block_hash_by_height(*block_height).unwrap(); + epoch_manager.get_epoch_id(&block_hash).unwrap() } } } } -enum Location { +pub(crate) enum Location { Files(PathBuf), S3 { bucket: String, region: String }, } impl Location { - fn new( + pub(crate) fn new( root_dir: Option, s3_bucket_and_region: (Option, Option), ) -> Self { @@ -186,18 +99,21 @@ impl Location { } } -/// Returns block hash of some block of the given `epoch_info` epoch. -fn get_any_block_hash_of_epoch(epoch_info: &EpochInfo, chain: &Chain) -> CryptoHash { - let head = chain.store().head().unwrap(); - let mut cur_block_info = chain.runtime_adapter.get_block_info(&head.last_block_hash).unwrap(); +/// Returns block hash of the last block of an epoch preceding the given `epoch_info`. +fn get_prev_hash_of_epoch( + epoch_info: &EpochInfo, + chain_store: &ChainStore, + epoch_manager: &EpochManager, +) -> CryptoHash { + let head = chain_store.head().unwrap(); + let mut cur_block_info = epoch_manager.get_block_info(&head.last_block_hash).unwrap(); // EpochManager doesn't have an API that maps EpochId to Blocks, and this function works // around that limitation by iterating over the epochs. // This workaround is acceptable because: // 1) Extending EpochManager's API is a major change. // 2) This use case is not critical at all. loop { - let cur_epoch_info = - chain.runtime_adapter.get_epoch_info(cur_block_info.epoch_id()).unwrap(); + let cur_epoch_info = epoch_manager.get_epoch_info(cur_block_info.epoch_id()).unwrap(); let cur_epoch_height = cur_epoch_info.epoch_height(); assert!( cur_epoch_height >= epoch_info.epoch_height(), @@ -206,58 +122,68 @@ fn get_any_block_hash_of_epoch(epoch_info: &EpochInfo, chain: &Chain) -> CryptoH epoch_info.epoch_height() ); let epoch_first_block_info = - chain.runtime_adapter.get_block_info(cur_block_info.epoch_first_block()).unwrap(); + epoch_manager.get_block_info(cur_block_info.epoch_first_block()).unwrap(); let prev_epoch_last_block_info = - chain.runtime_adapter.get_block_info(epoch_first_block_info.prev_hash()).unwrap(); + epoch_manager.get_block_info(epoch_first_block_info.prev_hash()).unwrap(); if cur_epoch_height == epoch_info.epoch_height() { - return *cur_block_info.hash(); + return *prev_epoch_last_block_info.hash(); } cur_block_info = prev_epoch_last_block_info; } } -fn apply_state_parts( +pub(crate) fn apply_state_parts( epoch_selection: EpochSelection, shard_id: ShardId, part_id: Option, - dry_run: bool, - maybe_state_root: Option, - chain: &mut Chain, - chain_id: &str, + home_dir: &Path, + near_config: NearConfig, store: Store, location: Location, ) { - let (state_root, epoch_height, epoch_id, sync_hash) = - if let (Some(state_root), EpochSelection::EpochHeight { epoch_height }) = - (maybe_state_root, &epoch_selection) - { - (state_root, *epoch_height, None, None) - } else { - let epoch_id = epoch_selection.to_epoch_id(store, &chain); - let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); - - let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); - let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); + let runtime_adapter: Arc = + NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); + let epoch_manager = + EpochManager::new_from_genesis_config(store.clone(), &near_config.genesis.config) + .expect("Failed to start Epoch Manager"); + let chain_store = ChainStore::new( + store.clone(), + near_config.genesis.config.genesis_height, + near_config.client_config.save_trie_changes, + ); - let state_header = chain.get_state_response_header(shard_id, sync_hash).unwrap(); - let state_root = state_header.chunk_prev_state_root(); + let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); + let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); + let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); + let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); - (state_root, epoch.epoch_height(), Some(epoch_id), Some(sync_hash)) - }; + assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); + assert!( + shard_id < sync_prev_block.chunks().len() as u64, + "shard_id: {}, #shards: {}", + shard_id, + sync_prev_block.chunks().len() + ); + let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); - let part_storage = get_state_part_reader(location, &chain_id, epoch_height, shard_id); + let part_storage = get_state_part_reader( + location, + &near_config.client_config.chain_id, + epoch.epoch_height(), + shard_id, + ); let num_parts = part_storage.num_parts(); - assert_ne!(num_parts, 0, "Too few num_parts: {}", num_parts); let part_ids = get_part_ids(part_id, part_id.map(|x| x + 1), num_parts); tracing::info!( target: "state-parts", - epoch_height, + epoch_height = epoch.epoch_height(), + epoch_id = ?epoch_id.0, shard_id, num_parts, - ?sync_hash, + ?sync_prev_hash, ?part_ids, "Applying state as seen at the beginning of the specified epoch.", ); @@ -266,58 +192,59 @@ fn apply_state_parts( for part_id in part_ids { let timer = Instant::now(); assert!(part_id < num_parts, "part_id: {}, num_parts: {}", part_id, num_parts); - let part = part_storage.read(part_id, num_parts); - - if dry_run { - assert!(chain.runtime_adapter.validate_state_part( + let part = part_storage.read(part_id); + runtime_adapter + .apply_state_part( + shard_id, &state_root, PartId::new(part_id, num_parts), - &part - )); - tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Validated a state part"); - } else { - chain - .set_state_part( - shard_id, - sync_hash.unwrap(), - PartId::new(part_id, num_parts), - &part, - ) - .unwrap(); - chain - .runtime_adapter - .apply_state_part( - shard_id, - &state_root, - PartId::new(part_id, num_parts), - &part, - epoch_id.as_ref().unwrap(), - ) - .unwrap(); - tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Applied a state part"); - } + &part, + &epoch_id, + ) + .unwrap(); + tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Applied a state part"); } tracing::info!(target: "state-parts", total_elapsed_sec = timer.elapsed().as_secs_f64(), "Applied all requested state parts"); } -fn dump_state_parts( +pub(crate) fn dump_state_parts( epoch_selection: EpochSelection, shard_id: ShardId, part_from: Option, part_to: Option, - chain: &Chain, - chain_id: &str, + home_dir: &Path, + near_config: NearConfig, store: Store, location: Location, ) { - let epoch_id = epoch_selection.to_epoch_id(store, &chain); - let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); - let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); - let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); - - let state_header = chain.compute_state_response_header(shard_id, sync_hash).unwrap(); - let state_root = state_header.chunk_prev_state_root(); - let num_parts = get_num_state_parts(state_header.state_root_node().memory_usage); + let runtime_adapter: Arc = + NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); + let epoch_manager = + EpochManager::new_from_genesis_config(store.clone(), &near_config.genesis.config) + .expect("Failed to start Epoch Manager"); + let chain_store = ChainStore::new( + store.clone(), + near_config.genesis.config.genesis_height, + near_config.client_config.save_trie_changes, + ); + + let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); + let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); + let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); + let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); + + assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); + assert!( + shard_id < sync_prev_block.chunks().len() as u64, + "shard_id: {}, #shards: {}", + shard_id, + sync_prev_block.chunks().len() + ); + let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); + let state_root_node = + runtime_adapter.get_state_root_node(shard_id, &sync_prev_hash, &state_root).unwrap(); + + let num_parts = get_num_state_parts(state_root_node.memory_usage); let part_ids = get_part_ids(part_from, part_to, num_parts); tracing::info!( @@ -326,107 +253,59 @@ fn dump_state_parts( epoch_id = ?epoch_id.0, shard_id, num_parts, - ?sync_hash, + ?sync_prev_hash, ?part_ids, - ?state_root, "Dumping state as seen at the beginning of the specified epoch.", ); - let part_storage = get_state_part_writer(location, chain_id, epoch.epoch_height(), shard_id); + let part_storage = get_state_part_writer( + location, + &near_config.client_config.chain_id, + epoch.epoch_height(), + shard_id, + ); let timer = Instant::now(); for part_id in part_ids { let timer = Instant::now(); assert!(part_id < num_parts, "part_id: {}, num_parts: {}", part_id, num_parts); - let state_part = chain - .runtime_adapter - .obtain_state_part(shard_id, &sync_hash, &state_root, PartId::new(part_id, num_parts)) + let state_part = runtime_adapter + .obtain_state_part( + shard_id, + &sync_prev_hash, + &state_root, + PartId::new(part_id, num_parts), + ) .unwrap(); - part_storage.write(&state_part, part_id, num_parts); - let elapsed_sec = timer.elapsed().as_secs_f64(); - let first_state_record = get_first_state_record(&state_root, &state_part); - tracing::info!( - target: "state-parts", - part_id, - part_length = state_part.len(), - elapsed_sec, - ?first_state_record, - "Wrote a state part"); + part_storage.write(&state_part, part_id); + tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote a state part"); } tracing::info!(target: "state-parts", total_elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote all requested state parts"); } -/// Returns the first `StateRecord` encountered while iterating over a sub-trie in the state part. -fn get_first_state_record(state_root: &StateRoot, data: &[u8]) -> Option { - let trie_nodes = BorshDeserialize::try_from_slice(data).unwrap(); - let trie = Trie::from_recorded_storage(PartialStorage { nodes: trie_nodes }, *state_root); - - for item in trie.iter().unwrap() { - if let Ok((key, value)) = item { - if let Some(sr) = StateRecord::from_raw_key_value(key, value) { - return Some(sr); - } - } - } - None -} - -/// Reads `StateHeader` stored in the DB. -fn read_state_header( - epoch_selection: EpochSelection, - shard_id: ShardId, - chain: &Chain, - store: Store, -) { - let epoch_id = epoch_selection.to_epoch_id(store, &chain); - let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); - - let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); - let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); - - let state_header = chain.store().get_state_header(shard_id, sync_hash); - tracing::info!(target: "state-parts", ?epoch_id, ?sync_hash, ?state_header); -} - fn get_part_ids(part_from: Option, part_to: Option, num_parts: u64) -> Range { part_from.unwrap_or(0)..part_to.unwrap_or(num_parts) } -// Needs to be in sync with `fn s3_location()`. fn location_prefix(chain_id: &str, epoch_height: u64, shard_id: u64) -> String { format!("chain_id={}/epoch_height={}/shard_id={}", chain_id, epoch_height, shard_id) } -fn match_filename(s: &str) -> Option { - let re = regex::Regex::new(r"^state_part_(\d{6})_of_(\d{6})$").unwrap(); - re.captures(s) -} - fn is_part_filename(s: &str) -> bool { - match_filename(s).is_some() -} - -fn get_num_parts_from_filename(s: &str) -> Option { - if let Some(captures) = match_filename(s) { - if let Some(num_parts) = captures.get(2) { - if let Ok(num_parts) = num_parts.as_str().parse::() { - return Some(num_parts); - } - } - } - None + let re = regex::Regex::new(r"^state_part_(\d{6})$").unwrap(); + re.is_match(s) } -fn part_filename(part_id: u64, num_parts: u64) -> String { - format!("state_part_{:06}_of_{:06}", part_id, num_parts) +fn part_filename(part_id: u64) -> String { + format!("state_part_{:06}", part_id) } trait StatePartWriter { - fn write(&self, state_part: &[u8], part_id: u64, num_parts: u64); + fn write(&self, state_part: &[u8], part_id: u64); } trait StatePartReader { - fn read(&self, part_id: u64, num_parts: u64) -> Vec; + fn read(&self, part_id: u64) -> Vec; fn num_parts(&self) -> u64; } @@ -480,58 +359,40 @@ impl FileSystemStorage { tracing::info!(target: "state-parts", ?root_dir, ?prefix, ?state_parts_dir, "Ensuring the directory exists"); std::fs::create_dir_all(&state_parts_dir).unwrap(); } - tracing::info!(target: "state-parts", ?state_parts_dir, "Initialized FileSystemStorage"); Self { state_parts_dir } } - fn get_location(&self, part_id: u64, num_parts: u64) -> PathBuf { - (&self.state_parts_dir).join(part_filename(part_id, num_parts)) + fn get_location(&self, part_id: u64) -> PathBuf { + (&self.state_parts_dir).join(part_filename(part_id)) } } impl StatePartWriter for FileSystemStorage { - fn write(&self, state_part: &[u8], part_id: u64, num_parts: u64) { - let filename = self.get_location(part_id, num_parts); + fn write(&self, state_part: &[u8], part_id: u64) { + let filename = self.get_location(part_id); std::fs::write(&filename, state_part).unwrap(); tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), ?filename, "Wrote a state part to disk"); } } impl StatePartReader for FileSystemStorage { - fn read(&self, part_id: u64, num_parts: u64) -> Vec { - let filename = self.get_location(part_id, num_parts); - tracing::debug!(target: "state-parts", part_id, num_parts, ?filename, "Reading state part file"); + fn read(&self, part_id: u64) -> Vec { + let filename = self.get_location(part_id); let part = std::fs::read(filename).unwrap(); part } fn num_parts(&self) -> u64 { let paths = std::fs::read_dir(&self.state_parts_dir).unwrap(); - let mut known_num_parts = None; - let num_files = paths + let num_parts = paths .filter(|path| { let full_path = path.as_ref().unwrap(); tracing::debug!(target: "state-parts", ?full_path); - let filename = full_path.file_name().to_str().unwrap().to_string(); - if let Some(num_parts) = get_num_parts_from_filename(&filename) { - if let Some(known_num_parts) = known_num_parts { - assert_eq!(known_num_parts, num_parts); - } - known_num_parts = Some(num_parts); - } - is_part_filename(&filename) + is_part_filename(full_path.file_name().to_str().unwrap()) }) .collect::>>() .len(); - if known_num_parts != Some(num_files as u64) { - // This is expected when a user saves time and downloads a few parts instead of all parts. - tracing::warn!(target: "state-parts", - dir = ?self.state_parts_dir, - ?known_num_parts, - num_files, - "Filename indicates that number of files expected doesn't match the number of files available"); - } - known_num_parts.unwrap() + num_parts as u64 } } @@ -560,22 +421,22 @@ impl S3Storage { Self { location, bucket } } - fn get_location(&self, part_id: u64, num_parts: u64) -> String { - format!("{}/{}", self.location, part_filename(part_id, num_parts)) + fn get_location(&self, part_id: u64) -> String { + format!("{}/{}", self.location, part_filename(part_id)) } } impl StatePartWriter for S3Storage { - fn write(&self, state_part: &[u8], part_id: u64, num_parts: u64) { - let location = self.get_location(part_id, num_parts); + fn write(&self, state_part: &[u8], part_id: u64) { + let location = self.get_location(part_id); self.bucket.put_object_blocking(&location, &state_part).unwrap(); tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); } } impl StatePartReader for S3Storage { - fn read(&self, part_id: u64, num_parts: u64) -> Vec { - let location = self.get_location(part_id, num_parts); + fn read(&self, part_id: u64) -> Vec { + let location = self.get_location(part_id); let response = self.bucket.get_object_blocking(location.clone()).unwrap(); tracing::info!(target: "state-parts", part_id, location, response_code = response.status_code(), "Got an object from S3"); assert_eq!(response.status_code(), 200); @@ -588,27 +449,17 @@ impl StatePartReader for S3Storage { let list: Vec = self.bucket.list_blocking(location, Some("/".to_string())).unwrap(); assert_eq!(list.len(), 1); - let mut known_num_parts = None; - let num_objects = list[0] + let num_parts = list[0] .contents .iter() .filter(|object| { let filename = Path::new(&object.key); let filename = filename.file_name().unwrap().to_str().unwrap(); tracing::debug!(target: "state-parts", object_key = ?object.key, ?filename); - if let Some(num_parts) = get_num_parts_from_filename(filename) { - if let Some(known_num_parts) = known_num_parts { - assert_eq!(known_num_parts, num_parts); - } - known_num_parts = Some(num_parts); - } is_part_filename(filename) }) .collect::>() .len(); - if let Some(known_num_parts) = known_num_parts { - assert_eq!(known_num_parts, num_objects as u64); - } - num_objects as u64 + num_parts as u64 } } From 8408ce7b328b6f01882e71c7c809c223c3193964 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 15 Mar 2023 16:18:10 +0100 Subject: [PATCH 25/88] Fix: Rework the state-parts tool API --- tools/state-viewer/src/cli.rs | 150 ++++++++---------- tools/state-viewer/src/state_parts.rs | 209 ++++++++++++++++++++------ 2 files changed, 224 insertions(+), 135 deletions(-) diff --git a/tools/state-viewer/src/cli.rs b/tools/state-viewer/src/cli.rs index fa4cafa5441..e0e6b26d860 100644 --- a/tools/state-viewer/src/cli.rs +++ b/tools/state-viewer/src/cli.rs @@ -1,8 +1,6 @@ use crate::commands::*; use crate::contract_accounts::ContractAccountFilter; use crate::rocksdb_stats::get_rocksdb_stats; -use crate::state_parts::{apply_state_parts, dump_state_parts}; -use crate::{epoch_info, state_parts}; use near_chain_configs::{GenesisChangeConfig, GenesisValidationMode}; use near_primitives::account::id::AccountId; use near_primitives::hash::CryptoHash; @@ -28,8 +26,6 @@ pub enum StateViewerSubCommand { /// even if it's not included in any block on disk #[clap(alias = "apply_receipt")] ApplyReceipt(ApplyReceiptCmd), - /// Apply all or a single state part of a shard. - ApplyStateParts(ApplyStatePartsCmd), /// Apply a transaction if it occurs in some chunk we know about, /// even if it's not included in any block on disk #[clap(alias = "apply_tx")] @@ -53,8 +49,6 @@ pub enum StateViewerSubCommand { /// Generate a genesis file from the current state of the DB. #[clap(alias = "dump_state")] DumpState(DumpStateCmd), - /// Dump all or a single state part of a shard. - DumpStateParts(DumpStatePartsCmd), /// Writes state to a remote redis server. #[clap(alias = "dump_state_redis")] DumpStateRedis(DumpStateRedisCmd), @@ -76,6 +70,11 @@ pub enum StateViewerSubCommand { RocksDBStats(RocksDBStatsCmd), /// Iterates over a trie and prints the StateRecords. State, + /// Dumps or applies StateChanges. + /// Experimental tool for shard shadowing development. + StateChanges(StateChangesCmd), + /// Dump or apply state parts. + StateParts(StatePartsCmd), /// View head of the storage. #[clap(alias = "view_chain")] ViewChain(ViewChainCmd), @@ -106,7 +105,8 @@ impl StateViewerSubCommand { let storage = store_opener.open_in_mode(mode).unwrap(); let store = match temperature { Temperature::Hot => storage.get_hot_store(), - Temperature::Cold => storage.get_cold_store().unwrap(), + // Cold store on it's own is useless in majority of subcommands + Temperature::Cold => storage.get_split_store().unwrap(), }; match self { @@ -114,7 +114,6 @@ impl StateViewerSubCommand { StateViewerSubCommand::ApplyChunk(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ApplyRange(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ApplyReceipt(cmd) => cmd.run(home_dir, near_config, store), - StateViewerSubCommand::ApplyStateParts(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ApplyTx(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::Chain(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::CheckBlock => check_block_chunk_existence(near_config, store), @@ -123,7 +122,6 @@ impl StateViewerSubCommand { StateViewerSubCommand::DumpAccountStorage(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpCode(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpState(cmd) => cmd.run(home_dir, near_config, store), - StateViewerSubCommand::DumpStateParts(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpStateRedis(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpTx(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::EpochInfo(cmd) => cmd.run(home_dir, near_config, store), @@ -132,6 +130,8 @@ impl StateViewerSubCommand { StateViewerSubCommand::Replay(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::RocksDBStats(cmd) => cmd.run(store_opener.path()), StateViewerSubCommand::State => state(home_dir, near_config, store), + StateViewerSubCommand::StateChanges(cmd) => cmd.run(home_dir, near_config, store), + StateViewerSubCommand::StateParts(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ViewChain(cmd) => cmd.run(near_config, store), StateViewerSubCommand::ViewTrie(cmd) => cmd.run(store), } @@ -215,42 +215,6 @@ impl ApplyReceiptCmd { } } -#[derive(clap::Parser)] -pub struct ApplyStatePartsCmd { - /// Selects an epoch. The dump will be of the state at the beginning of this epoch. - #[clap(subcommand)] - epoch_selection: state_parts::EpochSelection, - /// Shard id. - #[clap(long)] - shard_id: ShardId, - /// State part id. Leave empty to go through every part in the shard. - #[clap(long)] - part_id: Option, - /// Where to write the state parts to. - #[clap(long)] - root_dir: Option, - /// Store state parts in an S3 bucket. - #[clap(long)] - s3_bucket: Option, - /// Store state parts in an S3 bucket. - #[clap(long)] - s3_region: Option, -} - -impl ApplyStatePartsCmd { - pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { - apply_state_parts( - self.epoch_selection, - self.shard_id, - self.part_id, - home_dir, - near_config, - store, - state_parts::Location::new(self.root_dir, (self.s3_bucket, self.s3_region)), - ); - } -} - #[derive(clap::Parser)] pub struct ApplyTxCmd { #[clap(long)] @@ -398,49 +362,6 @@ impl DumpStateCmd { } } -#[derive(clap::Parser)] -pub struct DumpStatePartsCmd { - /// Selects an epoch. The dump will be of the state at the beginning of this epoch. - #[clap(subcommand)] - epoch_selection: state_parts::EpochSelection, - /// Shard id. - #[clap(long)] - shard_id: ShardId, - /// Dump a single part id. - #[clap(long)] - part_id: Option, - /// Dump part ids starting from this part. - #[clap(long)] - part_from: Option, - /// Dump part ids up to this part (exclusive). - #[clap(long)] - part_to: Option, - /// Where to write the state parts to. - #[clap(long)] - root_dir: Option, - /// Store state parts in an S3 bucket. - #[clap(long)] - s3_bucket: Option, - /// S3 region to store state parts. - #[clap(long)] - s3_region: Option, -} - -impl DumpStatePartsCmd { - pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { - dump_state_parts( - self.epoch_selection, - self.shard_id, - self.part_from.or(self.part_id), - self.part_to.or(self.part_id.map(|x| x + 1)), - home_dir, - near_config, - store, - state_parts::Location::new(self.root_dir, (self.s3_bucket, self.s3_region)), - ); - } -} - #[derive(clap::Parser)] pub struct DumpStateRedisCmd { /// Optionally, can specify at which height to dump state. @@ -489,7 +410,7 @@ impl DumpTxCmd { #[derive(clap::Args)] pub struct EpochInfoCmd { #[clap(subcommand)] - epoch_selection: epoch_info::EpochSelection, + epoch_selection: crate::epoch_info::EpochSelection, /// Displays kickouts of the given validator and expected and missed blocks and chunks produced. #[clap(long)] validator_account_id: Option, @@ -560,6 +481,18 @@ impl RocksDBStatsCmd { } } +#[derive(clap::Parser)] +pub struct StateChangesCmd { + #[clap(subcommand)] + command: crate::state_changes::StateChangesSubCommand, +} + +impl StateChangesCmd { + pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { + self.command.run(home_dir, near_config, store) + } +} + #[derive(clap::Parser)] pub struct ViewChainCmd { #[clap(long)] @@ -639,3 +572,42 @@ impl ViewTrieCmd { } } } + +#[derive(clap::Parser)] +pub struct StatePartsCmd { + /// Shard id. + #[clap(long)] + shard_id: ShardId, + /// Choose a single part id. + /// If None - affects all state parts. + #[clap(long)] + part_id: Option, + /// Location of serialized state parts. + #[clap(long)] + root_dir: Option, + /// Store state parts in an S3 bucket. + #[clap(long)] + s3_bucket: Option, + /// Store state parts in an S3 bucket. + #[clap(long)] + s3_region: Option, + + /// Dump or Apply state parts. + #[clap(subcommand)] + command: crate::state_parts::StatePartsSubCommand, +} + +impl StatePartsCmd { + pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { + self.command.run( + self.shard_id, + self.part_id, + self.root_dir, + self.s3_bucket, + self.s3_region, + home_dir, + near_config, + store, + ); + } +} diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index 945e8c6490e..63db6a1d4df 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -6,7 +6,7 @@ use near_epoch_manager::EpochManager; use near_primitives::epoch_manager::epoch_info::EpochInfo; use near_primitives::state_part::PartId; use near_primitives::syncing::get_num_state_parts; -use near_primitives::types::EpochId; +use near_primitives::types::{EpochId, StateRoot}; use near_primitives_core::hash::CryptoHash; use near_primitives_core::types::{BlockHeight, EpochHeight, ShardId}; use near_store::Store; @@ -20,7 +20,80 @@ use std::sync::Arc; use std::time::Instant; #[derive(Subcommand, Debug, Clone)] -pub(crate) enum EpochSelection { +pub enum StatePartsSubCommand { + /// Apply all or a single state part of a shard. + Apply { + /// If true, validate the state part but don't write it to the DB. + #[clap(long)] + dry_run: bool, + /// If provided, this value will be used instead of looking it up in the headers. + /// Use if those headers or blocks are not available. + #[clap(long)] + state_root: Option, + + /// Selects an epoch. The dump will be of the state at the beginning of this epoch. + #[clap(subcommand)] + epoch_selection: EpochSelection, + }, + /// Dump all or a single state part of a shard. + Dump { + /// Dump part ids starting from this part. + #[clap(long)] + part_from: Option, + /// Dump part ids up to this part (exclusive). + #[clap(long)] + part_to: Option, + + /// Selects an epoch. The dump will be of the state at the beginning of this epoch. + #[clap(subcommand)] + epoch_selection: EpochSelection, + }, +} + +impl StatePartsSubCommand { + pub(crate) fn run( + self, + shard_id: ShardId, + part_id: Option, + root_dir: Option, + s3_bucket: Option, + s3_region: Option, + home_dir: &Path, + near_config: NearConfig, + store: Store, + ) { + match self { + StatePartsSubCommand::Apply { dry_run, state_root, epoch_selection } => { + apply_state_parts( + epoch_selection, + shard_id, + part_id, + dry_run, + state_root, + home_dir, + near_config, + store, + Location::new(root_dir, (s3_bucket, s3_region)), + ); + } + StatePartsSubCommand::Dump { part_from, part_to, epoch_selection } => { + dump_state_parts( + epoch_selection, + shard_id, + part_from.or(part_id), + part_to.or(part_id.map(|x| x + 1)), + home_dir, + near_config, + store, + Location::new(root_dir, (s3_bucket, s3_region)), + ); + } + } + } +} + +#[derive(Subcommand, Debug, Clone)] +pub enum EpochSelection { /// Current epoch. Current, /// Fetch the given epoch. @@ -70,13 +143,13 @@ impl EpochSelection { } } -pub(crate) enum Location { +pub enum Location { Files(PathBuf), S3 { bucket: String, region: String }, } impl Location { - pub(crate) fn new( + pub fn new( root_dir: Option, s3_bucket_and_region: (Option, Option), ) -> Self { @@ -134,10 +207,12 @@ fn get_prev_hash_of_epoch( } } -pub(crate) fn apply_state_parts( +pub fn apply_state_parts( epoch_selection: EpochSelection, shard_id: ShardId, part_id: Option, + dry_run: bool, + maybe_state_root: Option, home_dir: &Path, near_config: NearConfig, store: Store, @@ -156,17 +231,23 @@ pub(crate) fn apply_state_parts( let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); - let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); - let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); - assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); - assert!( - shard_id < sync_prev_block.chunks().len() as u64, - "shard_id: {}, #shards: {}", - shard_id, - sync_prev_block.chunks().len() - ); - let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); + let (state_root, sync_prev_hash) = if let Some(state_root) = maybe_state_root { + (state_root, None) + } else { + let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); + let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); + + assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); + assert!( + shard_id < sync_prev_block.chunks().len() as u64, + "shard_id: {}, #shards: {}", + shard_id, + sync_prev_block.chunks().len() + ); + let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); + (state_root, Some(sync_prev_hash)) + }; let part_storage = get_state_part_reader( location, @@ -176,6 +257,7 @@ pub(crate) fn apply_state_parts( ); let num_parts = part_storage.num_parts(); + assert_ne!(num_parts, 0, "Too few num_parts: {}", num_parts); let part_ids = get_part_ids(part_id, part_id.map(|x| x + 1), num_parts); tracing::info!( target: "state-parts", @@ -192,22 +274,32 @@ pub(crate) fn apply_state_parts( for part_id in part_ids { let timer = Instant::now(); assert!(part_id < num_parts, "part_id: {}, num_parts: {}", part_id, num_parts); - let part = part_storage.read(part_id); - runtime_adapter - .apply_state_part( - shard_id, + let part = part_storage.read(part_id, num_parts); + + if dry_run { + assert!(runtime_adapter.validate_state_part( &state_root, PartId::new(part_id, num_parts), - &part, - &epoch_id, - ) - .unwrap(); - tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Applied a state part"); + &part + )); + tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Validated a state part"); + } else { + runtime_adapter + .apply_state_part( + shard_id, + &state_root, + PartId::new(part_id, num_parts), + &part, + &epoch_id, + ) + .unwrap(); + tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Applied a state part"); + } } tracing::info!(target: "state-parts", total_elapsed_sec = timer.elapsed().as_secs_f64(), "Applied all requested state parts"); } -pub(crate) fn dump_state_parts( +pub fn dump_state_parts( epoch_selection: EpochSelection, shard_id: ShardId, part_from: Option, @@ -277,7 +369,7 @@ pub(crate) fn dump_state_parts( PartId::new(part_id, num_parts), ) .unwrap(); - part_storage.write(&state_part, part_id); + part_storage.write(&state_part, part_id, num_parts); tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote a state part"); } tracing::info!(target: "state-parts", total_elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote all requested state parts"); @@ -291,21 +383,36 @@ fn location_prefix(chain_id: &str, epoch_height: u64, shard_id: u64) -> String { format!("chain_id={}/epoch_height={}/shard_id={}", chain_id, epoch_height, shard_id) } +fn match_filename(s: &str) -> Option { + let re = regex::Regex::new(r"^state_part_(\d{6})_of_(\d{6})$").unwrap(); + re.captures(s) +} + fn is_part_filename(s: &str) -> bool { - let re = regex::Regex::new(r"^state_part_(\d{6})$").unwrap(); - re.is_match(s) + match_filename(s).is_some() +} + +fn get_num_parts_from_filename(s: &str) -> Option { + if let Some(captures) = match_filename(s) { + if let Some(num_parts) = captures.get(2) { + if let Ok(num_parts) = num_parts.as_str().parse::() { + return Some(num_parts); + } + } + } + None } -fn part_filename(part_id: u64) -> String { - format!("state_part_{:06}", part_id) +fn part_filename(part_id: u64, num_parts: u64) -> String { + format!("state_part_{:06}_of_{:06}", part_id, num_parts) } trait StatePartWriter { - fn write(&self, state_part: &[u8], part_id: u64); + fn write(&self, state_part: &[u8], part_id: u64, num_parts: u64); } trait StatePartReader { - fn read(&self, part_id: u64) -> Vec; + fn read(&self, part_id: u64, num_parts: u64) -> Vec; fn num_parts(&self) -> u64; } @@ -362,22 +469,22 @@ impl FileSystemStorage { Self { state_parts_dir } } - fn get_location(&self, part_id: u64) -> PathBuf { - (&self.state_parts_dir).join(part_filename(part_id)) + fn get_location(&self, part_id: u64, num_parts: u64) -> PathBuf { + (&self.state_parts_dir).join(part_filename(part_id, num_parts)) } } impl StatePartWriter for FileSystemStorage { - fn write(&self, state_part: &[u8], part_id: u64) { - let filename = self.get_location(part_id); + fn write(&self, state_part: &[u8], part_id: u64, num_parts: u64) { + let filename = self.get_location(part_id, num_parts); std::fs::write(&filename, state_part).unwrap(); tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), ?filename, "Wrote a state part to disk"); } } impl StatePartReader for FileSystemStorage { - fn read(&self, part_id: u64) -> Vec { - let filename = self.get_location(part_id); + fn read(&self, part_id: u64, num_parts: u64) -> Vec { + let filename = self.get_location(part_id, num_parts); let part = std::fs::read(filename).unwrap(); part } @@ -421,22 +528,22 @@ impl S3Storage { Self { location, bucket } } - fn get_location(&self, part_id: u64) -> String { - format!("{}/{}", self.location, part_filename(part_id)) + fn get_location(&self, part_id: u64, num_parts: u64) -> String { + format!("{}/{}", self.location, part_filename(part_id, num_parts)) } } impl StatePartWriter for S3Storage { - fn write(&self, state_part: &[u8], part_id: u64) { - let location = self.get_location(part_id); + fn write(&self, state_part: &[u8], part_id: u64, num_parts: u64) { + let location = self.get_location(part_id, num_parts); self.bucket.put_object_blocking(&location, &state_part).unwrap(); tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); } } impl StatePartReader for S3Storage { - fn read(&self, part_id: u64) -> Vec { - let location = self.get_location(part_id); + fn read(&self, part_id: u64, num_parts: u64) -> Vec { + let location = self.get_location(part_id, num_parts); let response = self.bucket.get_object_blocking(location.clone()).unwrap(); tracing::info!(target: "state-parts", part_id, location, response_code = response.status_code(), "Got an object from S3"); assert_eq!(response.status_code(), 200); @@ -449,17 +556,27 @@ impl StatePartReader for S3Storage { let list: Vec = self.bucket.list_blocking(location, Some("/".to_string())).unwrap(); assert_eq!(list.len(), 1); - let num_parts = list[0] + let mut known_num_parts = None; + let num_objects = list[0] .contents .iter() .filter(|object| { let filename = Path::new(&object.key); let filename = filename.file_name().unwrap().to_str().unwrap(); tracing::debug!(target: "state-parts", object_key = ?object.key, ?filename); + if let Some(num_parts) = get_num_parts_from_filename(filename) { + if let Some(known_num_parts) = known_num_parts { + assert_eq!(known_num_parts, num_parts); + } + known_num_parts = Some(num_parts); + } is_part_filename(filename) }) .collect::>() .len(); - num_parts as u64 + if let Some(known_num_parts) = known_num_parts { + assert_eq!(known_num_parts, num_objects as u64); + } + num_objects as u64 } } From c61ff2a5dfc20e5d8b6d5a5fd04416a4bc89262b Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 15 Mar 2023 16:58:00 +0100 Subject: [PATCH 26/88] Merge --- chain/chain/src/store.rs | 6 +- chain/client/src/sync/state.rs | 6 +- core/chain-configs/src/client_config.rs | 1 + nearcore/src/config.rs | 9 +- nearcore/src/metrics.rs | 35 +++- nearcore/src/runtime/mod.rs | 96 +++------ tools/cold-store/src/cli.rs | 268 +++++++++++++++++++++++- tools/state-viewer/src/state_parts.rs | 8 +- 8 files changed, 338 insertions(+), 91 deletions(-) diff --git a/chain/chain/src/store.rs b/chain/chain/src/store.rs index cbdc3a37ff2..22ec2b5d0ea 100644 --- a/chain/chain/src/store.rs +++ b/chain/chain/src/store.rs @@ -47,7 +47,6 @@ use crate::chunks_store::ReadOnlyChunksStore; use crate::types::{Block, BlockHeader, LatestKnown}; use crate::{byzantine_assert, RuntimeWithEpochManagerAdapter}; use near_store::db::StoreStatistics; -use near_store::flat_state::{BlockInfo, ChainAccessForFlatStorage}; use std::sync::Arc; /// lru cache size @@ -2438,7 +2437,10 @@ impl<'a> ChainStoreUpdate<'a> { unreachable!(); } #[cfg(feature = "protocol_feature_flat_state")] - DBCol::FlatState | DBCol::FlatStateDeltas | DBCol::FlatStateMisc => { + DBCol::FlatState + | DBCol::FlatStateChanges + | DBCol::FlatStateDeltaMetadata + | DBCol::FlatStorageStatus => { unreachable!(); } } diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index e93952671f3..12efb580d2e 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -1236,17 +1236,17 @@ fn check_external_storage_part_response( .with_label_values(&[&shard_id.to_string()]) .inc(); tracing::warn!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to save a state part"); - err_to_retry = Some(Error::Other("Failed to save a state part".to_string())); + err_to_retry = Some(near_chain::Error::Other("Failed to save a state part".to_string())); } } } // Other HTTP status codes are considered errors. Ok((status_code, _)) => { - err_to_retry = Some(Error::Other(format!("status_code: {}", status_code).to_string())); + err_to_retry = Some(near_chain::Error::Other(format!("status_code: {}", status_code).to_string())); } // The request failed without reaching the external storage. Err(err) => { - err_to_retry = Some(Error::Other(err.to_string())); + err_to_retry = Some(near_chain::Error::Other(err.to_string())); } }; diff --git a/core/chain-configs/src/client_config.rs b/core/chain-configs/src/client_config.rs index c71ae93a895..8e66a3f7075 100644 --- a/core/chain-configs/src/client_config.rs +++ b/core/chain-configs/src/client_config.rs @@ -253,6 +253,7 @@ impl ClientConfig { state_sync_s3_bucket: String::new(), state_sync_s3_region: String::new(), state_sync_restart_dump_for_shards: vec![], + state_sync_from_s3_enabled: false, state_sync_num_s3_requests_per_shard: 10, } } diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index 2f61744bd48..cadab343a22 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -726,7 +726,6 @@ impl NearConfig { .state_sync .as_ref() .map(|x| x.sync_from_s3_enabled) - .flatten() .unwrap_or(false), state_sync_num_s3_requests_per_shard: config .state_sync @@ -1565,14 +1564,14 @@ pub struct StateSyncConfig { /// Region is very important on S3. pub s3_region: String, /// Whether a node should dump state of each epoch to the external storage. - #[serde(skip_serializing_if = "Option::is_none")] - pub dump_enabled: Option, + #[serde(skip_serializing_if = "is_false")] + pub dump_enabled: bool, /// Use carefully in case a node that dumps state to the external storage gets in trouble. #[serde(skip_serializing_if = "Option::is_none")] pub drop_state_of_dump: Option>, /// If enabled, will download state parts from external storage and not from the peers. - #[serde(skip_serializing_if = "Option::is_none")] - pub sync_from_s3_enabled: Option, + #[serde(skip_serializing_if = "is_false")] + pub sync_from_s3_enabled: bool, #[serde(skip_serializing_if = "Option::is_none")] pub num_s3_requests_per_shard: Option, } diff --git a/nearcore/src/metrics.rs b/nearcore/src/metrics.rs index 9b5d99bf01f..4fd4fa910c2 100644 --- a/nearcore/src/metrics.rs +++ b/nearcore/src/metrics.rs @@ -1,6 +1,7 @@ use near_o11y::metrics::{ exponential_buckets, linear_buckets, try_create_histogram_vec, try_create_int_counter_vec, - try_create_int_gauge, HistogramVec, IntCounterVec, IntGauge, + try_create_int_gauge, try_create_int_gauge_vec, HistogramVec, IntCounterVec, IntGauge, + IntGaugeVec, }; use once_cell::sync::Lazy; @@ -58,3 +59,35 @@ pub(crate) static STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED: Lazy = Lazy ) .unwrap() }); +pub(crate) static STATE_SYNC_DUMP_NUM_PARTS_TOTAL: Lazy = Lazy::new(|| { + try_create_int_gauge_vec( + "near_state_sync_dump_num_parts_total", + "Total number of parts in the epoch that being dumped", + &["shard_id"], + ) + .unwrap() +}); +pub(crate) static STATE_SYNC_DUMP_NUM_PARTS_DUMPED: Lazy = Lazy::new(|| { + try_create_int_gauge_vec( + "near_state_sync_dump_num_parts_dumped", + "Number of parts dumped in the epoch that is being dumped", + &["shard_id"], + ) + .unwrap() +}); +pub(crate) static STATE_SYNC_DUMP_SIZE_TOTAL: Lazy = Lazy::new(|| { + try_create_int_counter_vec( + "near_state_sync_dump_size_total", + "Total size of parts written to S3", + &["shard_id"], + ) + .unwrap() +}); +pub(crate) static STATE_SYNC_DUMP_EPOCH_HEIGHT: Lazy = Lazy::new(|| { + try_create_int_gauge_vec( + "near_state_sync_dump_epoch_height", + "Epoch Height of an epoch being dumped", + &["shard_id"], + ) + .unwrap() +}); diff --git a/nearcore/src/runtime/mod.rs b/nearcore/src/runtime/mod.rs index 32595e631d1..32a60e2a5c9 100644 --- a/nearcore/src/runtime/mod.rs +++ b/nearcore/src/runtime/mod.rs @@ -48,10 +48,7 @@ use near_primitives::views::{ AccessKeyInfoView, CallResult, QueryRequest, QueryResponse, QueryResponseKind, ViewApplyState, ViewStateResult, }; -use near_store::flat_state::ChainAccessForFlatStorage; -use near_store::flat_state::{ - store_helper, FlatStateFactory, FlatStorageCreationStatus, FlatStorageState, -}; +use near_store::flat::{store_helper, FlatStorage, FlatStorageManager, FlatStorageStatus}; use near_store::metadata::DbKind; use near_store::split_state::get_delayed_receipts; use near_store::{ @@ -89,7 +86,7 @@ pub struct NightshadeRuntime { store: Store, tries: ShardTries, trie_viewer: TrieViewer, - flat_state_factory: FlatStateFactory, + flat_storage_manager: FlatStorageManager, pub runtime: Runtime, epoch_manager: EpochManagerHandle, shard_tracker: ShardTracker, @@ -145,12 +142,12 @@ impl NightshadeRuntime { ); let state_roots = Self::initialize_genesis_state_if_needed(store.clone(), home_dir, genesis); - let flat_state_factory = FlatStateFactory::new(store.clone()); + let flat_storage_manager = FlatStorageManager::new(store.clone()); let tries = ShardTries::new( store.clone(), trie_config, &genesis_config.shard_layout.get_shard_uids(), - flat_state_factory.clone(), + flat_storage_manager.clone(), ); let epoch_manager = EpochManager::new_from_genesis_config(store.clone().into(), &genesis_config) @@ -166,7 +163,7 @@ impl NightshadeRuntime { trie_viewer, epoch_manager, shard_tracker, - flat_state_factory, + flat_storage_manager, genesis_state_roots: state_roots, migration_data: Arc::new(load_migration_data(&genesis.config.chain_id)), gc_num_epochs_to_keep: gc_num_epochs_to_keep.max(MIN_GC_NUM_EPOCHS_TO_KEEP), @@ -273,7 +270,7 @@ impl NightshadeRuntime { store.clone(), TrieConfig::default(), &genesis.config.shard_layout.get_shard_uids(), - FlatStateFactory::new(store), + FlatStorageManager::new(store), ); let runtime = Runtime::new(); let runtime_config_store = @@ -556,11 +553,6 @@ impl NightshadeRuntime { metrics::APPLY_CHUNK_DELAY .with_label_values(&[&format_total_gas_burnt(total_gas_burnt)]) .observe(elapsed.as_secs_f64()); - if total_gas_burnt > 0 { - metrics::SECONDS_PER_PETAGAS - .with_label_values(&[&shard_id.to_string()]) - .observe(elapsed.as_secs_f64() * 1e15 / total_gas_burnt as f64); - } let total_balance_burnt = apply_result .stats .tx_burnt_amount @@ -758,8 +750,8 @@ impl RuntimeAdapter for NightshadeRuntime { self.flat_storage_manager.get_flat_storage_for_shard(shard_id) } - fn get_flat_storage_creation_status(&self, shard_id: ShardId) -> FlatStorageCreationStatus { - store_helper::get_flat_storage_creation_status(&self.store, shard_id) + fn get_flat_storage_status(&self, shard_uid: ShardUId) -> FlatStorageStatus { + store_helper::get_flat_storage_status(&self.store, shard_uid) } // TODO (#7327): consider passing flat storage errors here to handle them gracefully @@ -768,7 +760,7 @@ impl RuntimeAdapter for NightshadeRuntime { self.flat_storage_manager.add_flat_storage_for_shard(shard_uid.shard_id(), flat_storage); } - fn remove_flat_storage_state_for_shard( + fn remove_flat_storage_for_shard( &self, shard_uid: ShardUId, epoch_id: &EpochId, @@ -780,17 +772,20 @@ impl RuntimeAdapter for NightshadeRuntime { Ok(()) } - fn set_flat_storage_state_for_genesis( + fn set_flat_storage_for_genesis( &self, genesis_block: &CryptoHash, + genesis_block_height: BlockHeight, genesis_epoch_id: &EpochId, ) -> Result { let mut store_update = self.store.store_update(); for shard_id in 0..self.num_shards(genesis_epoch_id)? { - self.flat_state_factory.set_flat_storage_state_for_genesis( + let shard_uid = self.shard_id_to_uid(shard_id, genesis_epoch_id)?; + self.flat_storage_manager.set_flat_storage_for_genesis( &mut store_update, - shard_id, + shard_uid, genesis_block, + genesis_block_height, ); } Ok(store_update) @@ -1398,8 +1393,10 @@ impl RuntimeAdapter for NightshadeRuntime { let shard_uid = self.get_shard_uid_from_epoch_id(shard_id, epoch_id)?; let mut store_update = tries.store_update(); tries.apply_all(&trie_changes, shard_uid, &mut store_update); - debug!(target: "chain", %shard_id, "Inserting {} values to flat storage", flat_state_delta.len()); - flat_state_delta.apply_to_flat_state(&mut store_update); + if cfg!(feature = "protocol_feature_flat_state") { + debug!(target: "chain", %shard_id, "Inserting {} values to flat storage", flat_state_delta.len()); + flat_state_delta.apply_to_flat_state(&mut store_update, shard_uid); + } self.precompile_contracts(epoch_id, contract_codes)?; Ok(store_update.commit()?) } @@ -1577,6 +1574,7 @@ mod test { use near_epoch_manager::shard_tracker::TrackedConfig; use near_primitives::test_utils::create_test_signer; use near_primitives::types::validator_stake::ValidatorStake; + use near_store::flat::{FlatStateChanges, FlatStateDelta, FlatStateDeltaMetadata}; use num_rational::Ratio; use crate::config::{GenesisExt, TESTING_INIT_BALANCE, TESTING_INIT_STAKE}; @@ -1595,7 +1593,7 @@ mod test { AccountView, CurrentEpochValidatorInfo, EpochValidatorInfo, NextEpochValidatorInfo, ValidatorKickoutView, }; - use near_store::{flat_state, FlatStateDelta, NodeStorage}; + use near_store::NodeStorage; use super::*; @@ -1657,8 +1655,8 @@ mod test { ) .unwrap(); let mut store_update = self.store.store_update(); - let flat_state_delta = - FlatStateDelta::from_state_changes(&result.trie_changes.state_changes()); + let flat_state_changes = + FlatStateChanges::from_state_changes(&result.trie_changes.state_changes()); result.trie_changes.insertions_into(&mut store_update); result.trie_changes.state_changes_into(&mut store_update); @@ -1674,9 +1672,7 @@ mod test { }, }, }; - let new_store_update = flat_storage_state - .add_block(&block_hash, flat_state_delta, block_info) - .unwrap(); + let new_store_update = flat_storage.add_delta(delta).unwrap(); store_update.merge(new_store_update); } None => {} @@ -1687,43 +1683,6 @@ mod test { } } - /// Stores chain data for genesis block to initialize flat storage in test environment. - struct MockChainForFlatStorage { - height_to_hashes: HashMap, - blocks: HashMap, - } - - impl ChainAccessForFlatStorage for MockChainForFlatStorage { - fn get_block_info(&self, block_hash: &CryptoHash) -> flat_state::BlockInfo { - self.blocks.get(block_hash).unwrap().clone() - } - - fn get_block_hashes_at_height(&self, block_height: BlockHeight) -> HashSet { - HashSet::from([self.get_block_hash(block_height)]) - } - } - - impl MockChainForFlatStorage { - /// Creates mock chain containing only genesis block data. - pub fn new(genesis_height: BlockHeight, genesis_hash: CryptoHash) -> Self { - Self { - height_to_hashes: HashMap::from([(genesis_height, genesis_hash)]), - blocks: HashMap::from([( - genesis_hash, - flat_state::BlockInfo { - hash: genesis_hash, - height: genesis_height, - prev_hash: CryptoHash::default(), - }, - )]), - } - } - - fn get_block_hash(&self, height: BlockHeight) -> CryptoHash { - *self.height_to_hashes.get(&height).unwrap() - } - } - /// Environment to test runtime behaviour separate from Chain. /// Runtime operates in a mock chain where i-th block is attached to (i-1)-th one, has height `i` and hash /// `hash([i])`. @@ -3106,13 +3065,16 @@ mod test { .runtime .get_trie_for_shard(0, &env.head.prev_block_hash, Trie::EMPTY_ROOT, true) .unwrap(); - assert_eq!(trie.flat_state.is_some(), cfg!(feature = "protocol_feature_flat_state")); + assert_eq!( + trie.flat_storage_chunk_view.is_some(), + cfg!(feature = "protocol_feature_flat_state") + ); let trie = env .runtime .get_view_trie_for_shard(0, &env.head.prev_block_hash, Trie::EMPTY_ROOT) .unwrap(); - assert!(trie.flat_state.is_none()); + assert!(trie.flat_storage_chunk_view.is_none()); } /// Check that querying trie and flat state gives the same result. diff --git a/tools/cold-store/src/cli.rs b/tools/cold-store/src/cli.rs index a5f11780d03..a6a2f976f33 100644 --- a/tools/cold-store/src/cli.rs +++ b/tools/cold-store/src/cli.rs @@ -1,13 +1,16 @@ +use crate::cli::SubCommand::CheckStateRoot; use anyhow; +use anyhow::Context; use clap; use near_epoch_manager::EpochManagerAdapter; use near_primitives::block::Tip; use near_primitives::hash::CryptoHash; use near_store::cold_storage::{copy_all_data_to_cold, update_cold_db, update_cold_head}; use near_store::metadata::DbKind; -use near_store::{DBCol, NodeStorage, Store}; +use near_store::{DBCol, NodeStorage, Store, StoreOpener}; use near_store::{COLD_HEAD_KEY, FINAL_HEAD_KEY, HEAD_KEY, TAIL_KEY}; use nearcore::{NearConfig, NightshadeRuntime}; +use rand::seq::SliceRandom; use std::io::Result; use std::path::Path; use std::sync::Arc; @@ -15,6 +18,10 @@ use strum::IntoEnumIterator; #[derive(clap::Parser)] pub struct ColdStoreCommand { + /// By default state viewer opens rocks DB in the read only mode, which allows it to run + /// multiple instances in parallel and be sure that no unintended changes get written to the DB. + #[clap(long, short = 'w')] + readwrite: bool, #[clap(subcommand)] subcmd: SubCommand, } @@ -39,23 +46,28 @@ enum SubCommand { /// - config.cold_store.path points to an existing database with kind Cold /// - store_relative_path points to an existing database with kind Rpc PrepareHot(PrepareHotCmd), + /// Traverse trie and check that every node is in cold db. + /// Can start from given state_root or compute previous roots for every chunk in provided block + /// and use them as starting point. + /// You can provide maximum depth and/or maximum number of vertices to traverse for each root. + /// Trie is traversed using DFS with randomly shuffled kids for every node. + CheckStateRoot(CheckStateRootCmd), } impl ColdStoreCommand { pub fn run(self, home_dir: &Path) -> anyhow::Result<()> { - let near_config = nearcore::config::load_config( + let mode = + if self.readwrite { near_store::Mode::ReadWrite } else { near_store::Mode::ReadOnly }; + let mut near_config = nearcore::config::load_config( &home_dir, near_chain_configs::GenesisValidationMode::Full, ) .unwrap_or_else(|e| panic!("Error loading config: {:#}", e)); - let opener = NodeStorage::opener( - home_dir, - true, - &near_config.config.store, - near_config.config.cold_store.as_ref(), - ); - let storage = opener.open().unwrap_or_else(|e| panic!("Error opening storage: {:#}", e)); + let opener = self.get_opener(home_dir, &mut near_config); + + let storage = + opener.open_in_mode(mode).unwrap_or_else(|e| panic!("Error opening storage: {:#}", e)); let hot_runtime = NightshadeRuntime::from_config(home_dir, storage.get_hot_store(), &near_config); @@ -73,8 +85,53 @@ impl ColdStoreCommand { Ok(()) } SubCommand::PrepareHot(cmd) => cmd.run(&storage, &home_dir, &near_config), + SubCommand::CheckStateRoot(cmd) => cmd.run(&storage), } } + + /// Returns opener suitable for subcommand. + /// If subcommand is CheckStateRoot, creates checkpoint for cold db + /// and modifies `near_config.config.cold_store.path` to path to that checkpoint. + /// Then returns opener for dbs at `store.path` and `cold_store.path`. + pub fn get_opener<'a>( + &'a self, + home_dir: &Path, + near_config: &'a mut NearConfig, + ) -> StoreOpener<'a> { + if !near_config.config.archive { + tracing::warn!("Expected archive option in config to be set to true."); + } + + let opener = NodeStorage::opener( + home_dir, + near_config.config.archive, + &near_config.config.store, + near_config.config.cold_store.as_ref(), + ); + + match self.subcmd { + CheckStateRoot(_) => { + let (hot_snapshot, cold_snapshot) = opener + .create_snapshots(near_store::Mode::ReadOnly) + .expect("Failed to create snapshots"); + if let Some(_) = &hot_snapshot.0 { + hot_snapshot.remove().expect("Failed to remove unnecessary hot snapshot"); + } + if let Some(cold_store_config) = near_config.config.cold_store.as_mut() { + cold_store_config.path = + Some(cold_snapshot.0.clone().expect("cold_snapshot should be Some")); + } + } + _ => {} + } + + NodeStorage::opener( + home_dir, + near_config.config.archive, + &near_config.config.store, + near_config.config.cold_store.as_ref(), + ) + } } #[derive(clap::Parser)] @@ -381,3 +438,196 @@ impl PrepareHotCmd { Ok(()) } } + +/// The StateRootSelector is a subcommand that allows the user to select the state root either by block height or by the state root hash. +#[derive(clap::Subcommand)] +enum StateRootSelector { + Height { height: near_primitives::types::BlockHeight }, + Hash { hash: CryptoHash }, +} + +impl StateRootSelector { + pub fn get_hashes( + &self, + storage: &NodeStorage, + cold_store: &Store, + ) -> anyhow::Result> { + match self { + // If height is provided, calculate previous state roots for this block's chunks. + StateRootSelector::Height { height } => { + let hash_key = { + let height_key = height.to_le_bytes(); + storage + .get_hot_store() + .get(DBCol::BlockHeight, &height_key)? + .ok_or(anyhow::anyhow!( + "Failed to find block hash for height {:?}", + height + ))? + .as_slice() + .to_vec() + }; + let block = cold_store + .get_ser::(DBCol::Block, &hash_key)? + .ok_or(anyhow::anyhow!("Failed to find Block: {:?}", hash_key))?; + let mut hashes = vec![]; + for chunk in block.chunks().iter() { + hashes.push( + cold_store + .get_ser::( + DBCol::Chunks, + chunk.chunk_hash().as_bytes(), + )? + .ok_or(anyhow::anyhow!( + "Failed to find Chunk: {:?}", + chunk.chunk_hash() + ))? + .take_header() + .prev_state_root(), + ); + } + Ok(hashes) + } + // If state root is provided, then just use it. + StateRootSelector::Hash { hash } => Ok(vec![*hash]), + } + } +} + +/// Struct that holds all conditions for node in Trie +/// to be checked by CheckStateRootCmd::check_trie. +#[derive(Debug)] +struct PruneCondition { + /// Maximum depth (measured in number of nodes, not trie key length). + max_depth: Option, + /// Maximum number of nodes checked for each state_root. + max_count: Option, +} + +/// Struct that holds data related to pruning of node in CheckStateRootCmd::check_trie. +#[derive(Debug)] +struct PruneState { + /// Depth of node in trie (measured in number of nodes, not trie key length). + depth: u64, + /// Number of already checked nodes. + count: u64, +} + +impl PruneState { + pub fn new() -> Self { + Self { depth: 0, count: 0 } + } + + /// Return `true` if node should be pruned. + pub fn should_prune(&self, condition: &PruneCondition) -> bool { + if let Some(md) = condition.max_depth { + if self.depth > md { + return true; + } + } + if let Some(mc) = condition.max_count { + if self.count > mc { + return true; + } + } + false + } + + /// Modify self to reflect going down a tree. + /// We increment node count, because we are visiting a new node. + pub fn down(&mut self) { + self.count += 1; + self.depth += 1; + } + + /// Modify self to reflect going up a tree. + /// We do not change node count, because we already visited parent node before. + pub fn up(&mut self) { + self.depth -= 1; + } +} + +#[derive(clap::Args)] +struct CheckStateRootCmd { + /// Maximum depth (measured in number of nodes, not trie key length) for checking trie. + #[clap(long)] + max_depth: Option, + /// Maximum number of nodes checked for each state_root. + #[clap(long)] + max_count: Option, + #[clap(subcommand)] + state_root_selector: StateRootSelector, +} + +impl CheckStateRootCmd { + pub fn run(self, storage: &NodeStorage) -> anyhow::Result<()> { + let cold_store = + storage.get_cold_store().ok_or(anyhow::anyhow!("Cold storage is not configured"))?; + + let hashes = self.state_root_selector.get_hashes(storage, &cold_store)?; + for hash in hashes.iter() { + Self::check_trie( + &cold_store, + &hash, + &mut PruneState::new(), + &PruneCondition { max_depth: self.max_depth, max_count: self.max_count }, + )?; + } + + Ok(()) + } + + /// Check that trie subtree of `hash` is fully present in `store`. + fn check_trie( + store: &Store, + hash: &CryptoHash, + prune_state: &mut PruneState, + prune_condition: &PruneCondition, + ) -> anyhow::Result<()> { + tracing::debug!(target: "check_trie", "Checking {:?} at {:?}", hash, prune_state); + if prune_state.should_prune(prune_condition) { + tracing::debug!(target: "check_trie", "Reached prune condition: {:?}", prune_condition); + return Ok(()); + } + + let bytes = Self::read_state(store, hash.as_ref()) + .with_context(|| format!("Failed to read raw bytes for hash {:?}", hash))? + .with_context(|| format!("Failed to find raw bytes for hash {:?}", hash))?; + let node = near_store::RawTrieNodeWithSize::decode(&bytes)?; + match node.node { + near_store::RawTrieNode::Leaf(..) => { + tracing::debug!(target: "check_trie", "Reached leaf node"); + return Ok(()); + } + near_store::RawTrieNode::Branch(mut children, _) => { + children.0.shuffle(&mut rand::thread_rng()); + for (_, child) in children.iter() { + // Record in prune state that we are visiting a child node + prune_state.down(); + // Visit a child node + Self::check_trie(store, child, prune_state, prune_condition)?; + // Record in prune state that we are returning from a child node + prune_state.up(); + } + } + near_store::RawTrieNode::Extension(_, child) => { + // Record in prune state that we are visiting a child node + prune_state.down(); + // Visit a child node + Self::check_trie(store, &child, prune_state, prune_condition)?; + // Record in prune state that we are returning from a child node + prune_state.up(); + } + } + Ok(()) + } + + fn read_state<'a>( + store: &'a Store, + trie_key: &'a [u8], + ) -> std::io::Result>> { + // As cold db strips shard_uid at the beginning of State key, we can add any 8 u8s as prefix. + let cold_state_key = [&[1; 8], trie_key.as_ref()].concat(); + store.get(DBCol::State, &cold_state_key) + } +} diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index 63db6a1d4df..64717e8c2d8 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -143,13 +143,13 @@ impl EpochSelection { } } -pub enum Location { +pub(crate) enum Location { Files(PathBuf), S3 { bucket: String, region: String }, } impl Location { - pub fn new( + fn new( root_dir: Option, s3_bucket_and_region: (Option, Option), ) -> Self { @@ -207,7 +207,7 @@ fn get_prev_hash_of_epoch( } } -pub fn apply_state_parts( +fn apply_state_parts( epoch_selection: EpochSelection, shard_id: ShardId, part_id: Option, @@ -299,7 +299,7 @@ pub fn apply_state_parts( tracing::info!(target: "state-parts", total_elapsed_sec = timer.elapsed().as_secs_f64(), "Applied all requested state parts"); } -pub fn dump_state_parts( +fn dump_state_parts( epoch_selection: EpochSelection, shard_id: ShardId, part_from: Option, From b68242468798d4edf6502119983d60a6a5c5c218 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 16 Mar 2023 18:50:31 +0100 Subject: [PATCH 27/88] Print state_root --- chain/chain/src/chain.rs | 5 ++++- chain/client/src/sync/state.rs | 6 ++++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/chain/chain/src/chain.rs b/chain/chain/src/chain.rs index e389a376007..81186452023 100644 --- a/chain/chain/src/chain.rs +++ b/chain/chain/src/chain.rs @@ -3139,7 +3139,10 @@ impl Chain { let state_root = *chunk.take_header().take_inner().prev_state_root(); if !self.runtime_adapter.validate_state_part(&state_root, part_id, data) { byzantine_assert!(false); - return Err(Error::Other("set_state_part failed: validate_state_part failed".into())); + return Err(Error::Other(format!( + "set_state_part failed: validate_state_part failed. state_root: {}", + state_root + ))); } // Saving the part data. diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 12efb580d2e..cdfdf731b43 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -1236,13 +1236,15 @@ fn check_external_storage_part_response( .with_label_values(&[&shard_id.to_string()]) .inc(); tracing::warn!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to save a state part"); - err_to_retry = Some(near_chain::Error::Other("Failed to save a state part".to_string())); + err_to_retry = + Some(near_chain::Error::Other("Failed to save a state part".to_string())); } } } // Other HTTP status codes are considered errors. Ok((status_code, _)) => { - err_to_retry = Some(near_chain::Error::Other(format!("status_code: {}", status_code).to_string())); + err_to_retry = + Some(near_chain::Error::Other(format!("status_code: {}", status_code).to_string())); } // The request failed without reaching the external storage. Err(err) => { From 4f43f8c3c82ab687b8578f0424fc3806b5ad2e0c Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 16 Mar 2023 18:57:58 +0100 Subject: [PATCH 28/88] Print state_root --- tools/state-viewer/src/state_parts.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index 64717e8c2d8..d10b5612c59 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -466,6 +466,7 @@ impl FileSystemStorage { tracing::info!(target: "state-parts", ?root_dir, ?prefix, ?state_parts_dir, "Ensuring the directory exists"); std::fs::create_dir_all(&state_parts_dir).unwrap(); } + tracing::info!(target: "state-parts", ?state_parts_dir, "Initialized FileSystemStorage"); Self { state_parts_dir } } From 161a3574bbc6ba614514028844750edbdb2babac Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 11:29:31 +0100 Subject: [PATCH 29/88] Changelog --- CHANGELOG.md | 23 ++--------------------- 1 file changed, 2 insertions(+), 21 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4eafa24785a..f4899e654d3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,28 +6,9 @@ ### Non-protocol Changes -## 1.33.0 - -### Protocol Changes - -### Non-protocol Changes -* State-viewer tool to dump and apply state changes from/to a range of blocks. [#8628](https://github.com/near/nearcore/pull/8628) * Experimental option to dump state of every epoch to external storage. [#8661](https://github.com/near/nearcore/pull/8661) -* Add prometheus metrics for tracked shards, block height within epoch, if is block/chunk producer. [#8728](https://github.com/near/nearcore/pull/8728) -* State sync is disabled by default [#8730](https://github.com/near/nearcore/pull/8730) -* Node can restart if State Sync gets interrupted. [#8732](https://github.com/near/nearcore/pull/8732) -* Merged two `neard view-state` commands: `apply-state-parts` and `dump-state-parts` into a single `state-parts` command. [#8739](https://github.com/near/nearcore/pull/8739) -* Add config.network.experimental.network_config_overrides to the JSON config. [#8871](https://github.com/near/nearcore/pull/8871) - -## 1.32.2 - -### Fixes -* Fix: rosetta zero balance accounts [#8833](https://github.com/near/nearcore/pull/8833) - -## 1.32.1 - -### Fixes -* Fix vulnerabilities in block outcome root validation and total supply validation [#8790](https://github.com/near/nearcore/pull/8790) +* State-viewer tool to dump and apply state changes from/to a range of blocks [#8628](https://github.com/near/nearcore/pull/8628) +* Node can sync State from S3. [#XXXX](https://github.com/near/nearcore/pull/XXXX) ## 1.32.0 From 020d7f9f5d404b9a79580993bd12e9f36ca9c0fe Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 12:25:39 +0100 Subject: [PATCH 30/88] Merge --- CHANGELOG.md | 5 +- chain/chain/src/chain.rs | 5 +- core/store/src/flat/storage.rs | 57 +++++++++++++------ core/store/src/trie/mod.rs | 29 ++++++---- core/store/src/trie/split_state.rs | 10 ++-- nearcore/src/runtime/mod.rs | 35 ++++++------ .../src/estimator_context.rs | 47 ++++++++++----- tools/state-viewer/src/state_parts.rs | 1 + 8 files changed, 123 insertions(+), 66 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f4899e654d3..5ee0bc2e1ef 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,8 +6,11 @@ ### Non-protocol Changes +* State-viewer tool to dump and apply state changes from/to a range of blocks. [#8628](https://github.com/near/nearcore/pull/8628) * Experimental option to dump state of every epoch to external storage. [#8661](https://github.com/near/nearcore/pull/8661) -* State-viewer tool to dump and apply state changes from/to a range of blocks [#8628](https://github.com/near/nearcore/pull/8628) +* Add prometheus metrics for tracked shards, block height within epoch, if is block/chunk producer. [#8728](https://github.com/near/nearcore/pull/8728) +* Node can restart if State Sync gets interrupted. [#8732](https://github.com/near/nearcore/pull/8732) +* Merged two `neard view-state` commands: `apply-state-parts` and `dump-state-parts` into a single `state-parts` command. [#8739](https://github.com/near/nearcore/pull/8739) * Node can sync State from S3. [#XXXX](https://github.com/near/nearcore/pull/XXXX) ## 1.32.0 diff --git a/chain/chain/src/chain.rs b/chain/chain/src/chain.rs index 81186452023..e389a376007 100644 --- a/chain/chain/src/chain.rs +++ b/chain/chain/src/chain.rs @@ -3139,10 +3139,7 @@ impl Chain { let state_root = *chunk.take_header().take_inner().prev_state_root(); if !self.runtime_adapter.validate_state_part(&state_root, part_id, data) { byzantine_assert!(false); - return Err(Error::Other(format!( - "set_state_part failed: validate_state_part failed. state_root: {}", - state_root - ))); + return Err(Error::Other("set_state_part failed: validate_state_part failed".into())); } // Saving the part data. diff --git a/core/store/src/flat/storage.rs b/core/store/src/flat/storage.rs index 93d5c8b5af6..01acf7d5658 100644 --- a/core/store/src/flat/storage.rs +++ b/core/store/src/flat/storage.rs @@ -1,6 +1,7 @@ use std::collections::HashMap; use std::sync::{Arc, RwLock}; +use near_o11y::metrics::IntGauge; use near_primitives::errors::StorageError; use near_primitives::hash::CryptoHash; use near_primitives::shard_layout::{ShardLayout, ShardUId}; @@ -10,10 +11,9 @@ use tracing::{info, warn}; use crate::flat::delta::CachedFlatStateChanges; use crate::flat::store_helper::FlatStateColumn; use crate::flat::{FlatStorageReadyStatus, FlatStorageStatus}; -use crate::{Store, StoreUpdate}; +use crate::{metrics, Store, StoreUpdate}; use super::delta::{CachedFlatStateDelta, FlatStateDelta}; -use super::metrics::FlatStorageMetrics; use super::types::FlatStorageError; use super::{store_helper, BlockInfo}; @@ -45,6 +45,14 @@ pub(crate) struct FlatStorageInner { metrics: FlatStorageMetrics, } +struct FlatStorageMetrics { + flat_head_height: IntGauge, + distance_to_head: IntGauge, + cached_deltas: IntGauge, + cached_changes_num_items: IntGauge, + cached_changes_size: IntGauge, +} + impl FlatStorageInner { /// Expected limits for in-memory stored changes, under which flat storage must keep working. /// If they are exceeded, warnings are displayed. Flat storage still will work, but its @@ -93,7 +101,7 @@ impl FlatStorageInner { .block .prev_hash; } - self.metrics.set_distance_to_head(blocks.len()); + self.metrics.distance_to_head.set(blocks.len() as i64); Ok(blocks) } @@ -108,18 +116,17 @@ impl FlatStorageInner { cached_changes_size += changes.changes.total_size(); } - self.metrics.set_cached_deltas( - cached_deltas, - cached_changes_num_items, - cached_changes_size, - ); + self.metrics.cached_deltas.set(cached_deltas as i64); + self.metrics.cached_changes_num_items.set(cached_changes_num_items as i64); + self.metrics.cached_changes_size.set(cached_changes_size as i64); let cached_changes_size_bytes = bytesize::ByteSize(cached_changes_size); + let shard_id = self.shard_uid.shard_id(); + let flat_head_height = self.flat_head.height; + if cached_deltas >= Self::CACHED_CHANGES_LIMIT || cached_changes_size_bytes >= Self::CACHED_CHANGES_SIZE_LIMIT { - let shard_id = self.shard_uid.shard_id(); - let flat_head_height = self.flat_head.height; warn!(target: "chain", %shard_id, %flat_head_height, %cached_deltas, %cached_changes_size_bytes, "Flat storage cached deltas exceeded expected limits"); } } @@ -137,8 +144,22 @@ impl FlatStorage { panic!("cannot create flat storage for shard {shard_id} with status {status:?}") } }; - let metrics = FlatStorageMetrics::new(shard_id); - metrics.set_flat_head_height(flat_head.height); + + // `itoa` is much faster for printing shard_id to a string than trivial alternatives. + let mut buffer = itoa::Buffer::new(); + let shard_id_label = buffer.format(shard_id); + let metrics = FlatStorageMetrics { + flat_head_height: metrics::FLAT_STORAGE_HEAD_HEIGHT + .with_label_values(&[shard_id_label]), + distance_to_head: metrics::FLAT_STORAGE_DISTANCE_TO_HEAD + .with_label_values(&[shard_id_label]), + cached_deltas: metrics::FLAT_STORAGE_CACHED_DELTAS.with_label_values(&[shard_id_label]), + cached_changes_num_items: metrics::FLAT_STORAGE_CACHED_CHANGES_NUM_ITEMS + .with_label_values(&[shard_id_label]), + cached_changes_size: metrics::FLAT_STORAGE_CACHED_CHANGES_SIZE + .with_label_values(&[shard_id_label]), + }; + metrics.flat_head_height.set(flat_head.height as i64); let deltas_metadata = store_helper::get_all_deltas_metadata(&store, shard_uid) .unwrap_or_else(|_| { @@ -160,9 +181,13 @@ impl FlatStorage { ); } - let inner = FlatStorageInner { store, shard_uid, flat_head, deltas, metrics }; - inner.update_delta_metrics(); - Self(Arc::new(RwLock::new(inner))) + Self(Arc::new(RwLock::new(FlatStorageInner { + store, + shard_uid, + flat_head, + deltas, + metrics, + }))) } /// Get sequence of blocks `target_block_hash` (inclusive) to flat head (exclusive) @@ -223,7 +248,7 @@ impl FlatStorage { FlatStorageStatus::Ready(FlatStorageReadyStatus { flat_head: block.clone() }), ); - guard.metrics.set_flat_head_height(block.height); + guard.metrics.flat_head_height.set(block.height as i64); guard.flat_head = block.clone(); // Remove old deltas from disk and memory. diff --git a/core/store/src/trie/mod.rs b/core/store/src/trie/mod.rs index 39e82b2b95a..fd12b911aaa 100644 --- a/core/store/src/trie/mod.rs +++ b/core/store/src/trie/mod.rs @@ -504,7 +504,7 @@ impl RawTrieNodeWithSize { } pub struct Trie { - pub storage: Box, + pub storage: Box, root: StateRoot, pub flat_storage_chunk_view: Option, } @@ -604,7 +604,7 @@ impl Trie { pub const EMPTY_ROOT: StateRoot = StateRoot::new(); pub fn new( - storage: Box, + storage: Box, root: StateRoot, flat_storage_chunk_view: Option, ) -> Self { @@ -961,20 +961,29 @@ impl Trie { /// storage for key lookup performed in `storage_write`, so we need /// the `use_flat_storage` to differentiate whether the lookup is performed for /// storage_write or not. + #[allow(unused)] pub fn get_ref( &self, key: &[u8], mode: KeyLookupMode, ) -> Result, StorageError> { - let use_flat_storage = - matches!(mode, KeyLookupMode::FlatStorage) && self.flat_storage_chunk_view.is_some(); - - if use_flat_storage { - self.flat_storage_chunk_view.as_ref().unwrap().get_ref(&key) - } else { - let key_nibbles = NibbleSlice::new(key); - self.lookup(key_nibbles) + let key_nibbles = NibbleSlice::new(key); + let result = self.lookup(key_nibbles); + + // For now, to test correctness, flat storage does double the work and + // compares the results. This needs to be changed when the features is + // stabilized. + if matches!(mode, KeyLookupMode::FlatStorage) { + if let Some(flat_storage_chunk_view) = &self.flat_storage_chunk_view { + let flat_result = flat_storage_chunk_view.get_ref(&key); + if matches!(flat_result, Err(StorageError::FlatStorageBlockNotSupported(_))) { + return flat_result; + } else { + assert_eq!(result, flat_result); + } + } } + result } pub fn get(&self, key: &[u8]) -> Result>, StorageError> { diff --git a/core/store/src/trie/split_state.rs b/core/store/src/trie/split_state.rs index 4f0af2f8580..40afdac3264 100644 --- a/core/store/src/trie/split_state.rs +++ b/core/store/src/trie/split_state.rs @@ -102,7 +102,7 @@ impl ShardTries { let mut trie_changes_map = HashMap::new(); for (shard_uid, update) in trie_updates { - let (_, trie_changes, _) = update.finalize()?; + let (trie_changes, _) = update.finalize()?; trie_changes_map.insert(shard_uid, trie_changes); } Ok(trie_changes_map) @@ -196,7 +196,7 @@ impl ShardTries { let mut new_state_roots = HashMap::new(); let mut store_update = self.store_update(); for (shard_uid, update) in updates { - let (_, trie_changes, state_changes) = update.finalize()?; + let (trie_changes, state_changes) = update.finalize()?; let state_root = self.apply_all(&trie_changes, shard_uid, &mut store_update); if cfg!(feature = "protocol_feature_flat_state") { FlatStateChanges::from_state_changes(&state_changes) @@ -525,7 +525,7 @@ mod tests { delayed_receipt_indices.next_available_index = all_receipts.len() as u64; set(&mut trie_update, TrieKey::DelayedReceiptIndices, &delayed_receipt_indices); trie_update.commit(StateChangeCause::Resharding); - let (_, trie_changes, _) = trie_update.finalize().unwrap(); + let (trie_changes, _) = trie_update.finalize().unwrap(); let mut store_update = tries.store_update(); let state_root = tries.apply_all(&trie_changes, ShardUId::single_shard(), &mut store_update); @@ -661,7 +661,7 @@ mod tests { }, ); trie_update.commit(StateChangeCause::Resharding); - let (_, trie_changes, _) = trie_update.finalize().unwrap(); + let (trie_changes, _) = trie_update.finalize().unwrap(); let mut store_update = tries.store_update(); let state_root = tries.apply_all(&trie_changes, ShardUId::single_shard(), &mut store_update); @@ -765,7 +765,7 @@ mod tests { ); set(&mut trie_update, TrieKey::DelayedReceiptIndices, &delayed_receipt_indices); trie_update.commit(StateChangeCause::Resharding); - let (_, trie_changes, state_changes) = trie_update.finalize().unwrap(); + let (trie_changes, state_changes) = trie_update.finalize().unwrap(); let mut store_update = tries.store_update(); let new_state_root = tries.apply_all(&trie_changes, ShardUId::single_shard(), &mut store_update); diff --git a/nearcore/src/runtime/mod.rs b/nearcore/src/runtime/mod.rs index 32a60e2a5c9..358387ae547 100644 --- a/nearcore/src/runtime/mod.rs +++ b/nearcore/src/runtime/mod.rs @@ -746,8 +746,8 @@ impl RuntimeAdapter for NightshadeRuntime { Ok(self.tries.get_view_trie_for_shard(shard_uid, state_root)) } - fn get_flat_storage_for_shard(&self, shard_id: ShardId) -> Option { - self.flat_storage_manager.get_flat_storage_for_shard(shard_id) + fn get_flat_storage_for_shard(&self, shard_uid: ShardUId) -> Option { + self.flat_storage_manager.get_flat_storage_for_shard(shard_uid) } fn get_flat_storage_status(&self, shard_uid: ShardUId) -> FlatStorageStatus { @@ -757,7 +757,7 @@ impl RuntimeAdapter for NightshadeRuntime { // TODO (#7327): consider passing flat storage errors here to handle them gracefully fn create_flat_storage_for_shard(&self, shard_uid: ShardUId) { let flat_storage = FlatStorage::new(self.store.clone(), shard_uid); - self.flat_storage_manager.add_flat_storage_for_shard(shard_uid.shard_id(), flat_storage); + self.flat_storage_manager.add_flat_storage_for_shard(shard_uid, flat_storage); } fn remove_flat_storage_for_shard( @@ -767,7 +767,7 @@ impl RuntimeAdapter for NightshadeRuntime { ) -> Result<(), Error> { let shard_layout = self.get_shard_layout(epoch_id)?; self.flat_storage_manager - .remove_flat_storage_for_shard(shard_id, shard_layout) + .remove_flat_storage_for_shard(shard_uid, shard_layout) .map_err(|e| Error::StorageError(e))?; Ok(()) } @@ -1660,7 +1660,8 @@ mod test { result.trie_changes.insertions_into(&mut store_update); result.trie_changes.state_changes_into(&mut store_update); - match self.get_flat_storage_for_shard(shard_id) { + let shard_uid = self.shard_id_to_uid(shard_id, &EpochId::default()).unwrap(); + match self.get_flat_storage_for_shard(shard_uid) { Some(flat_storage) => { let delta = FlatStateDelta { changes: flat_state_changes, @@ -1786,18 +1787,18 @@ mod test { // Create flat storage. Naturally it happens on Chain creation, but here we test only Runtime behaviour // and use a mock chain, so we need to initialize flat storage manually. - let store_update = runtime - .set_flat_storage_state_for_genesis(&genesis_hash, &EpochId::default()) - .unwrap(); - store_update.commit().unwrap(); - let mock_chain = MockChainForFlatStorage::new(0, genesis_hash); - for shard_id in 0..runtime.num_shards(&EpochId::default()).unwrap() { - let status = runtime.get_flat_storage_creation_status(shard_id); - if cfg!(feature = "protocol_feature_flat_state") { - assert_eq!(status, FlatStorageCreationStatus::Ready); - runtime.create_flat_storage_state_for_shard(shard_id, 0, &mock_chain); - } else { - assert_eq!(status, FlatStorageCreationStatus::DontCreate); + if cfg!(feature = "protocol_feature_flat_state") { + let store_update = runtime + .set_flat_storage_for_genesis(&genesis_hash, 0, &EpochId::default()) + .unwrap(); + store_update.commit().unwrap(); + for shard_id in 0..runtime.num_shards(&EpochId::default()).unwrap() { + let shard_uid = runtime.shard_id_to_uid(shard_id, &EpochId::default()).unwrap(); + assert!(matches!( + runtime.get_flat_storage_status(shard_uid), + FlatStorageStatus::Ready(_) + )); + runtime.create_flat_storage_for_shard(shard_uid); } } diff --git a/runtime/runtime-params-estimator/src/estimator_context.rs b/runtime/runtime-params-estimator/src/estimator_context.rs index ed2fcf83295..e2f98122f99 100644 --- a/runtime/runtime-params-estimator/src/estimator_context.rs +++ b/runtime/runtime-params-estimator/src/estimator_context.rs @@ -13,8 +13,8 @@ use near_primitives::transaction::{ExecutionStatus, SignedTransaction}; use near_primitives::types::{Gas, MerkleHash}; use near_primitives::version::PROTOCOL_VERSION; use near_store::flat::{ - BlockInfo, FlatStateChanges, FlatStateDelta, FlatStateDeltaMetadata, FlatStorage, - FlatStorageManager, + store_helper, BlockInfo, FlatStateChanges, FlatStateDelta, FlatStateDeltaMetadata, FlatStorage, + FlatStorageManager, FlatStorageReadyStatus, FlatStorageStatus, }; use near_store::{ShardTries, ShardUId, Store, StoreCompiledContractCache, TrieUpdate}; use near_store::{TrieCache, TrieCachingStorage, TrieConfig}; @@ -22,6 +22,7 @@ use near_vm_logic::{ExtCosts, VMLimitConfig}; use node_runtime::{ApplyState, Runtime}; use std::collections::HashMap; use std::iter; +use std::rc::Rc; use std::sync::Arc; /// Global context shared by all cost calculating functions. @@ -75,15 +76,12 @@ impl<'c> EstimatorContext<'c> { let mut trie_config = near_store::TrieConfig::default(); trie_config.enable_receipt_prefetching = true; - let flat_head = CryptoHash::hash_borsh(0usize); - let flat_storage_manager = FlatStorageManager::test(store.clone(), &shard_uids, flat_head); - if cfg!(feature = "protocol_feature_flat_state") { - let flat_storage = - flat_storage_manager.get_flat_storage_for_shard(shard_uids[0]).unwrap(); - self.generate_deltas(&flat_storage); - } - - let tries = ShardTries::new(store.clone(), trie_config, &shard_uids, flat_storage_manager); + let tries = ShardTries::new( + store.clone(), + trie_config, + &shard_uids, + self.create_flat_storage_manager(store.clone()), + ); Testbed { config: self.config, @@ -145,6 +143,29 @@ impl<'c> EstimatorContext<'c> { } } + fn create_flat_storage_manager(&self, store: Store) -> FlatStorageManager { + let flat_storage_manager = FlatStorageManager::new(store.clone()); + if !cfg!(feature = "protocol_feature_flat_state") { + return flat_storage_manager; + } + + let shard_uid = ShardUId::single_shard(); + // Set up flat head to be equal to the latest block height + let mut store_update = store.store_update(); + store_helper::set_flat_storage_status( + &mut store_update, + shard_uid, + FlatStorageStatus::Ready(FlatStorageReadyStatus { + flat_head: BlockInfo::genesis(CryptoHash::hash_borsh(0usize), 0), + }), + ); + store_update.commit().expect("failed to set flat storage status"); + let flat_storage = FlatStorage::new(store, shard_uid); + self.generate_deltas(&flat_storage); + flat_storage_manager.add_flat_storage_for_shard(shard_uid, flat_storage); + flat_storage_manager + } + /// Construct a chain of fake blocks with fake deltas for flat storage. /// /// Use `hash(height)` as the supposed block hash. @@ -352,7 +373,7 @@ impl Testbed<'_> { tx: &SignedTransaction, metric: GasMetric, ) -> GasCost { - let mut state_update = TrieUpdate::new(self.trie()); + let mut state_update = TrieUpdate::new(Rc::new(self.trie())); // gas price and block height can be anything, it doesn't affect performance // but making it too small affects max_depth and thus pessimistic inflation let gas_price = 100_000_000; @@ -378,7 +399,7 @@ impl Testbed<'_> { /// /// Use this method to estimate action exec costs. pub(crate) fn apply_action_receipt(&mut self, receipt: &Receipt, metric: GasMetric) -> GasCost { - let mut state_update = TrieUpdate::new(self.trie()); + let mut state_update = TrieUpdate::new(Rc::new(self.trie())); let mut outgoing_receipts = vec![]; let mut validator_proposals = vec![]; let mut stats = node_runtime::ApplyStats::default(); diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index d10b5612c59..d86d6070a22 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -486,6 +486,7 @@ impl StatePartWriter for FileSystemStorage { impl StatePartReader for FileSystemStorage { fn read(&self, part_id: u64, num_parts: u64) -> Vec { let filename = self.get_location(part_id, num_parts); + tracing::debug!(target: "state-parts", part_id, num_parts, ?filename, "Reading state part file"); let part = std::fs::read(filename).unwrap(); part } From 4910fcc8c7adc9c84f0310186026c173cf32cc84 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 12:28:14 +0100 Subject: [PATCH 31/88] Merge --- tools/state-viewer/src/state_parts.rs | 30 +++++++++++++-------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index d86d6070a22..4654fe82715 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -1,5 +1,4 @@ use crate::epoch_info::iterate_and_filter; -use clap::Subcommand; use near_chain::types::RuntimeAdapter; use near_chain::{ChainStore, ChainStoreAccess}; use near_epoch_manager::EpochManager; @@ -19,8 +18,8 @@ use std::str::FromStr; use std::sync::Arc; use std::time::Instant; -#[derive(Subcommand, Debug, Clone)] -pub enum StatePartsSubCommand { +#[derive(clap::Subcommand, Debug, Clone)] +pub(crate) enum StatePartsSubCommand { /// Apply all or a single state part of a shard. Apply { /// If true, validate the state part but don't write it to the DB. @@ -30,8 +29,11 @@ pub enum StatePartsSubCommand { /// Use if those headers or blocks are not available. #[clap(long)] state_root: Option, - - /// Selects an epoch. The dump will be of the state at the beginning of this epoch. + /// Choose a single part id. + /// If None - affects all state parts. + #[clap(long)] + part_id: Option, + /// Select an epoch to work on. #[clap(subcommand)] epoch_selection: EpochSelection, }, @@ -43,8 +45,7 @@ pub enum StatePartsSubCommand { /// Dump part ids up to this part (exclusive). #[clap(long)] part_to: Option, - - /// Selects an epoch. The dump will be of the state at the beginning of this epoch. + /// Select an epoch to work on. #[clap(subcommand)] epoch_selection: EpochSelection, }, @@ -54,7 +55,6 @@ impl StatePartsSubCommand { pub(crate) fn run( self, shard_id: ShardId, - part_id: Option, root_dir: Option, s3_bucket: Option, s3_region: Option, @@ -63,7 +63,7 @@ impl StatePartsSubCommand { store: Store, ) { match self { - StatePartsSubCommand::Apply { dry_run, state_root, epoch_selection } => { + StatePartsSubCommand::Apply { dry_run, state_root, part_id, epoch_selection } => { apply_state_parts( epoch_selection, shard_id, @@ -80,8 +80,8 @@ impl StatePartsSubCommand { dump_state_parts( epoch_selection, shard_id, - part_from.or(part_id), - part_to.or(part_id.map(|x| x + 1)), + part_from, + part_to, home_dir, near_config, store, @@ -92,8 +92,8 @@ impl StatePartsSubCommand { } } -#[derive(Subcommand, Debug, Clone)] -pub enum EpochSelection { +#[derive(clap::Subcommand, Debug, Clone)] +pub(crate) enum EpochSelection { /// Current epoch. Current, /// Fetch the given epoch. @@ -107,7 +107,7 @@ pub enum EpochSelection { } impl EpochSelection { - pub fn to_epoch_id( + fn to_epoch_id( &self, store: Store, chain_store: &ChainStore, @@ -143,7 +143,7 @@ impl EpochSelection { } } -pub(crate) enum Location { +enum Location { Files(PathBuf), S3 { bucket: String, region: String }, } From c2fa4a0315efd01759d81bf6093013c3fba8b17c Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 12:30:39 +0100 Subject: [PATCH 32/88] Merge --- tools/state-viewer/src/cli.rs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tools/state-viewer/src/cli.rs b/tools/state-viewer/src/cli.rs index e0e6b26d860..d73d12071b1 100644 --- a/tools/state-viewer/src/cli.rs +++ b/tools/state-viewer/src/cli.rs @@ -578,10 +578,6 @@ pub struct StatePartsCmd { /// Shard id. #[clap(long)] shard_id: ShardId, - /// Choose a single part id. - /// If None - affects all state parts. - #[clap(long)] - part_id: Option, /// Location of serialized state parts. #[clap(long)] root_dir: Option, @@ -601,7 +597,6 @@ impl StatePartsCmd { pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { self.command.run( self.shard_id, - self.part_id, self.root_dir, self.s3_bucket, self.s3_region, From 14e202ff7849b349fa4ee852e639225148ac6c4e Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 12:38:13 +0100 Subject: [PATCH 33/88] Handle partial state parts dumps --- tools/state-viewer/src/state_parts.rs | 22 +++++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index 4654fe82715..f22cd791cce 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -493,15 +493,31 @@ impl StatePartReader for FileSystemStorage { fn num_parts(&self) -> u64 { let paths = std::fs::read_dir(&self.state_parts_dir).unwrap(); - let num_parts = paths + let mut known_num_parts = None; + let num_files = paths .filter(|path| { let full_path = path.as_ref().unwrap(); tracing::debug!(target: "state-parts", ?full_path); - is_part_filename(full_path.file_name().to_str().unwrap()) + let filename = full_path.file_name().to_str().unwrap().to_string(); + if let Some(num_parts) = get_num_parts_from_filename(&filename) { + if let Some(known_num_parts) = known_num_parts { + assert_eq!(known_num_parts, num_parts); + } + known_num_parts = Some(num_parts); + } + is_part_filename(&filename) }) .collect::>>() .len(); - num_parts as u64 + if known_num_parts != Some(num_files as u64) { + // This is expected when a user saves time and downloads a few parts instead of all parts. + tracing::warn!(target: "state-parts", + dir = ?self.state_parts_dir, + ?known_num_parts, + num_files, + "Filename indicates that number of files expected doesn't match the number of files available"); + } + known_num_parts.unwrap() } } From 0da5f3b7e84bc7125be326a0839cc79899f40896 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 13:06:11 +0100 Subject: [PATCH 34/88] More output --- tools/state-viewer/src/state_parts.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index f22cd791cce..b2fc3c9356f 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -237,6 +237,11 @@ fn apply_state_parts( } else { let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); + tracing::info!( + target: "state-parts", + ?sync_prev_hash, + height = sync_prev_block.header().height(), + state_roots = ?sync_prev_block.chunks().iter().map(|chunk| chunk.prev_state_root()).collect::>()); assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); assert!( From 610e820a7e2deb8e4ddc17b5bf56aab8dc3fa864 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 14:12:03 +0100 Subject: [PATCH 35/88] More output --- nearcore/src/runtime/mod.rs | 36 +++++++++++---------------- tools/state-viewer/src/state_parts.rs | 31 +++++++++++++++-------- 2 files changed, 34 insertions(+), 33 deletions(-) diff --git a/nearcore/src/runtime/mod.rs b/nearcore/src/runtime/mod.rs index 358387ae547..2c8551adaae 100644 --- a/nearcore/src/runtime/mod.rs +++ b/nearcore/src/runtime/mod.rs @@ -1246,10 +1246,6 @@ impl RuntimeAdapter for NightshadeRuntime { %block_hash, num_parts = part_id.total) .entered(); - let _timer = metrics::STATE_SYNC_OBTAIN_PART_DELAY - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - let epoch_id = self.get_epoch_id(block_hash)?; let shard_uid = self.get_shard_uid_from_epoch_id(shard_id, &epoch_id)?; let trie = self.tries.get_view_trie_for_shard(shard_uid, *state_root); @@ -1275,16 +1271,16 @@ impl RuntimeAdapter for NightshadeRuntime { Ok(_) => true, // Storage error should not happen Err(err) => { - tracing::error!(target: "state-parts", ?err, "State part storage error"); + tracing::error!(target: "state-parts", ?err, "Storage error"); false } } } // Deserialization error means we've got the data from malicious peer Err(err) => { - tracing::error!(target: "state-parts", ?err, "State part deserialization error"); + tracing::error!(target: "state-parts", ?err, "Deserialization error"); false - } + }, } } @@ -1381,10 +1377,6 @@ impl RuntimeAdapter for NightshadeRuntime { data: &[u8], epoch_id: &EpochId, ) -> Result<(), Error> { - let _timer = metrics::STATE_SYNC_APPLY_PART_DELAY - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - let part = BorshDeserialize::try_from_slice(data) .expect("Part was already validated earlier, so could never fail here"); let ApplyStatePartResult { trie_changes, flat_state_delta, contract_codes } = @@ -1940,11 +1932,11 @@ mod test { * U256::from(self.runtime.genesis_config.total_supply) * U256::from(epoch_duration) / (U256::from(num_seconds_per_year) - * U256::from( - *self.runtime.genesis_config.max_inflation_rate.denom() as u128 - ) - * U256::from(num_ns_in_second))) - .as_u128(); + * U256::from( + *self.runtime.genesis_config.max_inflation_rate.denom() as u128 + ) + * U256::from(num_ns_in_second))) + .as_u128(); let per_epoch_protocol_treasury = per_epoch_total_reward * *self.runtime.genesis_config.protocol_reward_rate.numer() as u128 / *self.runtime.genesis_config.protocol_reward_rate.denom() as u128; @@ -2459,7 +2451,7 @@ mod test { block_producers[0].public_key(), 0, ) - .into()], + .into()], prev_epoch_kickout: Default::default(), epoch_start_height: 1, epoch_height: 1, @@ -2541,11 +2533,11 @@ mod test { assert!( env.runtime.cares_about_shard(Some(&validators[0]), &env.head.last_block_hash, 1, true) ^ env.runtime.cares_about_shard( - Some(&validators[1]), - &env.head.last_block_hash, - 1, - true - ) + Some(&validators[1]), + &env.head.last_block_hash, + 1, + true + ) ); assert!(env.runtime.cares_about_shard( Some(&validators[1]), diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index b2fc3c9356f..6d2fcff706b 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -63,7 +63,12 @@ impl StatePartsSubCommand { store: Store, ) { match self { - StatePartsSubCommand::Apply { dry_run, state_root, part_id, epoch_selection } => { + StatePartsSubCommand::Apply { + dry_run, + state_root, + part_id, + epoch_selection, + } => { apply_state_parts( epoch_selection, shard_id, @@ -229,12 +234,17 @@ fn apply_state_parts( near_config.client_config.save_trie_changes, ); - let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); - let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); - - let (state_root, sync_prev_hash) = if let Some(state_root) = maybe_state_root { - (state_root, None) + let (state_root, epoch_height, epoch_id, sync_prev_hash) = if let ( + Some(state_root), + EpochSelection::EpochHeight{epoch_height}, + ) = + (maybe_state_root, &epoch_selection) + { + (state_root, *epoch_height, None, None) } else { + let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); + let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); + let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); tracing::info!( @@ -251,13 +261,13 @@ fn apply_state_parts( sync_prev_block.chunks().len() ); let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); - (state_root, Some(sync_prev_hash)) + (state_root, epoch.epoch_height(), Some(epoch_id), Some(sync_prev_hash)) }; let part_storage = get_state_part_reader( location, &near_config.client_config.chain_id, - epoch.epoch_height(), + epoch_height, shard_id, ); @@ -266,8 +276,7 @@ fn apply_state_parts( let part_ids = get_part_ids(part_id, part_id.map(|x| x + 1), num_parts); tracing::info!( target: "state-parts", - epoch_height = epoch.epoch_height(), - epoch_id = ?epoch_id.0, + epoch_height, shard_id, num_parts, ?sync_prev_hash, @@ -295,7 +304,7 @@ fn apply_state_parts( &state_root, PartId::new(part_id, num_parts), &part, - &epoch_id, + epoch_id.as_ref().unwrap(), ) .unwrap(); tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Applied a state part"); From 296dda343f17c68e48ebcf4c7d902f9ed9fd206a Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 16:33:48 +0100 Subject: [PATCH 36/88] Fix off-by-one error in the state-part tool. --- tools/state-viewer/src/state_parts.rs | 187 +++++++++++--------------- 1 file changed, 80 insertions(+), 107 deletions(-) diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index 6d2fcff706b..1537240316d 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -1,7 +1,6 @@ use crate::epoch_info::iterate_and_filter; -use near_chain::types::RuntimeAdapter; -use near_chain::{ChainStore, ChainStoreAccess}; -use near_epoch_manager::EpochManager; +use near_chain::{Chain, ChainGenesis, ChainStoreAccess, DoomslugThresholdMode}; +use near_client::sync::state::StateSync; use near_primitives::epoch_manager::epoch_info::EpochInfo; use near_primitives::state_part::PartId; use near_primitives::syncing::get_num_state_parts; @@ -62,21 +61,27 @@ impl StatePartsSubCommand { near_config: NearConfig, store: Store, ) { + let runtime = + Arc::new(NightshadeRuntime::from_config(home_dir, store.clone(), &near_config)); + let chain_genesis = ChainGenesis::new(&near_config.genesis); + let mut chain = Chain::new_for_view_client( + runtime.clone(), + &chain_genesis, + DoomslugThresholdMode::TwoThirds, + false, + ) + .unwrap(); + let chain_id = &near_config.genesis.config.chain_id; match self { - StatePartsSubCommand::Apply { - dry_run, - state_root, - part_id, - epoch_selection, - } => { + StatePartsSubCommand::Apply { dry_run, state_root, part_id, epoch_selection } => { apply_state_parts( epoch_selection, shard_id, part_id, dry_run, state_root, - home_dir, - near_config, + &mut chain, + chain_id, store, Location::new(root_dir, (s3_bucket, s3_region)), ); @@ -87,8 +92,8 @@ impl StatePartsSubCommand { shard_id, part_from, part_to, - home_dir, - near_config, + &chain, + chain_id, store, Location::new(root_dir, (s3_bucket, s3_region)), ); @@ -112,15 +117,10 @@ pub(crate) enum EpochSelection { } impl EpochSelection { - fn to_epoch_id( - &self, - store: Store, - chain_store: &ChainStore, - epoch_manager: &EpochManager, - ) -> EpochId { + fn to_epoch_id(&self, store: Store, chain: &Chain) -> EpochId { match self { EpochSelection::Current => { - epoch_manager.get_epoch_id(&chain_store.head().unwrap().last_block_hash).unwrap() + chain.runtime_adapter.get_epoch_id(&chain.head().unwrap().last_block_hash).unwrap() } EpochSelection::EpochId { epoch_id } => { EpochId(CryptoHash::from_str(&epoch_id).unwrap()) @@ -137,12 +137,12 @@ impl EpochSelection { } EpochSelection::BlockHash { block_hash } => { let block_hash = CryptoHash::from_str(&block_hash).unwrap(); - epoch_manager.get_epoch_id(&block_hash).unwrap() + chain.runtime_adapter.get_epoch_id(&block_hash).unwrap() } EpochSelection::BlockHeight { block_height } => { // Fetch an epoch containing the given block height. - let block_hash = chain_store.get_block_hash_by_height(*block_height).unwrap(); - epoch_manager.get_epoch_id(&block_hash).unwrap() + let block_hash = chain.store().get_block_hash_by_height(*block_height).unwrap(); + chain.runtime_adapter.get_epoch_id(&block_hash).unwrap() } } } @@ -177,21 +177,18 @@ impl Location { } } -/// Returns block hash of the last block of an epoch preceding the given `epoch_info`. -fn get_prev_hash_of_epoch( - epoch_info: &EpochInfo, - chain_store: &ChainStore, - epoch_manager: &EpochManager, -) -> CryptoHash { - let head = chain_store.head().unwrap(); - let mut cur_block_info = epoch_manager.get_block_info(&head.last_block_hash).unwrap(); +/// Returns block hash of some block of the given `epoch_info` epoch. +fn get_any_block_hash_of_epoch(epoch_info: &EpochInfo, chain: &Chain) -> CryptoHash { + let head = chain.store().head().unwrap(); + let mut cur_block_info = chain.runtime_adapter.get_block_info(&head.last_block_hash).unwrap(); // EpochManager doesn't have an API that maps EpochId to Blocks, and this function works // around that limitation by iterating over the epochs. // This workaround is acceptable because: // 1) Extending EpochManager's API is a major change. // 2) This use case is not critical at all. loop { - let cur_epoch_info = epoch_manager.get_epoch_info(cur_block_info.epoch_id()).unwrap(); + let cur_epoch_info = + chain.runtime_adapter.get_epoch_info(cur_block_info.epoch_id()).unwrap(); let cur_epoch_height = cur_epoch_info.epoch_height(); assert!( cur_epoch_height >= epoch_info.epoch_height(), @@ -200,12 +197,12 @@ fn get_prev_hash_of_epoch( epoch_info.epoch_height() ); let epoch_first_block_info = - epoch_manager.get_block_info(cur_block_info.epoch_first_block()).unwrap(); + chain.runtime_adapter.get_block_info(cur_block_info.epoch_first_block()).unwrap(); let prev_epoch_last_block_info = - epoch_manager.get_block_info(epoch_first_block_info.prev_hash()).unwrap(); + chain.runtime_adapter.get_block_info(epoch_first_block_info.prev_hash()).unwrap(); if cur_epoch_height == epoch_info.epoch_height() { - return *prev_epoch_last_block_info.hash(); + return *cur_block_info.hash(); } cur_block_info = prev_epoch_last_block_info; @@ -218,58 +215,43 @@ fn apply_state_parts( part_id: Option, dry_run: bool, maybe_state_root: Option, - home_dir: &Path, - near_config: NearConfig, + chain: &mut Chain, + chain_id: &str, store: Store, location: Location, ) { - let runtime_adapter: Arc = - NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); - let epoch_manager = - EpochManager::new_from_genesis_config(store.clone(), &near_config.genesis.config) - .expect("Failed to start Epoch Manager"); - let chain_store = ChainStore::new( - store.clone(), - near_config.genesis.config.genesis_height, - near_config.client_config.save_trie_changes, - ); - - let (state_root, epoch_height, epoch_id, sync_prev_hash) = if let ( + let (state_root, epoch_height, epoch_id, sync_hash) = if let ( Some(state_root), - EpochSelection::EpochHeight{epoch_height}, + EpochSelection::EpochHeight { epoch_height }, ) = (maybe_state_root, &epoch_selection) { (state_root, *epoch_height, None, None) } else { - let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); - let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); + let epoch_id = epoch_selection.to_epoch_id(store, &chain); + let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); - let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); - let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); + let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); + let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); + let sync_block = chain.get_block(&sync_hash).unwrap(); tracing::info!( target: "state-parts", - ?sync_prev_hash, - height = sync_prev_block.header().height(), - state_roots = ?sync_prev_block.chunks().iter().map(|chunk| chunk.prev_state_root()).collect::>()); + ?sync_hash, + height = sync_block.header().height(), + state_roots = ?sync_block.chunks().iter().map(|chunk| chunk.prev_state_root()).collect::>()); - assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); + assert!(chain.runtime_adapter.is_next_block_epoch_start(&sync_hash).unwrap()); assert!( - shard_id < sync_prev_block.chunks().len() as u64, + shard_id < sync_block.chunks().len() as u64, "shard_id: {}, #shards: {}", shard_id, - sync_prev_block.chunks().len() + sync_block.chunks().len() ); - let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); - (state_root, epoch.epoch_height(), Some(epoch_id), Some(sync_prev_hash)) + let state_root = sync_block.chunks()[shard_id as usize].prev_state_root(); + (state_root, epoch.epoch_height(), Some(epoch_id), Some(sync_hash)) }; - let part_storage = get_state_part_reader( - location, - &near_config.client_config.chain_id, - epoch_height, - shard_id, - ); + let part_storage = get_state_part_reader(location, &chain_id, epoch_height, shard_id); let num_parts = part_storage.num_parts(); assert_ne!(num_parts, 0, "Too few num_parts: {}", num_parts); @@ -279,7 +261,7 @@ fn apply_state_parts( epoch_height, shard_id, num_parts, - ?sync_prev_hash, + ?sync_hash, ?part_ids, "Applying state as seen at the beginning of the specified epoch.", ); @@ -291,14 +273,23 @@ fn apply_state_parts( let part = part_storage.read(part_id, num_parts); if dry_run { - assert!(runtime_adapter.validate_state_part( + assert!(chain.runtime_adapter.validate_state_part( &state_root, PartId::new(part_id, num_parts), &part )); tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Validated a state part"); } else { - runtime_adapter + chain + .set_state_part( + shard_id, + sync_hash.unwrap(), + PartId::new(part_id, num_parts), + &part, + ) + .unwrap(); + chain + .runtime_adapter .apply_state_part( shard_id, &state_root, @@ -318,37 +309,27 @@ fn dump_state_parts( shard_id: ShardId, part_from: Option, part_to: Option, - home_dir: &Path, - near_config: NearConfig, + chain: &Chain, + chain_id: &str, store: Store, location: Location, ) { - let runtime_adapter: Arc = - NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); - let epoch_manager = - EpochManager::new_from_genesis_config(store.clone(), &near_config.genesis.config) - .expect("Failed to start Epoch Manager"); - let chain_store = ChainStore::new( - store.clone(), - near_config.genesis.config.genesis_height, - near_config.client_config.save_trie_changes, - ); + let epoch_id = epoch_selection.to_epoch_id(store, &chain); + let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); + let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); + let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); + let sync_block = chain.get_block(&sync_hash).unwrap(); - let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); - let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); - let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); - let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); - - assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); + assert!(chain.runtime_adapter.is_next_block_epoch_start(&sync_hash).unwrap()); assert!( - shard_id < sync_prev_block.chunks().len() as u64, + shard_id < sync_block.chunks().len() as u64, "shard_id: {}, #shards: {}", shard_id, - sync_prev_block.chunks().len() + sync_block.chunks().len() ); - let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); + let state_root = sync_block.chunks()[shard_id as usize].prev_state_root(); let state_root_node = - runtime_adapter.get_state_root_node(shard_id, &sync_prev_hash, &state_root).unwrap(); + chain.runtime_adapter.get_state_root_node(shard_id, &sync_hash, &state_root).unwrap(); let num_parts = get_num_state_parts(state_root_node.memory_usage); let part_ids = get_part_ids(part_from, part_to, num_parts); @@ -359,29 +340,21 @@ fn dump_state_parts( epoch_id = ?epoch_id.0, shard_id, num_parts, - ?sync_prev_hash, + ?sync_hash, ?part_ids, + ?state_root, "Dumping state as seen at the beginning of the specified epoch.", ); - let part_storage = get_state_part_writer( - location, - &near_config.client_config.chain_id, - epoch.epoch_height(), - shard_id, - ); + let part_storage = get_state_part_writer(location, chain_id, epoch.epoch_height(), shard_id); let timer = Instant::now(); for part_id in part_ids { let timer = Instant::now(); assert!(part_id < num_parts, "part_id: {}, num_parts: {}", part_id, num_parts); - let state_part = runtime_adapter - .obtain_state_part( - shard_id, - &sync_prev_hash, - &state_root, - PartId::new(part_id, num_parts), - ) + let state_part = chain + .runtime_adapter + .obtain_state_part(shard_id, &sync_hash, &state_root, PartId::new(part_id, num_parts)) .unwrap(); part_storage.write(&state_part, part_id, num_parts); tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote a state part"); From 11e965ababafae0262fe33acb3aa83f1654a55f0 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 16:35:46 +0100 Subject: [PATCH 37/88] fmt --- nearcore/src/runtime/mod.rs | 28 ++++++++++++++-------------- tools/state-viewer/src/cli.rs | 1 - 2 files changed, 14 insertions(+), 15 deletions(-) diff --git a/nearcore/src/runtime/mod.rs b/nearcore/src/runtime/mod.rs index 2c8551adaae..240383939c3 100644 --- a/nearcore/src/runtime/mod.rs +++ b/nearcore/src/runtime/mod.rs @@ -1271,16 +1271,16 @@ impl RuntimeAdapter for NightshadeRuntime { Ok(_) => true, // Storage error should not happen Err(err) => { - tracing::error!(target: "state-parts", ?err, "Storage error"); + tracing::error!(target: "state-parts", ?err, "State part storage error"); false } } } // Deserialization error means we've got the data from malicious peer Err(err) => { - tracing::error!(target: "state-parts", ?err, "Deserialization error"); + tracing::error!(target: "state-parts", ?err, "State part deserialization error"); false - }, + } } } @@ -1932,11 +1932,11 @@ mod test { * U256::from(self.runtime.genesis_config.total_supply) * U256::from(epoch_duration) / (U256::from(num_seconds_per_year) - * U256::from( - *self.runtime.genesis_config.max_inflation_rate.denom() as u128 - ) - * U256::from(num_ns_in_second))) - .as_u128(); + * U256::from( + *self.runtime.genesis_config.max_inflation_rate.denom() as u128 + ) + * U256::from(num_ns_in_second))) + .as_u128(); let per_epoch_protocol_treasury = per_epoch_total_reward * *self.runtime.genesis_config.protocol_reward_rate.numer() as u128 / *self.runtime.genesis_config.protocol_reward_rate.denom() as u128; @@ -2451,7 +2451,7 @@ mod test { block_producers[0].public_key(), 0, ) - .into()], + .into()], prev_epoch_kickout: Default::default(), epoch_start_height: 1, epoch_height: 1, @@ -2533,11 +2533,11 @@ mod test { assert!( env.runtime.cares_about_shard(Some(&validators[0]), &env.head.last_block_hash, 1, true) ^ env.runtime.cares_about_shard( - Some(&validators[1]), - &env.head.last_block_hash, - 1, - true - ) + Some(&validators[1]), + &env.head.last_block_hash, + 1, + true + ) ); assert!(env.runtime.cares_about_shard( Some(&validators[1]), diff --git a/tools/state-viewer/src/cli.rs b/tools/state-viewer/src/cli.rs index d73d12071b1..a3218119f07 100644 --- a/tools/state-viewer/src/cli.rs +++ b/tools/state-viewer/src/cli.rs @@ -587,7 +587,6 @@ pub struct StatePartsCmd { /// Store state parts in an S3 bucket. #[clap(long)] s3_region: Option, - /// Dump or Apply state parts. #[clap(subcommand)] command: crate::state_parts::StatePartsSubCommand, From 95c1ad5bd6f75050c89cafbf05065e2cda799608 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 19:12:03 +0100 Subject: [PATCH 38/88] fmt --- chain/chain/src/chain.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/chain/chain/src/chain.rs b/chain/chain/src/chain.rs index e389a376007..6e1e8b1c960 100644 --- a/chain/chain/src/chain.rs +++ b/chain/chain/src/chain.rs @@ -3139,7 +3139,7 @@ impl Chain { let state_root = *chunk.take_header().take_inner().prev_state_root(); if !self.runtime_adapter.validate_state_part(&state_root, part_id, data) { byzantine_assert!(false); - return Err(Error::Other("set_state_part failed: validate_state_part failed".into())); + return Err(Error::Other(format!("set_state_part failed: validate_state_part failed. state_root={:?}", state_root))); } // Saving the part data. From dc89915969e23e210a1e209e76cdd543a0f3c800 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 19:13:43 +0100 Subject: [PATCH 39/88] fmt --- chain/client/src/sync/state.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index cdfdf731b43..9eccc36688c 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -974,7 +974,7 @@ impl StateSync { let num_parts = shard_sync_download.downloads.len(); let mut num_parts_done = 0; for (part_id, part_download) in shard_sync_download.downloads.iter_mut().enumerate() { - tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, part_download.done, part_download.error, ?part_download); + // tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, part_download.done, part_download.error, ?part_download); if !part_download.done { // Check if a download from an external storage is finished. check_external_storage_part_response( From f14e8aa649360175e550a27b48c8729eb0d54bad Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 20:30:21 +0100 Subject: [PATCH 40/88] read-state-header --- chain/chain/src/chain.rs | 5 ++++- chain/client/src/adapter.rs | 4 +--- chain/client/src/sync/state.rs | 2 +- tools/state-viewer/src/state_parts.rs | 25 +++++++++++++++++++++++++ 4 files changed, 31 insertions(+), 5 deletions(-) diff --git a/chain/chain/src/chain.rs b/chain/chain/src/chain.rs index 6e1e8b1c960..dedc1369b1c 100644 --- a/chain/chain/src/chain.rs +++ b/chain/chain/src/chain.rs @@ -3139,7 +3139,10 @@ impl Chain { let state_root = *chunk.take_header().take_inner().prev_state_root(); if !self.runtime_adapter.validate_state_part(&state_root, part_id, data) { byzantine_assert!(false); - return Err(Error::Other(format!("set_state_part failed: validate_state_part failed. state_root={:?}", state_root))); + return Err(Error::Other(format!( + "set_state_part failed: validate_state_part failed. state_root={:?}", + state_root + ))); } // Saving the part data. diff --git a/chain/client/src/adapter.rs b/chain/client/src/adapter.rs index 7d01e46922c..dc0054dcefd 100644 --- a/chain/client/src/adapter.rs +++ b/chain/client/src/adapter.rs @@ -194,9 +194,7 @@ impl near_network::client::Client for Adapter { ) -> Result, ReasonForBan> { match self .view_client_addr - .send( - StateRequestHeader { shard_id: shard_id, sync_hash: sync_hash }.with_span_context(), - ) + .send(StateRequestHeader { shard_id, sync_hash }.with_span_context()) .await { Ok(Some(StateResponse(resp))) => Ok(Some(*resp)), diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 9eccc36688c..bb3ab6b82d6 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -356,7 +356,7 @@ impl StateSync { %shard_id, timeout_sec = self.timeout.num_seconds(), "State sync didn't download the state, sending StateRequest again"); - tracing::info!( + tracing::debug!( target: "sync", %shard_id, %sync_hash, diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index 1537240316d..0235ab6bd61 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -48,6 +48,12 @@ pub(crate) enum StatePartsSubCommand { #[clap(subcommand)] epoch_selection: EpochSelection, }, + /// Read State Header from the DB + ReadStateHeader { + /// Select an epoch to work on. + #[clap(subcommand)] + epoch_selection: EpochSelection, + }, } impl StatePartsSubCommand { @@ -98,6 +104,9 @@ impl StatePartsSubCommand { Location::new(root_dir, (s3_bucket, s3_region)), ); } + StatePartsSubCommand::ReadStateHeader { epoch_selection } => { + read_state_header(epoch_selection, shard_id, &chain, store) + } } } } @@ -362,6 +371,22 @@ fn dump_state_parts( tracing::info!(target: "state-parts", total_elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote all requested state parts"); } +fn read_state_header( + epoch_selection: EpochSelection, + shard_id: ShardId, + chain: &Chain, + store: Store, +) { + let epoch_id = epoch_selection.to_epoch_id(store, &chain); + let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); + + let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); + let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); + + let state_header = chain.store().get_state_header(shard_id, sync_hash); + tracing::info!(target: "state-parts", ?epoch_id, ?sync_hash, ?state_header); +} + fn get_part_ids(part_from: Option, part_to: Option, num_parts: u64) -> Range { part_from.unwrap_or(0)..part_to.unwrap_or(num_parts) } From 15d1fbbb3412d6410dfb5789090b3c0932b0c2f6 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 21:05:05 +0100 Subject: [PATCH 41/88] Dump correct state root in state dump --- nearcore/src/state_sync.rs | 18 +++++++----- tools/state-viewer/src/state_parts.rs | 42 +++++++++++++++++---------- 2 files changed, 37 insertions(+), 23 deletions(-) diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 458bff4e27e..357e40d298e 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -368,13 +368,17 @@ fn start_dumping( let epoch_info = runtime.get_epoch_info(&epoch_id)?; let epoch_height = epoch_info.epoch_height(); let num_shards = runtime.num_shards(&epoch_id)?; - let sync_hash_block = chain.get_block(&sync_hash)?; - if runtime.cares_about_shard(None, sync_hash_block.header().prev_hash(), shard_id, false) { - assert_eq!(num_shards, sync_hash_block.chunks().len() as u64); - let state_root = sync_hash_block.chunks()[shard_id as usize].prev_state_root(); - let state_root_node = runtime.get_state_root_node(shard_id, &sync_hash, &state_root)?; + let sync_prev_header = chain.get_block_header(&sync_hash)?; + let sync_prev_hash = sync_prev_header.prev_hash(); + let prev_sync_block = chain.get_block(&sync_prev_hash)?; + if runtime.cares_about_shard(None, prev_sync_block.header().prev_hash(), shard_id, false) { + assert_eq!(num_shards, prev_sync_block.chunks().len() as u64); + let state_root = prev_sync_block.chunks()[shard_id as usize].prev_state_root(); + // See `get_state_response_header()` for reference. + let state_root_node = + runtime.get_state_root_node(shard_id, &sync_prev_hash, &state_root)?; let num_parts = get_num_state_parts(state_root_node.memory_usage); - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, %state_root, num_parts, "Initialize dumping state of Epoch"); + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_prev_hash, %sync_hash, %state_root, num_parts, "Initialize dumping state of Epoch"); // Note that first the state of the state machines gets changes to // `InProgress` and it starts dumping state after a short interval. set_metrics(&shard_id, Some(0), Some(num_parts), Some(epoch_height)); @@ -387,7 +391,7 @@ fn start_dumping( num_parts, })) } else { - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, "Shard is not tracked, skip the epoch"); + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_prev_hash, %sync_hash, "Shard is not tracked, skip the epoch"); Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts: Some(0) })) } } diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index 0235ab6bd61..5e395e8ae58 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -229,35 +229,39 @@ fn apply_state_parts( store: Store, location: Location, ) { - let (state_root, epoch_height, epoch_id, sync_hash) = if let ( + let (state_root, epoch_height, epoch_id, sync_hash, sync_prev_hash) = if let ( Some(state_root), EpochSelection::EpochHeight { epoch_height }, ) = (maybe_state_root, &epoch_selection) { - (state_root, *epoch_height, None, None) + (state_root, *epoch_height, None, None, None) } else { let epoch_id = epoch_selection.to_epoch_id(store, &chain); let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); - let sync_block = chain.get_block(&sync_hash).unwrap(); + let sync_header = chain.get_block_header(&sync_hash).unwrap(); + // See `get_state_response_header()`. + let sync_prev_block = chain.get_block(sync_header.prev_hash()).unwrap(); + let sync_prev_hash = sync_prev_block.hash(); tracing::info!( target: "state-parts", ?sync_hash, - height = sync_block.header().height(), - state_roots = ?sync_block.chunks().iter().map(|chunk| chunk.prev_state_root()).collect::>()); + ?sync_prev_hash, + height = sync_prev_block.header().height(), + state_roots = ?sync_prev_block.chunks().iter().map(|chunk| chunk.prev_state_root()).collect::>()); - assert!(chain.runtime_adapter.is_next_block_epoch_start(&sync_hash).unwrap()); + assert!(chain.runtime_adapter.is_next_block_epoch_start(&sync_prev_hash).unwrap()); assert!( - shard_id < sync_block.chunks().len() as u64, + shard_id < sync_prev_block.chunks().len() as u64, "shard_id: {}, #shards: {}", shard_id, - sync_block.chunks().len() + sync_prev_block.chunks().len() ); - let state_root = sync_block.chunks()[shard_id as usize].prev_state_root(); - (state_root, epoch.epoch_height(), Some(epoch_id), Some(sync_hash)) + let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); + (state_root, epoch.epoch_height(), Some(epoch_id), Some(sync_hash), Some(*sync_prev_hash)) }; let part_storage = get_state_part_reader(location, &chain_id, epoch_height, shard_id); @@ -270,6 +274,7 @@ fn apply_state_parts( epoch_height, shard_id, num_parts, + ?sync_prev_hash, ?sync_hash, ?part_ids, "Applying state as seen at the beginning of the specified epoch.", @@ -327,18 +332,21 @@ fn dump_state_parts( let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); - let sync_block = chain.get_block(&sync_hash).unwrap(); + let sync_header = chain.get_block_header(&sync_hash).unwrap(); + // See `get_state_response_header()`. + let sync_prev_block = chain.get_block(sync_header.prev_hash()).unwrap(); + let sync_prev_hash = sync_prev_block.hash(); - assert!(chain.runtime_adapter.is_next_block_epoch_start(&sync_hash).unwrap()); + assert!(chain.runtime_adapter.is_next_block_epoch_start(&sync_prev_hash).unwrap()); assert!( - shard_id < sync_block.chunks().len() as u64, + shard_id < sync_prev_block.chunks().len() as u64, "shard_id: {}, #shards: {}", shard_id, - sync_block.chunks().len() + sync_prev_block.chunks().len() ); - let state_root = sync_block.chunks()[shard_id as usize].prev_state_root(); + let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); let state_root_node = - chain.runtime_adapter.get_state_root_node(shard_id, &sync_hash, &state_root).unwrap(); + chain.runtime_adapter.get_state_root_node(shard_id, &sync_prev_hash, &state_root).unwrap(); let num_parts = get_num_state_parts(state_root_node.memory_usage); let part_ids = get_part_ids(part_from, part_to, num_parts); @@ -350,6 +358,7 @@ fn dump_state_parts( shard_id, num_parts, ?sync_hash, + ?sync_prev_hash, ?part_ids, ?state_root, "Dumping state as seen at the beginning of the specified epoch.", @@ -371,6 +380,7 @@ fn dump_state_parts( tracing::info!(target: "state-parts", total_elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote all requested state parts"); } +/// Reads `StateHeader` stored in the DB. fn read_state_header( epoch_selection: EpochSelection, shard_id: ShardId, From f390b196ea3a8a03d1f4f5bc71c09fb2d1d5a19d Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 21:20:57 +0100 Subject: [PATCH 42/88] Merge --- chain/client/src/sync/state.rs | 93 ++++++++++--------- core/store/src/flat/storage.rs | 57 ++++-------- core/store/src/trie/mod.rs | 29 ++---- core/store/src/trie/split_state.rs | 10 +- .../src/estimator_context.rs | 47 +++------- 5 files changed, 93 insertions(+), 143 deletions(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index bb3ab6b82d6..24a6c363c8a 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -214,7 +214,7 @@ impl StateSync { target: "sync", %prev_hash, timeout_sec = self.timeout.num_seconds(), - "State sync: block request timed"); + "State sync: block request timed out"); (true, false) } else { (false, false) @@ -265,36 +265,38 @@ impl StateSync { for shard_id in tracking_shards { let mut download_timeout = false; - let mut need_shard = false; + let mut run_shard_state_download = false; let shard_sync_download = new_shard_sync.entry(shard_id).or_insert_with(|| { - need_shard = true; + run_shard_state_download = true; update_sync_status = true; ShardSyncDownload::new_download_state_header(now) }); let old_status = shard_sync_download.status.clone(); - let mut this_done = false; + let mut shard_sync_done = false; metrics::STATE_SYNC_STAGE .with_label_values(&[&shard_id.to_string()]) .set(shard_sync_download.status.repr() as i64); match &shard_sync_download.status { ShardSyncStatus::StateDownloadHeader => { - (download_timeout, need_shard) = self.sync_shards_download_header_status( - shard_id, - shard_sync_download, - sync_hash, - chain, - now, - )?; + (download_timeout, run_shard_state_download) = self + .sync_shards_download_header_status( + shard_id, + shard_sync_download, + sync_hash, + chain, + now, + )?; } ShardSyncStatus::StateDownloadParts => { - (download_timeout, need_shard) = self.sync_shards_download_parts_status( - shard_id, - shard_sync_download, - sync_hash, - chain, - now, - ); + (download_timeout, run_shard_state_download) = self + .sync_shards_download_parts_status( + shard_id, + shard_sync_download, + sync_hash, + chain, + now, + ); } ShardSyncStatus::StateDownloadScheduling => { self.sync_shards_download_scheduling_status( @@ -316,7 +318,7 @@ impl StateSync { )?; } ShardSyncStatus::StateDownloadComplete => { - this_done = self.sync_shards_download_complete_status( + shard_sync_done = self.sync_shards_download_complete_status( split_states, shard_id, shard_sync_download, @@ -337,7 +339,7 @@ impl StateSync { } ShardSyncStatus::StateSplitApplying(_status) => { debug_assert!(split_states); - this_done = self.sync_shards_state_split_applying_status( + shard_sync_done = self.sync_shards_state_split_applying_status( shard_id, shard_sync_download, sync_hash, @@ -345,10 +347,10 @@ impl StateSync { )?; } ShardSyncStatus::StateSyncDone => { - this_done = true; + shard_sync_done = true; } } - all_done &= this_done; + all_done &= shard_sync_done; if download_timeout { tracing::warn!( @@ -366,7 +368,7 @@ impl StateSync { } // Execute syncing for shard `shard_id` - if need_shard { + if run_shard_state_download { update_sync_status = true; self.request_shard( me, @@ -915,8 +917,11 @@ impl StateSync { } /// Checks if the header is downloaded. - /// If the download is complete, then moves forward to StateDownloadParts, otherwise retries the header request. - /// Returns (download_timeout, need_shard). + /// If the download is complete, then moves forward to `StateDownloadParts`, + /// otherwise retries the header request. + /// Returns `(download_timeout, run_shard_state_download)` where: + /// * `download_timeout` means that the state header request timed out (and needs to be retried). + /// * `run_shard_state_download` means that header or part download requests need to run for this shard. fn sync_shards_download_header_status( &mut self, shard_id: ShardId, @@ -926,17 +931,17 @@ impl StateSync { now: DateTime, ) -> Result<(bool, bool), near_chain::Error> { let mut download_timeout = false; - let mut need_shard = false; + let mut run_shard_state_download = false; // StateDownloadHeader is the first step. We want to fetch the basic information about the state (its size, hash etc). if shard_sync_download.downloads[0].done { - let shard_state_header = chain.get_state_header(shard_id.clone(), sync_hash)?; + let shard_state_header = chain.get_state_header(shard_id, sync_hash)?; let state_num_parts = get_num_state_parts(shard_state_header.state_root_node().memory_usage); // If the header was downloaded successfully - move to phase 2 (downloading parts). // Create the vector with entry for each part. *shard_sync_download = ShardSyncDownload::new_download_state_parts(now, state_num_parts); - need_shard = true; + run_shard_state_download = true; } else { let prev = shard_sync_download.downloads[0].prev_update_time; let error = shard_sync_download.downloads[0].error; @@ -948,16 +953,17 @@ impl StateSync { shard_sync_download.downloads[0].prev_update_time = now; } if shard_sync_download.downloads[0].run_me.load(Ordering::SeqCst) { - need_shard = true; + run_shard_state_download = true; } } - Ok((download_timeout, need_shard)) + Ok((download_timeout, run_shard_state_download)) } /// Checks if the parts are downloaded. - /// If download of all parts is complete, then moves forward to StateDownloadScheduling. - /// Otherwise, retries the failed part downloads. - /// Returns (download_timeout, need_shard). + /// If download of all parts is complete, then moves forward to `StateDownloadScheduling`. + /// Returns `(download_timeout, run_shard_state_download)` where: + /// * `download_timeout` means that the state header request timed out (and needs to be retried). + /// * `run_shard_state_download` means that header or part download requests need to run for this shard. fn sync_shards_download_parts_status( &mut self, shard_id: ShardId, @@ -968,7 +974,7 @@ impl StateSync { ) -> (bool, bool) { // Step 2 - download all the parts (each part is usually around 1MB). let mut download_timeout = false; - let mut need_shard = false; + let mut run_shard_state_download = false; let mut parts_done = true; let num_parts = shard_sync_download.downloads.len(); @@ -986,7 +992,6 @@ impl StateSync { chain, ); } - tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, part_download.done, part_download.error); if !part_download.done { parts_done = false; let prev = part_download.prev_update_time; @@ -1001,7 +1006,7 @@ impl StateSync { part_download.prev_update_time = now; } if part_download.run_me.load(Ordering::SeqCst) { - need_shard = true; + run_shard_state_download = true; } } if part_download.done { @@ -1022,7 +1027,7 @@ impl StateSync { status: ShardSyncStatus::StateDownloadScheduling, }; } - (download_timeout, need_shard) + (download_timeout, run_shard_state_download) } fn sync_shards_download_scheduling_status( @@ -1114,7 +1119,7 @@ impl StateSync { get_num_state_parts(shard_state_header.state_root_node().memory_usage); chain.clear_downloaded_parts(shard_id, sync_hash, state_num_parts)?; - let mut this_done = false; + let mut shard_sync_done = false; // If the shard layout is changing in this epoch - we have to apply it right now. if split_states { *shard_sync_download = ShardSyncDownload { @@ -1125,9 +1130,9 @@ impl StateSync { // If there is no layout change - we're done. *shard_sync_download = ShardSyncDownload { downloads: vec![], status: ShardSyncStatus::StateSyncDone }; - this_done = true; + shard_sync_done = true; } - Ok(this_done) + Ok(shard_sync_done) } fn sync_shards_state_split_scheduling_status( @@ -1154,7 +1159,7 @@ impl StateSync { Ok(()) } - /// Returns `this_done`. + /// Returns whether the State Sync for the given shard is complete. fn sync_shards_state_split_applying_status( &mut self, shard_id: ShardId, @@ -1163,14 +1168,14 @@ impl StateSync { chain: &mut Chain, ) -> Result { let result = self.split_state_roots.remove(&shard_id); - let mut this_done = false; + let mut shard_sync_done = false; if let Some(state_roots) = result { chain.build_state_for_split_shards_postprocessing(&sync_hash, state_roots)?; *shard_sync_download = ShardSyncDownload { downloads: vec![], status: ShardSyncStatus::StateSyncDone }; - this_done = true; + shard_sync_done = true; } - Ok(this_done) + Ok(shard_sync_done) } } diff --git a/core/store/src/flat/storage.rs b/core/store/src/flat/storage.rs index 01acf7d5658..93d5c8b5af6 100644 --- a/core/store/src/flat/storage.rs +++ b/core/store/src/flat/storage.rs @@ -1,7 +1,6 @@ use std::collections::HashMap; use std::sync::{Arc, RwLock}; -use near_o11y::metrics::IntGauge; use near_primitives::errors::StorageError; use near_primitives::hash::CryptoHash; use near_primitives::shard_layout::{ShardLayout, ShardUId}; @@ -11,9 +10,10 @@ use tracing::{info, warn}; use crate::flat::delta::CachedFlatStateChanges; use crate::flat::store_helper::FlatStateColumn; use crate::flat::{FlatStorageReadyStatus, FlatStorageStatus}; -use crate::{metrics, Store, StoreUpdate}; +use crate::{Store, StoreUpdate}; use super::delta::{CachedFlatStateDelta, FlatStateDelta}; +use super::metrics::FlatStorageMetrics; use super::types::FlatStorageError; use super::{store_helper, BlockInfo}; @@ -45,14 +45,6 @@ pub(crate) struct FlatStorageInner { metrics: FlatStorageMetrics, } -struct FlatStorageMetrics { - flat_head_height: IntGauge, - distance_to_head: IntGauge, - cached_deltas: IntGauge, - cached_changes_num_items: IntGauge, - cached_changes_size: IntGauge, -} - impl FlatStorageInner { /// Expected limits for in-memory stored changes, under which flat storage must keep working. /// If they are exceeded, warnings are displayed. Flat storage still will work, but its @@ -101,7 +93,7 @@ impl FlatStorageInner { .block .prev_hash; } - self.metrics.distance_to_head.set(blocks.len() as i64); + self.metrics.set_distance_to_head(blocks.len()); Ok(blocks) } @@ -116,17 +108,18 @@ impl FlatStorageInner { cached_changes_size += changes.changes.total_size(); } - self.metrics.cached_deltas.set(cached_deltas as i64); - self.metrics.cached_changes_num_items.set(cached_changes_num_items as i64); - self.metrics.cached_changes_size.set(cached_changes_size as i64); + self.metrics.set_cached_deltas( + cached_deltas, + cached_changes_num_items, + cached_changes_size, + ); let cached_changes_size_bytes = bytesize::ByteSize(cached_changes_size); - let shard_id = self.shard_uid.shard_id(); - let flat_head_height = self.flat_head.height; - if cached_deltas >= Self::CACHED_CHANGES_LIMIT || cached_changes_size_bytes >= Self::CACHED_CHANGES_SIZE_LIMIT { + let shard_id = self.shard_uid.shard_id(); + let flat_head_height = self.flat_head.height; warn!(target: "chain", %shard_id, %flat_head_height, %cached_deltas, %cached_changes_size_bytes, "Flat storage cached deltas exceeded expected limits"); } } @@ -144,22 +137,8 @@ impl FlatStorage { panic!("cannot create flat storage for shard {shard_id} with status {status:?}") } }; - - // `itoa` is much faster for printing shard_id to a string than trivial alternatives. - let mut buffer = itoa::Buffer::new(); - let shard_id_label = buffer.format(shard_id); - let metrics = FlatStorageMetrics { - flat_head_height: metrics::FLAT_STORAGE_HEAD_HEIGHT - .with_label_values(&[shard_id_label]), - distance_to_head: metrics::FLAT_STORAGE_DISTANCE_TO_HEAD - .with_label_values(&[shard_id_label]), - cached_deltas: metrics::FLAT_STORAGE_CACHED_DELTAS.with_label_values(&[shard_id_label]), - cached_changes_num_items: metrics::FLAT_STORAGE_CACHED_CHANGES_NUM_ITEMS - .with_label_values(&[shard_id_label]), - cached_changes_size: metrics::FLAT_STORAGE_CACHED_CHANGES_SIZE - .with_label_values(&[shard_id_label]), - }; - metrics.flat_head_height.set(flat_head.height as i64); + let metrics = FlatStorageMetrics::new(shard_id); + metrics.set_flat_head_height(flat_head.height); let deltas_metadata = store_helper::get_all_deltas_metadata(&store, shard_uid) .unwrap_or_else(|_| { @@ -181,13 +160,9 @@ impl FlatStorage { ); } - Self(Arc::new(RwLock::new(FlatStorageInner { - store, - shard_uid, - flat_head, - deltas, - metrics, - }))) + let inner = FlatStorageInner { store, shard_uid, flat_head, deltas, metrics }; + inner.update_delta_metrics(); + Self(Arc::new(RwLock::new(inner))) } /// Get sequence of blocks `target_block_hash` (inclusive) to flat head (exclusive) @@ -248,7 +223,7 @@ impl FlatStorage { FlatStorageStatus::Ready(FlatStorageReadyStatus { flat_head: block.clone() }), ); - guard.metrics.flat_head_height.set(block.height as i64); + guard.metrics.set_flat_head_height(block.height); guard.flat_head = block.clone(); // Remove old deltas from disk and memory. diff --git a/core/store/src/trie/mod.rs b/core/store/src/trie/mod.rs index fd12b911aaa..39e82b2b95a 100644 --- a/core/store/src/trie/mod.rs +++ b/core/store/src/trie/mod.rs @@ -504,7 +504,7 @@ impl RawTrieNodeWithSize { } pub struct Trie { - pub storage: Box, + pub storage: Box, root: StateRoot, pub flat_storage_chunk_view: Option, } @@ -604,7 +604,7 @@ impl Trie { pub const EMPTY_ROOT: StateRoot = StateRoot::new(); pub fn new( - storage: Box, + storage: Box, root: StateRoot, flat_storage_chunk_view: Option, ) -> Self { @@ -961,29 +961,20 @@ impl Trie { /// storage for key lookup performed in `storage_write`, so we need /// the `use_flat_storage` to differentiate whether the lookup is performed for /// storage_write or not. - #[allow(unused)] pub fn get_ref( &self, key: &[u8], mode: KeyLookupMode, ) -> Result, StorageError> { - let key_nibbles = NibbleSlice::new(key); - let result = self.lookup(key_nibbles); - - // For now, to test correctness, flat storage does double the work and - // compares the results. This needs to be changed when the features is - // stabilized. - if matches!(mode, KeyLookupMode::FlatStorage) { - if let Some(flat_storage_chunk_view) = &self.flat_storage_chunk_view { - let flat_result = flat_storage_chunk_view.get_ref(&key); - if matches!(flat_result, Err(StorageError::FlatStorageBlockNotSupported(_))) { - return flat_result; - } else { - assert_eq!(result, flat_result); - } - } + let use_flat_storage = + matches!(mode, KeyLookupMode::FlatStorage) && self.flat_storage_chunk_view.is_some(); + + if use_flat_storage { + self.flat_storage_chunk_view.as_ref().unwrap().get_ref(&key) + } else { + let key_nibbles = NibbleSlice::new(key); + self.lookup(key_nibbles) } - result } pub fn get(&self, key: &[u8]) -> Result>, StorageError> { diff --git a/core/store/src/trie/split_state.rs b/core/store/src/trie/split_state.rs index 40afdac3264..4f0af2f8580 100644 --- a/core/store/src/trie/split_state.rs +++ b/core/store/src/trie/split_state.rs @@ -102,7 +102,7 @@ impl ShardTries { let mut trie_changes_map = HashMap::new(); for (shard_uid, update) in trie_updates { - let (trie_changes, _) = update.finalize()?; + let (_, trie_changes, _) = update.finalize()?; trie_changes_map.insert(shard_uid, trie_changes); } Ok(trie_changes_map) @@ -196,7 +196,7 @@ impl ShardTries { let mut new_state_roots = HashMap::new(); let mut store_update = self.store_update(); for (shard_uid, update) in updates { - let (trie_changes, state_changes) = update.finalize()?; + let (_, trie_changes, state_changes) = update.finalize()?; let state_root = self.apply_all(&trie_changes, shard_uid, &mut store_update); if cfg!(feature = "protocol_feature_flat_state") { FlatStateChanges::from_state_changes(&state_changes) @@ -525,7 +525,7 @@ mod tests { delayed_receipt_indices.next_available_index = all_receipts.len() as u64; set(&mut trie_update, TrieKey::DelayedReceiptIndices, &delayed_receipt_indices); trie_update.commit(StateChangeCause::Resharding); - let (trie_changes, _) = trie_update.finalize().unwrap(); + let (_, trie_changes, _) = trie_update.finalize().unwrap(); let mut store_update = tries.store_update(); let state_root = tries.apply_all(&trie_changes, ShardUId::single_shard(), &mut store_update); @@ -661,7 +661,7 @@ mod tests { }, ); trie_update.commit(StateChangeCause::Resharding); - let (trie_changes, _) = trie_update.finalize().unwrap(); + let (_, trie_changes, _) = trie_update.finalize().unwrap(); let mut store_update = tries.store_update(); let state_root = tries.apply_all(&trie_changes, ShardUId::single_shard(), &mut store_update); @@ -765,7 +765,7 @@ mod tests { ); set(&mut trie_update, TrieKey::DelayedReceiptIndices, &delayed_receipt_indices); trie_update.commit(StateChangeCause::Resharding); - let (trie_changes, state_changes) = trie_update.finalize().unwrap(); + let (_, trie_changes, state_changes) = trie_update.finalize().unwrap(); let mut store_update = tries.store_update(); let new_state_root = tries.apply_all(&trie_changes, ShardUId::single_shard(), &mut store_update); diff --git a/runtime/runtime-params-estimator/src/estimator_context.rs b/runtime/runtime-params-estimator/src/estimator_context.rs index e2f98122f99..ed2fcf83295 100644 --- a/runtime/runtime-params-estimator/src/estimator_context.rs +++ b/runtime/runtime-params-estimator/src/estimator_context.rs @@ -13,8 +13,8 @@ use near_primitives::transaction::{ExecutionStatus, SignedTransaction}; use near_primitives::types::{Gas, MerkleHash}; use near_primitives::version::PROTOCOL_VERSION; use near_store::flat::{ - store_helper, BlockInfo, FlatStateChanges, FlatStateDelta, FlatStateDeltaMetadata, FlatStorage, - FlatStorageManager, FlatStorageReadyStatus, FlatStorageStatus, + BlockInfo, FlatStateChanges, FlatStateDelta, FlatStateDeltaMetadata, FlatStorage, + FlatStorageManager, }; use near_store::{ShardTries, ShardUId, Store, StoreCompiledContractCache, TrieUpdate}; use near_store::{TrieCache, TrieCachingStorage, TrieConfig}; @@ -22,7 +22,6 @@ use near_vm_logic::{ExtCosts, VMLimitConfig}; use node_runtime::{ApplyState, Runtime}; use std::collections::HashMap; use std::iter; -use std::rc::Rc; use std::sync::Arc; /// Global context shared by all cost calculating functions. @@ -76,12 +75,15 @@ impl<'c> EstimatorContext<'c> { let mut trie_config = near_store::TrieConfig::default(); trie_config.enable_receipt_prefetching = true; - let tries = ShardTries::new( - store.clone(), - trie_config, - &shard_uids, - self.create_flat_storage_manager(store.clone()), - ); + let flat_head = CryptoHash::hash_borsh(0usize); + let flat_storage_manager = FlatStorageManager::test(store.clone(), &shard_uids, flat_head); + if cfg!(feature = "protocol_feature_flat_state") { + let flat_storage = + flat_storage_manager.get_flat_storage_for_shard(shard_uids[0]).unwrap(); + self.generate_deltas(&flat_storage); + } + + let tries = ShardTries::new(store.clone(), trie_config, &shard_uids, flat_storage_manager); Testbed { config: self.config, @@ -143,29 +145,6 @@ impl<'c> EstimatorContext<'c> { } } - fn create_flat_storage_manager(&self, store: Store) -> FlatStorageManager { - let flat_storage_manager = FlatStorageManager::new(store.clone()); - if !cfg!(feature = "protocol_feature_flat_state") { - return flat_storage_manager; - } - - let shard_uid = ShardUId::single_shard(); - // Set up flat head to be equal to the latest block height - let mut store_update = store.store_update(); - store_helper::set_flat_storage_status( - &mut store_update, - shard_uid, - FlatStorageStatus::Ready(FlatStorageReadyStatus { - flat_head: BlockInfo::genesis(CryptoHash::hash_borsh(0usize), 0), - }), - ); - store_update.commit().expect("failed to set flat storage status"); - let flat_storage = FlatStorage::new(store, shard_uid); - self.generate_deltas(&flat_storage); - flat_storage_manager.add_flat_storage_for_shard(shard_uid, flat_storage); - flat_storage_manager - } - /// Construct a chain of fake blocks with fake deltas for flat storage. /// /// Use `hash(height)` as the supposed block hash. @@ -373,7 +352,7 @@ impl Testbed<'_> { tx: &SignedTransaction, metric: GasMetric, ) -> GasCost { - let mut state_update = TrieUpdate::new(Rc::new(self.trie())); + let mut state_update = TrieUpdate::new(self.trie()); // gas price and block height can be anything, it doesn't affect performance // but making it too small affects max_depth and thus pessimistic inflation let gas_price = 100_000_000; @@ -399,7 +378,7 @@ impl Testbed<'_> { /// /// Use this method to estimate action exec costs. pub(crate) fn apply_action_receipt(&mut self, receipt: &Receipt, metric: GasMetric) -> GasCost { - let mut state_update = TrieUpdate::new(Rc::new(self.trie())); + let mut state_update = TrieUpdate::new(self.trie()); let mut outgoing_receipts = vec![]; let mut validator_proposals = vec![]; let mut stats = node_runtime::ApplyStats::default(); From 89134c57fda2ef6122e5ea4986f54b73b9b153cf Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 21 Mar 2023 11:09:11 +0100 Subject: [PATCH 43/88] Don't retry S3 errors until the timeout --- chain/client-primitives/src/types.rs | 2 +- chain/client/src/sync/state.rs | 13 ++++++++----- chain/network/src/network_protocol/mod.rs | 2 +- .../network/src/peer_manager/peer_manager_actor.rs | 2 +- 4 files changed, 11 insertions(+), 8 deletions(-) diff --git a/chain/client-primitives/src/types.rs b/chain/client-primitives/src/types.rs index dee1bd94bc0..86ca8181966 100644 --- a/chain/client-primitives/src/types.rs +++ b/chain/client-primitives/src/types.rs @@ -50,7 +50,7 @@ pub enum AccountOrPeerIdOrHash { AccountId(AccountId), PeerId(PeerId), Hash(CryptoHash), - ExternalStorage(), + ExternalStorage, } #[derive(Debug, serde::Serialize)] diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 24a6c363c8a..35b7908f5c5 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -92,7 +92,7 @@ fn make_account_or_peer_id_or_hash( From::AccountId(a) => To::AccountId(a), From::PeerId(p) => To::PeerId(p), From::Hash(h) => To::Hash(h), - From::ExternalStorage() => To::ExternalStorage(), + From::ExternalStorage => To::ExternalStorage, } } @@ -709,7 +709,7 @@ impl StateSync { } download.state_requests_count += 1; download.last_target = - Some(make_account_or_peer_id_or_hash(AccountOrPeerIdOrHash::ExternalStorage())); + Some(make_account_or_peer_id_or_hash(AccountOrPeerIdOrHash::ExternalStorage)); let location = s3_location(chain_id, epoch_height, shard_id, part_id, num_parts); let download_response = download.response.clone(); @@ -1001,9 +1001,12 @@ impl StateSync { .with_label_values(&[&shard_id.to_string()]) .inc(); download_timeout |= part_timeout; - part_download.run_me.store(true, Ordering::SeqCst); - part_download.error = false; - part_download.prev_update_time = now; + if part_timeout || + part_download.last_target != Some(near_client_primitives::types::AccountOrPeerIdOrHash::ExternalStorage) { + part_download.run_me.store(true, Ordering::SeqCst); + part_download.error = false; + part_download.prev_update_time = now; + } } if part_download.run_me.load(Ordering::SeqCst) { run_shard_state_download = true; diff --git a/chain/network/src/network_protocol/mod.rs b/chain/network/src/network_protocol/mod.rs index ea231c1174c..9b3bb8502ec 100644 --- a/chain/network/src/network_protocol/mod.rs +++ b/chain/network/src/network_protocol/mod.rs @@ -795,7 +795,7 @@ pub enum AccountOrPeerIdOrHash { AccountId(AccountId), PeerId(PeerId), Hash(CryptoHash), - ExternalStorage(), + ExternalStorage, } pub(crate) struct RawRoutedMessage { diff --git a/chain/network/src/peer_manager/peer_manager_actor.rs b/chain/network/src/peer_manager/peer_manager_actor.rs index 22d85bc553d..c1fd90cead9 100644 --- a/chain/network/src/peer_manager/peer_manager_actor.rs +++ b/chain/network/src/peer_manager/peer_manager_actor.rs @@ -658,7 +658,7 @@ impl PeerManagerActor { } AccountOrPeerIdOrHash::PeerId(it) => PeerIdOrHash::PeerId(it.clone()), AccountOrPeerIdOrHash::Hash(it) => PeerIdOrHash::Hash(*it), - AccountOrPeerIdOrHash::ExternalStorage() => unreachable!(), + AccountOrPeerIdOrHash::ExternalStorage => unreachable!(), }; self.state.send_message_to_peer( From 5a1acff9df18932869a57d49302a77d89f4906f8 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 21 Mar 2023 11:35:54 +0100 Subject: [PATCH 44/88] Tune verbosity --- chain/client/src/sync/state.rs | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 35b7908f5c5..e5d8503ed29 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -806,7 +806,7 @@ impl StateSync { use_colour: bool, ) -> Result { let _span = tracing::debug_span!(target: "sync", "run", sync = "StateSync").entered(); - tracing::debug!(target: "sync", %sync_hash, ?tracking_shards, "syncing state"); + tracing::trace!(target: "sync", %sync_hash, ?tracking_shards, "syncing state"); let prev_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); let now = StaticClock::utc(); @@ -1016,7 +1016,7 @@ impl StateSync { num_parts_done += 1; } } - tracing::debug!(target: "sync", %shard_id, %sync_hash, num_parts_done, parts_done); + tracing::trace!(target: "sync", %shard_id, %sync_hash, num_parts_done, parts_done); metrics::STATE_SYNC_PARTS_DONE .with_label_values(&[&shard_id.to_string()]) .set(num_parts_done); @@ -1238,6 +1238,7 @@ fn check_external_storage_part_response( .with_label_values(&[&shard_id.to_string()]) .inc(); part_download.done = true; + tracing::debug!(target: "sync", %shard_id, part_id, ?part_download, "Set state part success"); } Err(err) => { metrics::STATE_SYNC_EXTERNAL_PARTS_FAILED @@ -1251,6 +1252,7 @@ fn check_external_storage_part_response( } // Other HTTP status codes are considered errors. Ok((status_code, _)) => { + tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, status_code, "Wrong response code, expected 200"); err_to_retry = Some(near_chain::Error::Other(format!("status_code: {}", status_code).to_string())); } @@ -1263,7 +1265,6 @@ fn check_external_storage_part_response( if let Some(err) = err_to_retry { tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to get a part from external storage, will retry"); part_download.error = true; - } else { } } @@ -1299,8 +1300,15 @@ fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload, use_colour: shard_sync_download.downloads[0].last_target ), ShardSyncStatus::StateDownloadParts => { + let mut num_parts_done = 0; + let mut num_parts_not_done = 0; let mut text = "".to_string(); for (i, download) in shard_sync_download.downloads.iter().enumerate() { + if download.done { + num_parts_done += 1; + continue; + } + num_parts_not_done += 1; text.push_str(&format!( "[{}: {}, {}, {:?}] ", paint(&i.to_string(), Yellow.bold(), use_colour), @@ -1310,10 +1318,12 @@ fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload, use_colour: )); } format!( - "{} [{}: is_done, requests sent, last target] {}", + "{} [{}: is_done, requests sent, last target] {} num_parts_done={} num_parts_not_done={}", paint("PARTS", Purple.bold(), use_colour), paint("part_id", Yellow.bold(), use_colour), - text + text, + num_parts_done, + num_parts_not_done ) } _ => unreachable!("timeout cannot happen when all state is downloaded"), From 4a9aa864fbb5e42078ea1b164c8789808d257d48 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 21 Mar 2023 12:33:15 +0100 Subject: [PATCH 45/88] Timing --- chain/chain/src/chain.rs | 17 ++++++++++++++++- chain/client/src/client_actor.rs | 1 + tools/delay-detector/src/lib.rs | 2 ++ 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/chain/chain/src/chain.rs b/chain/chain/src/chain.rs index dedc1369b1c..db2bc9d2541 100644 --- a/chain/chain/src/chain.rs +++ b/chain/chain/src/chain.rs @@ -1817,6 +1817,7 @@ impl Chain { } pub fn reset_data_pre_state_sync(&mut self, sync_hash: CryptoHash) -> Result<(), Error> { + let mut d = DelayDetector::new(|| "reset_data_pre_state_sync".into()); let _span = tracing::debug_span!(target: "sync", "reset_data_pre_state_sync").entered(); let head = self.head()?; // Get header we were syncing into. @@ -1829,7 +1830,12 @@ impl Chain { // there is no block, we need to make sure that the last block before tail is cleaned. let tail = self.store.tail()?; let mut tail_prev_block_cleaned = false; + tracing::debug!(target: "sync", tail, gc_height, "reset_data_pre_state_sync"); for height in tail..gc_height { + if height % 100 == 0 { + d.snapshot(&format!("reset_data_pre_state_sync height={}", height)); + tracing::debug!(target: "sync", height, tail, gc_height, "reset_data_pre_state_sync progress"); + } if let Ok(blocks_current_height) = self.store.get_all_block_hashes_by_height(height) { let blocks_current_height = blocks_current_height.values().flatten().cloned().collect::>(); @@ -1858,6 +1864,9 @@ impl Chain { } } + d.snapshot("reset_data_pre_state_sync loop done"); + tracing::debug!(target: "sync", "reset_data_pre_state_sync loop done"); + // Clear Chunks data let mut chain_store_update = self.mut_store().store_update(); // The largest height of chunk we have in storage is head.height + 1 @@ -1865,8 +1874,10 @@ impl Chain { chain_store_update.clear_chunk_data_and_headers(chunk_height)?; chain_store_update.commit()?; - // clear all trie data + d.snapshot("reset_data_pre_state_sync chunks data cleaned up"); + tracing::debug!(target: "sync", "reset_data_pre_state_sync chunks data cleaned up"); + // clear all trie data let tries = self.runtime_adapter.get_tries(); let mut chain_store_update = self.mut_store().store_update(); let mut store_update = StoreUpdate::new_with_tries(tries); @@ -1876,6 +1887,10 @@ impl Chain { // The reason to reset tail here is not to allow Tail be greater than Head chain_store_update.reset_tail(); chain_store_update.commit()?; + + d.snapshot("reset_data_pre_state_sync chunks trie data cleaned up"); + tracing::debug!(target: "sync", "reset_data_pre_state_sync chunks trie data cleaned up"); + Ok(()) } diff --git a/chain/client/src/client_actor.rs b/chain/client/src/client_actor.rs index 91d60fe908e..aa5afa9e4c2 100644 --- a/chain/client/src/client_actor.rs +++ b/chain/client/src/client_actor.rs @@ -1440,6 +1440,7 @@ impl ClientActor { /// Usually `state_fetch_horizon` is much less than the expected number of produced blocks on an epoch, /// so this is only relevant on epoch boundaries. fn find_sync_hash(&mut self) -> Result { + let _d = delay_detector::DelayDetector::new(|| "find_sync_hash".into()); let header_head = self.client.chain.header_head()?; let mut sync_hash = header_head.prev_block_hash; for _ in 0..self.client.config.state_fetch_horizon { diff --git a/tools/delay-detector/src/lib.rs b/tools/delay-detector/src/lib.rs index e33888a15bf..0ef9452d49f 100644 --- a/tools/delay-detector/src/lib.rs +++ b/tools/delay-detector/src/lib.rs @@ -12,6 +12,8 @@ mod disabled { pub fn new<'a>(_msg: impl FnOnce() -> Cow<'a, str>) -> Self { Self {} } + + pub fn snapshot(&mut self, _msg: &str) {} } } From 56dc868e60bcbadd4d946b8ef11019a930e9b302 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 21 Mar 2023 16:07:44 +0100 Subject: [PATCH 46/88] S3 downloads should now work. --- chain/client/src/client_actor.rs | 3 +- chain/client/src/info.rs | 26 ++++++----------- chain/client/src/sync/state.rs | 49 +++++++++++++++++++------------- 3 files changed, 39 insertions(+), 39 deletions(-) diff --git a/chain/client/src/client_actor.rs b/chain/client/src/client_actor.rs index aa5afa9e4c2..f011a431955 100644 --- a/chain/client/src/client_actor.rs +++ b/chain/client/src/client_actor.rs @@ -1278,7 +1278,8 @@ impl ClientActor { .runtime_adapter .get_epoch_start_height(&last_final_hash) .unwrap_or(last_final_block_height); - let last_final_block_height_in_epoch = last_final_block_height - epoch_start_height; + let last_final_block_height_in_epoch = + last_final_block_height.checked_sub(epoch_start_height); self.info_helper.block_processed( gas_used, diff --git a/chain/client/src/info.rs b/chain/client/src/info.rs index ea254d88e65..af7e2f45e6c 100644 --- a/chain/client/src/info.rs +++ b/chain/client/src/info.rs @@ -120,8 +120,6 @@ impl InfoHelper { metrics::FINAL_DOOMSLUG_BLOCK_HEIGHT.set(last_final_ds_block_height as i64); metrics::EPOCH_HEIGHT.set(epoch_height as i64); if let Some(last_final_block_height_in_epoch) = last_final_block_height_in_epoch { - // In rare cases cases the final height isn't updated, for example right after a state sync. - // Don't update the metric in such cases. metrics::FINAL_BLOCK_HEIGHT_IN_EPOCH.set(last_final_block_height_in_epoch as i64); } } @@ -146,16 +144,15 @@ impl InfoHelper { fn record_block_producers(head: &Tip, client: &crate::client::Client) { let me = client.validator_signer.as_ref().map(|x| x.validator_id().clone()); - if let Some(is_bp) = me.map_or(Some(false), |account_id| { - // In rare cases block producer information isn't available. - // Don't set the metric in this case. + let is_bp = me.map_or(false, |account_id| { client .runtime_adapter .get_epoch_block_producers_ordered(&head.epoch_id, &head.last_block_hash) - .map_or(None, |bp| Some(bp.iter().any(|bp| bp.0.account_id() == &account_id))) - }) { - metrics::IS_BLOCK_PRODUCER.set(if is_bp { 1 } else { 0 }); - } + .unwrap() + .iter() + .any(|bp| bp.0.account_id() == &account_id) + }); + metrics::IS_BLOCK_PRODUCER.set(if is_bp { 1 } else { 0 }); } fn record_chunk_producers(head: &Tip, client: &crate::client::Client) { @@ -533,14 +530,6 @@ pub fn display_sync_status(sync_status: &SyncStatus, head: &Tip) -> String { for (shard_id, shard_status) in shard_statuses { write!(res, "[{}: {}]", shard_id, shard_status.status.to_string(),).unwrap(); } - // TODO #8719 - tracing::warn!(target: "stats", - "The node is syncing its State. The current implementation of this mechanism is known to be unreliable. It may never complete, or fail randomly and corrupt the DB.\n\ - Suggestions:\n\ - * Download a recent data snapshot and restart the node.\n\ - * Disable state sync in the config. Add `\"state_sync_enabled\": false` to `config.json`.\n\ - \n\ - A better implementation of State Sync is work in progress."); res } SyncStatus::StateSyncDone => "State sync done".to_string(), @@ -747,7 +736,8 @@ mod tests { let store = near_store::test_utils::create_test_store(); let vs = ValidatorSchedule::new().block_producers_per_epoch(vec![vec!["test".parse().unwrap()]]); - let runtime = KeyValueRuntime::new_with_validators_and_no_gc(store, vs, 123, false); + let runtime = + Arc::new(KeyValueRuntime::new_with_validators_and_no_gc(store, vs, 123, false)); let chain_genesis = ChainGenesis { time: StaticClock::utc(), height: 0, diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index e5d8503ed29..dd2247b031b 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -57,6 +57,7 @@ pub const MAX_STATE_PART_REQUEST: u64 = 16; /// This number should not exceed MAX_STATE_PART_REQUEST times (number of peers in the network). pub const MAX_PENDING_PART: u64 = MAX_STATE_PART_REQUEST * 10000; +#[derive(Debug)] pub enum StateSyncResult { /// No shard has changed its status Unchanged, @@ -289,14 +290,16 @@ impl StateSync { )?; } ShardSyncStatus::StateDownloadParts => { - (download_timeout, run_shard_state_download) = self - .sync_shards_download_parts_status( - shard_id, - shard_sync_download, - sync_hash, - chain, - now, - ); + let res = self.sync_shards_download_parts_status( + shard_id, + shard_sync_download, + sync_hash, + chain, + now, + ); + download_timeout = res.0; + run_shard_state_download = res.1; + update_sync_status |= res.2; } ShardSyncStatus::StateDownloadScheduling => { self.sync_shards_download_scheduling_status( @@ -703,7 +706,8 @@ impl StateSync { return; } else { if !download.run_me.swap(false, Ordering::SeqCst) { - tracing::info!(target: "sync", %shard_id, part_id, "run_me is already false"); + tracing::info!(target: "sync", %shard_id, part_id, "External storage request is allowed but run_me is already false. Undoing"); + finished_request(&requests_remaining); return; } } @@ -961,9 +965,10 @@ impl StateSync { /// Checks if the parts are downloaded. /// If download of all parts is complete, then moves forward to `StateDownloadScheduling`. - /// Returns `(download_timeout, run_shard_state_download)` where: + /// Returns `(download_timeout, run_shard_state_download, update_sync_status)` where: /// * `download_timeout` means that the state header request timed out (and needs to be retried). /// * `run_shard_state_download` means that header or part download requests need to run for this shard. + /// * `update_sync_status` means that something changed in `ShardSyncDownload` and it needs to be persisted. fn sync_shards_download_parts_status( &mut self, shard_id: ShardId, @@ -971,19 +976,19 @@ impl StateSync { sync_hash: CryptoHash, chain: &mut Chain, now: DateTime, - ) -> (bool, bool) { + ) -> (bool, bool, bool) { // Step 2 - download all the parts (each part is usually around 1MB). let mut download_timeout = false; let mut run_shard_state_download = false; + let mut update_sync_status = false; let mut parts_done = true; let num_parts = shard_sync_download.downloads.len(); let mut num_parts_done = 0; for (part_id, part_download) in shard_sync_download.downloads.iter_mut().enumerate() { - // tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, part_download.done, part_download.error, ?part_download); if !part_download.done { // Check if a download from an external storage is finished. - check_external_storage_part_response( + update_sync_status |= check_external_storage_part_response( part_id as u64, num_parts as u64, shard_id, @@ -1006,6 +1011,7 @@ impl StateSync { part_download.run_me.store(true, Ordering::SeqCst); part_download.error = false; part_download.prev_update_time = now; + update_sync_status = true; } } if part_download.run_me.load(Ordering::SeqCst) { @@ -1016,7 +1022,6 @@ impl StateSync { num_parts_done += 1; } } - tracing::trace!(target: "sync", %shard_id, %sync_hash, num_parts_done, parts_done); metrics::STATE_SYNC_PARTS_DONE .with_label_values(&[&shard_id.to_string()]) .set(num_parts_done); @@ -1029,8 +1034,9 @@ impl StateSync { downloads: vec![], status: ShardSyncStatus::StateDownloadScheduling, }; + update_sync_status = true; } - (download_timeout, run_shard_state_download) + (download_timeout, run_shard_state_download, update_sync_status) } fn sync_shards_download_scheduling_status( @@ -1184,11 +1190,11 @@ impl StateSync { fn allow_request(requests_remaining: &AtomicI64) -> bool { let remaining = requests_remaining.fetch_sub(1, Ordering::SeqCst); - if remaining <= 0 { + if remaining >= 0 { + true + } else { requests_remaining.fetch_add(1, Ordering::SeqCst); false - } else { - true } } @@ -1200,6 +1206,8 @@ fn finished_request(requests_remaining: &AtomicI64) { /// The response is stored on the DownloadStatus object. /// This function investigates if the response is available and updates `done` and `error` appropriately. /// If the response is successful, then also writes the state part to the DB. +/// +/// Returns whether something changed in `DownloadStatus` which means it needs to be persisted. fn check_external_storage_part_response( part_id: u64, num_parts: u64, @@ -1207,7 +1215,7 @@ fn check_external_storage_part_response( sync_hash: CryptoHash, part_download: &mut DownloadStatus, chain: &mut Chain, -) { +) -> bool { let external_storage_response = { let mut lock = part_download.response.lock().unwrap(); if let Some(response) = lock.clone() { @@ -1218,7 +1226,7 @@ fn check_external_storage_part_response( *lock = None; response } else { - return; + return false; } }; @@ -1266,6 +1274,7 @@ fn check_external_storage_part_response( tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to get a part from external storage, will retry"); part_download.error = true; } + true } /// Construct a location on the external storage. From 74773e009cd7db3551ffeff8bdfdfb0592476b68 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 21 Mar 2023 16:16:15 +0100 Subject: [PATCH 47/88] Metrics for part application. --- chain/client/src/client_actor.rs | 3 +++ chain/client/src/metrics.rs | 10 ++++++++++ 2 files changed, 13 insertions(+) diff --git a/chain/client/src/client_actor.rs b/chain/client/src/client_actor.rs index f011a431955..0755aaba2ce 100644 --- a/chain/client/src/client_actor.rs +++ b/chain/client/src/client_actor.rs @@ -1714,6 +1714,9 @@ impl SyncJobsActor { let store = msg.runtime.store(); for part_id in 0..msg.num_parts { + let _timer = metrics::STATE_SYNC_APPLY_PART_DELAY + .with_label_values(&[&msg.shard_id.to_string()]) + .start_timer(); let key = StatePartKey(msg.sync_hash, msg.shard_id, part_id).try_to_vec()?; let part = store.get(DBCol::StateParts, &key)?.unwrap(); diff --git a/chain/client/src/metrics.rs b/chain/client/src/metrics.rs index 4df5ffb17ca..3ebf0f9a5e4 100644 --- a/chain/client/src/metrics.rs +++ b/chain/client/src/metrics.rs @@ -441,3 +441,13 @@ pub static STATE_SYNC_EXTERNAL_PARTS_REQUEST_DELAY: Lazy = Lazy::new(|| { + try_create_histogram_vec( + "near_state_sync_apply_part_delay", + "Latency of applying a state part", + &["shard_id"], + Some(exponential_buckets(0.001, 2.0, 20).unwrap()), + ) + .unwrap() +}); From 81d511f1140675b8cb4b79746f5920cdb9cc7325 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 21 Mar 2023 16:29:06 +0100 Subject: [PATCH 48/88] Metrics for part application. --- chain/client/src/info.rs | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/chain/client/src/info.rs b/chain/client/src/info.rs index af7e2f45e6c..8b3e8965354 100644 --- a/chain/client/src/info.rs +++ b/chain/client/src/info.rs @@ -120,6 +120,8 @@ impl InfoHelper { metrics::FINAL_DOOMSLUG_BLOCK_HEIGHT.set(last_final_ds_block_height as i64); metrics::EPOCH_HEIGHT.set(epoch_height as i64); if let Some(last_final_block_height_in_epoch) = last_final_block_height_in_epoch { + // In rare cases cases the final height isn't updated, for example right after a state sync. + // Don't update the metric in such cases. metrics::FINAL_BLOCK_HEIGHT_IN_EPOCH.set(last_final_block_height_in_epoch as i64); } } @@ -144,15 +146,16 @@ impl InfoHelper { fn record_block_producers(head: &Tip, client: &crate::client::Client) { let me = client.validator_signer.as_ref().map(|x| x.validator_id().clone()); - let is_bp = me.map_or(false, |account_id| { + if let Some(is_bp) = me.map_or(Some(false), |account_id| { + // In rare cases block producer information isn't available. + // Don't set the metric in this case. client .runtime_adapter .get_epoch_block_producers_ordered(&head.epoch_id, &head.last_block_hash) - .unwrap() - .iter() - .any(|bp| bp.0.account_id() == &account_id) - }); - metrics::IS_BLOCK_PRODUCER.set(if is_bp { 1 } else { 0 }); + .map_or(None, |bp| Some(bp.iter().any(|bp| bp.0.account_id() == &account_id))) + }) { + metrics::IS_BLOCK_PRODUCER.set(if is_bp { 1 } else { 0 }); + } } fn record_chunk_producers(head: &Tip, client: &crate::client::Client) { From 6db26381a4be032728ec201aee05772102fe11ef Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 21 Mar 2023 16:59:19 +0100 Subject: [PATCH 49/88] Metrics for part application. --- chain/client/src/metrics.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/chain/client/src/metrics.rs b/chain/client/src/metrics.rs index 3ebf0f9a5e4..f6f5f45f140 100644 --- a/chain/client/src/metrics.rs +++ b/chain/client/src/metrics.rs @@ -444,7 +444,7 @@ pub static STATE_SYNC_EXTERNAL_PARTS_REQUEST_DELAY: Lazy = Lazy::new(|| { try_create_histogram_vec( - "near_state_sync_apply_part_delay", + "near_state_sync_part_apply_delay_sec", "Latency of applying a state part", &["shard_id"], Some(exponential_buckets(0.001, 2.0, 20).unwrap()), From 51dbd552c423b38479b8d86c7aa39c7750aedfe7 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 22 Mar 2023 13:49:17 +0100 Subject: [PATCH 50/88] More metrics for part download --- chain/client/src/metrics.rs | 10 ++++++++++ chain/client/src/sync/state.rs | 3 +++ 2 files changed, 13 insertions(+) diff --git a/chain/client/src/metrics.rs b/chain/client/src/metrics.rs index f6f5f45f140..0bdaf977ac0 100644 --- a/chain/client/src/metrics.rs +++ b/chain/client/src/metrics.rs @@ -451,3 +451,13 @@ pub static STATE_SYNC_APPLY_PART_DELAY: Lazy = ) .unwrap() }); + +pub static STATE_SYNC_EXTERNAL_PARTS_SIZE_DOWNLOADED: Lazy = + Lazy::new(|| { + try_create_int_counter_vec( + "near_state_sync_external_parts_size_downloaded_bytes_total", + "Amount of bytes downloaded from an external storage when requesting state parts for a shard", + &["shard_id"], + ) + .unwrap() + }); diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index dd2247b031b..8dd8836b7a2 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -1245,6 +1245,9 @@ fn check_external_storage_part_response( metrics::STATE_SYNC_EXTERNAL_PARTS_DONE .with_label_values(&[&shard_id.to_string()]) .inc(); + metrics::STATE_SYNC_EXTERNAL_PARTS_SIZE_DOWNLOADED + .with_label_values(&[&shard_id.to_string()]) + .inc_by(data.len() as u64); part_download.done = true; tracing::debug!(target: "sync", %shard_id, part_id, ?part_download, "Set state part success"); } From 602232319cb589a0c0e9ee46931fe08719be1fe7 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 23 Mar 2023 14:34:53 +0100 Subject: [PATCH 51/88] Ready for review --- chain/chain/src/chain.rs | 20 +++++++++----------- chain/chain/src/metrics.rs | 10 ++++++++++ chain/client-primitives/src/types.rs | 9 ++++++--- chain/client/src/client_actor.rs | 25 ++++++++++++------------- chain/client/src/sync/state.rs | 28 ++++++++++++---------------- nearcore/src/config.rs | 14 ++++++++++---- tools/delay-detector/src/lib.rs | 2 -- 7 files changed, 59 insertions(+), 49 deletions(-) diff --git a/chain/chain/src/chain.rs b/chain/chain/src/chain.rs index db2bc9d2541..5572420ab77 100644 --- a/chain/chain/src/chain.rs +++ b/chain/chain/src/chain.rs @@ -1817,7 +1817,6 @@ impl Chain { } pub fn reset_data_pre_state_sync(&mut self, sync_hash: CryptoHash) -> Result<(), Error> { - let mut d = DelayDetector::new(|| "reset_data_pre_state_sync".into()); let _span = tracing::debug_span!(target: "sync", "reset_data_pre_state_sync").entered(); let head = self.head()?; // Get header we were syncing into. @@ -1830,11 +1829,10 @@ impl Chain { // there is no block, we need to make sure that the last block before tail is cleaned. let tail = self.store.tail()?; let mut tail_prev_block_cleaned = false; - tracing::debug!(target: "sync", tail, gc_height, "reset_data_pre_state_sync"); + tracing::trace!(target: "sync", tail, gc_height, "reset_data_pre_state_sync"); for height in tail..gc_height { - if height % 100 == 0 { - d.snapshot(&format!("reset_data_pre_state_sync height={}", height)); - tracing::debug!(target: "sync", height, tail, gc_height, "reset_data_pre_state_sync progress"); + if height % 1000 == 0 { + tracing::trace!(target: "sync", tail, gc_height, progress = height, "reset_data_pre_state_sync"); } if let Ok(blocks_current_height) = self.store.get_all_block_hashes_by_height(height) { let blocks_current_height = @@ -1864,8 +1862,7 @@ impl Chain { } } - d.snapshot("reset_data_pre_state_sync loop done"); - tracing::debug!(target: "sync", "reset_data_pre_state_sync loop done"); + tracing::trace!(target: "sync", progress = "loop done", "reset_data_pre_state_sync"); // Clear Chunks data let mut chain_store_update = self.mut_store().store_update(); @@ -1874,8 +1871,7 @@ impl Chain { chain_store_update.clear_chunk_data_and_headers(chunk_height)?; chain_store_update.commit()?; - d.snapshot("reset_data_pre_state_sync chunks data cleaned up"); - tracing::debug!(target: "sync", "reset_data_pre_state_sync chunks data cleaned up"); + tracing::trace!(target: "sync", progress = "chunks data cleaned up", "reset_data_pre_state_sync"); // clear all trie data let tries = self.runtime_adapter.get_tries(); @@ -1888,8 +1884,7 @@ impl Chain { chain_store_update.reset_tail(); chain_store_update.commit()?; - d.snapshot("reset_data_pre_state_sync chunks trie data cleaned up"); - tracing::debug!(target: "sync", "reset_data_pre_state_sync chunks trie data cleaned up"); + tracing::trace!(target: "sync", progress = "state data cleaned up", "reset_data_pre_state_sync"); Ok(()) } @@ -3149,6 +3144,9 @@ impl Chain { part_id: PartId, data: &[u8], ) -> Result<(), Error> { + let _timer = metrics::STATE_SYNC_SET_STATE_PART_DELAY + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); let shard_state_header = self.get_state_header(shard_id, sync_hash)?; let chunk = shard_state_header.take_chunk(); let state_root = *chunk.take_header().take_inner().prev_state_root(); diff --git a/chain/chain/src/metrics.rs b/chain/chain/src/metrics.rs index e38c70a68c1..1309ab3f938 100644 --- a/chain/chain/src/metrics.rs +++ b/chain/chain/src/metrics.rs @@ -110,3 +110,13 @@ pub static STATE_PART_ELAPSED: Lazy = Lazy::new(|| { pub static NUM_INVALID_BLOCKS: Lazy = Lazy::new(|| { try_create_int_gauge("near_num_invalid_blocks", "Number of invalid blocks").unwrap() }); +pub static STATE_SYNC_SET_STATE_PART_DELAY: Lazy = + Lazy::new(|| { + try_create_histogram_vec( + "near_state_sync_set_state_part_delay", + "Delay for saving a state part in the DB", + &["shard_id"], + Some(exponential_buckets(0.001, 2.0, 20).unwrap()), + ) + .unwrap() + }); diff --git a/chain/client-primitives/src/types.rs b/chain/client-primitives/src/types.rs index 86ca8181966..3a5e61c33a3 100644 --- a/chain/client-primitives/src/types.rs +++ b/chain/client-primitives/src/types.rs @@ -86,12 +86,13 @@ impl Clone for DownloadStatus { DownloadStatus { start_time: self.start_time, prev_update_time: self.prev_update_time, + // Creates a new `Arc` holding the same value. run_me: Arc::new(AtomicBool::new(self.run_me.load(Ordering::SeqCst))), error: self.error, done: self.done, state_requests_count: self.state_requests_count, last_target: self.last_target.clone(), - // Copy the contents of `response`, but make it an independent object. + // Clones the `Arc` containing the value. response: self.response.clone(), } } @@ -199,7 +200,7 @@ pub struct ShardSyncDownload { } impl ShardSyncDownload { - /// Creates a instance of self which includes initial statuses for shard sync and download at the given time. + /// Creates a instance of self which includes initial statuses for shard state header download at the given time. pub fn new_download_state_header(now: DateTime) -> Self { Self { downloads: vec![DownloadStatus::new(now); 1], @@ -207,8 +208,10 @@ impl ShardSyncDownload { } } + /// Creates a instance of self which includes initial statuses for shard state parts download at the given time. pub fn new_download_state_parts(now: DateTime, num_parts: u64) -> Self { - // Avoid using `vec![x; num_parts]`, because cloning DownloadStatus::response is a terrible idea. + // Avoid using `vec![x; num_parts]`, because each element needs to have + // its own independent value of `response`. let mut downloads = Vec::with_capacity(num_parts as usize); for _ in 0..num_parts { downloads.push(DownloadStatus::new(now)); diff --git a/chain/client/src/client_actor.rs b/chain/client/src/client_actor.rs index 0755aaba2ce..290c93bcd61 100644 --- a/chain/client/src/client_actor.rs +++ b/chain/client/src/client_actor.rs @@ -69,7 +69,7 @@ use std::sync::Arc; use std::thread; use std::time::{Duration, Instant}; use tokio::sync::broadcast; -use tracing::{error, info, warn}; +use tracing::{debug, error, info, trace, warn}; /// Multiplier on `max_block_time` to wait until deciding that chain stalled. const STATUS_WAIT_TIME_MULTIPLIER: u64 = 10; @@ -256,7 +256,7 @@ impl Actor for ClientActor { self.catchup(ctx); if let Err(err) = self.client.send_network_chain_info() { - tracing::error!(target: "client", ?err, "Failed to update network chain info"); + error!(target: "client", ?err, "Failed to update network chain info"); } } } @@ -500,7 +500,7 @@ impl Handler> for ClientActor { fn handle(&mut self, msg: WithSpanContext, ctx: &mut Context) { self.wrap(msg, ctx, "BlockApproval", |this, msg| { let BlockApproval(approval, peer_id) = msg; - tracing::debug!(target: "client", "Receive approval {:?} from peer {:?}", approval, peer_id); + debug!(target: "client", "Receive approval {:?} from peer {:?}", approval, peer_id); this.client.collect_block_approval(&approval, ApprovalType::PeerApproval(peer_id)); }) } @@ -518,7 +518,7 @@ impl Handler> for ClientActor { let hash = state_response_info.sync_hash(); let state_response = state_response_info.take_state_response(); - tracing::trace!(target: "sync", "Received state response shard_id: {} sync_hash: {:?} part(id/size): {:?}", + trace!(target: "sync", "Received state response shard_id: {} sync_hash: {:?} part(id/size): {:?}", shard_id, hash, state_response.part().as_ref().map(|(part_id, data)| (part_id, data.len())) @@ -815,7 +815,7 @@ impl ClientActor { fn check_send_announce_account(&mut self, prev_block_hash: CryptoHash) { // If no peers, there is no one to announce to. if self.network_info.num_connected_peers == 0 { - tracing::debug!(target: "client", "No peers: skip account announce"); + debug!(target: "client", "No peers: skip account announce"); return; } @@ -835,7 +835,7 @@ impl ClientActor { } } - tracing::debug!(target: "client", "Check announce account for {}, last announce time {:?}", validator_signer.validator_id(), self.last_validator_announce_time); + debug!(target: "client", "Check announce account for {}, last announce time {:?}", validator_signer.validator_id(), self.last_validator_announce_time); // Announce AccountId if client is becoming a validator soon. let next_epoch_id = unwrap_or_return!(self @@ -845,7 +845,7 @@ impl ClientActor { // Check client is part of the futures validators if self.client.is_validator(&next_epoch_id, &prev_block_hash) { - tracing::debug!(target: "client", "Sending announce account for {}", validator_signer.validator_id()); + debug!(target: "client", "Sending announce account for {}", validator_signer.validator_id()); self.last_validator_announce_time = Some(now); let signature = validator_signer.sign_account_announce( @@ -948,7 +948,7 @@ impl ClientActor { let _span = tracing::debug_span!(target: "client", "handle_block_production").entered(); // If syncing, don't try to produce blocks. if self.client.sync_status.is_syncing() { - tracing::debug!(target:"client", sync_status=?self.client.sync_status, "Syncing - block production disabled"); + debug!(target:"client", sync_status=?self.client.sync_status, "Syncing - block production disabled"); return Ok(()); } @@ -979,9 +979,9 @@ impl ClientActor { // We try to produce block for multiple heights (up to the highest height for which we've seen 2/3 of approvals). if latest_known.height + 1 <= self.client.doomslug.get_largest_height_crossing_threshold() { - tracing::debug!(target: "client", "Considering blocks for production between {} and {} ", latest_known.height + 1, self.client.doomslug.get_largest_height_crossing_threshold()); + debug!(target: "client", "Considering blocks for production between {} and {} ", latest_known.height + 1, self.client.doomslug.get_largest_height_crossing_threshold()); } else { - tracing::debug!(target: "client", "Cannot produce any block: not enough approvals beyond {}", latest_known.height); + debug!(target: "client", "Cannot produce any block: not enough approvals beyond {}", latest_known.height); } let me = if let Some(me) = &self.client.validator_signer { @@ -1331,7 +1331,7 @@ impl ClientActor { error!(target: "client", "Error processing sync blocks: {}", err); false } else { - tracing::debug!(target: "client", "Block headers refused by chain: {}", err); + debug!(target: "client", "Block headers refused by chain: {}", err); true } } @@ -1441,7 +1441,6 @@ impl ClientActor { /// Usually `state_fetch_horizon` is much less than the expected number of produced blocks on an epoch, /// so this is only relevant on epoch boundaries. fn find_sync_hash(&mut self) -> Result { - let _d = delay_detector::DelayDetector::new(|| "find_sync_hash".into()); let header_head = self.client.chain.header_head()?; let mut sync_hash = header_head.prev_block_hash; for _ in 0..self.client.config.state_fetch_horizon { @@ -1547,7 +1546,7 @@ impl ClientActor { if !self.needs_syncing(needs_syncing) { if currently_syncing { - tracing::debug!( + debug!( target: "client", "{:?} transitions to no sync", self.client.validator_signer.as_ref().map(|vs| vs.validator_id()), diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 8dd8836b7a2..6748d9ca3af 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -108,8 +108,6 @@ pub enum StateSyncMode { chain_id: String, /// Connection to the external storage. bucket: Arc, - /// Number state part requests allowed to be in-flight in parallel per shard. - num_s3_requests_per_shard: u64, }, } @@ -174,7 +172,6 @@ impl StateSync { StateSyncMode::HeaderFromPeersAndPartsFromExternal { chain_id: chain_id.to_string(), bucket, - num_s3_requests_per_shard, }, None, ) @@ -671,11 +668,7 @@ impl StateSync { sync_hash, ); } - StateSyncMode::HeaderFromPeersAndPartsFromExternal { - chain_id, - bucket, - num_s3_requests_per_shard: _, - } => { + StateSyncMode::HeaderFromPeersAndPartsFromExternal { chain_id, bucket } => { self.request_part_from_external_storage( part_id as u64, download, @@ -720,7 +713,7 @@ impl StateSync { let scheduled = StaticClock::utc(); near_performance_metrics::actix::spawn(std::any::type_name::(), { async move { - tracing::info!(target: "sync", %shard_id, part_id, location, "Getting an object from the external storage"); + tracing::debug!(target: "sync", %shard_id, part_id, location, "Getting an object from the external storage"); let started = StaticClock::utc(); metrics::STATE_SYNC_EXTERNAL_PARTS_SCHEDULING_DELAY .with_label_values(&[&shard_id.to_string()]) @@ -741,12 +734,12 @@ impl StateSync { ); match result { Ok(response) => { - tracing::info!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), "S3 request finished"); + tracing::debug!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), "S3 request finished"); let mut lock = download_response.lock().unwrap(); *lock = Some(Ok((response.status_code(), response.bytes().to_vec()))); } Err(err) => { - tracing::info!(target: "sync", %shard_id, part_id, location, ?err, "S3 request failed"); + tracing::debug!(target: "sync", %shard_id, part_id, location, ?err, "S3 request failed"); let mut lock = download_response.lock().unwrap(); *lock = Some(Err(err.to_string())); } @@ -1188,6 +1181,7 @@ impl StateSync { } } +/// Verifies that one more concurrent request can be started. fn allow_request(requests_remaining: &AtomicI64) -> bool { let remaining = requests_remaining.fetch_sub(1, Ordering::SeqCst); if remaining >= 0 { @@ -1198,6 +1192,7 @@ fn allow_request(requests_remaining: &AtomicI64) -> bool { } } +/// Notify about a request finishing. fn finished_request(requests_remaining: &AtomicI64) { requests_remaining.fetch_add(1, Ordering::SeqCst); } @@ -1221,8 +1216,8 @@ fn check_external_storage_part_response( if let Some(response) = lock.clone() { tracing::debug!(target: "sync", %shard_id, part_id, "Got response from external storage"); // Remove the response from DownloadStatus, because - // we're going to write positive responses to the DB - // and retry negative responses. + // we're going to write state parts to DB and don't need to keep + // them in `DownloadStatus`. *lock = None; response } else { @@ -1294,9 +1289,10 @@ pub fn s3_location( ) } -fn paint(s: &str, colour: Style, use_colour: bool) -> String { - if use_colour { - colour.paint(s).to_string() +/// Applies style if `use_colour` is enabled. +fn paint(s: &str, style: Style, use_style: bool) -> String { + if use_style { + style.paint(s).to_string() } else { s.to_string() } diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index cadab343a22..4b7e3efe76a 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -705,6 +705,7 @@ impl NearConfig { .state_sync .as_ref() .map(|x| x.dump_enabled) + .flatten() .unwrap_or(false), state_sync_s3_bucket: config .state_sync @@ -726,6 +727,7 @@ impl NearConfig { .state_sync .as_ref() .map(|x| x.sync_from_s3_enabled) + .flatten() .unwrap_or(false), state_sync_num_s3_requests_per_shard: config .state_sync @@ -1565,13 +1567,17 @@ pub struct StateSyncConfig { pub s3_region: String, /// Whether a node should dump state of each epoch to the external storage. #[serde(skip_serializing_if = "is_false")] - pub dump_enabled: bool, - /// Use carefully in case a node that dumps state to the external storage gets in trouble. + pub dump_enabled: Option, + /// Use carefully in case a node that dumps state to the external storage + /// gets in trouble. #[serde(skip_serializing_if = "Option::is_none")] pub drop_state_of_dump: Option>, - /// If enabled, will download state parts from external storage and not from the peers. + /// If enabled, will download state parts from external storage and not from + /// the peers. #[serde(skip_serializing_if = "is_false")] - pub sync_from_s3_enabled: bool, + pub sync_from_s3_enabled: Option, + /// When syncing state from S3, throttle requests to this many concurrent + /// requests per shard. #[serde(skip_serializing_if = "Option::is_none")] pub num_s3_requests_per_shard: Option, } diff --git a/tools/delay-detector/src/lib.rs b/tools/delay-detector/src/lib.rs index 0ef9452d49f..e33888a15bf 100644 --- a/tools/delay-detector/src/lib.rs +++ b/tools/delay-detector/src/lib.rs @@ -12,8 +12,6 @@ mod disabled { pub fn new<'a>(_msg: impl FnOnce() -> Cow<'a, str>) -> Self { Self {} } - - pub fn snapshot(&mut self, _msg: &str) {} } } From cca8bdc934b13314b8a4e1983450bd4372ccbbcc Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 23 Mar 2023 14:44:52 +0100 Subject: [PATCH 52/88] Rename config field --- chain/client/src/client.rs | 4 ++-- chain/client/src/sync/state.rs | 4 ++-- core/chain-configs/src/client_config.rs | 4 ++-- nearcore/src/config.rs | 6 +++--- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/chain/client/src/client.rs b/chain/client/src/client.rs index 115090d8f6e..995a5542e8f 100644 --- a/chain/client/src/client.rs +++ b/chain/client/src/client.rs @@ -248,7 +248,7 @@ impl Client { config.state_sync_from_s3_enabled, &config.state_sync_s3_bucket, &config.state_sync_s3_region, - config.state_sync_num_s3_requests_per_shard, + config.state_sync_num_concurrent_s3_requests, ); let num_block_producer_seats = config.num_block_producer_seats as usize; let data_parts = runtime_adapter.num_data_parts(); @@ -2127,7 +2127,7 @@ impl Client { self.config.state_sync_from_s3_enabled, &self.config.state_sync_s3_bucket, &self.config.state_sync_s3_region, - self.config.state_sync_num_s3_requests_per_shard, + self.config.state_sync_num_concurrent_s3_requests, ), new_shard_sync, BlocksCatchUpState::new(sync_hash, epoch_id), diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 6748d9ca3af..22e174169ee 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -154,7 +154,7 @@ impl StateSync { state_sync_from_s3_enabled: bool, s3_bucket: &str, s3_region: &str, - num_s3_requests_per_shard: u64, + num_concurrent_s3_requests: u64, ) -> Self { let (mode, parts_request_state) = if state_sync_from_s3_enabled { tracing::debug!(target: "sync", s3_bucket, s3_region, "Initializing S3 bucket connection."); @@ -193,7 +193,7 @@ impl StateSync { timeout, state_parts_apply_results: HashMap::new(), split_state_roots: HashMap::new(), - requests_remaining: Arc::new(AtomicI64::new(num_s3_requests_per_shard as i64)), + requests_remaining: Arc::new(AtomicI64::new(num_concurrent_s3_requests as i64)), } } diff --git a/core/chain-configs/src/client_config.rs b/core/chain-configs/src/client_config.rs index 8e66a3f7075..408e7b31022 100644 --- a/core/chain-configs/src/client_config.rs +++ b/core/chain-configs/src/client_config.rs @@ -179,7 +179,7 @@ pub struct ClientConfig { /// If disabled will perform state sync from the peers. pub state_sync_from_s3_enabled: bool, /// Number of parallel in-flight requests allowed per shard. - pub state_sync_num_s3_requests_per_shard: u64, + pub state_sync_num_concurrent_s3_requests: u64, } impl ClientConfig { @@ -254,7 +254,7 @@ impl ClientConfig { state_sync_s3_region: String::new(), state_sync_restart_dump_for_shards: vec![], state_sync_from_s3_enabled: false, - state_sync_num_s3_requests_per_shard: 10, + state_sync_num_concurrent_s3_requests: 10, } } } diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index 4b7e3efe76a..94da1e52529 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -729,10 +729,10 @@ impl NearConfig { .map(|x| x.sync_from_s3_enabled) .flatten() .unwrap_or(false), - state_sync_num_s3_requests_per_shard: config + state_sync_num_concurrent_s3_requests: config .state_sync .as_ref() - .map(|x| x.num_s3_requests_per_shard) + .map(|x| x.num_concurrent_s3_requests) .flatten() .unwrap_or(100), }, @@ -1579,7 +1579,7 @@ pub struct StateSyncConfig { /// When syncing state from S3, throttle requests to this many concurrent /// requests per shard. #[serde(skip_serializing_if = "Option::is_none")] - pub num_s3_requests_per_shard: Option, + pub num_concurrent_s3_requests: Option, } #[test] From b800bdc736292221b3ee5b6c3a001c4dbd01ac84 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 23 Mar 2023 15:08:05 +0100 Subject: [PATCH 53/88] Fix retry metric --- chain/client/src/sync/state.rs | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 22e174169ee..ad2c092e1b4 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -995,12 +995,16 @@ impl StateSync { let prev = part_download.prev_update_time; let part_timeout = now - prev > self.timeout; // Retry parts that failed. if part_timeout || part_download.error { - metrics::STATE_SYNC_RETRY_PART - .with_label_values(&[&shard_id.to_string()]) - .inc(); download_timeout |= part_timeout; if part_timeout || part_download.last_target != Some(near_client_primitives::types::AccountOrPeerIdOrHash::ExternalStorage) { + // Don't immediately retry failed requests from external + // storage. Most often error is a state part not + // available. That error doesn't get fixed by retrying, + // but rather by waiting. + metrics::STATE_SYNC_RETRY_PART + .with_label_values(&[&shard_id.to_string()]) + .inc(); part_download.run_me.store(true, Ordering::SeqCst); part_download.error = false; part_download.prev_update_time = now; From 1cd3a72ca4dff407cb5d172c9342123aac438be4 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 23 Mar 2023 16:12:01 +0100 Subject: [PATCH 54/88] Minimize --- chain/chain/src/chain.rs | 6 ----- chain/chain/src/metrics.rs | 10 -------- chain/client-primitives/src/types.rs | 2 +- chain/client/src/adapter.rs | 4 ++- chain/client/src/client_actor.rs | 3 --- chain/client/src/metrics.rs | 10 -------- nearcore/src/config.rs | 4 +-- nearcore/src/metrics.rs | 27 +++++++++++++------- nearcore/src/runtime/mod.rs | 8 ++++++ nearcore/src/state_sync.rs | 38 ++++++++-------------------- 10 files changed, 42 insertions(+), 70 deletions(-) diff --git a/chain/chain/src/chain.rs b/chain/chain/src/chain.rs index 5572420ab77..533dfbb4716 100644 --- a/chain/chain/src/chain.rs +++ b/chain/chain/src/chain.rs @@ -1861,7 +1861,6 @@ impl Chain { } } } - tracing::trace!(target: "sync", progress = "loop done", "reset_data_pre_state_sync"); // Clear Chunks data @@ -1870,7 +1869,6 @@ impl Chain { let chunk_height = std::cmp::min(head.height + 2, sync_height); chain_store_update.clear_chunk_data_and_headers(chunk_height)?; chain_store_update.commit()?; - tracing::trace!(target: "sync", progress = "chunks data cleaned up", "reset_data_pre_state_sync"); // clear all trie data @@ -1883,7 +1881,6 @@ impl Chain { // The reason to reset tail here is not to allow Tail be greater than Head chain_store_update.reset_tail(); chain_store_update.commit()?; - tracing::trace!(target: "sync", progress = "state data cleaned up", "reset_data_pre_state_sync"); Ok(()) @@ -3144,9 +3141,6 @@ impl Chain { part_id: PartId, data: &[u8], ) -> Result<(), Error> { - let _timer = metrics::STATE_SYNC_SET_STATE_PART_DELAY - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); let shard_state_header = self.get_state_header(shard_id, sync_hash)?; let chunk = shard_state_header.take_chunk(); let state_root = *chunk.take_header().take_inner().prev_state_root(); diff --git a/chain/chain/src/metrics.rs b/chain/chain/src/metrics.rs index 1309ab3f938..e38c70a68c1 100644 --- a/chain/chain/src/metrics.rs +++ b/chain/chain/src/metrics.rs @@ -110,13 +110,3 @@ pub static STATE_PART_ELAPSED: Lazy = Lazy::new(|| { pub static NUM_INVALID_BLOCKS: Lazy = Lazy::new(|| { try_create_int_gauge("near_num_invalid_blocks", "Number of invalid blocks").unwrap() }); -pub static STATE_SYNC_SET_STATE_PART_DELAY: Lazy = - Lazy::new(|| { - try_create_histogram_vec( - "near_state_sync_set_state_part_delay", - "Delay for saving a state part in the DB", - &["shard_id"], - Some(exponential_buckets(0.001, 2.0, 20).unwrap()), - ) - .unwrap() - }); diff --git a/chain/client-primitives/src/types.rs b/chain/client-primitives/src/types.rs index 3a5e61c33a3..3a104050d0b 100644 --- a/chain/client-primitives/src/types.rs +++ b/chain/client-primitives/src/types.rs @@ -203,7 +203,7 @@ impl ShardSyncDownload { /// Creates a instance of self which includes initial statuses for shard state header download at the given time. pub fn new_download_state_header(now: DateTime) -> Self { Self { - downloads: vec![DownloadStatus::new(now); 1], + downloads: vec![DownloadStatus::new(now)], status: ShardSyncStatus::StateDownloadHeader, } } diff --git a/chain/client/src/adapter.rs b/chain/client/src/adapter.rs index dc0054dcefd..7d01e46922c 100644 --- a/chain/client/src/adapter.rs +++ b/chain/client/src/adapter.rs @@ -194,7 +194,9 @@ impl near_network::client::Client for Adapter { ) -> Result, ReasonForBan> { match self .view_client_addr - .send(StateRequestHeader { shard_id, sync_hash }.with_span_context()) + .send( + StateRequestHeader { shard_id: shard_id, sync_hash: sync_hash }.with_span_context(), + ) .await { Ok(Some(StateResponse(resp))) => Ok(Some(*resp)), diff --git a/chain/client/src/client_actor.rs b/chain/client/src/client_actor.rs index 290c93bcd61..47c1f629d3a 100644 --- a/chain/client/src/client_actor.rs +++ b/chain/client/src/client_actor.rs @@ -1713,9 +1713,6 @@ impl SyncJobsActor { let store = msg.runtime.store(); for part_id in 0..msg.num_parts { - let _timer = metrics::STATE_SYNC_APPLY_PART_DELAY - .with_label_values(&[&msg.shard_id.to_string()]) - .start_timer(); let key = StatePartKey(msg.sync_hash, msg.shard_id, part_id).try_to_vec()?; let part = store.get(DBCol::StateParts, &key)?.unwrap(); diff --git a/chain/client/src/metrics.rs b/chain/client/src/metrics.rs index 0bdaf977ac0..4683ca3ca40 100644 --- a/chain/client/src/metrics.rs +++ b/chain/client/src/metrics.rs @@ -442,16 +442,6 @@ pub static STATE_SYNC_EXTERNAL_PARTS_REQUEST_DELAY: Lazy = Lazy::new(|| { - try_create_histogram_vec( - "near_state_sync_part_apply_delay_sec", - "Latency of applying a state part", - &["shard_id"], - Some(exponential_buckets(0.001, 2.0, 20).unwrap()), - ) - .unwrap() -}); - pub static STATE_SYNC_EXTERNAL_PARTS_SIZE_DOWNLOADED: Lazy = Lazy::new(|| { try_create_int_counter_vec( diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index 94da1e52529..b611f18d893 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -1566,7 +1566,7 @@ pub struct StateSyncConfig { /// Region is very important on S3. pub s3_region: String, /// Whether a node should dump state of each epoch to the external storage. - #[serde(skip_serializing_if = "is_false")] + #[serde(skip_serializing_if = "Option::is_none")] pub dump_enabled: Option, /// Use carefully in case a node that dumps state to the external storage /// gets in trouble. @@ -1574,7 +1574,7 @@ pub struct StateSyncConfig { pub drop_state_of_dump: Option>, /// If enabled, will download state parts from external storage and not from /// the peers. - #[serde(skip_serializing_if = "is_false")] + #[serde(skip_serializing_if = "Option::is_none")] pub sync_from_s3_enabled: Option, /// When syncing state from S3, throttle requests to this many concurrent /// requests per shard. diff --git a/nearcore/src/metrics.rs b/nearcore/src/metrics.rs index 4fd4fa910c2..19605ceeca1 100644 --- a/nearcore/src/metrics.rs +++ b/nearcore/src/metrics.rs @@ -50,15 +50,6 @@ pub(crate) static STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED: Lazy = Lazy: ) .unwrap() }); -pub(crate) static STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED: Lazy = Lazy::new(|| { - try_create_histogram_vec( - "near_state_sync_dump_obtain_part_elapsed_sec", - "Time needed to obtain a part", - &["shard_id"], - Some(exponential_buckets(0.001, 1.6, 25).unwrap()), - ) - .unwrap() -}); pub(crate) static STATE_SYNC_DUMP_NUM_PARTS_TOTAL: Lazy = Lazy::new(|| { try_create_int_gauge_vec( "near_state_sync_dump_num_parts_total", @@ -91,3 +82,21 @@ pub(crate) static STATE_SYNC_DUMP_EPOCH_HEIGHT: Lazy = Lazy::new(|| ) .unwrap() }); +pub static STATE_SYNC_APPLY_PART_DELAY: Lazy = Lazy::new(|| { + try_create_histogram_vec( + "near_state_sync_apply_part_delay_sec", + "Latency of applying a state part", + &["shard_id"], + Some(exponential_buckets(0.001, 2.0, 20).unwrap()), + ) + .unwrap() +}); +pub static STATE_SYNC_OBTAIN_PART_DELAY: Lazy = Lazy::new(|| { + try_create_histogram_vec( + "near_state_sync_obtain_part_delay_sec", + "Latency of applying a state part", + &["shard_id"], + Some(exponential_buckets(0.001, 2.0, 20).unwrap()), + ) + .unwrap() +}); diff --git a/nearcore/src/runtime/mod.rs b/nearcore/src/runtime/mod.rs index 240383939c3..358387ae547 100644 --- a/nearcore/src/runtime/mod.rs +++ b/nearcore/src/runtime/mod.rs @@ -1246,6 +1246,10 @@ impl RuntimeAdapter for NightshadeRuntime { %block_hash, num_parts = part_id.total) .entered(); + let _timer = metrics::STATE_SYNC_OBTAIN_PART_DELAY + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + let epoch_id = self.get_epoch_id(block_hash)?; let shard_uid = self.get_shard_uid_from_epoch_id(shard_id, &epoch_id)?; let trie = self.tries.get_view_trie_for_shard(shard_uid, *state_root); @@ -1377,6 +1381,10 @@ impl RuntimeAdapter for NightshadeRuntime { data: &[u8], epoch_id: &EpochId, ) -> Result<(), Error> { + let _timer = metrics::STATE_SYNC_APPLY_PART_DELAY + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + let part = BorshDeserialize::try_from_slice(data) .expect("Part was already validated earlier, so could never fail here"); let ApplyStatePartResult { trie_changes, flat_state_delta, contract_codes } = diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 357e40d298e..465fb987841 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -3,7 +3,7 @@ use borsh::BorshSerialize; use near_chain::types::RuntimeAdapter; use near_chain::{Chain, ChainGenesis, ChainStoreAccess, DoomslugThresholdMode, Error}; use near_chain_configs::ClientConfig; -use near_client::sync::state::StateSync; +use near_client::sync::state::{s3_location, StateSync}; use near_crypto::PublicKey; use near_epoch_manager::EpochManagerAdapter; use near_primitives::hash::CryptoHash; @@ -175,7 +175,7 @@ async fn state_sync_dump( .with_label_values(&[&shard_id.to_string()]) .start_timer(); - let state_part = match get_state_part( + let state_part = match obtain_and_store_state_part( &runtime, &shard_id, &sync_hash, @@ -328,7 +328,8 @@ fn set_metrics( } } -fn get_state_part( +/// Obtains and then saves the part data. +fn obtain_and_store_state_part( runtime: &Arc, shard_id: &ShardId, sync_hash: &CryptoHash, @@ -337,19 +338,13 @@ fn get_state_part( num_parts: u64, chain: &Chain, ) -> Result, Error> { - let state_part = { - let _timer = metrics::STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - runtime.obtain_state_part( - *shard_id, - &sync_hash, - &state_root, - PartId::new(part_id, num_parts), - )? - }; + let state_part = runtime.obtain_state_part( + *shard_id, + &sync_hash, + &state_root, + PartId::new(part_id, num_parts), + )?; - // Save the part data. let key = StatePartKey(*sync_hash, *shard_id, part_id).try_to_vec()?; let mut store_update = chain.store().store().store_update(); store_update.set(DBCol::StateParts, &key, &state_part); @@ -426,16 +421,3 @@ fn check_new_epoch( } } } - -fn s3_location( - chain_id: &str, - epoch_height: u64, - shard_id: u64, - part_id: u64, - num_parts: u64, -) -> String { - format!( - "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}_of_{:06}", - chain_id, epoch_height, shard_id, part_id, num_parts - ) -} From b319c5144d3dae9f06063fd36cba8ed2c64d3993 Mon Sep 17 00:00:00 2001 From: nikurt <86772482+nikurt@users.noreply.github.com> Date: Tue, 21 Mar 2023 16:51:37 +0100 Subject: [PATCH 55/88] fix: apply `log_summary_style` to state sync phase message (#8735) * Use colour for state sync phase display * fix * fix * debug verbosity * fix --- chain/client/src/sync/state.rs | 584 +++++++-------------------------- 1 file changed, 117 insertions(+), 467 deletions(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index ad2c092e1b4..378be79df37 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -20,7 +20,6 @@ //! here to depend more on local peers instead. //! -use crate::metrics; use ansi_term::Color::{Purple, Yellow}; use ansi_term::Style; use chrono::{DateTime, Duration, Utc}; @@ -42,12 +41,12 @@ use near_primitives::shard_layout::ShardUId; use near_primitives::state_part::PartId; use near_primitives::static_clock::StaticClock; use near_primitives::syncing::{get_num_state_parts, ShardStateSyncResponse}; -use near_primitives::types::{AccountId, EpochHeight, ShardId, StateRoot}; +use near_primitives::types::{AccountId, ShardId, StateRoot}; use rand::seq::SliceRandom; use rand::{thread_rng, Rng}; use std::collections::HashMap; use std::ops::Add; -use std::sync::atomic::{AtomicI64, Ordering}; +use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use std::time::Duration as TimeDuration; @@ -57,7 +56,6 @@ pub const MAX_STATE_PART_REQUEST: u64 = 16; /// This number should not exceed MAX_STATE_PART_REQUEST times (number of peers in the network). pub const MAX_PENDING_PART: u64 = MAX_STATE_PART_REQUEST * 10000; -#[derive(Debug)] pub enum StateSyncResult { /// No shard has changed its status Unchanged, @@ -93,43 +91,18 @@ fn make_account_or_peer_id_or_hash( From::AccountId(a) => To::AccountId(a), From::PeerId(p) => To::PeerId(p), From::Hash(h) => To::Hash(h), - From::ExternalStorage => To::ExternalStorage, } } -/// How to retrieve the state data. -pub enum StateSyncMode { - /// Request both the state header and state parts from the peers. - Peers, - /// Requests the state header from peers but gets the state parts from an - /// external storage. - HeaderFromPeersAndPartsFromExternal { - /// Chain ID. - chain_id: String, - /// Connection to the external storage. - bucket: Arc, - }, -} - -/// Information about which parts were requested from which peer and when. -pub struct PartsRequestState { - last_part_id_requested: HashMap<(AccountOrPeerIdOrHash, ShardId), PendingRequestStatus>, - /// Map from which part we requested to whom. - requested_target: lru::LruCache<(u64, CryptoHash), AccountOrPeerIdOrHash>, -} - /// Helper to track state sync. pub struct StateSync { - /// How to retrieve the state data. - mode: StateSyncMode, - - /// Is used for communication with the peers. network_adapter: PeerManagerAdapter, last_time_block_requested: Option>, - /// Information about which parts were requested from which peer and when. - parts_request_state: Option, + last_part_id_requested: HashMap<(AccountOrPeerIdOrHash, ShardId), PendingRequestStatus>, + /// Map from which part we requested to whom. + requested_target: lru::LruCache<(u64, CryptoHash), AccountOrPeerIdOrHash>, /// Timeout (set in config - by default to 60 seconds) is used to figure out how long we should wait /// for the answer from the other node before giving up. @@ -140,60 +113,18 @@ pub struct StateSync { /// Maps shard_id to result of splitting state for resharding split_state_roots: HashMap, near_chain::Error>>, - - /// The number of requests for state parts from external storage that are - /// allowed to be started for this shard. - requests_remaining: Arc, } impl StateSync { - pub fn new( - network_adapter: PeerManagerAdapter, - timeout: TimeDuration, - chain_id: &str, - state_sync_from_s3_enabled: bool, - s3_bucket: &str, - s3_region: &str, - num_concurrent_s3_requests: u64, - ) -> Self { - let (mode, parts_request_state) = if state_sync_from_s3_enabled { - tracing::debug!(target: "sync", s3_bucket, s3_region, "Initializing S3 bucket connection."); - assert!(!s3_bucket.is_empty() && !s3_region.is_empty(), "State sync from S3 is enabled. This requires that both `s3_bucket and `s3_region` and specified and non-empty"); - let mut bucket = s3::Bucket::new( - s3_bucket, - s3_region.parse::().unwrap(), - s3::creds::Credentials::default().unwrap(), - ) - .unwrap(); - // Ensure requests finish in finite amount of time. - bucket.set_request_timeout(Some(timeout)); - let bucket = Arc::new(bucket); - ( - StateSyncMode::HeaderFromPeersAndPartsFromExternal { - chain_id: chain_id.to_string(), - bucket, - }, - None, - ) - } else { - ( - StateSyncMode::Peers, - Some(PartsRequestState { - last_part_id_requested: Default::default(), - requested_target: lru::LruCache::new(MAX_PENDING_PART as usize), - }), - ) - }; - let timeout = Duration::from_std(timeout).unwrap(); + pub fn new(network_adapter: PeerManagerAdapter, timeout: TimeDuration) -> Self { StateSync { - mode, network_adapter, last_time_block_requested: None, - parts_request_state, - timeout, + last_part_id_requested: Default::default(), + requested_target: lru::LruCache::new(MAX_PENDING_PART as usize), + timeout: Duration::from_std(timeout).unwrap(), state_parts_apply_results: HashMap::new(), split_state_roots: HashMap::new(), - requests_remaining: Arc::new(AtomicI64::new(num_concurrent_s3_requests as i64)), } } @@ -267,14 +198,11 @@ impl StateSync { let shard_sync_download = new_shard_sync.entry(shard_id).or_insert_with(|| { run_shard_state_download = true; update_sync_status = true; - ShardSyncDownload::new_download_state_header(now) + ShardSyncDownload::new(now) }); let old_status = shard_sync_download.status.clone(); let mut shard_sync_done = false; - metrics::STATE_SYNC_STAGE - .with_label_values(&[&shard_id.to_string()]) - .set(shard_sync_download.status.repr() as i64); match &shard_sync_download.status { ShardSyncStatus::StateDownloadHeader => { (download_timeout, run_shard_state_download) = self @@ -287,16 +215,8 @@ impl StateSync { )?; } ShardSyncStatus::StateDownloadParts => { - let res = self.sync_shards_download_parts_status( - shard_id, - shard_sync_download, - sync_hash, - chain, - now, - ); - download_timeout = res.0; - run_shard_state_download = res.1; - update_sync_status |= res.2; + (download_timeout, run_shard_state_download) = + self.sync_shards_download_parts_status(shard_sync_download, now); } ShardSyncStatus::StateDownloadScheduling => { self.sync_shards_download_scheduling_status( @@ -370,13 +290,13 @@ impl StateSync { // Execute syncing for shard `shard_id` if run_shard_state_download { update_sync_status = true; - self.request_shard( + *shard_sync_download = self.request_shard( me, shard_id, chain, runtime_adapter, sync_hash, - shard_sync_download, + shard_sync_download.clone(), highest_height_peers, )?; } @@ -435,15 +355,15 @@ impl StateSync { sync_hash: CryptoHash, ) { // FIXME: something is wrong - the index should have a shard_id too. - let parts_request_state = self.parts_request_state.as_mut().unwrap(); - parts_request_state.requested_target.put((part_id, sync_hash), target.clone()); - parts_request_state - .last_part_id_requested + self.requested_target.put((part_id, sync_hash), target.clone()); + + let timeout = self.timeout; + self.last_part_id_requested .entry((target, shard_id)) .and_modify(|pending_request| { pending_request.missing_parts += 1; }) - .or_insert_with(|| PendingRequestStatus::new(self.timeout)); + .or_insert_with(|| PendingRequestStatus::new(timeout)); } // Function called when our node receives the network response with a part. @@ -455,17 +375,15 @@ impl StateSync { ) { let key = (part_id, sync_hash); // Check that it came from the target that we requested it from. - let parts_request_state = self.parts_request_state.as_mut().unwrap(); - if let Some(target) = parts_request_state.requested_target.get(&key) { - if parts_request_state - .last_part_id_requested - .get_mut(&(target.clone(), shard_id)) - .map_or(false, |request| { + if let Some(target) = self.requested_target.get(&key) { + if self.last_part_id_requested.get_mut(&(target.clone(), shard_id)).map_or( + false, + |request| { request.missing_parts = request.missing_parts.saturating_sub(1); request.missing_parts == 0 - }) - { - parts_request_state.last_part_id_requested.remove(&(target.clone(), shard_id)); + }, + ) { + self.last_part_id_requested.remove(&(target.clone(), shard_id)); } } } @@ -483,16 +401,13 @@ impl StateSync { highest_height_peers: &[HighestHeightPeerInfo], ) -> Result, near_chain::Error> { // Remove candidates from pending list if request expired due to timeout - self.parts_request_state.as_mut().map(|parts_request_state| { - parts_request_state.last_part_id_requested.retain(|_, request| !request.expired()) - }); + self.last_part_id_requested.retain(|_, request| !request.expired()); let prev_block_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); let epoch_hash = runtime_adapter.get_epoch_id_from_prev_block(&prev_block_hash)?; - let block_producers = - runtime_adapter.get_epoch_block_producers_ordered(&epoch_hash, &sync_hash)?; - let peers = block_producers + Ok(runtime_adapter + .get_epoch_block_producers_ordered(&epoch_hash, &sync_hash)? .iter() .filter_map(|(validator_stake, _slashed)| { let account_id = validator_stake.account_id(); @@ -522,20 +437,12 @@ impl StateSync { } else { None } - })); - let result = if let Some(parts_request_state) = &self.parts_request_state { - peers - .filter(|candidate| { - // If we still have a pending request from this node - don't add another one. - !parts_request_state - .last_part_id_requested - .contains_key(&(candidate.clone(), shard_id)) - }) - .collect::>() - } else { - peers.collect::>() - }; - Ok(result) + })) + .filter(|candidate| { + // If we still have a pending request from this node - don't add another one. + !self.last_part_id_requested.contains_key(&(candidate.clone(), shard_id)) + }) + .collect::>()) } /// Returns new ShardSyncDownload if successful, otherwise returns given shard_sync_download @@ -546,9 +453,9 @@ impl StateSync { chain: &Chain, runtime_adapter: &Arc, sync_hash: CryptoHash, - shard_sync_download: &mut ShardSyncDownload, + shard_sync_download: ShardSyncDownload, highest_height_peers: &[HighestHeightPeerInfo], - ) -> Result<(), near_chain::Error> { + ) -> Result { let possible_targets = self.possible_targets( me, shard_id, @@ -560,17 +467,19 @@ impl StateSync { if possible_targets.is_empty() { // In most cases it means that all the targets are currently busy (that we have a pending request with them). - return Ok(()); + return Ok(shard_sync_download); } // Downloading strategy starts here + let mut new_shard_sync_download = shard_sync_download.clone(); + match shard_sync_download.status { ShardSyncStatus::StateDownloadHeader => { self.request_shard_header( shard_id, sync_hash, &possible_targets, - shard_sync_download, + &mut new_shard_sync_download, ); } ShardSyncStatus::StateDownloadParts => { @@ -578,17 +487,15 @@ impl StateSync { shard_id, sync_hash, possible_targets, - shard_sync_download, - chain, + &mut new_shard_sync_download, ); } _ => {} } - Ok(()) + Ok(new_shard_sync_download) } - /// Makes a StateRequestHeader header to one of the peers. fn request_shard_header( &mut self, shard_id: ShardId, @@ -621,167 +528,62 @@ impl StateSync { ); } - /// Makes requests to download state parts for the given epoch of the given shard. fn request_shard_parts( &mut self, shard_id: ShardId, sync_hash: CryptoHash, possible_targets: Vec, new_shard_sync_download: &mut ShardSyncDownload, - chain: &Chain, ) { // We'll select all the 'highest' peers + validators as candidates (excluding those that gave us timeout in the past). // And for each one of them, we'll ask for up to 16 (MAX_STATE_PART_REQUEST) parts. - let mut possible_targets_sampler = + let possible_targets_sampler = SamplerLimited::new(possible_targets, MAX_STATE_PART_REQUEST); - let sync_block_header = chain.get_block_header(&sync_hash).unwrap(); - let epoch_id = sync_block_header.epoch_id(); - let epoch_info = chain.runtime_adapter.get_epoch_info(epoch_id).unwrap(); - let epoch_height = epoch_info.epoch_height(); - - let shard_state_header = chain.get_state_header(shard_id, sync_hash).unwrap(); - let state_num_parts = - get_num_state_parts(shard_state_header.state_root_node().memory_usage); - // Iterate over all parts that needs to be requested (i.e. download.run_me is true). // Parts are ordered such that its index match its part_id. - for (part_id, download) in new_shard_sync_download + // Finally, for every part that needs to be requested it is selected one peer (target) randomly + // to request the part from. + // IMPORTANT: here we use 'zip' with possible_target_sampler - which is limited. So at any moment we'll not request more than + // possible_targets.len() * MAX_STATE_PART_REQUEST parts. + for ((part_id, download), target) in new_shard_sync_download .downloads .iter_mut() .enumerate() .filter(|(_, download)| download.run_me.load(Ordering::SeqCst)) + .zip(possible_targets_sampler) { - match &self.mode { - StateSyncMode::Peers => { - // For every part that needs to be requested it is selected one - // peer (target) randomly to request the part from. - // IMPORTANT: here we use 'zip' with possible_target_sampler - - // which is limited. So at any moment we'll not request more - // than possible_targets.len() * MAX_STATE_PART_REQUEST parts. - let target = possible_targets_sampler.next().unwrap(); - self.request_part_from_peers( - part_id as u64, - target, - download, - shard_id, - sync_hash, - ); - } - StateSyncMode::HeaderFromPeersAndPartsFromExternal { chain_id, bucket } => { - self.request_part_from_external_storage( - part_id as u64, - download, - shard_id, - epoch_height, - state_num_parts, - &chain_id, - bucket.clone(), - ); - } - } - } - } - - /// Starts an asynchronous network request to external storage to obtain the given state part. - fn request_part_from_external_storage( - &self, - part_id: u64, - download: &mut DownloadStatus, - shard_id: ShardId, - epoch_height: EpochHeight, - num_parts: u64, - chain_id: &str, - bucket: Arc, - ) { - let requests_remaining = self.requests_remaining.clone(); - if !allow_request(&requests_remaining) { - return; - } else { - if !download.run_me.swap(false, Ordering::SeqCst) { - tracing::info!(target: "sync", %shard_id, part_id, "External storage request is allowed but run_me is already false. Undoing"); - finished_request(&requests_remaining); - return; - } + self.sent_request_part(target.clone(), part_id as u64, shard_id, sync_hash); + download.run_me.store(false, Ordering::SeqCst); + download.state_requests_count += 1; + download.last_target = Some(make_account_or_peer_id_or_hash(target.clone())); + let run_me = download.run_me.clone(); + + near_performance_metrics::actix::spawn( + std::any::type_name::(), + self.network_adapter + .send_async(PeerManagerMessageRequest::NetworkRequests( + NetworkRequests::StateRequestPart { + shard_id, + sync_hash, + part_id: part_id as u64, + target: target.clone(), + }, + )) + .then(move |result| { + // TODO: possible optimization - in the current code, even if one of the targets it not present in the network graph + // (so we keep getting RouteNotFound) - we'll still keep trying to assign parts to it. + // Fortunately only once every 60 seconds (timeout value). + if let Ok(NetworkResponses::RouteNotFound) = + result.map(|f| f.as_network_response()) + { + // Send a StateRequestPart on the next iteration + run_me.store(true, Ordering::SeqCst); + } + future::ready(()) + }), + ); } - download.state_requests_count += 1; - download.last_target = - Some(make_account_or_peer_id_or_hash(AccountOrPeerIdOrHash::ExternalStorage)); - - let location = s3_location(chain_id, epoch_height, shard_id, part_id, num_parts); - let download_response = download.response.clone(); - let scheduled = StaticClock::utc(); - near_performance_metrics::actix::spawn(std::any::type_name::(), { - async move { - tracing::debug!(target: "sync", %shard_id, part_id, location, "Getting an object from the external storage"); - let started = StaticClock::utc(); - metrics::STATE_SYNC_EXTERNAL_PARTS_SCHEDULING_DELAY - .with_label_values(&[&shard_id.to_string()]) - .observe( - started.signed_duration_since(scheduled).num_nanoseconds().unwrap_or(0) - as f64 - / 1e9, - ); - let result = bucket.get_object(location.clone()).await; - let completed = StaticClock::utc(); - finished_request(&requests_remaining); - metrics::STATE_SYNC_EXTERNAL_PARTS_REQUEST_DELAY - .with_label_values(&[&shard_id.to_string()]) - .observe( - completed.signed_duration_since(started).num_nanoseconds().unwrap_or(0) - as f64 - / 1e9, - ); - match result { - Ok(response) => { - tracing::debug!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), "S3 request finished"); - let mut lock = download_response.lock().unwrap(); - *lock = Some(Ok((response.status_code(), response.bytes().to_vec()))); - } - Err(err) => { - tracing::debug!(target: "sync", %shard_id, part_id, location, ?err, "S3 request failed"); - let mut lock = download_response.lock().unwrap(); - *lock = Some(Err(err.to_string())); - } - } - } - }); - } - - /// Asynchronously requests a state part from a suitable peer. - fn request_part_from_peers( - &mut self, - part_id: u64, - target: AccountOrPeerIdOrHash, - download: &mut DownloadStatus, - shard_id: ShardId, - sync_hash: CryptoHash, - ) { - self.sent_request_part(target.clone(), part_id, shard_id, sync_hash); - download.run_me.store(false, Ordering::SeqCst); - download.state_requests_count += 1; - download.last_target = Some(make_account_or_peer_id_or_hash(target.clone())); - let run_me = download.run_me.clone(); - - near_performance_metrics::actix::spawn( - std::any::type_name::(), - self.network_adapter - .send_async(PeerManagerMessageRequest::NetworkRequests( - NetworkRequests::StateRequestPart { shard_id, sync_hash, part_id, target }, - )) - .then(move |result| { - // TODO: possible optimization - in the current code, even if one of the targets it not present in the network graph - // (so we keep getting RouteNotFound) - we'll still keep trying to assign parts to it. - // Fortunately only once every 60 seconds (timeout value). - if let Ok(NetworkResponses::RouteNotFound) = - result.map(|f| f.as_network_response()) - { - // Send a StateRequestPart on the next iteration - run_me.store(true, Ordering::SeqCst); - } - future::ready(()) - }), - ); } /// The main 'step' function that should be called periodically to check and update the sync process. @@ -803,7 +605,7 @@ impl StateSync { use_colour: bool, ) -> Result { let _span = tracing::debug_span!(target: "sync", "run", sync = "StateSync").entered(); - tracing::trace!(target: "sync", %sync_hash, ?tracking_shards, "syncing state"); + tracing::debug!(target: "sync", %sync_hash, ?tracking_shards, "syncing state"); let prev_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); let now = StaticClock::utc(); @@ -936,8 +738,21 @@ impl StateSync { get_num_state_parts(shard_state_header.state_root_node().memory_usage); // If the header was downloaded successfully - move to phase 2 (downloading parts). // Create the vector with entry for each part. - *shard_sync_download = - ShardSyncDownload::new_download_state_parts(now, state_num_parts); + *shard_sync_download = ShardSyncDownload { + downloads: vec![ + DownloadStatus { + start_time: now, + prev_update_time: now, + run_me: Arc::new(AtomicBool::new(true)), + error: false, + done: false, + state_requests_count: 0, + last_target: None, + }; + state_num_parts as usize + ], + status: ShardSyncStatus::StateDownloadParts, + }; run_shard_state_download = true; } else { let prev = shard_sync_download.downloads[0].prev_update_time; @@ -958,82 +773,45 @@ impl StateSync { /// Checks if the parts are downloaded. /// If download of all parts is complete, then moves forward to `StateDownloadScheduling`. - /// Returns `(download_timeout, run_shard_state_download, update_sync_status)` where: + /// Returns `(download_timeout, run_shard_state_download)` where: /// * `download_timeout` means that the state header request timed out (and needs to be retried). /// * `run_shard_state_download` means that header or part download requests need to run for this shard. - /// * `update_sync_status` means that something changed in `ShardSyncDownload` and it needs to be persisted. fn sync_shards_download_parts_status( &mut self, - shard_id: ShardId, shard_sync_download: &mut ShardSyncDownload, - sync_hash: CryptoHash, - chain: &mut Chain, now: DateTime, - ) -> (bool, bool, bool) { + ) -> (bool, bool) { // Step 2 - download all the parts (each part is usually around 1MB). let mut download_timeout = false; let mut run_shard_state_download = false; - let mut update_sync_status = false; let mut parts_done = true; - let num_parts = shard_sync_download.downloads.len(); - let mut num_parts_done = 0; - for (part_id, part_download) in shard_sync_download.downloads.iter_mut().enumerate() { - if !part_download.done { - // Check if a download from an external storage is finished. - update_sync_status |= check_external_storage_part_response( - part_id as u64, - num_parts as u64, - shard_id, - sync_hash, - part_download, - chain, - ); - } + for part_download in shard_sync_download.downloads.iter_mut() { if !part_download.done { parts_done = false; let prev = part_download.prev_update_time; - let part_timeout = now - prev > self.timeout; // Retry parts that failed. - if part_timeout || part_download.error { + let error = part_download.error; + let part_timeout = now - prev > self.timeout; + // Retry parts that failed. + if part_timeout || error { download_timeout |= part_timeout; - if part_timeout || - part_download.last_target != Some(near_client_primitives::types::AccountOrPeerIdOrHash::ExternalStorage) { - // Don't immediately retry failed requests from external - // storage. Most often error is a state part not - // available. That error doesn't get fixed by retrying, - // but rather by waiting. - metrics::STATE_SYNC_RETRY_PART - .with_label_values(&[&shard_id.to_string()]) - .inc(); - part_download.run_me.store(true, Ordering::SeqCst); - part_download.error = false; - part_download.prev_update_time = now; - update_sync_status = true; - } + part_download.run_me.store(true, Ordering::SeqCst); + part_download.error = false; + part_download.prev_update_time = now; } if part_download.run_me.load(Ordering::SeqCst) { run_shard_state_download = true; } } - if part_download.done { - num_parts_done += 1; - } } - metrics::STATE_SYNC_PARTS_DONE - .with_label_values(&[&shard_id.to_string()]) - .set(num_parts_done); - metrics::STATE_SYNC_PARTS_TOTAL - .with_label_values(&[&shard_id.to_string()]) - .set(num_parts as i64); // If all parts are done - we can move towards scheduling. if parts_done { *shard_sync_download = ShardSyncDownload { downloads: vec![], status: ShardSyncStatus::StateDownloadScheduling, }; - update_sync_status = true; } - (download_timeout, run_shard_state_download, update_sync_status) + (download_timeout, run_shard_state_download) } fn sync_shards_download_scheduling_status( @@ -1066,9 +844,8 @@ impl StateSync { Err(err) => { // Cannot finalize the downloaded state. // The reasonable behavior here is to start from the very beginning. - metrics::STATE_SYNC_DISCARD_PARTS.with_label_values(&[&shard_id.to_string()]).inc(); tracing::error!(target: "sync", %shard_id, %sync_hash, ?err, "State sync finalizing error"); - *shard_sync_download = ShardSyncDownload::new_download_state_header(now); + *shard_sync_download = ShardSyncDownload::new(now); chain.clear_downloaded_parts(shard_id, sync_hash, state_num_parts)?; } } @@ -1097,11 +874,8 @@ impl StateSync { Err(err) => { // Cannot finalize the downloaded state. // The reasonable behavior here is to start from the very beginning. - metrics::STATE_SYNC_DISCARD_PARTS - .with_label_values(&[&shard_id.to_string()]) - .inc(); tracing::error!(target: "sync", %shard_id, %sync_hash, ?err, "State sync finalizing error"); - *shard_sync_download = ShardSyncDownload::new_download_state_header(now); + *shard_sync_download = ShardSyncDownload::new(now); let shard_state_header = chain.get_state_header(shard_id, sync_hash)?; let state_num_parts = get_num_state_parts(shard_state_header.state_root_node().memory_usage); @@ -1185,118 +959,9 @@ impl StateSync { } } -/// Verifies that one more concurrent request can be started. -fn allow_request(requests_remaining: &AtomicI64) -> bool { - let remaining = requests_remaining.fetch_sub(1, Ordering::SeqCst); - if remaining >= 0 { - true - } else { - requests_remaining.fetch_add(1, Ordering::SeqCst); - false - } -} - -/// Notify about a request finishing. -fn finished_request(requests_remaining: &AtomicI64) { - requests_remaining.fetch_add(1, Ordering::SeqCst); -} - -/// Works around how data requests to external storage are done. -/// The response is stored on the DownloadStatus object. -/// This function investigates if the response is available and updates `done` and `error` appropriately. -/// If the response is successful, then also writes the state part to the DB. -/// -/// Returns whether something changed in `DownloadStatus` which means it needs to be persisted. -fn check_external_storage_part_response( - part_id: u64, - num_parts: u64, - shard_id: ShardId, - sync_hash: CryptoHash, - part_download: &mut DownloadStatus, - chain: &mut Chain, -) -> bool { - let external_storage_response = { - let mut lock = part_download.response.lock().unwrap(); - if let Some(response) = lock.clone() { - tracing::debug!(target: "sync", %shard_id, part_id, "Got response from external storage"); - // Remove the response from DownloadStatus, because - // we're going to write state parts to DB and don't need to keep - // them in `DownloadStatus`. - *lock = None; - response - } else { - return false; - } - }; - - let mut err_to_retry = None; - match external_storage_response { - // HTTP status code 200 means success. - Ok((200, data)) => { - tracing::debug!(target: "sync", %shard_id, part_id, "Got 200 response from external storage"); - match chain.set_state_part( - shard_id, - sync_hash, - PartId::new(part_id as u64, num_parts as u64), - &data, - ) { - Ok(_) => { - metrics::STATE_SYNC_EXTERNAL_PARTS_DONE - .with_label_values(&[&shard_id.to_string()]) - .inc(); - metrics::STATE_SYNC_EXTERNAL_PARTS_SIZE_DOWNLOADED - .with_label_values(&[&shard_id.to_string()]) - .inc_by(data.len() as u64); - part_download.done = true; - tracing::debug!(target: "sync", %shard_id, part_id, ?part_download, "Set state part success"); - } - Err(err) => { - metrics::STATE_SYNC_EXTERNAL_PARTS_FAILED - .with_label_values(&[&shard_id.to_string()]) - .inc(); - tracing::warn!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to save a state part"); - err_to_retry = - Some(near_chain::Error::Other("Failed to save a state part".to_string())); - } - } - } - // Other HTTP status codes are considered errors. - Ok((status_code, _)) => { - tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, status_code, "Wrong response code, expected 200"); - err_to_retry = - Some(near_chain::Error::Other(format!("status_code: {}", status_code).to_string())); - } - // The request failed without reaching the external storage. - Err(err) => { - err_to_retry = Some(near_chain::Error::Other(err.to_string())); - } - }; - - if let Some(err) = err_to_retry { - tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to get a part from external storage, will retry"); - part_download.error = true; - } - true -} - -/// Construct a location on the external storage. -pub fn s3_location( - chain_id: &str, - epoch_height: u64, - shard_id: u64, - part_id: u64, - num_parts: u64, -) -> String { - format!( - "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}_of_{:06}", - chain_id, epoch_height, shard_id, part_id, num_parts - ) -} - -/// Applies style if `use_colour` is enabled. -fn paint(s: &str, style: Style, use_style: bool) -> String { - if use_style { - style.paint(s).to_string() +fn paint(s: &str, colour: Style, use_colour: bool) -> String { + if use_colour { + colour.paint(s).to_string() } else { s.to_string() } @@ -1312,15 +977,8 @@ fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload, use_colour: shard_sync_download.downloads[0].last_target ), ShardSyncStatus::StateDownloadParts => { - let mut num_parts_done = 0; - let mut num_parts_not_done = 0; let mut text = "".to_string(); for (i, download) in shard_sync_download.downloads.iter().enumerate() { - if download.done { - num_parts_done += 1; - continue; - } - num_parts_not_done += 1; text.push_str(&format!( "[{}: {}, {}, {:?}] ", paint(&i.to_string(), Yellow.bold(), use_colour), @@ -1330,12 +988,10 @@ fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload, use_colour: )); } format!( - "{} [{}: is_done, requests sent, last target] {} num_parts_done={} num_parts_not_done={}", + "{} [{}: is_done, requests sent, last target] {}", paint("PARTS", Purple.bold(), use_colour), paint("part_id", Yellow.bold(), use_colour), - text, - num_parts_done, - num_parts_not_done + text ) } _ => unreachable!("timeout cannot happen when all state is downloaded"), @@ -1427,15 +1083,8 @@ mod test { // Start a new state sync - and check that it asks for a header. fn test_ask_for_header() { let mock_peer_manager = Arc::new(MockPeerManagerAdapter::default()); - let mut state_sync = StateSync::new( - mock_peer_manager.clone().into(), - TimeDuration::from_secs(1), - "chain_id".to_string(), - false, - "".to_string(), - "".to_string(), - 100, - ); + let mut state_sync = + StateSync::new(mock_peer_manager.clone().into(), TimeDuration::from_secs(1)); let mut new_shard_sync = HashMap::new(); let (mut chain, kv, signer) = test_utils::setup(); @@ -1485,6 +1134,7 @@ mod test { vec![0], &apply_parts_fn, &state_split_fn, + false, ) .unwrap(); From 6b09e0d47bfc0d422452979e0922119d85e005fd Mon Sep 17 00:00:00 2001 From: robin-near <111538878+robin-near@users.noreply.github.com> Date: Wed, 22 Mar 2023 10:02:00 -0700 Subject: [PATCH 56/88] [Runtime Epoch Split] (3/n) Add ability to get Arc out of an &RuntimeWithEpochManagerAdapter. (#8768) --- chain/chain/src/test_utils/kv_runtime.rs | 79 +++----- chain/chain/src/types.rs | 2 - chain/client/src/info.rs | 3 +- nearcore/src/lib.rs | 5 +- nearcore/src/runtime/mod.rs | 9 +- tools/state-viewer/src/state_parts.rs | 238 ++++++++++------------- 6 files changed, 127 insertions(+), 209 deletions(-) diff --git a/chain/chain/src/test_utils/kv_runtime.rs b/chain/chain/src/test_utils/kv_runtime.rs index 7af4586d8b2..5952346a471 100644 --- a/chain/chain/src/test_utils/kv_runtime.rs +++ b/chain/chain/src/test_utils/kv_runtime.rs @@ -4,7 +4,6 @@ use std::sync::{Arc, RwLock, Weak}; use borsh::{BorshDeserialize, BorshSerialize}; -use near_epoch_manager::shard_tracker::{ShardTracker, TrackedConfig}; use near_epoch_manager::{EpochManagerAdapter, RngSeed}; use near_primitives::sandbox::state_patch::SandboxStatePatch; use near_primitives::state_part::PartId; @@ -845,45 +844,6 @@ impl EpochManagerAdapter for KeyValueRuntime { Ok(()) } } - - fn cares_about_shard_from_prev_block( - &self, - parent_hash: &CryptoHash, - account_id: &AccountId, - shard_id: ShardId, - ) -> Result { - // This `unwrap` here tests that in all code paths we check that the epoch exists before - // we check if we care about a shard. Please do not remove the unwrap, fix the logic of - // the calling function. - let epoch_valset = self.get_epoch_and_valset(*parent_hash).unwrap(); - let chunk_producers = self.get_chunk_producers(epoch_valset.1, shard_id); - for validator in chunk_producers { - if validator.account_id() == account_id { - return Ok(true); - } - } - Ok(false) - } - - fn cares_about_shard_next_epoch_from_prev_block( - &self, - parent_hash: &CryptoHash, - account_id: &AccountId, - shard_id: ShardId, - ) -> Result { - // This `unwrap` here tests that in all code paths we check that the epoch exists before - // we check if we care about a shard. Please do not remove the unwrap, fix the logic of - // the calling function. - let epoch_valset = self.get_epoch_and_valset(*parent_hash).unwrap(); - let chunk_producers = self - .get_chunk_producers((epoch_valset.1 + 1) % self.validators_by_valset.len(), shard_id); - for validator in chunk_producers { - if validator.account_id() == account_id { - return Ok(true); - } - } - Ok(false) - } } impl RuntimeAdapter for KeyValueRuntime { @@ -962,12 +922,19 @@ impl RuntimeAdapter for KeyValueRuntime { if self.tracks_all_shards { return true; } + // This `unwrap` here tests that in all code paths we check that the epoch exists before + // we check if we care about a shard. Please do not remove the unwrap, fix the logic of + // the calling function. + let epoch_valset = self.get_epoch_and_valset(*parent_hash).unwrap(); + let chunk_producers = self.get_chunk_producers(epoch_valset.1, shard_id); if let Some(account_id) = account_id { - self.cares_about_shard_from_prev_block(parent_hash, account_id, shard_id) - .unwrap_or(false) - } else { - false + for validator in chunk_producers { + if validator.account_id() == account_id { + return true; + } + } } + false } fn will_care_about_shard( @@ -980,12 +947,20 @@ impl RuntimeAdapter for KeyValueRuntime { if self.tracks_all_shards { return true; } + // This `unwrap` here tests that in all code paths we check that the epoch exists before + // we check if we care about a shard. Please do not remove the unwrap, fix the logic of + // the calling function. + let epoch_valset = self.get_epoch_and_valset(*parent_hash).unwrap(); + let chunk_producers = self + .get_chunk_producers((epoch_valset.1 + 1) % self.validators_by_valset.len(), shard_id); if let Some(account_id) = account_id { - self.cares_about_shard_next_epoch_from_prev_block(parent_hash, account_id, shard_id) - .unwrap_or(false) - } else { - false + for validator in chunk_producers { + if validator.account_id() == account_id { + return true; + } + } } + false } fn validate_tx( @@ -1436,12 +1411,4 @@ impl RuntimeWithEpochManagerAdapter for KeyValueRuntime { fn epoch_manager_adapter_arc(&self) -> Arc { self.myself.upgrade().unwrap() } - fn shard_tracker(&self) -> ShardTracker { - let config = if self.tracks_all_shards { - TrackedConfig::AllShards - } else { - TrackedConfig::new_empty() - }; - ShardTracker::new(config, self.epoch_manager_adapter_arc()) - } } diff --git a/chain/chain/src/types.rs b/chain/chain/src/types.rs index a88be70c394..bfa69391e8d 100644 --- a/chain/chain/src/types.rs +++ b/chain/chain/src/types.rs @@ -4,7 +4,6 @@ use std::sync::Arc; use borsh::{BorshDeserialize, BorshSerialize}; use chrono::DateTime; use chrono::Utc; -use near_epoch_manager::shard_tracker::ShardTracker; use near_primitives::sandbox::state_patch::SandboxStatePatch; use num_rational::Rational32; @@ -574,7 +573,6 @@ pub trait RuntimeAdapter: Send + Sync { pub trait RuntimeWithEpochManagerAdapter: RuntimeAdapter + EpochManagerAdapter { fn epoch_manager_adapter(&self) -> &dyn EpochManagerAdapter; fn epoch_manager_adapter_arc(&self) -> Arc; - fn shard_tracker(&self) -> ShardTracker; } /// The last known / checked height and time when we have processed it. diff --git a/chain/client/src/info.rs b/chain/client/src/info.rs index 8b3e8965354..cd14dedce87 100644 --- a/chain/client/src/info.rs +++ b/chain/client/src/info.rs @@ -739,8 +739,7 @@ mod tests { let store = near_store::test_utils::create_test_store(); let vs = ValidatorSchedule::new().block_producers_per_epoch(vec![vec!["test".parse().unwrap()]]); - let runtime = - Arc::new(KeyValueRuntime::new_with_validators_and_no_gc(store, vs, 123, false)); + let runtime = KeyValueRuntime::new_with_validators_and_no_gc(store, vs, 123, false); let chain_genesis = ChainGenesis { time: StaticClock::utc(), height: 0, diff --git a/nearcore/src/lib.rs b/nearcore/src/lib.rs index 3da5e6a2766..000ccf43d19 100644 --- a/nearcore/src/lib.rs +++ b/nearcore/src/lib.rs @@ -208,14 +208,13 @@ pub fn start_with_config_and_synchronization( ) -> anyhow::Result { let store = open_storage(home_dir, &mut config)?; - let runtime = - Arc::new(NightshadeRuntime::from_config(home_dir, store.get_hot_store(), &config)); + let runtime = NightshadeRuntime::from_config(home_dir, store.get_hot_store(), &config); // Get the split store. If split store is some then create a new runtime for // the view client. Otherwise just re-use the existing runtime. let split_store = get_split_store(&config, &store)?; let view_runtime = if let Some(split_store) = split_store { - Arc::new(NightshadeRuntime::from_config(home_dir, split_store, &config)) + NightshadeRuntime::from_config(home_dir, split_store, &config) } else { runtime.clone() }; diff --git a/nearcore/src/runtime/mod.rs b/nearcore/src/runtime/mod.rs index 358387ae547..3233443f589 100644 --- a/nearcore/src/runtime/mod.rs +++ b/nearcore/src/runtime/mod.rs @@ -1,5 +1,6 @@ use crate::metrics; use crate::migrations::load_migration_data; +use crate::shard_tracker::{ShardTracker, TrackedConfig}; use crate::NearConfig; use borsh::ser::BorshSerialize; use borsh::BorshDeserialize; @@ -14,7 +15,6 @@ use near_chain_configs::{ }; use near_client_primitives::types::StateSplitApplyingStatus; use near_crypto::PublicKey; -use near_epoch_manager::shard_tracker::{ShardTracker, TrackedConfig}; use near_epoch_manager::{EpochManager, EpochManagerAdapter, EpochManagerHandle}; use near_o11y::log_assert; use near_pool::types::PoolIterator; @@ -153,7 +153,7 @@ impl NightshadeRuntime { EpochManager::new_from_genesis_config(store.clone().into(), &genesis_config) .expect("Failed to start Epoch Manager") .into_handle(); - let shard_tracker = ShardTracker::new(tracked_config, Arc::new(epoch_manager.clone())); + let shard_tracker = ShardTracker::new(tracked_config, epoch_manager.clone()); Arc::new_cyclic(|myself| NightshadeRuntime { genesis_config, runtime_config_store, @@ -1465,10 +1465,6 @@ impl RuntimeWithEpochManagerAdapter for NightshadeRuntime { fn epoch_manager_adapter_arc(&self) -> Arc { self.myself.upgrade().unwrap() } - - fn shard_tracker(&self) -> ShardTracker { - self.shard_tracker.clone() - } } impl node_runtime::adapter::ViewRuntimeAdapter for NightshadeRuntime { @@ -1571,7 +1567,6 @@ mod test { use std::collections::BTreeSet; use near_chain::{Chain, ChainGenesis}; - use near_epoch_manager::shard_tracker::TrackedConfig; use near_primitives::test_utils::create_test_signer; use near_primitives::types::validator_stake::ValidatorStake; use near_store::flat::{FlatStateChanges, FlatStateDelta, FlatStateDeltaMetadata}; diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index 5e395e8ae58..27e117c1332 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -1,6 +1,7 @@ use crate::epoch_info::iterate_and_filter; -use near_chain::{Chain, ChainGenesis, ChainStoreAccess, DoomslugThresholdMode}; -use near_client::sync::state::StateSync; +use near_chain::types::RuntimeAdapter; +use near_chain::{ChainStore, ChainStoreAccess}; +use near_epoch_manager::EpochManager; use near_primitives::epoch_manager::epoch_info::EpochInfo; use near_primitives::state_part::PartId; use near_primitives::syncing::get_num_state_parts; @@ -48,12 +49,6 @@ pub(crate) enum StatePartsSubCommand { #[clap(subcommand)] epoch_selection: EpochSelection, }, - /// Read State Header from the DB - ReadStateHeader { - /// Select an epoch to work on. - #[clap(subcommand)] - epoch_selection: EpochSelection, - }, } impl StatePartsSubCommand { @@ -67,17 +62,6 @@ impl StatePartsSubCommand { near_config: NearConfig, store: Store, ) { - let runtime = - Arc::new(NightshadeRuntime::from_config(home_dir, store.clone(), &near_config)); - let chain_genesis = ChainGenesis::new(&near_config.genesis); - let mut chain = Chain::new_for_view_client( - runtime.clone(), - &chain_genesis, - DoomslugThresholdMode::TwoThirds, - false, - ) - .unwrap(); - let chain_id = &near_config.genesis.config.chain_id; match self { StatePartsSubCommand::Apply { dry_run, state_root, part_id, epoch_selection } => { apply_state_parts( @@ -86,8 +70,8 @@ impl StatePartsSubCommand { part_id, dry_run, state_root, - &mut chain, - chain_id, + home_dir, + near_config, store, Location::new(root_dir, (s3_bucket, s3_region)), ); @@ -98,15 +82,12 @@ impl StatePartsSubCommand { shard_id, part_from, part_to, - &chain, - chain_id, + home_dir, + near_config, store, Location::new(root_dir, (s3_bucket, s3_region)), ); } - StatePartsSubCommand::ReadStateHeader { epoch_selection } => { - read_state_header(epoch_selection, shard_id, &chain, store) - } } } } @@ -126,10 +107,15 @@ pub(crate) enum EpochSelection { } impl EpochSelection { - fn to_epoch_id(&self, store: Store, chain: &Chain) -> EpochId { + fn to_epoch_id( + &self, + store: Store, + chain_store: &ChainStore, + epoch_manager: &EpochManager, + ) -> EpochId { match self { EpochSelection::Current => { - chain.runtime_adapter.get_epoch_id(&chain.head().unwrap().last_block_hash).unwrap() + epoch_manager.get_epoch_id(&chain_store.head().unwrap().last_block_hash).unwrap() } EpochSelection::EpochId { epoch_id } => { EpochId(CryptoHash::from_str(&epoch_id).unwrap()) @@ -146,12 +132,12 @@ impl EpochSelection { } EpochSelection::BlockHash { block_hash } => { let block_hash = CryptoHash::from_str(&block_hash).unwrap(); - chain.runtime_adapter.get_epoch_id(&block_hash).unwrap() + epoch_manager.get_epoch_id(&block_hash).unwrap() } EpochSelection::BlockHeight { block_height } => { // Fetch an epoch containing the given block height. - let block_hash = chain.store().get_block_hash_by_height(*block_height).unwrap(); - chain.runtime_adapter.get_epoch_id(&block_hash).unwrap() + let block_hash = chain_store.get_block_hash_by_height(*block_height).unwrap(); + epoch_manager.get_epoch_id(&block_hash).unwrap() } } } @@ -186,18 +172,21 @@ impl Location { } } -/// Returns block hash of some block of the given `epoch_info` epoch. -fn get_any_block_hash_of_epoch(epoch_info: &EpochInfo, chain: &Chain) -> CryptoHash { - let head = chain.store().head().unwrap(); - let mut cur_block_info = chain.runtime_adapter.get_block_info(&head.last_block_hash).unwrap(); +/// Returns block hash of the last block of an epoch preceding the given `epoch_info`. +fn get_prev_hash_of_epoch( + epoch_info: &EpochInfo, + chain_store: &ChainStore, + epoch_manager: &EpochManager, +) -> CryptoHash { + let head = chain_store.head().unwrap(); + let mut cur_block_info = epoch_manager.get_block_info(&head.last_block_hash).unwrap(); // EpochManager doesn't have an API that maps EpochId to Blocks, and this function works // around that limitation by iterating over the epochs. // This workaround is acceptable because: // 1) Extending EpochManager's API is a major change. // 2) This use case is not critical at all. loop { - let cur_epoch_info = - chain.runtime_adapter.get_epoch_info(cur_block_info.epoch_id()).unwrap(); + let cur_epoch_info = epoch_manager.get_epoch_info(cur_block_info.epoch_id()).unwrap(); let cur_epoch_height = cur_epoch_info.epoch_height(); assert!( cur_epoch_height >= epoch_info.epoch_height(), @@ -206,12 +195,12 @@ fn get_any_block_hash_of_epoch(epoch_info: &EpochInfo, chain: &Chain) -> CryptoH epoch_info.epoch_height() ); let epoch_first_block_info = - chain.runtime_adapter.get_block_info(cur_block_info.epoch_first_block()).unwrap(); + epoch_manager.get_block_info(cur_block_info.epoch_first_block()).unwrap(); let prev_epoch_last_block_info = - chain.runtime_adapter.get_block_info(epoch_first_block_info.prev_hash()).unwrap(); + epoch_manager.get_block_info(epoch_first_block_info.prev_hash()).unwrap(); if cur_epoch_height == epoch_info.epoch_height() { - return *cur_block_info.hash(); + return *prev_epoch_last_block_info.hash(); } cur_block_info = prev_epoch_last_block_info; @@ -224,36 +213,32 @@ fn apply_state_parts( part_id: Option, dry_run: bool, maybe_state_root: Option, - chain: &mut Chain, - chain_id: &str, + home_dir: &Path, + near_config: NearConfig, store: Store, location: Location, ) { - let (state_root, epoch_height, epoch_id, sync_hash, sync_prev_hash) = if let ( - Some(state_root), - EpochSelection::EpochHeight { epoch_height }, - ) = - (maybe_state_root, &epoch_selection) - { - (state_root, *epoch_height, None, None, None) + let runtime_adapter: Arc = + NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); + let epoch_manager = + EpochManager::new_from_genesis_config(store.clone(), &near_config.genesis.config) + .expect("Failed to start Epoch Manager"); + let chain_store = ChainStore::new( + store.clone(), + near_config.genesis.config.genesis_height, + near_config.client_config.save_trie_changes, + ); + + let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); + let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); + + let (state_root, sync_prev_hash) = if let Some(state_root) = maybe_state_root { + (state_root, None) } else { - let epoch_id = epoch_selection.to_epoch_id(store, &chain); - let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); - - let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); - let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); - let sync_header = chain.get_block_header(&sync_hash).unwrap(); - // See `get_state_response_header()`. - let sync_prev_block = chain.get_block(sync_header.prev_hash()).unwrap(); - let sync_prev_hash = sync_prev_block.hash(); - tracing::info!( - target: "state-parts", - ?sync_hash, - ?sync_prev_hash, - height = sync_prev_block.header().height(), - state_roots = ?sync_prev_block.chunks().iter().map(|chunk| chunk.prev_state_root()).collect::>()); - - assert!(chain.runtime_adapter.is_next_block_epoch_start(&sync_prev_hash).unwrap()); + let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); + let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); + + assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); assert!( shard_id < sync_prev_block.chunks().len() as u64, "shard_id: {}, #shards: {}", @@ -261,21 +246,26 @@ fn apply_state_parts( sync_prev_block.chunks().len() ); let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); - (state_root, epoch.epoch_height(), Some(epoch_id), Some(sync_hash), Some(*sync_prev_hash)) + (state_root, Some(sync_prev_hash)) }; - let part_storage = get_state_part_reader(location, &chain_id, epoch_height, shard_id); + let part_storage = get_state_part_reader( + location, + &near_config.client_config.chain_id, + epoch.epoch_height(), + shard_id, + ); let num_parts = part_storage.num_parts(); assert_ne!(num_parts, 0, "Too few num_parts: {}", num_parts); let part_ids = get_part_ids(part_id, part_id.map(|x| x + 1), num_parts); tracing::info!( target: "state-parts", - epoch_height, + epoch_height = epoch.epoch_height(), + epoch_id = ?epoch_id.0, shard_id, num_parts, ?sync_prev_hash, - ?sync_hash, ?part_ids, "Applying state as seen at the beginning of the specified epoch.", ); @@ -287,29 +277,20 @@ fn apply_state_parts( let part = part_storage.read(part_id, num_parts); if dry_run { - assert!(chain.runtime_adapter.validate_state_part( + assert!(runtime_adapter.validate_state_part( &state_root, PartId::new(part_id, num_parts), &part )); tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Validated a state part"); } else { - chain - .set_state_part( - shard_id, - sync_hash.unwrap(), - PartId::new(part_id, num_parts), - &part, - ) - .unwrap(); - chain - .runtime_adapter + runtime_adapter .apply_state_part( shard_id, &state_root, PartId::new(part_id, num_parts), &part, - epoch_id.as_ref().unwrap(), + &epoch_id, ) .unwrap(); tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Applied a state part"); @@ -323,21 +304,28 @@ fn dump_state_parts( shard_id: ShardId, part_from: Option, part_to: Option, - chain: &Chain, - chain_id: &str, + home_dir: &Path, + near_config: NearConfig, store: Store, location: Location, ) { - let epoch_id = epoch_selection.to_epoch_id(store, &chain); - let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); - let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); - let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); - let sync_header = chain.get_block_header(&sync_hash).unwrap(); - // See `get_state_response_header()`. - let sync_prev_block = chain.get_block(sync_header.prev_hash()).unwrap(); - let sync_prev_hash = sync_prev_block.hash(); - - assert!(chain.runtime_adapter.is_next_block_epoch_start(&sync_prev_hash).unwrap()); + let runtime_adapter: Arc = + NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); + let epoch_manager = + EpochManager::new_from_genesis_config(store.clone(), &near_config.genesis.config) + .expect("Failed to start Epoch Manager"); + let chain_store = ChainStore::new( + store.clone(), + near_config.genesis.config.genesis_height, + near_config.client_config.save_trie_changes, + ); + + let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); + let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); + let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); + let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); + + assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); assert!( shard_id < sync_prev_block.chunks().len() as u64, "shard_id: {}, #shards: {}", @@ -346,7 +334,7 @@ fn dump_state_parts( ); let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); let state_root_node = - chain.runtime_adapter.get_state_root_node(shard_id, &sync_prev_hash, &state_root).unwrap(); + runtime_adapter.get_state_root_node(shard_id, &sync_prev_hash, &state_root).unwrap(); let num_parts = get_num_state_parts(state_root_node.memory_usage); let part_ids = get_part_ids(part_from, part_to, num_parts); @@ -357,22 +345,29 @@ fn dump_state_parts( epoch_id = ?epoch_id.0, shard_id, num_parts, - ?sync_hash, ?sync_prev_hash, ?part_ids, - ?state_root, "Dumping state as seen at the beginning of the specified epoch.", ); - let part_storage = get_state_part_writer(location, chain_id, epoch.epoch_height(), shard_id); + let part_storage = get_state_part_writer( + location, + &near_config.client_config.chain_id, + epoch.epoch_height(), + shard_id, + ); let timer = Instant::now(); for part_id in part_ids { let timer = Instant::now(); assert!(part_id < num_parts, "part_id: {}, num_parts: {}", part_id, num_parts); - let state_part = chain - .runtime_adapter - .obtain_state_part(shard_id, &sync_hash, &state_root, PartId::new(part_id, num_parts)) + let state_part = runtime_adapter + .obtain_state_part( + shard_id, + &sync_prev_hash, + &state_root, + PartId::new(part_id, num_parts), + ) .unwrap(); part_storage.write(&state_part, part_id, num_parts); tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote a state part"); @@ -380,23 +375,6 @@ fn dump_state_parts( tracing::info!(target: "state-parts", total_elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote all requested state parts"); } -/// Reads `StateHeader` stored in the DB. -fn read_state_header( - epoch_selection: EpochSelection, - shard_id: ShardId, - chain: &Chain, - store: Store, -) { - let epoch_id = epoch_selection.to_epoch_id(store, &chain); - let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); - - let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); - let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); - - let state_header = chain.store().get_state_header(shard_id, sync_hash); - tracing::info!(target: "state-parts", ?epoch_id, ?sync_hash, ?state_header); -} - fn get_part_ids(part_from: Option, part_to: Option, num_parts: u64) -> Range { part_from.unwrap_or(0)..part_to.unwrap_or(num_parts) } @@ -488,7 +466,6 @@ impl FileSystemStorage { tracing::info!(target: "state-parts", ?root_dir, ?prefix, ?state_parts_dir, "Ensuring the directory exists"); std::fs::create_dir_all(&state_parts_dir).unwrap(); } - tracing::info!(target: "state-parts", ?state_parts_dir, "Initialized FileSystemStorage"); Self { state_parts_dir } } @@ -508,38 +485,21 @@ impl StatePartWriter for FileSystemStorage { impl StatePartReader for FileSystemStorage { fn read(&self, part_id: u64, num_parts: u64) -> Vec { let filename = self.get_location(part_id, num_parts); - tracing::debug!(target: "state-parts", part_id, num_parts, ?filename, "Reading state part file"); let part = std::fs::read(filename).unwrap(); part } fn num_parts(&self) -> u64 { let paths = std::fs::read_dir(&self.state_parts_dir).unwrap(); - let mut known_num_parts = None; - let num_files = paths + let num_parts = paths .filter(|path| { let full_path = path.as_ref().unwrap(); tracing::debug!(target: "state-parts", ?full_path); - let filename = full_path.file_name().to_str().unwrap().to_string(); - if let Some(num_parts) = get_num_parts_from_filename(&filename) { - if let Some(known_num_parts) = known_num_parts { - assert_eq!(known_num_parts, num_parts); - } - known_num_parts = Some(num_parts); - } - is_part_filename(&filename) + is_part_filename(full_path.file_name().to_str().unwrap()) }) .collect::>>() .len(); - if known_num_parts != Some(num_files as u64) { - // This is expected when a user saves time and downloads a few parts instead of all parts. - tracing::warn!(target: "state-parts", - dir = ?self.state_parts_dir, - ?known_num_parts, - num_files, - "Filename indicates that number of files expected doesn't match the number of files available"); - } - known_num_parts.unwrap() + num_parts as u64 } } From fdaa79ea85f5d4d9fc0c55df1ec12004c3729d49 Mon Sep 17 00:00:00 2001 From: nikurt <86772482+nikurt@users.noreply.github.com> Date: Thu, 23 Mar 2023 10:53:05 +0100 Subject: [PATCH 57/88] feat: Disable state sync by default because it's unreliable (#8730) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Add a config option `state_sync_enabled`. Default value is `false`. If the option is `true`, state sync works as usual, but prints this message every 10 seconds: ``` Mar 15 07:31:11 nikurt-4 neard[48582]: 2023-03-15T07:31:11.352458Z WARN stats: The node is syncing its State. The current implementation of this mechanism is known to be unreliable. It may never complete, or fail randomly and corrupt the DB. Mar 15 07:31:11 nikurt-4 neard[48582]: Suggestions: Mar 15 07:31:11 nikurt-4 neard[48582]: * Download a recent data snapshot and restart the node. Mar 15 07:31:11 nikurt-4 neard[48582]: * Disable state sync in the config. Add `"state_sync_enabled": false` to `config.json`. Mar 15 07:31:11 nikurt-4 neard[48582]: A better implementation of State Sync is work in progress. ``` If the option is `false`, the node proceeds to download and apply blocks. Started a node from a month-old snapshot. After about 24 hours of running I see this, which confirms that the Block Sync is enabled and State Sync wasn't enabled: ``` Mar 15 07:32:27 nikurt-3 neard[36289]: 2023-03-15T07:32:27.565700Z INFO stats: #118440788 Downloading blocks 4.70% (1951625 left; at 118440788) 23 peers ⬇ 638 kB/s ⬆ 86.9 kB/s 2.00 bps 70.7 Tgas/s CPU: 62%, Mem: 4.27 GB ``` Fix #8719 --- CHANGELOG.md | 1 + chain/client/src/client.rs | 24 ++++---------- chain/client/src/info.rs | 8 +++++ core/chain-configs/src/client_config.rs | 11 +++---- nearcore/src/config.rs | 44 +++++-------------------- 5 files changed, 28 insertions(+), 60 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5ee0bc2e1ef..fad2605bd40 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -9,6 +9,7 @@ * State-viewer tool to dump and apply state changes from/to a range of blocks. [#8628](https://github.com/near/nearcore/pull/8628) * Experimental option to dump state of every epoch to external storage. [#8661](https://github.com/near/nearcore/pull/8661) * Add prometheus metrics for tracked shards, block height within epoch, if is block/chunk producer. [#8728](https://github.com/near/nearcore/pull/8728) +* State sync is disabled by default [#8730](https://github.com/near/nearcore/pull/8730) * Node can restart if State Sync gets interrupted. [#8732](https://github.com/near/nearcore/pull/8732) * Merged two `neard view-state` commands: `apply-state-parts` and `dump-state-parts` into a single `state-parts` command. [#8739](https://github.com/near/nearcore/pull/8739) * Node can sync State from S3. [#XXXX](https://github.com/near/nearcore/pull/XXXX) diff --git a/chain/client/src/client.rs b/chain/client/src/client.rs index 995a5542e8f..1ce060d05e6 100644 --- a/chain/client/src/client.rs +++ b/chain/client/src/client.rs @@ -239,17 +239,13 @@ impl Client { config.header_sync_stall_ban_timeout, config.header_sync_expected_height_per_second, ); - let block_sync = - BlockSync::new(network_adapter.clone(), config.block_fetch_horizon, config.archive); - let state_sync = StateSync::new( + let block_sync = BlockSync::new( network_adapter.clone(), - config.state_sync_timeout, - &config.chain_id, - config.state_sync_from_s3_enabled, - &config.state_sync_s3_bucket, - &config.state_sync_s3_region, - config.state_sync_num_concurrent_s3_requests, + config.block_fetch_horizon, + config.archive, + config.state_sync_enabled, ); + let state_sync = StateSync::new(network_adapter.clone(), config.state_sync_timeout); let num_block_producer_seats = config.num_block_producer_seats as usize; let data_parts = runtime_adapter.num_data_parts(); let parity_parts = runtime_adapter.num_total_parts() - data_parts; @@ -2120,15 +2116,7 @@ impl Client { let (state_sync, new_shard_sync, blocks_catch_up_state) = self.catchup_state_syncs.entry(sync_hash).or_insert_with(|| { ( - StateSync::new( - network_adapter1, - state_sync_timeout, - &self.config.chain_id, - self.config.state_sync_from_s3_enabled, - &self.config.state_sync_s3_bucket, - &self.config.state_sync_s3_region, - self.config.state_sync_num_concurrent_s3_requests, - ), + StateSync::new(network_adapter1, state_sync_timeout), new_shard_sync, BlocksCatchUpState::new(sync_hash, epoch_id), ) diff --git a/chain/client/src/info.rs b/chain/client/src/info.rs index cd14dedce87..ea254d88e65 100644 --- a/chain/client/src/info.rs +++ b/chain/client/src/info.rs @@ -533,6 +533,14 @@ pub fn display_sync_status(sync_status: &SyncStatus, head: &Tip) -> String { for (shard_id, shard_status) in shard_statuses { write!(res, "[{}: {}]", shard_id, shard_status.status.to_string(),).unwrap(); } + // TODO #8719 + tracing::warn!(target: "stats", + "The node is syncing its State. The current implementation of this mechanism is known to be unreliable. It may never complete, or fail randomly and corrupt the DB.\n\ + Suggestions:\n\ + * Download a recent data snapshot and restart the node.\n\ + * Disable state sync in the config. Add `\"state_sync_enabled\": false` to `config.json`.\n\ + \n\ + A better implementation of State Sync is work in progress."); res } SyncStatus::StateSyncDone => "State sync done".to_string(), diff --git a/core/chain-configs/src/client_config.rs b/core/chain-configs/src/client_config.rs index 408e7b31022..3d7ee29b827 100644 --- a/core/chain-configs/src/client_config.rs +++ b/core/chain-configs/src/client_config.rs @@ -175,11 +175,9 @@ pub struct ClientConfig { /// Restart dumping state of selected shards. /// Use for troubleshooting of the state dumping process. pub state_sync_restart_dump_for_shards: Vec, - /// Whether to enable state sync from S3. - /// If disabled will perform state sync from the peers. - pub state_sync_from_s3_enabled: bool, - /// Number of parallel in-flight requests allowed per shard. - pub state_sync_num_concurrent_s3_requests: u64, + /// Whether to use the State Sync mechanism. + /// If disabled, the node will do Block Sync instead of State Sync. + pub state_sync_enabled: bool, } impl ClientConfig { @@ -253,8 +251,7 @@ impl ClientConfig { state_sync_s3_bucket: String::new(), state_sync_s3_region: String::new(), state_sync_restart_dump_for_shards: vec![], - state_sync_from_s3_enabled: false, - state_sync_num_concurrent_s3_requests: 10, + state_sync_enabled: true, } } } diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index b611f18d893..7dc7a8a7177 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -356,6 +356,9 @@ pub struct Config { /// Options for dumping state of every epoch to S3. #[serde(skip_serializing_if = "Option::is_none")] pub state_sync: Option, + /// Whether to use state sync (unreliable and corrupts the DB if fails) or do a block sync instead. + #[serde(skip_serializing_if = "is_false")] + pub state_sync_enabled: bool, } fn is_false(value: &bool) -> bool { @@ -393,6 +396,7 @@ impl Default for Config { split_storage: None, expected_shutdown: None, state_sync: None, + state_sync_enabled: false, } } } @@ -704,37 +708,20 @@ impl NearConfig { state_sync_dump_enabled: config .state_sync .as_ref() - .map(|x| x.dump_enabled) - .flatten() - .unwrap_or(false), + .map_or(false, |x| x.dump_enabled.unwrap_or(false)), state_sync_s3_bucket: config .state_sync .as_ref() - .map(|x| x.s3_bucket.clone()) - .unwrap_or(String::new()), + .map_or(String::new(), |x| x.s3_bucket.clone()), state_sync_s3_region: config .state_sync .as_ref() - .map(|x| x.s3_region.clone()) - .unwrap_or(String::new()), + .map_or(String::new(), |x| x.s3_region.clone()), state_sync_restart_dump_for_shards: config .state_sync .as_ref() - .map(|x| x.drop_state_of_dump.clone()) - .flatten() - .unwrap_or(vec![]), - state_sync_from_s3_enabled: config - .state_sync - .as_ref() - .map(|x| x.sync_from_s3_enabled) - .flatten() - .unwrap_or(false), - state_sync_num_concurrent_s3_requests: config - .state_sync - .as_ref() - .map(|x| x.num_concurrent_s3_requests) - .flatten() - .unwrap_or(100), + .map_or(vec![], |x| x.drop_state_of_dump.clone().unwrap_or(vec![])), + state_sync_enabled: config.state_sync_enabled, }, network_config: NetworkConfig::new( config.network, @@ -1561,25 +1548,12 @@ pub fn load_test_config(seed: &str, addr: tcp::ListenerAddr, genesis: Genesis) - #[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default)] /// Options for dumping state to S3. pub struct StateSyncConfig { - /// Location of state dumps on S3. pub s3_bucket: String, - /// Region is very important on S3. pub s3_region: String, - /// Whether a node should dump state of each epoch to the external storage. #[serde(skip_serializing_if = "Option::is_none")] pub dump_enabled: Option, - /// Use carefully in case a node that dumps state to the external storage - /// gets in trouble. #[serde(skip_serializing_if = "Option::is_none")] pub drop_state_of_dump: Option>, - /// If enabled, will download state parts from external storage and not from - /// the peers. - #[serde(skip_serializing_if = "Option::is_none")] - pub sync_from_s3_enabled: Option, - /// When syncing state from S3, throttle requests to this many concurrent - /// requests per shard. - #[serde(skip_serializing_if = "Option::is_none")] - pub num_concurrent_s3_requests: Option, } #[test] From 89297d42b9caab6d6b684343f820be4e9ef9d050 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 23 Mar 2023 16:39:03 +0100 Subject: [PATCH 58/88] Merge --- CHANGELOG.md | 2 +- chain/chain/src/test_utils/kv_runtime.rs | 79 +++++++++++++++++------- chain/chain/src/types.rs | 2 + chain/client/src/client.rs | 5 +- chain/client/src/client_actor.rs | 11 +++- nearcore/src/config.rs | 4 +- nearcore/src/lib.rs | 2 - nearcore/src/runtime/mod.rs | 9 ++- 8 files changed, 79 insertions(+), 35 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index fad2605bd40..b336e9ea5da 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -12,7 +12,7 @@ * State sync is disabled by default [#8730](https://github.com/near/nearcore/pull/8730) * Node can restart if State Sync gets interrupted. [#8732](https://github.com/near/nearcore/pull/8732) * Merged two `neard view-state` commands: `apply-state-parts` and `dump-state-parts` into a single `state-parts` command. [#8739](https://github.com/near/nearcore/pull/8739) -* Node can sync State from S3. [#XXXX](https://github.com/near/nearcore/pull/XXXX) +* Node can sync State from S3. [#8789](https://github.com/near/nearcore/pull/8789) ## 1.32.0 diff --git a/chain/chain/src/test_utils/kv_runtime.rs b/chain/chain/src/test_utils/kv_runtime.rs index 5952346a471..7af4586d8b2 100644 --- a/chain/chain/src/test_utils/kv_runtime.rs +++ b/chain/chain/src/test_utils/kv_runtime.rs @@ -4,6 +4,7 @@ use std::sync::{Arc, RwLock, Weak}; use borsh::{BorshDeserialize, BorshSerialize}; +use near_epoch_manager::shard_tracker::{ShardTracker, TrackedConfig}; use near_epoch_manager::{EpochManagerAdapter, RngSeed}; use near_primitives::sandbox::state_patch::SandboxStatePatch; use near_primitives::state_part::PartId; @@ -844,6 +845,45 @@ impl EpochManagerAdapter for KeyValueRuntime { Ok(()) } } + + fn cares_about_shard_from_prev_block( + &self, + parent_hash: &CryptoHash, + account_id: &AccountId, + shard_id: ShardId, + ) -> Result { + // This `unwrap` here tests that in all code paths we check that the epoch exists before + // we check if we care about a shard. Please do not remove the unwrap, fix the logic of + // the calling function. + let epoch_valset = self.get_epoch_and_valset(*parent_hash).unwrap(); + let chunk_producers = self.get_chunk_producers(epoch_valset.1, shard_id); + for validator in chunk_producers { + if validator.account_id() == account_id { + return Ok(true); + } + } + Ok(false) + } + + fn cares_about_shard_next_epoch_from_prev_block( + &self, + parent_hash: &CryptoHash, + account_id: &AccountId, + shard_id: ShardId, + ) -> Result { + // This `unwrap` here tests that in all code paths we check that the epoch exists before + // we check if we care about a shard. Please do not remove the unwrap, fix the logic of + // the calling function. + let epoch_valset = self.get_epoch_and_valset(*parent_hash).unwrap(); + let chunk_producers = self + .get_chunk_producers((epoch_valset.1 + 1) % self.validators_by_valset.len(), shard_id); + for validator in chunk_producers { + if validator.account_id() == account_id { + return Ok(true); + } + } + Ok(false) + } } impl RuntimeAdapter for KeyValueRuntime { @@ -922,19 +962,12 @@ impl RuntimeAdapter for KeyValueRuntime { if self.tracks_all_shards { return true; } - // This `unwrap` here tests that in all code paths we check that the epoch exists before - // we check if we care about a shard. Please do not remove the unwrap, fix the logic of - // the calling function. - let epoch_valset = self.get_epoch_and_valset(*parent_hash).unwrap(); - let chunk_producers = self.get_chunk_producers(epoch_valset.1, shard_id); if let Some(account_id) = account_id { - for validator in chunk_producers { - if validator.account_id() == account_id { - return true; - } - } + self.cares_about_shard_from_prev_block(parent_hash, account_id, shard_id) + .unwrap_or(false) + } else { + false } - false } fn will_care_about_shard( @@ -947,20 +980,12 @@ impl RuntimeAdapter for KeyValueRuntime { if self.tracks_all_shards { return true; } - // This `unwrap` here tests that in all code paths we check that the epoch exists before - // we check if we care about a shard. Please do not remove the unwrap, fix the logic of - // the calling function. - let epoch_valset = self.get_epoch_and_valset(*parent_hash).unwrap(); - let chunk_producers = self - .get_chunk_producers((epoch_valset.1 + 1) % self.validators_by_valset.len(), shard_id); if let Some(account_id) = account_id { - for validator in chunk_producers { - if validator.account_id() == account_id { - return true; - } - } + self.cares_about_shard_next_epoch_from_prev_block(parent_hash, account_id, shard_id) + .unwrap_or(false) + } else { + false } - false } fn validate_tx( @@ -1411,4 +1436,12 @@ impl RuntimeWithEpochManagerAdapter for KeyValueRuntime { fn epoch_manager_adapter_arc(&self) -> Arc { self.myself.upgrade().unwrap() } + fn shard_tracker(&self) -> ShardTracker { + let config = if self.tracks_all_shards { + TrackedConfig::AllShards + } else { + TrackedConfig::new_empty() + }; + ShardTracker::new(config, self.epoch_manager_adapter_arc()) + } } diff --git a/chain/chain/src/types.rs b/chain/chain/src/types.rs index bfa69391e8d..a88be70c394 100644 --- a/chain/chain/src/types.rs +++ b/chain/chain/src/types.rs @@ -4,6 +4,7 @@ use std::sync::Arc; use borsh::{BorshDeserialize, BorshSerialize}; use chrono::DateTime; use chrono::Utc; +use near_epoch_manager::shard_tracker::ShardTracker; use near_primitives::sandbox::state_patch::SandboxStatePatch; use num_rational::Rational32; @@ -573,6 +574,7 @@ pub trait RuntimeAdapter: Send + Sync { pub trait RuntimeWithEpochManagerAdapter: RuntimeAdapter + EpochManagerAdapter { fn epoch_manager_adapter(&self) -> &dyn EpochManagerAdapter; fn epoch_manager_adapter_arc(&self) -> Arc; + fn shard_tracker(&self) -> ShardTracker; } /// The last known / checked height and time when we have processed it. diff --git a/chain/client/src/client.rs b/chain/client/src/client.rs index 1ce060d05e6..899c2507c58 100644 --- a/chain/client/src/client.rs +++ b/chain/client/src/client.rs @@ -41,6 +41,7 @@ use near_primitives::block::{Approval, ApprovalInner, ApprovalMessage, Block, Bl use near_primitives::block_header::ApprovalType; use near_primitives::challenge::{Challenge, ChallengeBody}; use near_primitives::epoch_manager::RngSeed; +use near_primitives::errors::EpochError; use near_primitives::hash::CryptoHash; use near_primitives::merkle::{merklize, MerklePath, PartialMerkleTree}; use near_primitives::network::PeerId; @@ -1758,7 +1759,7 @@ impl Client { let next_block_epoch_id = match self.runtime_adapter.get_epoch_id_from_prev_block(&parent_hash) { Err(e) => { - self.handle_process_approval_error(approval, approval_type, true, e); + self.handle_process_approval_error(approval, approval_type, true, e.into()); return; } Ok(next_epoch_id) => next_epoch_id, @@ -1779,7 +1780,7 @@ impl Client { account_id, ) { Ok(_) => next_block_epoch_id.clone(), - Err(near_chain::Error::NotAValidator) => { + Err(EpochError::NotAValidator(_, _)) => { match self.runtime_adapter.get_next_epoch_id_from_prev_block(&parent_hash) { Ok(next_block_next_epoch_id) => next_block_next_epoch_id, Err(_) => return, diff --git a/chain/client/src/client_actor.rs b/chain/client/src/client_actor.rs index 47c1f629d3a..b2297bd0a3e 100644 --- a/chain/client/src/client_actor.rs +++ b/chain/client/src/client_actor.rs @@ -33,6 +33,7 @@ use near_chain::{ ChainGenesis, DoneApplyChunkCallback, Provenance, RuntimeWithEpochManagerAdapter, }; use near_chain_configs::{ClientConfig, LogSummaryStyle}; +use near_chain_primitives::error::EpochErrorResultToChainError; use near_chunks::adapter::ShardsManagerRequestFromClient; use near_chunks::client::ShardsManagerResponse; use near_chunks::logic::cares_about_shard_this_or_next_epoch; @@ -652,7 +653,8 @@ impl Handler> for ClientActor { let validators: Vec = self .client .runtime_adapter - .get_epoch_block_producers_ordered(&head.epoch_id, &head.last_block_hash)? + .get_epoch_block_producers_ordered(&head.epoch_id, &head.last_block_hash) + .into_chain_error()? .into_iter() .map(|(validator_stake, is_slashed)| ValidatorInfo { account_id: validator_stake.take_account_id(), @@ -663,8 +665,11 @@ impl Handler> for ClientActor { let epoch_start_height = self.client.runtime_adapter.get_epoch_start_height(&head.last_block_hash).ok(); - let protocol_version = - self.client.runtime_adapter.get_epoch_protocol_version(&head.epoch_id)?; + let protocol_version = self + .client + .runtime_adapter + .get_epoch_protocol_version(&head.epoch_id) + .into_chain_error()?; let node_public_key = self.node_id.public_key().clone(); let (validator_account_id, validator_public_key) = match &self.client.validator_signer { diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index 7dc7a8a7177..e70a8e9cd7c 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -313,8 +313,8 @@ pub struct Config { /// save_trie_changes = !archive /// save_trie_changes should be set to true iff /// - archive if false - non-archival nodes need trie changes to perform garbage collection - /// - archive is true, cold_store is configured and migration to split_storage is finished - node - /// working in split storage mode needs trie changes in order to do garbage collection on hot. + /// - archive is true and cold_store is configured - node working in split storage mode + /// needs trie changes in order to do garbage collection on hot and populate cold State column. #[serde(skip_serializing_if = "Option::is_none")] pub save_trie_changes: Option, pub log_summary_style: LogSummaryStyle, diff --git a/nearcore/src/lib.rs b/nearcore/src/lib.rs index 000ccf43d19..b10afeefcaf 100644 --- a/nearcore/src/lib.rs +++ b/nearcore/src/lib.rs @@ -1,7 +1,6 @@ use crate::cold_storage::spawn_cold_store_loop; pub use crate::config::{init_configs, load_config, load_test_config, NearConfig, NEAR_BASE}; pub use crate::runtime::NightshadeRuntime; -pub use crate::shard_tracker::TrackedConfig; use crate::state_sync::{spawn_state_sync_dump, StateSyncDumpHandle}; use actix::{Actor, Addr}; use actix_rt::ArbiterHandle; @@ -33,7 +32,6 @@ pub mod dyn_config; mod metrics; pub mod migrations; mod runtime; -mod shard_tracker; mod state_sync; pub fn get_default_home() -> PathBuf { diff --git a/nearcore/src/runtime/mod.rs b/nearcore/src/runtime/mod.rs index 3233443f589..358387ae547 100644 --- a/nearcore/src/runtime/mod.rs +++ b/nearcore/src/runtime/mod.rs @@ -1,6 +1,5 @@ use crate::metrics; use crate::migrations::load_migration_data; -use crate::shard_tracker::{ShardTracker, TrackedConfig}; use crate::NearConfig; use borsh::ser::BorshSerialize; use borsh::BorshDeserialize; @@ -15,6 +14,7 @@ use near_chain_configs::{ }; use near_client_primitives::types::StateSplitApplyingStatus; use near_crypto::PublicKey; +use near_epoch_manager::shard_tracker::{ShardTracker, TrackedConfig}; use near_epoch_manager::{EpochManager, EpochManagerAdapter, EpochManagerHandle}; use near_o11y::log_assert; use near_pool::types::PoolIterator; @@ -153,7 +153,7 @@ impl NightshadeRuntime { EpochManager::new_from_genesis_config(store.clone().into(), &genesis_config) .expect("Failed to start Epoch Manager") .into_handle(); - let shard_tracker = ShardTracker::new(tracked_config, epoch_manager.clone()); + let shard_tracker = ShardTracker::new(tracked_config, Arc::new(epoch_manager.clone())); Arc::new_cyclic(|myself| NightshadeRuntime { genesis_config, runtime_config_store, @@ -1465,6 +1465,10 @@ impl RuntimeWithEpochManagerAdapter for NightshadeRuntime { fn epoch_manager_adapter_arc(&self) -> Arc { self.myself.upgrade().unwrap() } + + fn shard_tracker(&self) -> ShardTracker { + self.shard_tracker.clone() + } } impl node_runtime::adapter::ViewRuntimeAdapter for NightshadeRuntime { @@ -1567,6 +1571,7 @@ mod test { use std::collections::BTreeSet; use near_chain::{Chain, ChainGenesis}; + use near_epoch_manager::shard_tracker::TrackedConfig; use near_primitives::test_utils::create_test_signer; use near_primitives::types::validator_stake::ValidatorStake; use near_store::flat::{FlatStateChanges, FlatStateDelta, FlatStateDeltaMetadata}; From 699954f00207f8feaf50338e0095d1f063e21df5 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 23 Mar 2023 16:41:40 +0100 Subject: [PATCH 59/88] Merge --- chain/chain/src/chain.rs | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/chain/chain/src/chain.rs b/chain/chain/src/chain.rs index 533dfbb4716..dedc1369b1c 100644 --- a/chain/chain/src/chain.rs +++ b/chain/chain/src/chain.rs @@ -1829,11 +1829,7 @@ impl Chain { // there is no block, we need to make sure that the last block before tail is cleaned. let tail = self.store.tail()?; let mut tail_prev_block_cleaned = false; - tracing::trace!(target: "sync", tail, gc_height, "reset_data_pre_state_sync"); for height in tail..gc_height { - if height % 1000 == 0 { - tracing::trace!(target: "sync", tail, gc_height, progress = height, "reset_data_pre_state_sync"); - } if let Ok(blocks_current_height) = self.store.get_all_block_hashes_by_height(height) { let blocks_current_height = blocks_current_height.values().flatten().cloned().collect::>(); @@ -1861,7 +1857,6 @@ impl Chain { } } } - tracing::trace!(target: "sync", progress = "loop done", "reset_data_pre_state_sync"); // Clear Chunks data let mut chain_store_update = self.mut_store().store_update(); @@ -1869,9 +1864,9 @@ impl Chain { let chunk_height = std::cmp::min(head.height + 2, sync_height); chain_store_update.clear_chunk_data_and_headers(chunk_height)?; chain_store_update.commit()?; - tracing::trace!(target: "sync", progress = "chunks data cleaned up", "reset_data_pre_state_sync"); // clear all trie data + let tries = self.runtime_adapter.get_tries(); let mut chain_store_update = self.mut_store().store_update(); let mut store_update = StoreUpdate::new_with_tries(tries); @@ -1881,8 +1876,6 @@ impl Chain { // The reason to reset tail here is not to allow Tail be greater than Head chain_store_update.reset_tail(); chain_store_update.commit()?; - tracing::trace!(target: "sync", progress = "state data cleaned up", "reset_data_pre_state_sync"); - Ok(()) } From 1244ac3ed2e7cbb401a2c57950d287fceb2b5fe4 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 23 Mar 2023 17:17:10 +0100 Subject: [PATCH 60/88] Merge --- chain/client/src/client.rs | 20 +- chain/client/src/sync/state.rs | 584 +++++++++++++++++++----- core/chain-configs/src/client_config.rs | 7 + nearcore/src/config.rs | 45 +- tools/state-viewer/src/state_parts.rs | 233 ++++++---- 5 files changed, 667 insertions(+), 222 deletions(-) diff --git a/chain/client/src/client.rs b/chain/client/src/client.rs index 899c2507c58..38838c3dbfd 100644 --- a/chain/client/src/client.rs +++ b/chain/client/src/client.rs @@ -246,7 +246,15 @@ impl Client { config.archive, config.state_sync_enabled, ); - let state_sync = StateSync::new(network_adapter.clone(), config.state_sync_timeout); + let state_sync = StateSync::new( + network_adapter.clone(), + config.state_sync_timeout, + &config.chain_id, + config.state_sync_from_s3_enabled, + &config.state_sync_s3_bucket, + &config.state_sync_s3_region, + config.state_sync_num_concurrent_s3_requests, + ); let num_block_producer_seats = config.num_block_producer_seats as usize; let data_parts = runtime_adapter.num_data_parts(); let parity_parts = runtime_adapter.num_total_parts() - data_parts; @@ -2117,7 +2125,15 @@ impl Client { let (state_sync, new_shard_sync, blocks_catch_up_state) = self.catchup_state_syncs.entry(sync_hash).or_insert_with(|| { ( - StateSync::new(network_adapter1, state_sync_timeout), + StateSync::new( + network_adapter1, + state_sync_timeout, + &self.config.chain_id, + self.config.state_sync_from_s3_enabled, + &self.config.state_sync_s3_bucket, + &self.config.state_sync_s3_region, + self.config.state_sync_num_concurrent_s3_requests, + ), new_shard_sync, BlocksCatchUpState::new(sync_hash, epoch_id), ) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 378be79df37..ad2c092e1b4 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -20,6 +20,7 @@ //! here to depend more on local peers instead. //! +use crate::metrics; use ansi_term::Color::{Purple, Yellow}; use ansi_term::Style; use chrono::{DateTime, Duration, Utc}; @@ -41,12 +42,12 @@ use near_primitives::shard_layout::ShardUId; use near_primitives::state_part::PartId; use near_primitives::static_clock::StaticClock; use near_primitives::syncing::{get_num_state_parts, ShardStateSyncResponse}; -use near_primitives::types::{AccountId, ShardId, StateRoot}; +use near_primitives::types::{AccountId, EpochHeight, ShardId, StateRoot}; use rand::seq::SliceRandom; use rand::{thread_rng, Rng}; use std::collections::HashMap; use std::ops::Add; -use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::atomic::{AtomicI64, Ordering}; use std::sync::Arc; use std::time::Duration as TimeDuration; @@ -56,6 +57,7 @@ pub const MAX_STATE_PART_REQUEST: u64 = 16; /// This number should not exceed MAX_STATE_PART_REQUEST times (number of peers in the network). pub const MAX_PENDING_PART: u64 = MAX_STATE_PART_REQUEST * 10000; +#[derive(Debug)] pub enum StateSyncResult { /// No shard has changed its status Unchanged, @@ -91,18 +93,43 @@ fn make_account_or_peer_id_or_hash( From::AccountId(a) => To::AccountId(a), From::PeerId(p) => To::PeerId(p), From::Hash(h) => To::Hash(h), + From::ExternalStorage => To::ExternalStorage, } } +/// How to retrieve the state data. +pub enum StateSyncMode { + /// Request both the state header and state parts from the peers. + Peers, + /// Requests the state header from peers but gets the state parts from an + /// external storage. + HeaderFromPeersAndPartsFromExternal { + /// Chain ID. + chain_id: String, + /// Connection to the external storage. + bucket: Arc, + }, +} + +/// Information about which parts were requested from which peer and when. +pub struct PartsRequestState { + last_part_id_requested: HashMap<(AccountOrPeerIdOrHash, ShardId), PendingRequestStatus>, + /// Map from which part we requested to whom. + requested_target: lru::LruCache<(u64, CryptoHash), AccountOrPeerIdOrHash>, +} + /// Helper to track state sync. pub struct StateSync { + /// How to retrieve the state data. + mode: StateSyncMode, + + /// Is used for communication with the peers. network_adapter: PeerManagerAdapter, last_time_block_requested: Option>, - last_part_id_requested: HashMap<(AccountOrPeerIdOrHash, ShardId), PendingRequestStatus>, - /// Map from which part we requested to whom. - requested_target: lru::LruCache<(u64, CryptoHash), AccountOrPeerIdOrHash>, + /// Information about which parts were requested from which peer and when. + parts_request_state: Option, /// Timeout (set in config - by default to 60 seconds) is used to figure out how long we should wait /// for the answer from the other node before giving up. @@ -113,18 +140,60 @@ pub struct StateSync { /// Maps shard_id to result of splitting state for resharding split_state_roots: HashMap, near_chain::Error>>, + + /// The number of requests for state parts from external storage that are + /// allowed to be started for this shard. + requests_remaining: Arc, } impl StateSync { - pub fn new(network_adapter: PeerManagerAdapter, timeout: TimeDuration) -> Self { + pub fn new( + network_adapter: PeerManagerAdapter, + timeout: TimeDuration, + chain_id: &str, + state_sync_from_s3_enabled: bool, + s3_bucket: &str, + s3_region: &str, + num_concurrent_s3_requests: u64, + ) -> Self { + let (mode, parts_request_state) = if state_sync_from_s3_enabled { + tracing::debug!(target: "sync", s3_bucket, s3_region, "Initializing S3 bucket connection."); + assert!(!s3_bucket.is_empty() && !s3_region.is_empty(), "State sync from S3 is enabled. This requires that both `s3_bucket and `s3_region` and specified and non-empty"); + let mut bucket = s3::Bucket::new( + s3_bucket, + s3_region.parse::().unwrap(), + s3::creds::Credentials::default().unwrap(), + ) + .unwrap(); + // Ensure requests finish in finite amount of time. + bucket.set_request_timeout(Some(timeout)); + let bucket = Arc::new(bucket); + ( + StateSyncMode::HeaderFromPeersAndPartsFromExternal { + chain_id: chain_id.to_string(), + bucket, + }, + None, + ) + } else { + ( + StateSyncMode::Peers, + Some(PartsRequestState { + last_part_id_requested: Default::default(), + requested_target: lru::LruCache::new(MAX_PENDING_PART as usize), + }), + ) + }; + let timeout = Duration::from_std(timeout).unwrap(); StateSync { + mode, network_adapter, last_time_block_requested: None, - last_part_id_requested: Default::default(), - requested_target: lru::LruCache::new(MAX_PENDING_PART as usize), - timeout: Duration::from_std(timeout).unwrap(), + parts_request_state, + timeout, state_parts_apply_results: HashMap::new(), split_state_roots: HashMap::new(), + requests_remaining: Arc::new(AtomicI64::new(num_concurrent_s3_requests as i64)), } } @@ -198,11 +267,14 @@ impl StateSync { let shard_sync_download = new_shard_sync.entry(shard_id).or_insert_with(|| { run_shard_state_download = true; update_sync_status = true; - ShardSyncDownload::new(now) + ShardSyncDownload::new_download_state_header(now) }); let old_status = shard_sync_download.status.clone(); let mut shard_sync_done = false; + metrics::STATE_SYNC_STAGE + .with_label_values(&[&shard_id.to_string()]) + .set(shard_sync_download.status.repr() as i64); match &shard_sync_download.status { ShardSyncStatus::StateDownloadHeader => { (download_timeout, run_shard_state_download) = self @@ -215,8 +287,16 @@ impl StateSync { )?; } ShardSyncStatus::StateDownloadParts => { - (download_timeout, run_shard_state_download) = - self.sync_shards_download_parts_status(shard_sync_download, now); + let res = self.sync_shards_download_parts_status( + shard_id, + shard_sync_download, + sync_hash, + chain, + now, + ); + download_timeout = res.0; + run_shard_state_download = res.1; + update_sync_status |= res.2; } ShardSyncStatus::StateDownloadScheduling => { self.sync_shards_download_scheduling_status( @@ -290,13 +370,13 @@ impl StateSync { // Execute syncing for shard `shard_id` if run_shard_state_download { update_sync_status = true; - *shard_sync_download = self.request_shard( + self.request_shard( me, shard_id, chain, runtime_adapter, sync_hash, - shard_sync_download.clone(), + shard_sync_download, highest_height_peers, )?; } @@ -355,15 +435,15 @@ impl StateSync { sync_hash: CryptoHash, ) { // FIXME: something is wrong - the index should have a shard_id too. - self.requested_target.put((part_id, sync_hash), target.clone()); - - let timeout = self.timeout; - self.last_part_id_requested + let parts_request_state = self.parts_request_state.as_mut().unwrap(); + parts_request_state.requested_target.put((part_id, sync_hash), target.clone()); + parts_request_state + .last_part_id_requested .entry((target, shard_id)) .and_modify(|pending_request| { pending_request.missing_parts += 1; }) - .or_insert_with(|| PendingRequestStatus::new(timeout)); + .or_insert_with(|| PendingRequestStatus::new(self.timeout)); } // Function called when our node receives the network response with a part. @@ -375,15 +455,17 @@ impl StateSync { ) { let key = (part_id, sync_hash); // Check that it came from the target that we requested it from. - if let Some(target) = self.requested_target.get(&key) { - if self.last_part_id_requested.get_mut(&(target.clone(), shard_id)).map_or( - false, - |request| { + let parts_request_state = self.parts_request_state.as_mut().unwrap(); + if let Some(target) = parts_request_state.requested_target.get(&key) { + if parts_request_state + .last_part_id_requested + .get_mut(&(target.clone(), shard_id)) + .map_or(false, |request| { request.missing_parts = request.missing_parts.saturating_sub(1); request.missing_parts == 0 - }, - ) { - self.last_part_id_requested.remove(&(target.clone(), shard_id)); + }) + { + parts_request_state.last_part_id_requested.remove(&(target.clone(), shard_id)); } } } @@ -401,13 +483,16 @@ impl StateSync { highest_height_peers: &[HighestHeightPeerInfo], ) -> Result, near_chain::Error> { // Remove candidates from pending list if request expired due to timeout - self.last_part_id_requested.retain(|_, request| !request.expired()); + self.parts_request_state.as_mut().map(|parts_request_state| { + parts_request_state.last_part_id_requested.retain(|_, request| !request.expired()) + }); let prev_block_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); let epoch_hash = runtime_adapter.get_epoch_id_from_prev_block(&prev_block_hash)?; - Ok(runtime_adapter - .get_epoch_block_producers_ordered(&epoch_hash, &sync_hash)? + let block_producers = + runtime_adapter.get_epoch_block_producers_ordered(&epoch_hash, &sync_hash)?; + let peers = block_producers .iter() .filter_map(|(validator_stake, _slashed)| { let account_id = validator_stake.account_id(); @@ -437,12 +522,20 @@ impl StateSync { } else { None } - })) - .filter(|candidate| { - // If we still have a pending request from this node - don't add another one. - !self.last_part_id_requested.contains_key(&(candidate.clone(), shard_id)) - }) - .collect::>()) + })); + let result = if let Some(parts_request_state) = &self.parts_request_state { + peers + .filter(|candidate| { + // If we still have a pending request from this node - don't add another one. + !parts_request_state + .last_part_id_requested + .contains_key(&(candidate.clone(), shard_id)) + }) + .collect::>() + } else { + peers.collect::>() + }; + Ok(result) } /// Returns new ShardSyncDownload if successful, otherwise returns given shard_sync_download @@ -453,9 +546,9 @@ impl StateSync { chain: &Chain, runtime_adapter: &Arc, sync_hash: CryptoHash, - shard_sync_download: ShardSyncDownload, + shard_sync_download: &mut ShardSyncDownload, highest_height_peers: &[HighestHeightPeerInfo], - ) -> Result { + ) -> Result<(), near_chain::Error> { let possible_targets = self.possible_targets( me, shard_id, @@ -467,19 +560,17 @@ impl StateSync { if possible_targets.is_empty() { // In most cases it means that all the targets are currently busy (that we have a pending request with them). - return Ok(shard_sync_download); + return Ok(()); } // Downloading strategy starts here - let mut new_shard_sync_download = shard_sync_download.clone(); - match shard_sync_download.status { ShardSyncStatus::StateDownloadHeader => { self.request_shard_header( shard_id, sync_hash, &possible_targets, - &mut new_shard_sync_download, + shard_sync_download, ); } ShardSyncStatus::StateDownloadParts => { @@ -487,15 +578,17 @@ impl StateSync { shard_id, sync_hash, possible_targets, - &mut new_shard_sync_download, + shard_sync_download, + chain, ); } _ => {} } - Ok(new_shard_sync_download) + Ok(()) } + /// Makes a StateRequestHeader header to one of the peers. fn request_shard_header( &mut self, shard_id: ShardId, @@ -528,62 +621,167 @@ impl StateSync { ); } + /// Makes requests to download state parts for the given epoch of the given shard. fn request_shard_parts( &mut self, shard_id: ShardId, sync_hash: CryptoHash, possible_targets: Vec, new_shard_sync_download: &mut ShardSyncDownload, + chain: &Chain, ) { // We'll select all the 'highest' peers + validators as candidates (excluding those that gave us timeout in the past). // And for each one of them, we'll ask for up to 16 (MAX_STATE_PART_REQUEST) parts. - let possible_targets_sampler = + let mut possible_targets_sampler = SamplerLimited::new(possible_targets, MAX_STATE_PART_REQUEST); + let sync_block_header = chain.get_block_header(&sync_hash).unwrap(); + let epoch_id = sync_block_header.epoch_id(); + let epoch_info = chain.runtime_adapter.get_epoch_info(epoch_id).unwrap(); + let epoch_height = epoch_info.epoch_height(); + + let shard_state_header = chain.get_state_header(shard_id, sync_hash).unwrap(); + let state_num_parts = + get_num_state_parts(shard_state_header.state_root_node().memory_usage); + // Iterate over all parts that needs to be requested (i.e. download.run_me is true). // Parts are ordered such that its index match its part_id. - // Finally, for every part that needs to be requested it is selected one peer (target) randomly - // to request the part from. - // IMPORTANT: here we use 'zip' with possible_target_sampler - which is limited. So at any moment we'll not request more than - // possible_targets.len() * MAX_STATE_PART_REQUEST parts. - for ((part_id, download), target) in new_shard_sync_download + for (part_id, download) in new_shard_sync_download .downloads .iter_mut() .enumerate() .filter(|(_, download)| download.run_me.load(Ordering::SeqCst)) - .zip(possible_targets_sampler) { - self.sent_request_part(target.clone(), part_id as u64, shard_id, sync_hash); - download.run_me.store(false, Ordering::SeqCst); - download.state_requests_count += 1; - download.last_target = Some(make_account_or_peer_id_or_hash(target.clone())); - let run_me = download.run_me.clone(); - - near_performance_metrics::actix::spawn( - std::any::type_name::(), - self.network_adapter - .send_async(PeerManagerMessageRequest::NetworkRequests( - NetworkRequests::StateRequestPart { - shard_id, - sync_hash, - part_id: part_id as u64, - target: target.clone(), - }, - )) - .then(move |result| { - // TODO: possible optimization - in the current code, even if one of the targets it not present in the network graph - // (so we keep getting RouteNotFound) - we'll still keep trying to assign parts to it. - // Fortunately only once every 60 seconds (timeout value). - if let Ok(NetworkResponses::RouteNotFound) = - result.map(|f| f.as_network_response()) - { - // Send a StateRequestPart on the next iteration - run_me.store(true, Ordering::SeqCst); - } - future::ready(()) - }), - ); + match &self.mode { + StateSyncMode::Peers => { + // For every part that needs to be requested it is selected one + // peer (target) randomly to request the part from. + // IMPORTANT: here we use 'zip' with possible_target_sampler - + // which is limited. So at any moment we'll not request more + // than possible_targets.len() * MAX_STATE_PART_REQUEST parts. + let target = possible_targets_sampler.next().unwrap(); + self.request_part_from_peers( + part_id as u64, + target, + download, + shard_id, + sync_hash, + ); + } + StateSyncMode::HeaderFromPeersAndPartsFromExternal { chain_id, bucket } => { + self.request_part_from_external_storage( + part_id as u64, + download, + shard_id, + epoch_height, + state_num_parts, + &chain_id, + bucket.clone(), + ); + } + } + } + } + + /// Starts an asynchronous network request to external storage to obtain the given state part. + fn request_part_from_external_storage( + &self, + part_id: u64, + download: &mut DownloadStatus, + shard_id: ShardId, + epoch_height: EpochHeight, + num_parts: u64, + chain_id: &str, + bucket: Arc, + ) { + let requests_remaining = self.requests_remaining.clone(); + if !allow_request(&requests_remaining) { + return; + } else { + if !download.run_me.swap(false, Ordering::SeqCst) { + tracing::info!(target: "sync", %shard_id, part_id, "External storage request is allowed but run_me is already false. Undoing"); + finished_request(&requests_remaining); + return; + } } + download.state_requests_count += 1; + download.last_target = + Some(make_account_or_peer_id_or_hash(AccountOrPeerIdOrHash::ExternalStorage)); + + let location = s3_location(chain_id, epoch_height, shard_id, part_id, num_parts); + let download_response = download.response.clone(); + let scheduled = StaticClock::utc(); + near_performance_metrics::actix::spawn(std::any::type_name::(), { + async move { + tracing::debug!(target: "sync", %shard_id, part_id, location, "Getting an object from the external storage"); + let started = StaticClock::utc(); + metrics::STATE_SYNC_EXTERNAL_PARTS_SCHEDULING_DELAY + .with_label_values(&[&shard_id.to_string()]) + .observe( + started.signed_duration_since(scheduled).num_nanoseconds().unwrap_or(0) + as f64 + / 1e9, + ); + let result = bucket.get_object(location.clone()).await; + let completed = StaticClock::utc(); + finished_request(&requests_remaining); + metrics::STATE_SYNC_EXTERNAL_PARTS_REQUEST_DELAY + .with_label_values(&[&shard_id.to_string()]) + .observe( + completed.signed_duration_since(started).num_nanoseconds().unwrap_or(0) + as f64 + / 1e9, + ); + match result { + Ok(response) => { + tracing::debug!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), "S3 request finished"); + let mut lock = download_response.lock().unwrap(); + *lock = Some(Ok((response.status_code(), response.bytes().to_vec()))); + } + Err(err) => { + tracing::debug!(target: "sync", %shard_id, part_id, location, ?err, "S3 request failed"); + let mut lock = download_response.lock().unwrap(); + *lock = Some(Err(err.to_string())); + } + } + } + }); + } + + /// Asynchronously requests a state part from a suitable peer. + fn request_part_from_peers( + &mut self, + part_id: u64, + target: AccountOrPeerIdOrHash, + download: &mut DownloadStatus, + shard_id: ShardId, + sync_hash: CryptoHash, + ) { + self.sent_request_part(target.clone(), part_id, shard_id, sync_hash); + download.run_me.store(false, Ordering::SeqCst); + download.state_requests_count += 1; + download.last_target = Some(make_account_or_peer_id_or_hash(target.clone())); + let run_me = download.run_me.clone(); + + near_performance_metrics::actix::spawn( + std::any::type_name::(), + self.network_adapter + .send_async(PeerManagerMessageRequest::NetworkRequests( + NetworkRequests::StateRequestPart { shard_id, sync_hash, part_id, target }, + )) + .then(move |result| { + // TODO: possible optimization - in the current code, even if one of the targets it not present in the network graph + // (so we keep getting RouteNotFound) - we'll still keep trying to assign parts to it. + // Fortunately only once every 60 seconds (timeout value). + if let Ok(NetworkResponses::RouteNotFound) = + result.map(|f| f.as_network_response()) + { + // Send a StateRequestPart on the next iteration + run_me.store(true, Ordering::SeqCst); + } + future::ready(()) + }), + ); } /// The main 'step' function that should be called periodically to check and update the sync process. @@ -605,7 +803,7 @@ impl StateSync { use_colour: bool, ) -> Result { let _span = tracing::debug_span!(target: "sync", "run", sync = "StateSync").entered(); - tracing::debug!(target: "sync", %sync_hash, ?tracking_shards, "syncing state"); + tracing::trace!(target: "sync", %sync_hash, ?tracking_shards, "syncing state"); let prev_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); let now = StaticClock::utc(); @@ -738,21 +936,8 @@ impl StateSync { get_num_state_parts(shard_state_header.state_root_node().memory_usage); // If the header was downloaded successfully - move to phase 2 (downloading parts). // Create the vector with entry for each part. - *shard_sync_download = ShardSyncDownload { - downloads: vec![ - DownloadStatus { - start_time: now, - prev_update_time: now, - run_me: Arc::new(AtomicBool::new(true)), - error: false, - done: false, - state_requests_count: 0, - last_target: None, - }; - state_num_parts as usize - ], - status: ShardSyncStatus::StateDownloadParts, - }; + *shard_sync_download = + ShardSyncDownload::new_download_state_parts(now, state_num_parts); run_shard_state_download = true; } else { let prev = shard_sync_download.downloads[0].prev_update_time; @@ -773,45 +958,82 @@ impl StateSync { /// Checks if the parts are downloaded. /// If download of all parts is complete, then moves forward to `StateDownloadScheduling`. - /// Returns `(download_timeout, run_shard_state_download)` where: + /// Returns `(download_timeout, run_shard_state_download, update_sync_status)` where: /// * `download_timeout` means that the state header request timed out (and needs to be retried). /// * `run_shard_state_download` means that header or part download requests need to run for this shard. + /// * `update_sync_status` means that something changed in `ShardSyncDownload` and it needs to be persisted. fn sync_shards_download_parts_status( &mut self, + shard_id: ShardId, shard_sync_download: &mut ShardSyncDownload, + sync_hash: CryptoHash, + chain: &mut Chain, now: DateTime, - ) -> (bool, bool) { + ) -> (bool, bool, bool) { // Step 2 - download all the parts (each part is usually around 1MB). let mut download_timeout = false; let mut run_shard_state_download = false; + let mut update_sync_status = false; let mut parts_done = true; - for part_download in shard_sync_download.downloads.iter_mut() { + let num_parts = shard_sync_download.downloads.len(); + let mut num_parts_done = 0; + for (part_id, part_download) in shard_sync_download.downloads.iter_mut().enumerate() { + if !part_download.done { + // Check if a download from an external storage is finished. + update_sync_status |= check_external_storage_part_response( + part_id as u64, + num_parts as u64, + shard_id, + sync_hash, + part_download, + chain, + ); + } if !part_download.done { parts_done = false; let prev = part_download.prev_update_time; - let error = part_download.error; - let part_timeout = now - prev > self.timeout; - // Retry parts that failed. - if part_timeout || error { + let part_timeout = now - prev > self.timeout; // Retry parts that failed. + if part_timeout || part_download.error { download_timeout |= part_timeout; - part_download.run_me.store(true, Ordering::SeqCst); - part_download.error = false; - part_download.prev_update_time = now; + if part_timeout || + part_download.last_target != Some(near_client_primitives::types::AccountOrPeerIdOrHash::ExternalStorage) { + // Don't immediately retry failed requests from external + // storage. Most often error is a state part not + // available. That error doesn't get fixed by retrying, + // but rather by waiting. + metrics::STATE_SYNC_RETRY_PART + .with_label_values(&[&shard_id.to_string()]) + .inc(); + part_download.run_me.store(true, Ordering::SeqCst); + part_download.error = false; + part_download.prev_update_time = now; + update_sync_status = true; + } } if part_download.run_me.load(Ordering::SeqCst) { run_shard_state_download = true; } } + if part_download.done { + num_parts_done += 1; + } } + metrics::STATE_SYNC_PARTS_DONE + .with_label_values(&[&shard_id.to_string()]) + .set(num_parts_done); + metrics::STATE_SYNC_PARTS_TOTAL + .with_label_values(&[&shard_id.to_string()]) + .set(num_parts as i64); // If all parts are done - we can move towards scheduling. if parts_done { *shard_sync_download = ShardSyncDownload { downloads: vec![], status: ShardSyncStatus::StateDownloadScheduling, }; + update_sync_status = true; } - (download_timeout, run_shard_state_download) + (download_timeout, run_shard_state_download, update_sync_status) } fn sync_shards_download_scheduling_status( @@ -844,8 +1066,9 @@ impl StateSync { Err(err) => { // Cannot finalize the downloaded state. // The reasonable behavior here is to start from the very beginning. + metrics::STATE_SYNC_DISCARD_PARTS.with_label_values(&[&shard_id.to_string()]).inc(); tracing::error!(target: "sync", %shard_id, %sync_hash, ?err, "State sync finalizing error"); - *shard_sync_download = ShardSyncDownload::new(now); + *shard_sync_download = ShardSyncDownload::new_download_state_header(now); chain.clear_downloaded_parts(shard_id, sync_hash, state_num_parts)?; } } @@ -874,8 +1097,11 @@ impl StateSync { Err(err) => { // Cannot finalize the downloaded state. // The reasonable behavior here is to start from the very beginning. + metrics::STATE_SYNC_DISCARD_PARTS + .with_label_values(&[&shard_id.to_string()]) + .inc(); tracing::error!(target: "sync", %shard_id, %sync_hash, ?err, "State sync finalizing error"); - *shard_sync_download = ShardSyncDownload::new(now); + *shard_sync_download = ShardSyncDownload::new_download_state_header(now); let shard_state_header = chain.get_state_header(shard_id, sync_hash)?; let state_num_parts = get_num_state_parts(shard_state_header.state_root_node().memory_usage); @@ -959,9 +1185,118 @@ impl StateSync { } } -fn paint(s: &str, colour: Style, use_colour: bool) -> String { - if use_colour { - colour.paint(s).to_string() +/// Verifies that one more concurrent request can be started. +fn allow_request(requests_remaining: &AtomicI64) -> bool { + let remaining = requests_remaining.fetch_sub(1, Ordering::SeqCst); + if remaining >= 0 { + true + } else { + requests_remaining.fetch_add(1, Ordering::SeqCst); + false + } +} + +/// Notify about a request finishing. +fn finished_request(requests_remaining: &AtomicI64) { + requests_remaining.fetch_add(1, Ordering::SeqCst); +} + +/// Works around how data requests to external storage are done. +/// The response is stored on the DownloadStatus object. +/// This function investigates if the response is available and updates `done` and `error` appropriately. +/// If the response is successful, then also writes the state part to the DB. +/// +/// Returns whether something changed in `DownloadStatus` which means it needs to be persisted. +fn check_external_storage_part_response( + part_id: u64, + num_parts: u64, + shard_id: ShardId, + sync_hash: CryptoHash, + part_download: &mut DownloadStatus, + chain: &mut Chain, +) -> bool { + let external_storage_response = { + let mut lock = part_download.response.lock().unwrap(); + if let Some(response) = lock.clone() { + tracing::debug!(target: "sync", %shard_id, part_id, "Got response from external storage"); + // Remove the response from DownloadStatus, because + // we're going to write state parts to DB and don't need to keep + // them in `DownloadStatus`. + *lock = None; + response + } else { + return false; + } + }; + + let mut err_to_retry = None; + match external_storage_response { + // HTTP status code 200 means success. + Ok((200, data)) => { + tracing::debug!(target: "sync", %shard_id, part_id, "Got 200 response from external storage"); + match chain.set_state_part( + shard_id, + sync_hash, + PartId::new(part_id as u64, num_parts as u64), + &data, + ) { + Ok(_) => { + metrics::STATE_SYNC_EXTERNAL_PARTS_DONE + .with_label_values(&[&shard_id.to_string()]) + .inc(); + metrics::STATE_SYNC_EXTERNAL_PARTS_SIZE_DOWNLOADED + .with_label_values(&[&shard_id.to_string()]) + .inc_by(data.len() as u64); + part_download.done = true; + tracing::debug!(target: "sync", %shard_id, part_id, ?part_download, "Set state part success"); + } + Err(err) => { + metrics::STATE_SYNC_EXTERNAL_PARTS_FAILED + .with_label_values(&[&shard_id.to_string()]) + .inc(); + tracing::warn!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to save a state part"); + err_to_retry = + Some(near_chain::Error::Other("Failed to save a state part".to_string())); + } + } + } + // Other HTTP status codes are considered errors. + Ok((status_code, _)) => { + tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, status_code, "Wrong response code, expected 200"); + err_to_retry = + Some(near_chain::Error::Other(format!("status_code: {}", status_code).to_string())); + } + // The request failed without reaching the external storage. + Err(err) => { + err_to_retry = Some(near_chain::Error::Other(err.to_string())); + } + }; + + if let Some(err) = err_to_retry { + tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to get a part from external storage, will retry"); + part_download.error = true; + } + true +} + +/// Construct a location on the external storage. +pub fn s3_location( + chain_id: &str, + epoch_height: u64, + shard_id: u64, + part_id: u64, + num_parts: u64, +) -> String { + format!( + "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}_of_{:06}", + chain_id, epoch_height, shard_id, part_id, num_parts + ) +} + +/// Applies style if `use_colour` is enabled. +fn paint(s: &str, style: Style, use_style: bool) -> String { + if use_style { + style.paint(s).to_string() } else { s.to_string() } @@ -977,8 +1312,15 @@ fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload, use_colour: shard_sync_download.downloads[0].last_target ), ShardSyncStatus::StateDownloadParts => { + let mut num_parts_done = 0; + let mut num_parts_not_done = 0; let mut text = "".to_string(); for (i, download) in shard_sync_download.downloads.iter().enumerate() { + if download.done { + num_parts_done += 1; + continue; + } + num_parts_not_done += 1; text.push_str(&format!( "[{}: {}, {}, {:?}] ", paint(&i.to_string(), Yellow.bold(), use_colour), @@ -988,10 +1330,12 @@ fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload, use_colour: )); } format!( - "{} [{}: is_done, requests sent, last target] {}", + "{} [{}: is_done, requests sent, last target] {} num_parts_done={} num_parts_not_done={}", paint("PARTS", Purple.bold(), use_colour), paint("part_id", Yellow.bold(), use_colour), - text + text, + num_parts_done, + num_parts_not_done ) } _ => unreachable!("timeout cannot happen when all state is downloaded"), @@ -1083,8 +1427,15 @@ mod test { // Start a new state sync - and check that it asks for a header. fn test_ask_for_header() { let mock_peer_manager = Arc::new(MockPeerManagerAdapter::default()); - let mut state_sync = - StateSync::new(mock_peer_manager.clone().into(), TimeDuration::from_secs(1)); + let mut state_sync = StateSync::new( + mock_peer_manager.clone().into(), + TimeDuration::from_secs(1), + "chain_id".to_string(), + false, + "".to_string(), + "".to_string(), + 100, + ); let mut new_shard_sync = HashMap::new(); let (mut chain, kv, signer) = test_utils::setup(); @@ -1134,7 +1485,6 @@ mod test { vec![0], &apply_parts_fn, &state_split_fn, - false, ) .unwrap(); diff --git a/core/chain-configs/src/client_config.rs b/core/chain-configs/src/client_config.rs index 3d7ee29b827..da3624314aa 100644 --- a/core/chain-configs/src/client_config.rs +++ b/core/chain-configs/src/client_config.rs @@ -175,6 +175,11 @@ pub struct ClientConfig { /// Restart dumping state of selected shards. /// Use for troubleshooting of the state dumping process. pub state_sync_restart_dump_for_shards: Vec, + /// Whether to enable state sync from S3. + /// If disabled will perform state sync from the peers. + pub state_sync_from_s3_enabled: bool, + /// Number of parallel in-flight requests allowed per shard. + pub state_sync_num_concurrent_s3_requests: u64, /// Whether to use the State Sync mechanism. /// If disabled, the node will do Block Sync instead of State Sync. pub state_sync_enabled: bool, @@ -251,6 +256,8 @@ impl ClientConfig { state_sync_s3_bucket: String::new(), state_sync_s3_region: String::new(), state_sync_restart_dump_for_shards: vec![], + state_sync_from_s3_enabled: false, + state_sync_num_concurrent_s3_requests: 10, state_sync_enabled: true, } } diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index e70a8e9cd7c..f02a2765922 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -357,8 +357,8 @@ pub struct Config { #[serde(skip_serializing_if = "Option::is_none")] pub state_sync: Option, /// Whether to use state sync (unreliable and corrupts the DB if fails) or do a block sync instead. - #[serde(skip_serializing_if = "is_false")] - pub state_sync_enabled: bool, + #[serde(skip_serializing_if = "Option::is_none")] + pub state_sync_enabled: Option, } fn is_false(value: &bool) -> bool { @@ -708,19 +708,37 @@ impl NearConfig { state_sync_dump_enabled: config .state_sync .as_ref() - .map_or(false, |x| x.dump_enabled.unwrap_or(false)), + .map(|x| x.dump_enabled) + .flatten() + .unwrap_or(false), state_sync_s3_bucket: config .state_sync .as_ref() - .map_or(String::new(), |x| x.s3_bucket.clone()), + .map(|x| x.s3_bucket.clone()) + .unwrap_or(String::new()), state_sync_s3_region: config .state_sync .as_ref() - .map_or(String::new(), |x| x.s3_region.clone()), + .map(|x| x.s3_region.clone()) + .unwrap_or(String::new()), state_sync_restart_dump_for_shards: config .state_sync .as_ref() - .map_or(vec![], |x| x.drop_state_of_dump.clone().unwrap_or(vec![])), + .map(|x| x.restart_dump_for_shards.clone()) + .flatten() + .unwrap_or(vec![]), + state_sync_from_s3_enabled: config + .state_sync + .as_ref() + .map(|x| x.sync_from_s3_enabled) + .flatten() + .unwrap_or(false), + state_sync_num_concurrent_s3_requests: config + .state_sync + .as_ref() + .map(|x| x.num_concurrent_s3_requests) + .flatten() + .unwrap_or(100), state_sync_enabled: config.state_sync_enabled, }, network_config: NetworkConfig::new( @@ -1548,12 +1566,25 @@ pub fn load_test_config(seed: &str, addr: tcp::ListenerAddr, genesis: Genesis) - #[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default)] /// Options for dumping state to S3. pub struct StateSyncConfig { + /// Location of state dumps on S3. pub s3_bucket: String, + /// Region is very important on S3. pub s3_region: String, + /// Whether a node should dump state of each epoch to the external storage. #[serde(skip_serializing_if = "Option::is_none")] pub dump_enabled: Option, + /// Use carefully in case a node that dumps state to the external storage + /// gets in trouble. + #[serde(skip_serializing_if = "Option::is_none")] + pub restart_dump_for_shards: Option>, + /// If enabled, will download state parts from external storage and not from + /// the peers. + #[serde(skip_serializing_if = "Option::is_none")] + pub sync_from_s3_enabled: Option, + /// When syncing state from S3, throttle requests to this many concurrent + /// requests per shard. #[serde(skip_serializing_if = "Option::is_none")] - pub drop_state_of_dump: Option>, + pub num_concurrent_s3_requests: Option, } #[test] diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index 27e117c1332..8523cc27597 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -49,6 +49,12 @@ pub(crate) enum StatePartsSubCommand { #[clap(subcommand)] epoch_selection: EpochSelection, }, + /// Read State Header from the DB + ReadStateHeader { + /// Select an epoch to work on. + #[clap(subcommand)] + epoch_selection: EpochSelection, + }, } impl StatePartsSubCommand { @@ -62,6 +68,17 @@ impl StatePartsSubCommand { near_config: NearConfig, store: Store, ) { + let runtime = + Arc::new(NightshadeRuntime::from_config(home_dir, store.clone(), &near_config)); + let chain_genesis = ChainGenesis::new(&near_config.genesis); + let mut chain = Chain::new_for_view_client( + runtime.clone(), + &chain_genesis, + DoomslugThresholdMode::TwoThirds, + false, + ) + .unwrap(); + let chain_id = &near_config.genesis.config.chain_id; match self { StatePartsSubCommand::Apply { dry_run, state_root, part_id, epoch_selection } => { apply_state_parts( @@ -70,8 +87,8 @@ impl StatePartsSubCommand { part_id, dry_run, state_root, - home_dir, - near_config, + &mut chain, + chain_id, store, Location::new(root_dir, (s3_bucket, s3_region)), ); @@ -82,12 +99,15 @@ impl StatePartsSubCommand { shard_id, part_from, part_to, - home_dir, - near_config, + &chain, + chain_id, store, Location::new(root_dir, (s3_bucket, s3_region)), ); } + StatePartsSubCommand::ReadStateHeader { epoch_selection } => { + read_state_header(epoch_selection, shard_id, &chain, store) + } } } } @@ -107,15 +127,10 @@ pub(crate) enum EpochSelection { } impl EpochSelection { - fn to_epoch_id( - &self, - store: Store, - chain_store: &ChainStore, - epoch_manager: &EpochManager, - ) -> EpochId { + fn to_epoch_id(&self, store: Store, chain: &Chain) -> EpochId { match self { EpochSelection::Current => { - epoch_manager.get_epoch_id(&chain_store.head().unwrap().last_block_hash).unwrap() + chain.runtime_adapter.get_epoch_id(&chain.head().unwrap().last_block_hash).unwrap() } EpochSelection::EpochId { epoch_id } => { EpochId(CryptoHash::from_str(&epoch_id).unwrap()) @@ -132,12 +147,12 @@ impl EpochSelection { } EpochSelection::BlockHash { block_hash } => { let block_hash = CryptoHash::from_str(&block_hash).unwrap(); - epoch_manager.get_epoch_id(&block_hash).unwrap() + chain.runtime_adapter.get_epoch_id(&block_hash).unwrap() } EpochSelection::BlockHeight { block_height } => { // Fetch an epoch containing the given block height. - let block_hash = chain_store.get_block_hash_by_height(*block_height).unwrap(); - epoch_manager.get_epoch_id(&block_hash).unwrap() + let block_hash = chain.store().get_block_hash_by_height(*block_height).unwrap(); + chain.runtime_adapter.get_epoch_id(&block_hash).unwrap() } } } @@ -172,21 +187,18 @@ impl Location { } } -/// Returns block hash of the last block of an epoch preceding the given `epoch_info`. -fn get_prev_hash_of_epoch( - epoch_info: &EpochInfo, - chain_store: &ChainStore, - epoch_manager: &EpochManager, -) -> CryptoHash { - let head = chain_store.head().unwrap(); - let mut cur_block_info = epoch_manager.get_block_info(&head.last_block_hash).unwrap(); +/// Returns block hash of some block of the given `epoch_info` epoch. +fn get_any_block_hash_of_epoch(epoch_info: &EpochInfo, chain: &Chain) -> CryptoHash { + let head = chain.store().head().unwrap(); + let mut cur_block_info = chain.runtime_adapter.get_block_info(&head.last_block_hash).unwrap(); // EpochManager doesn't have an API that maps EpochId to Blocks, and this function works // around that limitation by iterating over the epochs. // This workaround is acceptable because: // 1) Extending EpochManager's API is a major change. // 2) This use case is not critical at all. loop { - let cur_epoch_info = epoch_manager.get_epoch_info(cur_block_info.epoch_id()).unwrap(); + let cur_epoch_info = + chain.runtime_adapter.get_epoch_info(cur_block_info.epoch_id()).unwrap(); let cur_epoch_height = cur_epoch_info.epoch_height(); assert!( cur_epoch_height >= epoch_info.epoch_height(), @@ -195,12 +207,12 @@ fn get_prev_hash_of_epoch( epoch_info.epoch_height() ); let epoch_first_block_info = - epoch_manager.get_block_info(cur_block_info.epoch_first_block()).unwrap(); + chain.runtime_adapter.get_block_info(cur_block_info.epoch_first_block()).unwrap(); let prev_epoch_last_block_info = - epoch_manager.get_block_info(epoch_first_block_info.prev_hash()).unwrap(); + chain.runtime_adapter.get_block_info(epoch_first_block_info.prev_hash()).unwrap(); if cur_epoch_height == epoch_info.epoch_height() { - return *prev_epoch_last_block_info.hash(); + return *cur_block_info.hash(); } cur_block_info = prev_epoch_last_block_info; @@ -213,32 +225,36 @@ fn apply_state_parts( part_id: Option, dry_run: bool, maybe_state_root: Option, - home_dir: &Path, - near_config: NearConfig, + chain: &mut Chain, + chain_id: &str, store: Store, location: Location, ) { - let runtime_adapter: Arc = - NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); - let epoch_manager = - EpochManager::new_from_genesis_config(store.clone(), &near_config.genesis.config) - .expect("Failed to start Epoch Manager"); - let chain_store = ChainStore::new( - store.clone(), - near_config.genesis.config.genesis_height, - near_config.client_config.save_trie_changes, - ); - - let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); - let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); - - let (state_root, sync_prev_hash) = if let Some(state_root) = maybe_state_root { - (state_root, None) + let (state_root, epoch_height, epoch_id, sync_hash, sync_prev_hash) = if let ( + Some(state_root), + EpochSelection::EpochHeight { epoch_height }, + ) = + (maybe_state_root, &epoch_selection) + { + (state_root, *epoch_height, None, None, None) } else { - let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); - let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); - - assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); + let epoch_id = epoch_selection.to_epoch_id(store, &chain); + let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); + + let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); + let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); + let sync_header = chain.get_block_header(&sync_hash).unwrap(); + // See `get_state_response_header()`. + let sync_prev_block = chain.get_block(sync_header.prev_hash()).unwrap(); + let sync_prev_hash = sync_prev_block.hash(); + tracing::info!( + target: "state-parts", + ?sync_hash, + ?sync_prev_hash, + height = sync_prev_block.header().height(), + state_roots = ?sync_prev_block.chunks().iter().map(|chunk| chunk.prev_state_root()).collect::>()); + + assert!(chain.runtime_adapter.is_next_block_epoch_start(&sync_prev_hash).unwrap()); assert!( shard_id < sync_prev_block.chunks().len() as u64, "shard_id: {}, #shards: {}", @@ -246,26 +262,21 @@ fn apply_state_parts( sync_prev_block.chunks().len() ); let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); - (state_root, Some(sync_prev_hash)) + (state_root, epoch.epoch_height(), Some(epoch_id), Some(sync_hash), Some(*sync_prev_hash)) }; - let part_storage = get_state_part_reader( - location, - &near_config.client_config.chain_id, - epoch.epoch_height(), - shard_id, - ); + let part_storage = get_state_part_reader(location, &chain_id, epoch_height, shard_id); let num_parts = part_storage.num_parts(); assert_ne!(num_parts, 0, "Too few num_parts: {}", num_parts); let part_ids = get_part_ids(part_id, part_id.map(|x| x + 1), num_parts); tracing::info!( target: "state-parts", - epoch_height = epoch.epoch_height(), - epoch_id = ?epoch_id.0, + epoch_height, shard_id, num_parts, ?sync_prev_hash, + ?sync_hash, ?part_ids, "Applying state as seen at the beginning of the specified epoch.", ); @@ -277,20 +288,29 @@ fn apply_state_parts( let part = part_storage.read(part_id, num_parts); if dry_run { - assert!(runtime_adapter.validate_state_part( + assert!(chain.runtime_adapter.validate_state_part( &state_root, PartId::new(part_id, num_parts), &part )); tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Validated a state part"); } else { - runtime_adapter + chain + .set_state_part( + shard_id, + sync_hash.unwrap(), + PartId::new(part_id, num_parts), + &part, + ) + .unwrap(); + chain + .runtime_adapter .apply_state_part( shard_id, &state_root, PartId::new(part_id, num_parts), &part, - &epoch_id, + epoch_id.as_ref().unwrap(), ) .unwrap(); tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Applied a state part"); @@ -304,28 +324,21 @@ fn dump_state_parts( shard_id: ShardId, part_from: Option, part_to: Option, - home_dir: &Path, - near_config: NearConfig, + chain: &Chain, + chain_id: &str, store: Store, location: Location, ) { - let runtime_adapter: Arc = - NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); - let epoch_manager = - EpochManager::new_from_genesis_config(store.clone(), &near_config.genesis.config) - .expect("Failed to start Epoch Manager"); - let chain_store = ChainStore::new( - store.clone(), - near_config.genesis.config.genesis_height, - near_config.client_config.save_trie_changes, - ); - - let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); - let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); - let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); - let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); - - assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); + let epoch_id = epoch_selection.to_epoch_id(store, &chain); + let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); + let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); + let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); + let sync_header = chain.get_block_header(&sync_hash).unwrap(); + // See `get_state_response_header()`. + let sync_prev_block = chain.get_block(sync_header.prev_hash()).unwrap(); + let sync_prev_hash = sync_prev_block.hash(); + + assert!(chain.runtime_adapter.is_next_block_epoch_start(&sync_prev_hash).unwrap()); assert!( shard_id < sync_prev_block.chunks().len() as u64, "shard_id: {}, #shards: {}", @@ -334,7 +347,7 @@ fn dump_state_parts( ); let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); let state_root_node = - runtime_adapter.get_state_root_node(shard_id, &sync_prev_hash, &state_root).unwrap(); + chain.runtime_adapter.get_state_root_node(shard_id, &sync_prev_hash, &state_root).unwrap(); let num_parts = get_num_state_parts(state_root_node.memory_usage); let part_ids = get_part_ids(part_from, part_to, num_parts); @@ -345,29 +358,22 @@ fn dump_state_parts( epoch_id = ?epoch_id.0, shard_id, num_parts, + ?sync_hash, ?sync_prev_hash, ?part_ids, + ?state_root, "Dumping state as seen at the beginning of the specified epoch.", ); - let part_storage = get_state_part_writer( - location, - &near_config.client_config.chain_id, - epoch.epoch_height(), - shard_id, - ); + let part_storage = get_state_part_writer(location, chain_id, epoch.epoch_height(), shard_id); let timer = Instant::now(); for part_id in part_ids { let timer = Instant::now(); assert!(part_id < num_parts, "part_id: {}, num_parts: {}", part_id, num_parts); - let state_part = runtime_adapter - .obtain_state_part( - shard_id, - &sync_prev_hash, - &state_root, - PartId::new(part_id, num_parts), - ) + let state_part = chain + .runtime_adapter + .obtain_state_part(shard_id, &sync_hash, &state_root, PartId::new(part_id, num_parts)) .unwrap(); part_storage.write(&state_part, part_id, num_parts); tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote a state part"); @@ -375,6 +381,23 @@ fn dump_state_parts( tracing::info!(target: "state-parts", total_elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote all requested state parts"); } +/// Reads `StateHeader` stored in the DB. +fn read_state_header( + epoch_selection: EpochSelection, + shard_id: ShardId, + chain: &Chain, + store: Store, +) { + let epoch_id = epoch_selection.to_epoch_id(store, &chain); + let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); + + let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); + let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); + + let state_header = chain.store().get_state_header(shard_id, sync_hash); + tracing::info!(target: "state-parts", ?epoch_id, ?sync_hash, ?state_header); +} + fn get_part_ids(part_from: Option, part_to: Option, num_parts: u64) -> Range { part_from.unwrap_or(0)..part_to.unwrap_or(num_parts) } @@ -466,6 +489,7 @@ impl FileSystemStorage { tracing::info!(target: "state-parts", ?root_dir, ?prefix, ?state_parts_dir, "Ensuring the directory exists"); std::fs::create_dir_all(&state_parts_dir).unwrap(); } + tracing::info!(target: "state-parts", ?state_parts_dir, "Initialized FileSystemStorage"); Self { state_parts_dir } } @@ -485,21 +509,38 @@ impl StatePartWriter for FileSystemStorage { impl StatePartReader for FileSystemStorage { fn read(&self, part_id: u64, num_parts: u64) -> Vec { let filename = self.get_location(part_id, num_parts); + tracing::debug!(target: "state-parts", part_id, num_parts, ?filename, "Reading state part file"); let part = std::fs::read(filename).unwrap(); part } fn num_parts(&self) -> u64 { let paths = std::fs::read_dir(&self.state_parts_dir).unwrap(); - let num_parts = paths + let mut known_num_parts = None; + let num_files = paths .filter(|path| { let full_path = path.as_ref().unwrap(); tracing::debug!(target: "state-parts", ?full_path); - is_part_filename(full_path.file_name().to_str().unwrap()) + let filename = full_path.file_name().to_str().unwrap().to_string(); + if let Some(num_parts) = get_num_parts_from_filename(&filename) { + if let Some(known_num_parts) = known_num_parts { + assert_eq!(known_num_parts, num_parts); + } + known_num_parts = Some(num_parts); + } + is_part_filename(&filename) }) .collect::>>() .len(); - num_parts as u64 + if known_num_parts != Some(num_files as u64) { + // This is expected when a user saves time and downloads a few parts instead of all parts. + tracing::warn!(target: "state-parts", + dir = ?self.state_parts_dir, + ?known_num_parts, + num_files, + "Filename indicates that number of files expected doesn't match the number of files available"); + } + known_num_parts.unwrap() } } From ad7ded8c823fe6b21b2d01e0ec9317902155cb68 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 23 Mar 2023 17:24:28 +0100 Subject: [PATCH 61/88] Merge --- chain/client/src/sync/state.rs | 1 + nearcore/src/config.rs | 4 ++-- tools/state-viewer/src/state_parts.rs | 10 ++++------ 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index ad2c092e1b4..73ba460e685 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -1485,6 +1485,7 @@ mod test { vec![0], &apply_parts_fn, &state_split_fn, + false, ) .unwrap(); diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index f02a2765922..195f85e6ce4 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -396,7 +396,7 @@ impl Default for Config { split_storage: None, expected_shutdown: None, state_sync: None, - state_sync_enabled: false, + state_sync_enabled: None, } } } @@ -739,7 +739,7 @@ impl NearConfig { .map(|x| x.num_concurrent_s3_requests) .flatten() .unwrap_or(100), - state_sync_enabled: config.state_sync_enabled, + state_sync_enabled: config.state_sync_enabled.unwrap_or(false), }, network_config: NetworkConfig::new( config.network, diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index 8523cc27597..c44943dede1 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -1,7 +1,6 @@ use crate::epoch_info::iterate_and_filter; -use near_chain::types::RuntimeAdapter; -use near_chain::{ChainStore, ChainStoreAccess}; -use near_epoch_manager::EpochManager; +use near_chain::{Chain, ChainGenesis, ChainStoreAccess, DoomslugThresholdMode}; +use near_client::sync::state::StateSync; use near_primitives::epoch_manager::epoch_info::EpochInfo; use near_primitives::state_part::PartId; use near_primitives::syncing::get_num_state_parts; @@ -15,7 +14,6 @@ use std::fs::DirEntry; use std::ops::Range; use std::path::{Path, PathBuf}; use std::str::FromStr; -use std::sync::Arc; use std::time::Instant; #[derive(clap::Subcommand, Debug, Clone)] @@ -68,8 +66,7 @@ impl StatePartsSubCommand { near_config: NearConfig, store: Store, ) { - let runtime = - Arc::new(NightshadeRuntime::from_config(home_dir, store.clone(), &near_config)); + let runtime = NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); let chain_genesis = ChainGenesis::new(&near_config.genesis); let mut chain = Chain::new_for_view_client( runtime.clone(), @@ -402,6 +399,7 @@ fn get_part_ids(part_from: Option, part_to: Option, num_parts: u64) -> part_from.unwrap_or(0)..part_to.unwrap_or(num_parts) } +// Needs to be in sync with `fn s3_location()`. fn location_prefix(chain_id: &str, epoch_height: u64, shard_id: u64) -> String { format!("chain_id={}/epoch_height={}/shard_id={}", chain_id, epoch_height, shard_id) } From 6691bf784ffc8c117edf1c78920316a036373e6f Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 23 Mar 2023 17:53:46 +0100 Subject: [PATCH 62/88] Doc --- docs/misc/state_sync_from_s3.md | 59 +++++++++++++++++++++++++++++++++ 1 file changed, 59 insertions(+) create mode 100644 docs/misc/state_sync_from_s3.md diff --git a/docs/misc/state_sync_from_s3.md b/docs/misc/state_sync_from_s3.md new file mode 100644 index 00000000000..ff1929b981b --- /dev/null +++ b/docs/misc/state_sync_from_s3.md @@ -0,0 +1,59 @@ +# Experimental: Sync state from External Storage + +## Purpose + +Current implementation of state sync (see +https://github.com/near/nearcore/blob/master/docs/architecture/how/sync.md for +details) doesn't allow the nodes to reliably perform state sync for testnet or +mainnet. + +That's why a new solution for state sync is being designed. +The experimental code is likely going to be a part of solution to greatly +improve both reliability and speed of state sync. + +The new solution will probably involve making the state available on external +storage, making downloading the state both low latency and reliable process, +thanks to the robust infrastructure of external storage such as S3. + +## How-to + +[#8789](https://github.com/near/nearcore/pull/8789) adds an experimental option +to sync state from external storage. At the moment only S3 is +supported as external storage. + +To enable, add this to your `config.json` file: + +```json +"state_sync_enabled": true, +"state_sync": { + "s3_bucket": "my-bucket", + "s3_region": "eu-central-1", + "sync_from_s3_enabled": true +} +``` + +And run your node with environment variables `AWS_ACCESS_KEY_ID` and +`AWS_SECRET_ACCESS_KEY`: +```shell +AWS_ACCESS_KEY_ID="MY_ACCESS_KEY" AWS_SECRET_ACCESS_KEY="MY_AWS_SECRET_ACCESS_KEY" ./neard run +``` + +## Implementation Details + +The experimental option replaces how a node fetches state parts. +The legacy implementation asks peer nodes to create and share a state part over network. +The new implementation expects to find state parts as files on an S3 storage. + +The sync mechanism proceeds to download state parts mostly-sequentially from S3. +In case the state part is not available, the request will be retried after a +delay defined by `state_sync_timeout`, which by default is 1 minute. + +State parts are location on S3 at the following location: +``` +"chain_id={chain_id}/epoch_height={epoch_height}/shard_id={shard_id}/state_part_{part_id:06}_of_{num_parts:06}", +``` +for example `chain_id=testnet/epoch_height=1790/shard_id=2/state_part_032642_of_065402` + +After all state parts are downloaded, the node applies them, which replaces the existing State of the node. + +Currently, both downloading and applying state parts work rather quickly. From 1cc6f0538d8a91aad79cf6b3815d67a7b8ed77ef Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 23 Mar 2023 18:03:54 +0100 Subject: [PATCH 63/88] Fix test --- chain/client/src/sync/state.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 73ba460e685..a41fb21a830 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -1430,10 +1430,10 @@ mod test { let mut state_sync = StateSync::new( mock_peer_manager.clone().into(), TimeDuration::from_secs(1), - "chain_id".to_string(), + "chain_id", false, - "".to_string(), - "".to_string(), + "", + "", 100, ); let mut new_shard_sync = HashMap::new(); From 0ad29cd894befad3f486363f6b2233c04d1b752c Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Fri, 24 Mar 2023 11:54:23 +0100 Subject: [PATCH 64/88] Fix test --- chain/client/src/sync/state.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index a41fb21a830..4b047deb393 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -1263,12 +1263,11 @@ fn check_external_storage_part_response( // Other HTTP status codes are considered errors. Ok((status_code, _)) => { tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, status_code, "Wrong response code, expected 200"); - err_to_retry = - Some(near_chain::Error::Other(format!("status_code: {}", status_code).to_string())); + err_to_retry = Some(near_chain::Error::Other(format!("status_code: {}", status_code))); } // The request failed without reaching the external storage. Err(err) => { - err_to_retry = Some(near_chain::Error::Other(err.to_string())); + err_to_retry = Some(near_chain::Error::Other(err)); } }; From 438e3431f68b42e07e99595fb7390e21f299b8cb Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Fri, 24 Mar 2023 12:39:16 +0100 Subject: [PATCH 65/88] Move some code to #8794 --- nearcore/src/metrics.rs | 27 +-- nearcore/src/runtime/mod.rs | 18 +- nearcore/src/state_sync.rs | 56 +++--- tools/state-viewer/src/state_parts.rs | 239 +++++++++++--------------- 4 files changed, 146 insertions(+), 194 deletions(-) diff --git a/nearcore/src/metrics.rs b/nearcore/src/metrics.rs index 19605ceeca1..4fd4fa910c2 100644 --- a/nearcore/src/metrics.rs +++ b/nearcore/src/metrics.rs @@ -50,6 +50,15 @@ pub(crate) static STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED: Lazy = Lazy: ) .unwrap() }); +pub(crate) static STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED: Lazy = Lazy::new(|| { + try_create_histogram_vec( + "near_state_sync_dump_obtain_part_elapsed_sec", + "Time needed to obtain a part", + &["shard_id"], + Some(exponential_buckets(0.001, 1.6, 25).unwrap()), + ) + .unwrap() +}); pub(crate) static STATE_SYNC_DUMP_NUM_PARTS_TOTAL: Lazy = Lazy::new(|| { try_create_int_gauge_vec( "near_state_sync_dump_num_parts_total", @@ -82,21 +91,3 @@ pub(crate) static STATE_SYNC_DUMP_EPOCH_HEIGHT: Lazy = Lazy::new(|| ) .unwrap() }); -pub static STATE_SYNC_APPLY_PART_DELAY: Lazy = Lazy::new(|| { - try_create_histogram_vec( - "near_state_sync_apply_part_delay_sec", - "Latency of applying a state part", - &["shard_id"], - Some(exponential_buckets(0.001, 2.0, 20).unwrap()), - ) - .unwrap() -}); -pub static STATE_SYNC_OBTAIN_PART_DELAY: Lazy = Lazy::new(|| { - try_create_histogram_vec( - "near_state_sync_obtain_part_delay_sec", - "Latency of applying a state part", - &["shard_id"], - Some(exponential_buckets(0.001, 2.0, 20).unwrap()), - ) - .unwrap() -}); diff --git a/nearcore/src/runtime/mod.rs b/nearcore/src/runtime/mod.rs index 358387ae547..5fa3543c190 100644 --- a/nearcore/src/runtime/mod.rs +++ b/nearcore/src/runtime/mod.rs @@ -1246,10 +1246,6 @@ impl RuntimeAdapter for NightshadeRuntime { %block_hash, num_parts = part_id.total) .entered(); - let _timer = metrics::STATE_SYNC_OBTAIN_PART_DELAY - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - let epoch_id = self.get_epoch_id(block_hash)?; let shard_uid = self.get_shard_uid_from_epoch_id(shard_id, &epoch_id)?; let trie = self.tries.get_view_trie_for_shard(shard_uid, *state_root); @@ -1274,17 +1270,11 @@ impl RuntimeAdapter for NightshadeRuntime { match Trie::validate_trie_nodes_for_part(state_root, part_id, trie_nodes) { Ok(_) => true, // Storage error should not happen - Err(err) => { - tracing::error!(target: "state-parts", ?err, "State part storage error"); - false - } + Err(_) => false, } } // Deserialization error means we've got the data from malicious peer - Err(err) => { - tracing::error!(target: "state-parts", ?err, "State part deserialization error"); - false - } + Err(_) => false, } } @@ -1381,10 +1371,6 @@ impl RuntimeAdapter for NightshadeRuntime { data: &[u8], epoch_id: &EpochId, ) -> Result<(), Error> { - let _timer = metrics::STATE_SYNC_APPLY_PART_DELAY - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - let part = BorshDeserialize::try_from_slice(data) .expect("Part was already validated earlier, so could never fail here"); let ApplyStatePartResult { trie_changes, flat_state_delta, contract_codes } = diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 465fb987841..458bff4e27e 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -3,7 +3,7 @@ use borsh::BorshSerialize; use near_chain::types::RuntimeAdapter; use near_chain::{Chain, ChainGenesis, ChainStoreAccess, DoomslugThresholdMode, Error}; use near_chain_configs::ClientConfig; -use near_client::sync::state::{s3_location, StateSync}; +use near_client::sync::state::StateSync; use near_crypto::PublicKey; use near_epoch_manager::EpochManagerAdapter; use near_primitives::hash::CryptoHash; @@ -175,7 +175,7 @@ async fn state_sync_dump( .with_label_values(&[&shard_id.to_string()]) .start_timer(); - let state_part = match obtain_and_store_state_part( + let state_part = match get_state_part( &runtime, &shard_id, &sync_hash, @@ -328,8 +328,7 @@ fn set_metrics( } } -/// Obtains and then saves the part data. -fn obtain_and_store_state_part( +fn get_state_part( runtime: &Arc, shard_id: &ShardId, sync_hash: &CryptoHash, @@ -338,13 +337,19 @@ fn obtain_and_store_state_part( num_parts: u64, chain: &Chain, ) -> Result, Error> { - let state_part = runtime.obtain_state_part( - *shard_id, - &sync_hash, - &state_root, - PartId::new(part_id, num_parts), - )?; + let state_part = { + let _timer = metrics::STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + runtime.obtain_state_part( + *shard_id, + &sync_hash, + &state_root, + PartId::new(part_id, num_parts), + )? + }; + // Save the part data. let key = StatePartKey(*sync_hash, *shard_id, part_id).try_to_vec()?; let mut store_update = chain.store().store().store_update(); store_update.set(DBCol::StateParts, &key, &state_part); @@ -363,17 +368,13 @@ fn start_dumping( let epoch_info = runtime.get_epoch_info(&epoch_id)?; let epoch_height = epoch_info.epoch_height(); let num_shards = runtime.num_shards(&epoch_id)?; - let sync_prev_header = chain.get_block_header(&sync_hash)?; - let sync_prev_hash = sync_prev_header.prev_hash(); - let prev_sync_block = chain.get_block(&sync_prev_hash)?; - if runtime.cares_about_shard(None, prev_sync_block.header().prev_hash(), shard_id, false) { - assert_eq!(num_shards, prev_sync_block.chunks().len() as u64); - let state_root = prev_sync_block.chunks()[shard_id as usize].prev_state_root(); - // See `get_state_response_header()` for reference. - let state_root_node = - runtime.get_state_root_node(shard_id, &sync_prev_hash, &state_root)?; + let sync_hash_block = chain.get_block(&sync_hash)?; + if runtime.cares_about_shard(None, sync_hash_block.header().prev_hash(), shard_id, false) { + assert_eq!(num_shards, sync_hash_block.chunks().len() as u64); + let state_root = sync_hash_block.chunks()[shard_id as usize].prev_state_root(); + let state_root_node = runtime.get_state_root_node(shard_id, &sync_hash, &state_root)?; let num_parts = get_num_state_parts(state_root_node.memory_usage); - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_prev_hash, %sync_hash, %state_root, num_parts, "Initialize dumping state of Epoch"); + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, %state_root, num_parts, "Initialize dumping state of Epoch"); // Note that first the state of the state machines gets changes to // `InProgress` and it starts dumping state after a short interval. set_metrics(&shard_id, Some(0), Some(num_parts), Some(epoch_height)); @@ -386,7 +387,7 @@ fn start_dumping( num_parts, })) } else { - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_prev_hash, %sync_hash, "Shard is not tracked, skip the epoch"); + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, "Shard is not tracked, skip the epoch"); Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts: Some(0) })) } } @@ -421,3 +422,16 @@ fn check_new_epoch( } } } + +fn s3_location( + chain_id: &str, + epoch_height: u64, + shard_id: u64, + part_id: u64, + num_parts: u64, +) -> String { + format!( + "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}_of_{:06}", + chain_id, epoch_height, shard_id, part_id, num_parts + ) +} diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index c44943dede1..27e117c1332 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -1,6 +1,7 @@ use crate::epoch_info::iterate_and_filter; -use near_chain::{Chain, ChainGenesis, ChainStoreAccess, DoomslugThresholdMode}; -use near_client::sync::state::StateSync; +use near_chain::types::RuntimeAdapter; +use near_chain::{ChainStore, ChainStoreAccess}; +use near_epoch_manager::EpochManager; use near_primitives::epoch_manager::epoch_info::EpochInfo; use near_primitives::state_part::PartId; use near_primitives::syncing::get_num_state_parts; @@ -14,6 +15,7 @@ use std::fs::DirEntry; use std::ops::Range; use std::path::{Path, PathBuf}; use std::str::FromStr; +use std::sync::Arc; use std::time::Instant; #[derive(clap::Subcommand, Debug, Clone)] @@ -47,12 +49,6 @@ pub(crate) enum StatePartsSubCommand { #[clap(subcommand)] epoch_selection: EpochSelection, }, - /// Read State Header from the DB - ReadStateHeader { - /// Select an epoch to work on. - #[clap(subcommand)] - epoch_selection: EpochSelection, - }, } impl StatePartsSubCommand { @@ -66,16 +62,6 @@ impl StatePartsSubCommand { near_config: NearConfig, store: Store, ) { - let runtime = NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); - let chain_genesis = ChainGenesis::new(&near_config.genesis); - let mut chain = Chain::new_for_view_client( - runtime.clone(), - &chain_genesis, - DoomslugThresholdMode::TwoThirds, - false, - ) - .unwrap(); - let chain_id = &near_config.genesis.config.chain_id; match self { StatePartsSubCommand::Apply { dry_run, state_root, part_id, epoch_selection } => { apply_state_parts( @@ -84,8 +70,8 @@ impl StatePartsSubCommand { part_id, dry_run, state_root, - &mut chain, - chain_id, + home_dir, + near_config, store, Location::new(root_dir, (s3_bucket, s3_region)), ); @@ -96,15 +82,12 @@ impl StatePartsSubCommand { shard_id, part_from, part_to, - &chain, - chain_id, + home_dir, + near_config, store, Location::new(root_dir, (s3_bucket, s3_region)), ); } - StatePartsSubCommand::ReadStateHeader { epoch_selection } => { - read_state_header(epoch_selection, shard_id, &chain, store) - } } } } @@ -124,10 +107,15 @@ pub(crate) enum EpochSelection { } impl EpochSelection { - fn to_epoch_id(&self, store: Store, chain: &Chain) -> EpochId { + fn to_epoch_id( + &self, + store: Store, + chain_store: &ChainStore, + epoch_manager: &EpochManager, + ) -> EpochId { match self { EpochSelection::Current => { - chain.runtime_adapter.get_epoch_id(&chain.head().unwrap().last_block_hash).unwrap() + epoch_manager.get_epoch_id(&chain_store.head().unwrap().last_block_hash).unwrap() } EpochSelection::EpochId { epoch_id } => { EpochId(CryptoHash::from_str(&epoch_id).unwrap()) @@ -144,12 +132,12 @@ impl EpochSelection { } EpochSelection::BlockHash { block_hash } => { let block_hash = CryptoHash::from_str(&block_hash).unwrap(); - chain.runtime_adapter.get_epoch_id(&block_hash).unwrap() + epoch_manager.get_epoch_id(&block_hash).unwrap() } EpochSelection::BlockHeight { block_height } => { // Fetch an epoch containing the given block height. - let block_hash = chain.store().get_block_hash_by_height(*block_height).unwrap(); - chain.runtime_adapter.get_epoch_id(&block_hash).unwrap() + let block_hash = chain_store.get_block_hash_by_height(*block_height).unwrap(); + epoch_manager.get_epoch_id(&block_hash).unwrap() } } } @@ -184,18 +172,21 @@ impl Location { } } -/// Returns block hash of some block of the given `epoch_info` epoch. -fn get_any_block_hash_of_epoch(epoch_info: &EpochInfo, chain: &Chain) -> CryptoHash { - let head = chain.store().head().unwrap(); - let mut cur_block_info = chain.runtime_adapter.get_block_info(&head.last_block_hash).unwrap(); +/// Returns block hash of the last block of an epoch preceding the given `epoch_info`. +fn get_prev_hash_of_epoch( + epoch_info: &EpochInfo, + chain_store: &ChainStore, + epoch_manager: &EpochManager, +) -> CryptoHash { + let head = chain_store.head().unwrap(); + let mut cur_block_info = epoch_manager.get_block_info(&head.last_block_hash).unwrap(); // EpochManager doesn't have an API that maps EpochId to Blocks, and this function works // around that limitation by iterating over the epochs. // This workaround is acceptable because: // 1) Extending EpochManager's API is a major change. // 2) This use case is not critical at all. loop { - let cur_epoch_info = - chain.runtime_adapter.get_epoch_info(cur_block_info.epoch_id()).unwrap(); + let cur_epoch_info = epoch_manager.get_epoch_info(cur_block_info.epoch_id()).unwrap(); let cur_epoch_height = cur_epoch_info.epoch_height(); assert!( cur_epoch_height >= epoch_info.epoch_height(), @@ -204,12 +195,12 @@ fn get_any_block_hash_of_epoch(epoch_info: &EpochInfo, chain: &Chain) -> CryptoH epoch_info.epoch_height() ); let epoch_first_block_info = - chain.runtime_adapter.get_block_info(cur_block_info.epoch_first_block()).unwrap(); + epoch_manager.get_block_info(cur_block_info.epoch_first_block()).unwrap(); let prev_epoch_last_block_info = - chain.runtime_adapter.get_block_info(epoch_first_block_info.prev_hash()).unwrap(); + epoch_manager.get_block_info(epoch_first_block_info.prev_hash()).unwrap(); if cur_epoch_height == epoch_info.epoch_height() { - return *cur_block_info.hash(); + return *prev_epoch_last_block_info.hash(); } cur_block_info = prev_epoch_last_block_info; @@ -222,36 +213,32 @@ fn apply_state_parts( part_id: Option, dry_run: bool, maybe_state_root: Option, - chain: &mut Chain, - chain_id: &str, + home_dir: &Path, + near_config: NearConfig, store: Store, location: Location, ) { - let (state_root, epoch_height, epoch_id, sync_hash, sync_prev_hash) = if let ( - Some(state_root), - EpochSelection::EpochHeight { epoch_height }, - ) = - (maybe_state_root, &epoch_selection) - { - (state_root, *epoch_height, None, None, None) + let runtime_adapter: Arc = + NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); + let epoch_manager = + EpochManager::new_from_genesis_config(store.clone(), &near_config.genesis.config) + .expect("Failed to start Epoch Manager"); + let chain_store = ChainStore::new( + store.clone(), + near_config.genesis.config.genesis_height, + near_config.client_config.save_trie_changes, + ); + + let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); + let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); + + let (state_root, sync_prev_hash) = if let Some(state_root) = maybe_state_root { + (state_root, None) } else { - let epoch_id = epoch_selection.to_epoch_id(store, &chain); - let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); - - let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); - let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); - let sync_header = chain.get_block_header(&sync_hash).unwrap(); - // See `get_state_response_header()`. - let sync_prev_block = chain.get_block(sync_header.prev_hash()).unwrap(); - let sync_prev_hash = sync_prev_block.hash(); - tracing::info!( - target: "state-parts", - ?sync_hash, - ?sync_prev_hash, - height = sync_prev_block.header().height(), - state_roots = ?sync_prev_block.chunks().iter().map(|chunk| chunk.prev_state_root()).collect::>()); - - assert!(chain.runtime_adapter.is_next_block_epoch_start(&sync_prev_hash).unwrap()); + let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); + let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); + + assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); assert!( shard_id < sync_prev_block.chunks().len() as u64, "shard_id: {}, #shards: {}", @@ -259,21 +246,26 @@ fn apply_state_parts( sync_prev_block.chunks().len() ); let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); - (state_root, epoch.epoch_height(), Some(epoch_id), Some(sync_hash), Some(*sync_prev_hash)) + (state_root, Some(sync_prev_hash)) }; - let part_storage = get_state_part_reader(location, &chain_id, epoch_height, shard_id); + let part_storage = get_state_part_reader( + location, + &near_config.client_config.chain_id, + epoch.epoch_height(), + shard_id, + ); let num_parts = part_storage.num_parts(); assert_ne!(num_parts, 0, "Too few num_parts: {}", num_parts); let part_ids = get_part_ids(part_id, part_id.map(|x| x + 1), num_parts); tracing::info!( target: "state-parts", - epoch_height, + epoch_height = epoch.epoch_height(), + epoch_id = ?epoch_id.0, shard_id, num_parts, ?sync_prev_hash, - ?sync_hash, ?part_ids, "Applying state as seen at the beginning of the specified epoch.", ); @@ -285,29 +277,20 @@ fn apply_state_parts( let part = part_storage.read(part_id, num_parts); if dry_run { - assert!(chain.runtime_adapter.validate_state_part( + assert!(runtime_adapter.validate_state_part( &state_root, PartId::new(part_id, num_parts), &part )); tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Validated a state part"); } else { - chain - .set_state_part( - shard_id, - sync_hash.unwrap(), - PartId::new(part_id, num_parts), - &part, - ) - .unwrap(); - chain - .runtime_adapter + runtime_adapter .apply_state_part( shard_id, &state_root, PartId::new(part_id, num_parts), &part, - epoch_id.as_ref().unwrap(), + &epoch_id, ) .unwrap(); tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Applied a state part"); @@ -321,21 +304,28 @@ fn dump_state_parts( shard_id: ShardId, part_from: Option, part_to: Option, - chain: &Chain, - chain_id: &str, + home_dir: &Path, + near_config: NearConfig, store: Store, location: Location, ) { - let epoch_id = epoch_selection.to_epoch_id(store, &chain); - let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); - let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); - let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); - let sync_header = chain.get_block_header(&sync_hash).unwrap(); - // See `get_state_response_header()`. - let sync_prev_block = chain.get_block(sync_header.prev_hash()).unwrap(); - let sync_prev_hash = sync_prev_block.hash(); - - assert!(chain.runtime_adapter.is_next_block_epoch_start(&sync_prev_hash).unwrap()); + let runtime_adapter: Arc = + NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); + let epoch_manager = + EpochManager::new_from_genesis_config(store.clone(), &near_config.genesis.config) + .expect("Failed to start Epoch Manager"); + let chain_store = ChainStore::new( + store.clone(), + near_config.genesis.config.genesis_height, + near_config.client_config.save_trie_changes, + ); + + let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); + let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); + let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); + let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); + + assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); assert!( shard_id < sync_prev_block.chunks().len() as u64, "shard_id: {}, #shards: {}", @@ -344,7 +334,7 @@ fn dump_state_parts( ); let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); let state_root_node = - chain.runtime_adapter.get_state_root_node(shard_id, &sync_prev_hash, &state_root).unwrap(); + runtime_adapter.get_state_root_node(shard_id, &sync_prev_hash, &state_root).unwrap(); let num_parts = get_num_state_parts(state_root_node.memory_usage); let part_ids = get_part_ids(part_from, part_to, num_parts); @@ -355,22 +345,29 @@ fn dump_state_parts( epoch_id = ?epoch_id.0, shard_id, num_parts, - ?sync_hash, ?sync_prev_hash, ?part_ids, - ?state_root, "Dumping state as seen at the beginning of the specified epoch.", ); - let part_storage = get_state_part_writer(location, chain_id, epoch.epoch_height(), shard_id); + let part_storage = get_state_part_writer( + location, + &near_config.client_config.chain_id, + epoch.epoch_height(), + shard_id, + ); let timer = Instant::now(); for part_id in part_ids { let timer = Instant::now(); assert!(part_id < num_parts, "part_id: {}, num_parts: {}", part_id, num_parts); - let state_part = chain - .runtime_adapter - .obtain_state_part(shard_id, &sync_hash, &state_root, PartId::new(part_id, num_parts)) + let state_part = runtime_adapter + .obtain_state_part( + shard_id, + &sync_prev_hash, + &state_root, + PartId::new(part_id, num_parts), + ) .unwrap(); part_storage.write(&state_part, part_id, num_parts); tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote a state part"); @@ -378,28 +375,10 @@ fn dump_state_parts( tracing::info!(target: "state-parts", total_elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote all requested state parts"); } -/// Reads `StateHeader` stored in the DB. -fn read_state_header( - epoch_selection: EpochSelection, - shard_id: ShardId, - chain: &Chain, - store: Store, -) { - let epoch_id = epoch_selection.to_epoch_id(store, &chain); - let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); - - let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); - let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); - - let state_header = chain.store().get_state_header(shard_id, sync_hash); - tracing::info!(target: "state-parts", ?epoch_id, ?sync_hash, ?state_header); -} - fn get_part_ids(part_from: Option, part_to: Option, num_parts: u64) -> Range { part_from.unwrap_or(0)..part_to.unwrap_or(num_parts) } -// Needs to be in sync with `fn s3_location()`. fn location_prefix(chain_id: &str, epoch_height: u64, shard_id: u64) -> String { format!("chain_id={}/epoch_height={}/shard_id={}", chain_id, epoch_height, shard_id) } @@ -487,7 +466,6 @@ impl FileSystemStorage { tracing::info!(target: "state-parts", ?root_dir, ?prefix, ?state_parts_dir, "Ensuring the directory exists"); std::fs::create_dir_all(&state_parts_dir).unwrap(); } - tracing::info!(target: "state-parts", ?state_parts_dir, "Initialized FileSystemStorage"); Self { state_parts_dir } } @@ -507,38 +485,21 @@ impl StatePartWriter for FileSystemStorage { impl StatePartReader for FileSystemStorage { fn read(&self, part_id: u64, num_parts: u64) -> Vec { let filename = self.get_location(part_id, num_parts); - tracing::debug!(target: "state-parts", part_id, num_parts, ?filename, "Reading state part file"); let part = std::fs::read(filename).unwrap(); part } fn num_parts(&self) -> u64 { let paths = std::fs::read_dir(&self.state_parts_dir).unwrap(); - let mut known_num_parts = None; - let num_files = paths + let num_parts = paths .filter(|path| { let full_path = path.as_ref().unwrap(); tracing::debug!(target: "state-parts", ?full_path); - let filename = full_path.file_name().to_str().unwrap().to_string(); - if let Some(num_parts) = get_num_parts_from_filename(&filename) { - if let Some(known_num_parts) = known_num_parts { - assert_eq!(known_num_parts, num_parts); - } - known_num_parts = Some(num_parts); - } - is_part_filename(&filename) + is_part_filename(full_path.file_name().to_str().unwrap()) }) .collect::>>() .len(); - if known_num_parts != Some(num_files as u64) { - // This is expected when a user saves time and downloads a few parts instead of all parts. - tracing::warn!(target: "state-parts", - dir = ?self.state_parts_dir, - ?known_num_parts, - num_files, - "Filename indicates that number of files expected doesn't match the number of files available"); - } - known_num_parts.unwrap() + num_parts as u64 } } From ea0b82a9cef34159fc49c3689e878550ba3b87fe Mon Sep 17 00:00:00 2001 From: wacban Date: Mon, 3 Apr 2023 15:01:22 +0100 Subject: [PATCH 66/88] chore: update changelog with 1.32.1 and 1.32.2 (#8869) Not entirely clear if we want minor releases tracked in changelog, do we? If so here is the PR for 1.32.1 and 1.32.2. --- CHANGELOG.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index b336e9ea5da..0522faece4e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -14,6 +14,16 @@ * Merged two `neard view-state` commands: `apply-state-parts` and `dump-state-parts` into a single `state-parts` command. [#8739](https://github.com/near/nearcore/pull/8739) * Node can sync State from S3. [#8789](https://github.com/near/nearcore/pull/8789) +## 1.32.2 + +### Fixes +* Fix: rosetta zero balance accounts [#8833](https://github.com/near/nearcore/pull/8833) + +## 1.32.1 + +### Fixes +* Fix vulnerabilities in block outcome root validation and total supply validation [#8790](https://github.com/near/nearcore/pull/8790) + ## 1.32.0 ### Protocol Changes From e3baa8eabce39ce153f9edfaa6ee06ace076f75f Mon Sep 17 00:00:00 2001 From: Razvan Barbascu Date: Tue, 4 Apr 2023 19:29:42 +0100 Subject: [PATCH 67/88] feat: override NetworkConfig from JSON config (#8871) Added config.experimental.network_config_overrides field. It contains the overrides for the currently default values from NetworkConfig. The JSON config override is done before the CLI overrides. --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0522faece4e..cfd30d49342 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -12,7 +12,7 @@ * State sync is disabled by default [#8730](https://github.com/near/nearcore/pull/8730) * Node can restart if State Sync gets interrupted. [#8732](https://github.com/near/nearcore/pull/8732) * Merged two `neard view-state` commands: `apply-state-parts` and `dump-state-parts` into a single `state-parts` command. [#8739](https://github.com/near/nearcore/pull/8739) -* Node can sync State from S3. [#8789](https://github.com/near/nearcore/pull/8789) +* Add config.network.experimental.network_config_overrides to the JSON config. [#8871](https://github.com/near/nearcore/pull/8871) ## 1.32.2 From 5a0a4fb781d8ae7592b758587377ec7a588a6a6a Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 28 Feb 2023 17:25:18 +0100 Subject: [PATCH 68/88] feat: Dump state to S3 --- chain/chain/src/store.rs | 21 +- core/chain-configs/src/client_config.rs | 26 +- core/primitives/src/syncing.rs | 19 +- nearcore/src/config.rs | 107 +------- nearcore/src/lib.rs | 59 ++--- nearcore/src/metrics.rs | 49 ++-- nearcore/src/state_sync.rs | 313 ++++++++---------------- 7 files changed, 162 insertions(+), 432 deletions(-) diff --git a/chain/chain/src/store.rs b/chain/chain/src/store.rs index 22ec2b5d0ea..a59793cbd82 100644 --- a/chain/chain/src/store.rs +++ b/chain/chain/src/store.rs @@ -47,6 +47,7 @@ use crate::chunks_store::ReadOnlyChunksStore; use crate::types::{Block, BlockHeader, LatestKnown}; use crate::{byzantine_assert, RuntimeWithEpochManagerAdapter}; use near_store::db::StoreStatistics; +use near_store::flat_state::{BlockInfo, ChainAccessForFlatStorage}; use std::sync::Arc; /// lru cache size @@ -841,7 +842,6 @@ impl ChainStore { /// Constructs key 'STATE_SYNC_DUMP:', /// for example 'STATE_SYNC_DUMP:2' for shard_id=2. - /// Doesn't contain epoch_id, because only one dump process per shard is allowed. fn state_sync_dump_progress_key(shard_id: ShardId) -> Vec { let mut key = b"STATE_SYNC_DUMP:".to_vec(); key.extend(shard_id.to_le_bytes()); @@ -1165,6 +1165,20 @@ impl ChainStoreAccess for ChainStore { } } +impl ChainAccessForFlatStorage for ChainStore { + fn get_block_info(&self, block_hash: &CryptoHash) -> BlockInfo { + let header = self.get_block_header(block_hash).unwrap(); + BlockInfo { hash: *block_hash, height: header.height(), prev_hash: *header.prev_hash() } + } + + fn get_block_hashes_at_height(&self, height: BlockHeight) -> HashSet { + match self.get_all_block_hashes_by_height(height) { + Ok(hashes) => hashes.values().flatten().copied().collect::>(), + Err(_) => Default::default(), + } + } +} + /// Cache update for ChainStore #[derive(Default)] struct ChainStoreCacheUpdate { @@ -2437,10 +2451,7 @@ impl<'a> ChainStoreUpdate<'a> { unreachable!(); } #[cfg(feature = "protocol_feature_flat_state")] - DBCol::FlatState - | DBCol::FlatStateChanges - | DBCol::FlatStateDeltaMetadata - | DBCol::FlatStorageStatus => { + DBCol::FlatState | DBCol::FlatStateDeltas | DBCol::FlatStateMisc => { unreachable!(); } } diff --git a/core/chain-configs/src/client_config.rs b/core/chain-configs/src/client_config.rs index da3624314aa..202363eb6a6 100644 --- a/core/chain-configs/src/client_config.rs +++ b/core/chain-configs/src/client_config.rs @@ -166,23 +166,10 @@ pub struct ClientConfig { pub client_background_migration_threads: usize, /// Duration to perform background flat storage creation step. pub flat_storage_creation_period: Duration, - /// If enabled, will dump state of every epoch to external storage. - pub state_sync_dump_enabled: bool, /// S3 bucket for storing state dumps. - pub state_sync_s3_bucket: String, + pub state_sync_s3_bucket: Option, /// S3 region for storing state dumps. - pub state_sync_s3_region: String, - /// Restart dumping state of selected shards. - /// Use for troubleshooting of the state dumping process. - pub state_sync_restart_dump_for_shards: Vec, - /// Whether to enable state sync from S3. - /// If disabled will perform state sync from the peers. - pub state_sync_from_s3_enabled: bool, - /// Number of parallel in-flight requests allowed per shard. - pub state_sync_num_concurrent_s3_requests: u64, - /// Whether to use the State Sync mechanism. - /// If disabled, the node will do Block Sync instead of State Sync. - pub state_sync_enabled: bool, + pub state_sync_s3_region: Option, } impl ClientConfig { @@ -252,13 +239,8 @@ impl ClientConfig { enable_statistics_export: true, client_background_migration_threads: 1, flat_storage_creation_period: Duration::from_secs(1), - state_sync_dump_enabled: false, - state_sync_s3_bucket: String::new(), - state_sync_s3_region: String::new(), - state_sync_restart_dump_for_shards: vec![], - state_sync_from_s3_enabled: false, - state_sync_num_concurrent_s3_requests: 10, - state_sync_enabled: true, + state_sync_s3_bucket: None, + state_sync_s3_region: None, } } } diff --git a/core/primitives/src/syncing.rs b/core/primitives/src/syncing.rs index 4dc68f3b7a6..c948fd7a895 100644 --- a/core/primitives/src/syncing.rs +++ b/core/primitives/src/syncing.rs @@ -229,29 +229,14 @@ pub fn get_num_state_parts(memory_usage: u64) -> u64 { } #[derive(BorshSerialize, BorshDeserialize, Debug, Clone)] -/// Represents the progress of dumps state of a shard. +/// Represents the state of the state machine that dumps state. pub enum StateSyncDumpProgress { - /// Represents two cases: - /// * An epoch dump is complete - /// * The node is running its first epoch and there is nothing to dump. - AllDumped { - /// The dumped state corresponds to the state at the beginning of the specified epoch. - epoch_id: EpochId, - epoch_height: EpochHeight, - // Missing in case of a node running the first epoch. - num_parts: Option, - }, - /// Represents the case of an epoch being partially dumped. + AllDumped(EpochId), InProgress { - /// The dumped state corresponds to the state at the beginning of the specified epoch. epoch_id: EpochId, epoch_height: EpochHeight, - /// Block hash of the first block of the epoch. - /// The dumped state corresponds to the state before applying this block. sync_hash: CryptoHash, - /// Root of the state being dumped. state_root: StateRoot, - /// Progress made. parts_dumped: u64, num_parts: u64, }, diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index 195f85e6ce4..456ec1cee17 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -313,8 +313,8 @@ pub struct Config { /// save_trie_changes = !archive /// save_trie_changes should be set to true iff /// - archive if false - non-archival nodes need trie changes to perform garbage collection - /// - archive is true and cold_store is configured - node working in split storage mode - /// needs trie changes in order to do garbage collection on hot and populate cold State column. + /// - archive is true, cold_store is configured and migration to split_storage is finished - node + /// working in split storage mode needs trie changes in order to do garbage collection on hot. #[serde(skip_serializing_if = "Option::is_none")] pub save_trie_changes: Option, pub log_summary_style: LogSummaryStyle, @@ -337,9 +337,7 @@ pub struct Config { /// This feature is under development, do not use in production. #[serde(default, skip_serializing_if = "Option::is_none")] pub cold_store: Option, - /// Configuration for the - #[serde(default, skip_serializing_if = "Option::is_none")] - pub split_storage: Option, + // TODO(mina86): Remove those two altogether at some point. We need to be // somewhat careful though and make sure that we don’t start silently // ignoring this option without users setting corresponding store option. @@ -354,11 +352,7 @@ pub struct Config { #[serde(default, skip_serializing_if = "Option::is_none")] pub expected_shutdown: Option, /// Options for dumping state of every epoch to S3. - #[serde(skip_serializing_if = "Option::is_none")] pub state_sync: Option, - /// Whether to use state sync (unreliable and corrupts the DB if fails) or do a block sync instead. - #[serde(skip_serializing_if = "Option::is_none")] - pub state_sync_enabled: Option, } fn is_false(value: &bool) -> bool { @@ -393,53 +387,8 @@ impl Default for Config { use_db_migration_snapshot: None, store: near_store::StoreConfig::default(), cold_store: None, - split_storage: None, expected_shutdown: None, state_sync: None, - state_sync_enabled: None, - } - } -} - -fn default_enable_split_storage_view_client() -> bool { - false -} - -fn default_cold_store_initial_migration_batch_size() -> usize { - 500_000_000 -} - -fn default_cold_store_initial_migration_loop_sleep_duration() -> Duration { - Duration::from_secs(30) -} - -fn default_cold_store_loop_sleep_duration() -> Duration { - Duration::from_secs(1) -} - -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] -pub struct SplitStorageConfig { - #[serde(default = "default_enable_split_storage_view_client")] - pub enable_split_storage_view_client: bool, - - #[serde(default = "default_cold_store_initial_migration_batch_size")] - pub cold_store_initial_migration_batch_size: usize, - #[serde(default = "default_cold_store_initial_migration_loop_sleep_duration")] - pub cold_store_initial_migration_loop_sleep_duration: Duration, - - #[serde(default = "default_cold_store_loop_sleep_duration")] - pub cold_store_loop_sleep_duration: Duration, -} - -impl Default for SplitStorageConfig { - fn default() -> Self { - SplitStorageConfig { - enable_split_storage_view_client: default_enable_split_storage_view_client(), - cold_store_initial_migration_batch_size: - default_cold_store_initial_migration_batch_size(), - cold_store_initial_migration_loop_sleep_duration: - default_cold_store_initial_migration_loop_sleep_duration(), - cold_store_loop_sleep_duration: default_cold_store_loop_sleep_duration(), } } } @@ -705,41 +654,14 @@ impl NearConfig { enable_statistics_export: config.store.enable_statistics_export, client_background_migration_threads: config.store.background_migration_threads, flat_storage_creation_period: config.store.flat_storage_creation_period, - state_sync_dump_enabled: config - .state_sync - .as_ref() - .map(|x| x.dump_enabled) - .flatten() - .unwrap_or(false), state_sync_s3_bucket: config .state_sync .as_ref() - .map(|x| x.s3_bucket.clone()) - .unwrap_or(String::new()), + .map_or(None, |x| Some(x.s3_bucket.clone())), state_sync_s3_region: config .state_sync .as_ref() - .map(|x| x.s3_region.clone()) - .unwrap_or(String::new()), - state_sync_restart_dump_for_shards: config - .state_sync - .as_ref() - .map(|x| x.restart_dump_for_shards.clone()) - .flatten() - .unwrap_or(vec![]), - state_sync_from_s3_enabled: config - .state_sync - .as_ref() - .map(|x| x.sync_from_s3_enabled) - .flatten() - .unwrap_or(false), - state_sync_num_concurrent_s3_requests: config - .state_sync - .as_ref() - .map(|x| x.num_concurrent_s3_requests) - .flatten() - .unwrap_or(100), - state_sync_enabled: config.state_sync_enabled.unwrap_or(false), + .map_or(None, |x| Some(x.s3_region.clone())), }, network_config: NetworkConfig::new( config.network, @@ -1563,28 +1485,11 @@ pub fn load_test_config(seed: &str, addr: tcp::ListenerAddr, genesis: Genesis) - NearConfig::new(config, genesis, signer.into(), validator_signer).unwrap() } -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default)] +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] /// Options for dumping state to S3. pub struct StateSyncConfig { - /// Location of state dumps on S3. pub s3_bucket: String, - /// Region is very important on S3. pub s3_region: String, - /// Whether a node should dump state of each epoch to the external storage. - #[serde(skip_serializing_if = "Option::is_none")] - pub dump_enabled: Option, - /// Use carefully in case a node that dumps state to the external storage - /// gets in trouble. - #[serde(skip_serializing_if = "Option::is_none")] - pub restart_dump_for_shards: Option>, - /// If enabled, will download state parts from external storage and not from - /// the peers. - #[serde(skip_serializing_if = "Option::is_none")] - pub sync_from_s3_enabled: Option, - /// When syncing state from S3, throttle requests to this many concurrent - /// requests per shard. - #[serde(skip_serializing_if = "Option::is_none")] - pub num_concurrent_s3_requests: Option, } #[test] diff --git a/nearcore/src/lib.rs b/nearcore/src/lib.rs index b10afeefcaf..7ea9b4f5eed 100644 --- a/nearcore/src/lib.rs +++ b/nearcore/src/lib.rs @@ -1,6 +1,7 @@ use crate::cold_storage::spawn_cold_store_loop; pub use crate::config::{init_configs, load_config, load_test_config, NearConfig, NEAR_BASE}; pub use crate::runtime::NightshadeRuntime; +pub use crate::shard_tracker::TrackedConfig; use crate::state_sync::{spawn_state_sync_dump, StateSyncDumpHandle}; use actix::{Actor, Addr}; use actix_rt::ArbiterHandle; @@ -15,8 +16,7 @@ use near_client::{start_client, start_view_client, ClientActor, ConfigUpdater, V use near_network::PeerManagerActor; use near_primitives::block::GenesisId; use near_primitives::time; -use near_store::metadata::DbKind; -use near_store::{DBCol, Mode, NodeStorage, Store, StoreOpenerError}; +use near_store::{DBCol, Mode, NodeStorage, StoreOpenerError, Temperature}; use near_telemetry::TelemetryActor; use std::path::{Path, PathBuf}; use std::sync::Arc; @@ -32,6 +32,7 @@ pub mod dyn_config; mod metrics; pub mod migrations; mod runtime; +mod shard_tracker; mod state_sync; pub fn get_default_home() -> PathBuf { @@ -153,33 +154,6 @@ fn open_storage(home_dir: &Path, near_config: &mut NearConfig) -> anyhow::Result Ok(storage) } -// Safely get the split store while checking that all conditions to use it are met. -fn get_split_store(config: &NearConfig, storage: &NodeStorage) -> anyhow::Result> { - // SplitStore should only be used on archival nodes. - if !config.config.archive { - return Ok(None); - } - - // SplitStore should only be used if cold store is configured. - if config.config.cold_store.is_none() { - return Ok(None); - } - - // SplitStore should only be used in the view client if it is enabled. - if !config.config.split_storage.as_ref().map_or(false, |c| c.enable_split_storage_view_client) { - return Ok(None); - } - - // SplitStore should only be used if the migration is finished. The - // migration to cold store is finished when the db kind of the hot store is - // changed from Archive to Hot. - if storage.get_hot_store().get_db_kind()? != Some(DbKind::Hot) { - return Ok(None); - } - - Ok(storage.get_split_store()) -} - pub struct NearNode { pub client: Addr, pub view_client: Addr, @@ -206,16 +180,11 @@ pub fn start_with_config_and_synchronization( ) -> anyhow::Result { let store = open_storage(home_dir, &mut config)?; - let runtime = NightshadeRuntime::from_config(home_dir, store.get_hot_store(), &config); - - // Get the split store. If split store is some then create a new runtime for - // the view client. Otherwise just re-use the existing runtime. - let split_store = get_split_store(&config, &store)?; - let view_runtime = if let Some(split_store) = split_store { - NightshadeRuntime::from_config(home_dir, split_store, &config) - } else { - runtime.clone() - }; + let runtime = Arc::new(NightshadeRuntime::from_config( + home_dir, + store.get_store(Temperature::Hot), + &config, + )); let cold_store_loop_handle = spawn_cold_store_loop(&config, &store, runtime.clone())?; @@ -236,7 +205,7 @@ pub fn start_with_config_and_synchronization( let view_client = start_view_client( config.validator_signer.as_ref().map(|signer| signer.validator_id().clone()), chain_genesis.clone(), - view_runtime, + runtime.clone(), network_adapter.clone().into(), config.client_config.clone(), adv.clone(), @@ -260,15 +229,15 @@ pub fn start_with_config_and_synchronization( network_adapter.as_sender(), client_adapter_for_shards_manager.as_sender(), config.validator_signer.as_ref().map(|signer| signer.validator_id().clone()), - store.get_hot_store(), + store.get_store(Temperature::Hot), config.client_config.chunk_request_retry_period, ); shards_manager_adapter.bind(shards_manager_actor); let state_sync_dump_handle = spawn_state_sync_dump( &config, - chain_genesis, - runtime, + &chain_genesis.clone(), + runtime.clone(), config.network_config.node_id().public_key(), )?; @@ -365,7 +334,7 @@ pub fn recompress_storage(home_dir: &Path, opts: RecompressOpts) -> anyhow::Resu "Recompressing database"); info!("Opening database at {}", src_path.display()); - let src_store = src_opener.open_in_mode(Mode::ReadOnly)?.get_hot_store(); + let src_store = src_opener.open_in_mode(Mode::ReadOnly)?.get_store(Temperature::Hot); let final_head_height = if skip_columns.contains(&DBCol::PartialChunks) { let tip: Option = @@ -382,7 +351,7 @@ pub fn recompress_storage(home_dir: &Path, opts: RecompressOpts) -> anyhow::Resu }; info!("Creating database at {}", dst_path.display()); - let dst_store = dst_opener.open_in_mode(Mode::Create)?.get_hot_store(); + let dst_store = dst_opener.open_in_mode(Mode::Create)?.get_store(Temperature::Hot); const BATCH_SIZE_BYTES: u64 = 150_000_000; diff --git a/nearcore/src/metrics.rs b/nearcore/src/metrics.rs index 4fd4fa910c2..2983683b16e 100644 --- a/nearcore/src/metrics.rs +++ b/nearcore/src/metrics.rs @@ -1,7 +1,6 @@ use near_o11y::metrics::{ exponential_buckets, linear_buckets, try_create_histogram_vec, try_create_int_counter_vec, - try_create_int_gauge, try_create_int_gauge_vec, HistogramVec, IntCounterVec, IntGauge, - IntGaugeVec, + try_create_int_gauge, HistogramVec, IntCounterVec, IntGauge, }; use once_cell::sync::Lazy; @@ -15,6 +14,20 @@ pub static APPLY_CHUNK_DELAY: Lazy = Lazy::new(|| { .unwrap() }); +pub static SECONDS_PER_PETAGAS: Lazy = Lazy::new(|| { + try_create_histogram_vec( + "near_execution_seconds_per_petagas_ratio", + "Execution time per unit of gas, measured in seconds per petagas. Ignore label 'label'.", + &["shard_id"], + // Non-linear buckets with higher resolution around 1.0. + Some(vec![ + 0.0, 0.1, 0.2, 0.5, 0.7, 0.8, 0.9, 0.95, 0.97, 0.99, 1.0, 1.01, 1.03, 1.05, 1.1, 1.2, + 1.3, 1.5, 2.0, 5.0, 10.0, + ]), + ) + .unwrap() +}); + pub(crate) static CONFIG_CORRECT: Lazy = Lazy::new(|| { try_create_int_gauge( "near_config_correct", @@ -59,35 +72,3 @@ pub(crate) static STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED: Lazy = Lazy ) .unwrap() }); -pub(crate) static STATE_SYNC_DUMP_NUM_PARTS_TOTAL: Lazy = Lazy::new(|| { - try_create_int_gauge_vec( - "near_state_sync_dump_num_parts_total", - "Total number of parts in the epoch that being dumped", - &["shard_id"], - ) - .unwrap() -}); -pub(crate) static STATE_SYNC_DUMP_NUM_PARTS_DUMPED: Lazy = Lazy::new(|| { - try_create_int_gauge_vec( - "near_state_sync_dump_num_parts_dumped", - "Number of parts dumped in the epoch that is being dumped", - &["shard_id"], - ) - .unwrap() -}); -pub(crate) static STATE_SYNC_DUMP_SIZE_TOTAL: Lazy = Lazy::new(|| { - try_create_int_counter_vec( - "near_state_sync_dump_size_total", - "Total size of parts written to S3", - &["shard_id"], - ) - .unwrap() -}); -pub(crate) static STATE_SYNC_DUMP_EPOCH_HEIGHT: Lazy = Lazy::new(|| { - try_create_int_gauge_vec( - "near_state_sync_dump_epoch_height", - "Epoch Height of an epoch being dumped", - &["shard_id"], - ) - .unwrap() -}); diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 458bff4e27e..44483f28c5c 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -1,61 +1,52 @@ use crate::{metrics, NearConfig, NightshadeRuntime}; -use borsh::BorshSerialize; use near_chain::types::RuntimeAdapter; -use near_chain::{Chain, ChainGenesis, ChainStoreAccess, DoomslugThresholdMode, Error}; +use near_chain::{Chain, ChainGenesis, DoomslugThresholdMode, Error}; use near_chain_configs::ClientConfig; use near_client::sync::state::StateSync; use near_crypto::PublicKey; use near_epoch_manager::EpochManagerAdapter; use near_primitives::hash::CryptoHash; use near_primitives::state_part::PartId; -use near_primitives::syncing::{get_num_state_parts, StatePartKey, StateSyncDumpProgress}; -use near_primitives::types::{EpochHeight, EpochId, ShardId, StateRoot}; -use near_store::DBCol; +use near_primitives::syncing::{get_num_state_parts, StateSyncDumpProgress}; +use near_primitives::types::{EpochId, ShardId}; use std::sync::Arc; -/// Starts one a thread per tracked shard. -/// Each started thread will be dumping state parts of a single epoch to external storage. pub fn spawn_state_sync_dump( config: &NearConfig, - chain_genesis: ChainGenesis, + chain_genesis: &ChainGenesis, runtime: Arc, node_key: &PublicKey, ) -> anyhow::Result> { - if !config.client_config.state_sync_dump_enabled { - return Ok(None); - } - if config.client_config.state_sync_s3_bucket.is_empty() - || config.client_config.state_sync_s3_region.is_empty() + if config.client_config.state_sync_s3_bucket.is_none() + || config.client_config.state_sync_s3_region.is_none() { - panic!("Enabled dumps of state to external storage. Please specify state_sync.s3_bucket and state_sync.s3_region"); + return Ok(None); } tracing::info!(target: "state_sync_dump", "Spawning the state sync dump loop"); // Create a connection to S3. - let s3_bucket = config.client_config.state_sync_s3_bucket.clone(); - let s3_region = config.client_config.state_sync_s3_region.clone(); - - // Credentials to establish a connection are taken from environment variables: AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY. + let s3_bucket = config.client_config.state_sync_s3_bucket.as_ref().unwrap(); + let s3_region = config.client_config.state_sync_s3_region.as_ref().unwrap(); let bucket = s3::Bucket::new( &s3_bucket, s3_region .parse::() .map_err(|err| >::into(err))?, s3::creds::Credentials::default().map_err(|err| { - tracing::error!(target: "state_sync_dump", "Failed to create a connection to S3. Did you provide environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY?"); >::into(err) })?, - ).map_err(|err| >::into(err))?; + ) + .map_err(|err| >::into(err))?; // Determine how many threads to start. - // TODO: Handle the case of changing the shard layout. + // Doesn't handle the case of changing the shard layout. let num_shards = { // Sadly, `Chain` is not `Send` and each thread needs to create its own `Chain` instance. let chain = Chain::new_for_view_client( runtime.clone(), - &chain_genesis, + chain_genesis, DoomslugThresholdMode::TwoThirds, - false, + config.client_config.save_trie_changes, )?; let epoch_id = chain.head()?.epoch_id; runtime.num_shards(&epoch_id) @@ -66,12 +57,13 @@ pub fn spawn_state_sync_dump( .map(|shard_id| { let client_config = config.client_config.clone(); let runtime = runtime.clone(); + let save_trie_changes = client_config.save_trie_changes; let chain_genesis = chain_genesis.clone(); let chain = Chain::new_for_view_client( runtime.clone(), &chain_genesis, DoomslugThresholdMode::TwoThirds, - false, + save_trie_changes, ) .unwrap(); let arbiter_handle = actix_rt::Arbiter::new().handle(); @@ -102,7 +94,7 @@ impl Drop for StateSyncDumpHandle { } impl StateSyncDumpHandle { - pub fn stop(&self) { + pub fn stop(&mut self) { let _: Vec = self.handles.iter().map(|handle| handle.stop()).collect(); } } @@ -116,16 +108,11 @@ async fn state_sync_dump( runtime: Arc, config: ClientConfig, bucket: s3::Bucket, - _node_key: PublicKey, + node_key: PublicKey, ) { tracing::info!(target: "state_sync_dump", shard_id, "Running StateSyncDump loop"); let mut interval = actix_rt::time::interval(std::time::Duration::from_secs(10)); - if config.state_sync_restart_dump_for_shards.contains(&shard_id) { - tracing::debug!(target: "state_sync_dump", shard_id, "Dropped existing progress"); - chain.store().set_state_sync_dump_progress(shard_id, None).unwrap(); - } - loop { // Avoid a busy-loop when there is nothing to do. interval.tick().await; @@ -134,26 +121,19 @@ async fn state_sync_dump( tracing::debug!(target: "state_sync_dump", shard_id, ?progress, "Running StateSyncDump loop iteration"); // The `match` returns the next state of the state machine. let next_state = match progress { - Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts })) => { + Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) => { // The latest epoch was dumped. Check if a newer epoch is available. - check_new_epoch( - Some(epoch_id), - Some(epoch_height), - num_parts, - shard_id, - &chain, - &runtime, - ) + check_new_epoch(Some(epoch_id), shard_id, &chain, &runtime, &config) } Err(Error::DBNotFoundErr(_)) | Ok(None) => { - // First invocation of this state-machine. See if at least one epoch is available for dumping. - check_new_epoch(None, None, None, shard_id, &chain, &runtime) + // First invokation of this state-machine. See if at least one epoch is available for dumping. + check_new_epoch(None, shard_id, &chain, &runtime, &config) } Err(err) => { // Something went wrong, let's retry. - tracing::warn!(target: "state_sync_dump", shard_id, ?err, "Failed to read the progress, will now delete and retry"); + tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to read the progress, delete and retry"); if let Err(err) = chain.store().set_state_sync_dump_progress(shard_id, None) { - tracing::warn!(target: "state_sync_dump", shard_id, ?err, "and failed to delete the progress. Will later retry."); + tracing::debug!(target: "state_sync_dump", shard_id, ?err, "And failed to delete it too :("); } Ok(None) } @@ -166,7 +146,7 @@ async fn state_sync_dump( num_parts, })) => { // The actual dumping of state to S3. - tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, %sync_hash, %state_root, parts_dumped, num_parts, "Creating parts and dumping them"); + tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, ?sync_hash, ?state_root, parts_dumped, num_parts, "Creating parts and dumping them"); let mut res = None; for part_id in parts_dumped..num_parts { // Dump parts sequentially synchronously. @@ -174,50 +154,86 @@ async fn state_sync_dump( let _timer = metrics::STATE_SYNC_DUMP_ITERATION_ELAPSED .with_label_values(&[&shard_id.to_string()]) .start_timer(); - - let state_part = match get_state_part( - &runtime, - &shard_id, - &sync_hash, - &state_root, - part_id, - num_parts, - &chain, - ) { + let state_part = { + let _timer = metrics::STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + runtime.obtain_state_part( + shard_id, + &sync_hash, + &state_root, + PartId::new(part_id, num_parts), + ) + }; + let state_part = match state_part { Ok(state_part) => state_part, Err(err) => { res = Some(err); break; } }; - let location = - s3_location(&config.chain_id, epoch_height, shard_id, part_id, num_parts); - if let Err(err) = - put_state_part(&location, &state_part, &shard_id, &bucket).await + let location = s3_location(&config.chain_id, epoch_height, shard_id, part_id); + { - res = Some(err); - break; + let _timer = metrics::STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + let put = bucket + .put_object(&location, &state_part) + .await + .map_err(|err| Error::Other(err.to_string())); + if let Err(err) = put { + res = Some(err); + break; + } + + // Optional, we probably don't need this. + let put = bucket + .put_object_tagging( + &location, + &[ + ("chain_id", &config.chain_id), + ("epoch_id", &format!("{:?}", epoch_id.0)), + ("epoch_height", &epoch_height.to_string()), + ("state_root", &format!("{:?}", state_root)), + ("sync_hash", &format!("{:?}", sync_hash)), + ("node_key", &format!("{:?}", node_key)), + ], + ) + .await + .map_err(|err| Error::Other(err.to_string())); + if let Err(err) = put { + res = Some(err); + break; + } } - update_progress( - &shard_id, - &epoch_id, + + // Record that a part was obtained and dumped. + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, ?sync_hash, ?state_root, part_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); + let next_progress = StateSyncDumpProgress::InProgress { + epoch_id: epoch_id.clone(), epoch_height, - &sync_hash, - &state_root, - part_id, + sync_hash, + state_root, + parts_dumped: part_id + 1, num_parts, - state_part.len(), - &chain, - ); + }; + match chain + .store() + .set_state_sync_dump_progress(shard_id, Some(next_progress.clone())) + { + Ok(_) => { + tracing::debug!(target: "state_sync_dump", shard_id, ?next_progress, "Updated dump progress"); + } + Err(err) => { + tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to update dump progress, continue"); + } + } } if let Some(err) = res { Err(err) } else { - Ok(Some(StateSyncDumpProgress::AllDumped { - epoch_id, - epoch_height, - num_parts: Some(num_parts), - })) + Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) } } }; @@ -246,117 +262,6 @@ async fn state_sync_dump( } } -async fn put_state_part( - location: &str, - state_part: &[u8], - shard_id: &ShardId, - bucket: &s3::Bucket, -) -> Result { - let _timer = metrics::STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - let put = bucket - .put_object(&location, &state_part) - .await - .map_err(|err| Error::Other(err.to_string())); - tracing::debug!(target: "state_sync_dump", shard_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); - put -} - -fn update_progress( - shard_id: &ShardId, - epoch_id: &EpochId, - epoch_height: EpochHeight, - sync_hash: &CryptoHash, - state_root: &StateRoot, - part_id: u64, - num_parts: u64, - part_len: usize, - chain: &Chain, -) { - // Record that a part was obtained and dumped. - metrics::STATE_SYNC_DUMP_SIZE_TOTAL - .with_label_values(&[&shard_id.to_string()]) - .inc_by(part_len as u64); - let next_progress = StateSyncDumpProgress::InProgress { - epoch_id: epoch_id.clone(), - epoch_height, - sync_hash: *sync_hash, - state_root: *state_root, - parts_dumped: part_id + 1, - num_parts, - }; - match chain.store().set_state_sync_dump_progress(*shard_id, Some(next_progress.clone())) { - Ok(_) => { - tracing::debug!(target: "state_sync_dump", shard_id, ?next_progress, "Updated dump progress"); - } - Err(err) => { - tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to update dump progress, continue"); - } - } - set_metrics(shard_id, Some(part_id + 1), Some(num_parts), Some(epoch_height)); -} - -fn set_metrics( - shard_id: &ShardId, - parts_dumped: Option, - num_parts: Option, - epoch_height: Option, -) { - if let Some(parts_dumped) = parts_dumped { - metrics::STATE_SYNC_DUMP_NUM_PARTS_DUMPED - .with_label_values(&[&shard_id.to_string()]) - .set(parts_dumped as i64); - } - if let Some(num_parts) = num_parts { - metrics::STATE_SYNC_DUMP_NUM_PARTS_TOTAL - .with_label_values(&[&shard_id.to_string()]) - .set(num_parts as i64); - } - if let Some(epoch_height) = epoch_height { - assert!( - epoch_height < 10000, - "Impossible: {:?} {:?} {:?} {:?}", - shard_id, - parts_dumped, - num_parts, - epoch_height - ); - metrics::STATE_SYNC_DUMP_EPOCH_HEIGHT - .with_label_values(&[&shard_id.to_string()]) - .set(epoch_height as i64); - } -} - -fn get_state_part( - runtime: &Arc, - shard_id: &ShardId, - sync_hash: &CryptoHash, - state_root: &StateRoot, - part_id: u64, - num_parts: u64, - chain: &Chain, -) -> Result, Error> { - let state_part = { - let _timer = metrics::STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - runtime.obtain_state_part( - *shard_id, - &sync_hash, - &state_root, - PartId::new(part_id, num_parts), - )? - }; - - // Save the part data. - let key = StatePartKey(*sync_hash, *shard_id, part_id).try_to_vec()?; - let mut store_update = chain.store().store().store_update(); - store_update.set(DBCol::StateParts, &key, &state_part); - store_update.commit()?; - Ok(state_part) -} - /// Gets basic information about the epoch to be dumped. fn start_dumping( epoch_id: EpochId, @@ -374,10 +279,9 @@ fn start_dumping( let state_root = sync_hash_block.chunks()[shard_id as usize].prev_state_root(); let state_root_node = runtime.get_state_root_node(shard_id, &sync_hash, &state_root)?; let num_parts = get_num_state_parts(state_root_node.memory_usage); - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, %state_root, num_parts, "Initialize dumping state of Epoch"); + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, ?state_root, num_parts, "Initialize dumping state of Epoch"); // Note that first the state of the state machines gets changes to // `InProgress` and it starts dumping state after a short interval. - set_metrics(&shard_id, Some(0), Some(num_parts), Some(epoch_height)); Ok(Some(StateSyncDumpProgress::InProgress { epoch_id, epoch_height, @@ -387,8 +291,8 @@ fn start_dumping( num_parts, })) } else { - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, "Shard is not tracked, skip the epoch"); - Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts: Some(0) })) + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, "Shard is not tracked, skip the epoch"); + Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) } } @@ -396,42 +300,35 @@ fn start_dumping( /// `epoch_id` represents the last fully dumped epoch. fn check_new_epoch( epoch_id: Option, - epoch_height: Option, - num_parts: Option, shard_id: ShardId, chain: &Chain, runtime: &Arc, + config: &ClientConfig, ) -> Result, Error> { let head = chain.head()?; if Some(&head.epoch_id) == epoch_id.as_ref() { - set_metrics(&shard_id, num_parts, num_parts, epoch_height); Ok(None) } else { - // Check if the final block is now in the next epoch. tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, "Check if a new complete epoch is available"); - let hash = head.last_block_hash; - let header = chain.get_block_header(&hash)?; - let final_hash = header.last_final_block(); - let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &final_hash)?; + let mut sync_hash = head.prev_block_hash; + // Step back a few blocks to avoid dealing with forks. + for _ in 0..config.state_fetch_horizon { + sync_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); + } + let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash)?; let header = chain.get_block_header(&sync_hash)?; if Some(header.epoch_id()) == epoch_id.as_ref() { // Still in the latest dumped epoch. Do nothing. Ok(None) } else { - start_dumping(head.epoch_id, sync_hash, shard_id, &chain, runtime) + start_dumping(head.epoch_id.clone(), sync_hash, shard_id, &chain, runtime) } } } -fn s3_location( - chain_id: &str, - epoch_height: u64, - shard_id: u64, - part_id: u64, - num_parts: u64, -) -> String { +fn s3_location(chain_id: &str, epoch_height: u64, shard_id: u64, part_id: u64) -> String { format!( - "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}_of_{:06}", - chain_id, epoch_height, shard_id, part_id, num_parts + "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}", + chain_id, epoch_height, shard_id, part_id ) } From 526225d96a3c36df1fb1f928c34998c7b408fc9e Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 15 Mar 2023 14:45:08 +0100 Subject: [PATCH 69/88] Fix DB corruption --- chain/client/src/client_actor.rs | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/chain/client/src/client_actor.rs b/chain/client/src/client_actor.rs index b2297bd0a3e..ef43582ed27 100644 --- a/chain/client/src/client_actor.rs +++ b/chain/client/src/client_actor.rs @@ -70,7 +70,7 @@ use std::sync::Arc; use std::thread; use std::time::{Duration, Instant}; use tokio::sync::broadcast; -use tracing::{debug, error, info, trace, warn}; +use tracing::{error, info, warn}; /// Multiplier on `max_block_time` to wait until deciding that chain stalled. const STATUS_WAIT_TIME_MULTIPLIER: u64 = 10; @@ -257,7 +257,7 @@ impl Actor for ClientActor { self.catchup(ctx); if let Err(err) = self.client.send_network_chain_info() { - error!(target: "client", ?err, "Failed to update network chain info"); + tracing::error!(target: "client", ?err, "Failed to update network chain info"); } } } @@ -501,7 +501,7 @@ impl Handler> for ClientActor { fn handle(&mut self, msg: WithSpanContext, ctx: &mut Context) { self.wrap(msg, ctx, "BlockApproval", |this, msg| { let BlockApproval(approval, peer_id) = msg; - debug!(target: "client", "Receive approval {:?} from peer {:?}", approval, peer_id); + tracing::debug!(target: "client", "Receive approval {:?} from peer {:?}", approval, peer_id); this.client.collect_block_approval(&approval, ApprovalType::PeerApproval(peer_id)); }) } @@ -519,7 +519,7 @@ impl Handler> for ClientActor { let hash = state_response_info.sync_hash(); let state_response = state_response_info.take_state_response(); - trace!(target: "sync", "Received state response shard_id: {} sync_hash: {:?} part(id/size): {:?}", + tracing::trace!(target: "sync", "Received state response shard_id: {} sync_hash: {:?} part(id/size): {:?}", shard_id, hash, state_response.part().as_ref().map(|(part_id, data)| (part_id, data.len())) @@ -820,7 +820,7 @@ impl ClientActor { fn check_send_announce_account(&mut self, prev_block_hash: CryptoHash) { // If no peers, there is no one to announce to. if self.network_info.num_connected_peers == 0 { - debug!(target: "client", "No peers: skip account announce"); + tracing::debug!(target: "client", "No peers: skip account announce"); return; } @@ -840,7 +840,7 @@ impl ClientActor { } } - debug!(target: "client", "Check announce account for {}, last announce time {:?}", validator_signer.validator_id(), self.last_validator_announce_time); + tracing::debug!(target: "client", "Check announce account for {}, last announce time {:?}", validator_signer.validator_id(), self.last_validator_announce_time); // Announce AccountId if client is becoming a validator soon. let next_epoch_id = unwrap_or_return!(self @@ -850,7 +850,7 @@ impl ClientActor { // Check client is part of the futures validators if self.client.is_validator(&next_epoch_id, &prev_block_hash) { - debug!(target: "client", "Sending announce account for {}", validator_signer.validator_id()); + tracing::debug!(target: "client", "Sending announce account for {}", validator_signer.validator_id()); self.last_validator_announce_time = Some(now); let signature = validator_signer.sign_account_announce( @@ -953,7 +953,7 @@ impl ClientActor { let _span = tracing::debug_span!(target: "client", "handle_block_production").entered(); // If syncing, don't try to produce blocks. if self.client.sync_status.is_syncing() { - debug!(target:"client", sync_status=?self.client.sync_status, "Syncing - block production disabled"); + tracing::debug!(target:"client", sync_status=?self.client.sync_status, "Syncing - block production disabled"); return Ok(()); } @@ -984,9 +984,9 @@ impl ClientActor { // We try to produce block for multiple heights (up to the highest height for which we've seen 2/3 of approvals). if latest_known.height + 1 <= self.client.doomslug.get_largest_height_crossing_threshold() { - debug!(target: "client", "Considering blocks for production between {} and {} ", latest_known.height + 1, self.client.doomslug.get_largest_height_crossing_threshold()); + tracing::debug!(target: "client", "Considering blocks for production between {} and {} ", latest_known.height + 1, self.client.doomslug.get_largest_height_crossing_threshold()); } else { - debug!(target: "client", "Cannot produce any block: not enough approvals beyond {}", latest_known.height); + tracing::debug!(target: "client", "Cannot produce any block: not enough approvals beyond {}", latest_known.height); } let me = if let Some(me) = &self.client.validator_signer { @@ -1336,7 +1336,7 @@ impl ClientActor { error!(target: "client", "Error processing sync blocks: {}", err); false } else { - debug!(target: "client", "Block headers refused by chain: {}", err); + tracing::debug!(target: "client", "Block headers refused by chain: {}", err); true } } @@ -1551,7 +1551,7 @@ impl ClientActor { if !self.needs_syncing(needs_syncing) { if currently_syncing { - debug!( + tracing::debug!( target: "client", "{:?} transitions to no sync", self.client.validator_signer.as_ref().map(|vs| vs.validator_id()), From f5d9545adfb54d4c27fdb9b4d19bebcb7dde8fc3 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 15 Mar 2023 16:35:29 +0100 Subject: [PATCH 70/88] Merge --- chain/chain/src/store.rs | 15 +- chain/client/src/sync/state.rs | 202 +++++++-------- core/chain-configs/src/client_config.rs | 21 +- core/primitives/src/syncing.rs | 19 +- nearcore/src/config.rs | 93 ++++++- nearcore/src/lib.rs | 58 ++++- nearcore/src/metrics.rs | 14 -- nearcore/src/state_sync.rs | 313 ++++++++++++++++-------- tools/state-viewer/src/cli.rs | 144 ++++++----- tools/state-viewer/src/state_parts.rs | 213 ++++------------ 10 files changed, 600 insertions(+), 492 deletions(-) diff --git a/chain/chain/src/store.rs b/chain/chain/src/store.rs index a59793cbd82..cbdc3a37ff2 100644 --- a/chain/chain/src/store.rs +++ b/chain/chain/src/store.rs @@ -842,6 +842,7 @@ impl ChainStore { /// Constructs key 'STATE_SYNC_DUMP:', /// for example 'STATE_SYNC_DUMP:2' for shard_id=2. + /// Doesn't contain epoch_id, because only one dump process per shard is allowed. fn state_sync_dump_progress_key(shard_id: ShardId) -> Vec { let mut key = b"STATE_SYNC_DUMP:".to_vec(); key.extend(shard_id.to_le_bytes()); @@ -1165,20 +1166,6 @@ impl ChainStoreAccess for ChainStore { } } -impl ChainAccessForFlatStorage for ChainStore { - fn get_block_info(&self, block_hash: &CryptoHash) -> BlockInfo { - let header = self.get_block_header(block_hash).unwrap(); - BlockInfo { hash: *block_hash, height: header.height(), prev_hash: *header.prev_hash() } - } - - fn get_block_hashes_at_height(&self, height: BlockHeight) -> HashSet { - match self.get_all_block_hashes_by_height(height) { - Ok(hashes) => hashes.values().flatten().copied().collect::>(), - Err(_) => Default::default(), - } - } -} - /// Cache update for ChainStore #[derive(Default)] struct ChainStoreCacheUpdate { diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 4b047deb393..e93952671f3 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -57,7 +57,6 @@ pub const MAX_STATE_PART_REQUEST: u64 = 16; /// This number should not exceed MAX_STATE_PART_REQUEST times (number of peers in the network). pub const MAX_PENDING_PART: u64 = MAX_STATE_PART_REQUEST * 10000; -#[derive(Debug)] pub enum StateSyncResult { /// No shard has changed its status Unchanged, @@ -93,7 +92,7 @@ fn make_account_or_peer_id_or_hash( From::AccountId(a) => To::AccountId(a), From::PeerId(p) => To::PeerId(p), From::Hash(h) => To::Hash(h), - From::ExternalStorage => To::ExternalStorage, + From::ExternalStorage() => To::ExternalStorage(), } } @@ -108,6 +107,8 @@ pub enum StateSyncMode { chain_id: String, /// Connection to the external storage. bucket: Arc, + /// Number state part requests allowed to be in-flight in parallel per shard. + num_s3_requests_per_shard: u64, }, } @@ -154,7 +155,7 @@ impl StateSync { state_sync_from_s3_enabled: bool, s3_bucket: &str, s3_region: &str, - num_concurrent_s3_requests: u64, + num_s3_requests_per_shard: u64, ) -> Self { let (mode, parts_request_state) = if state_sync_from_s3_enabled { tracing::debug!(target: "sync", s3_bucket, s3_region, "Initializing S3 bucket connection."); @@ -172,6 +173,7 @@ impl StateSync { StateSyncMode::HeaderFromPeersAndPartsFromExternal { chain_id: chain_id.to_string(), bucket, + num_s3_requests_per_shard, }, None, ) @@ -193,7 +195,7 @@ impl StateSync { timeout, state_parts_apply_results: HashMap::new(), split_state_roots: HashMap::new(), - requests_remaining: Arc::new(AtomicI64::new(num_concurrent_s3_requests as i64)), + requests_remaining: Arc::new(AtomicI64::new(num_s3_requests_per_shard as i64)), } } @@ -212,7 +214,7 @@ impl StateSync { target: "sync", %prev_hash, timeout_sec = self.timeout.num_seconds(), - "State sync: block request timed out"); + "State sync: block request timed"); (true, false) } else { (false, false) @@ -263,40 +265,36 @@ impl StateSync { for shard_id in tracking_shards { let mut download_timeout = false; - let mut run_shard_state_download = false; + let mut need_shard = false; let shard_sync_download = new_shard_sync.entry(shard_id).or_insert_with(|| { - run_shard_state_download = true; + need_shard = true; update_sync_status = true; ShardSyncDownload::new_download_state_header(now) }); let old_status = shard_sync_download.status.clone(); - let mut shard_sync_done = false; + let mut this_done = false; metrics::STATE_SYNC_STAGE .with_label_values(&[&shard_id.to_string()]) .set(shard_sync_download.status.repr() as i64); match &shard_sync_download.status { ShardSyncStatus::StateDownloadHeader => { - (download_timeout, run_shard_state_download) = self - .sync_shards_download_header_status( - shard_id, - shard_sync_download, - sync_hash, - chain, - now, - )?; + (download_timeout, need_shard) = self.sync_shards_download_header_status( + shard_id, + shard_sync_download, + sync_hash, + chain, + now, + )?; } ShardSyncStatus::StateDownloadParts => { - let res = self.sync_shards_download_parts_status( + (download_timeout, need_shard) = self.sync_shards_download_parts_status( shard_id, shard_sync_download, sync_hash, chain, now, ); - download_timeout = res.0; - run_shard_state_download = res.1; - update_sync_status |= res.2; } ShardSyncStatus::StateDownloadScheduling => { self.sync_shards_download_scheduling_status( @@ -318,7 +316,7 @@ impl StateSync { )?; } ShardSyncStatus::StateDownloadComplete => { - shard_sync_done = self.sync_shards_download_complete_status( + this_done = self.sync_shards_download_complete_status( split_states, shard_id, shard_sync_download, @@ -339,7 +337,7 @@ impl StateSync { } ShardSyncStatus::StateSplitApplying(_status) => { debug_assert!(split_states); - shard_sync_done = self.sync_shards_state_split_applying_status( + this_done = self.sync_shards_state_split_applying_status( shard_id, shard_sync_download, sync_hash, @@ -347,10 +345,10 @@ impl StateSync { )?; } ShardSyncStatus::StateSyncDone => { - shard_sync_done = true; + this_done = true; } } - all_done &= shard_sync_done; + all_done &= this_done; if download_timeout { tracing::warn!( @@ -358,7 +356,7 @@ impl StateSync { %shard_id, timeout_sec = self.timeout.num_seconds(), "State sync didn't download the state, sending StateRequest again"); - tracing::debug!( + tracing::info!( target: "sync", %shard_id, %sync_hash, @@ -368,7 +366,7 @@ impl StateSync { } // Execute syncing for shard `shard_id` - if run_shard_state_download { + if need_shard { update_sync_status = true; self.request_shard( me, @@ -668,7 +666,11 @@ impl StateSync { sync_hash, ); } - StateSyncMode::HeaderFromPeersAndPartsFromExternal { chain_id, bucket } => { + StateSyncMode::HeaderFromPeersAndPartsFromExternal { + chain_id, + bucket, + num_s3_requests_per_shard: _, + } => { self.request_part_from_external_storage( part_id as u64, download, @@ -699,21 +701,20 @@ impl StateSync { return; } else { if !download.run_me.swap(false, Ordering::SeqCst) { - tracing::info!(target: "sync", %shard_id, part_id, "External storage request is allowed but run_me is already false. Undoing"); - finished_request(&requests_remaining); + tracing::info!(target: "sync", %shard_id, part_id, "run_me is already false"); return; } } download.state_requests_count += 1; download.last_target = - Some(make_account_or_peer_id_or_hash(AccountOrPeerIdOrHash::ExternalStorage)); + Some(make_account_or_peer_id_or_hash(AccountOrPeerIdOrHash::ExternalStorage())); let location = s3_location(chain_id, epoch_height, shard_id, part_id, num_parts); let download_response = download.response.clone(); let scheduled = StaticClock::utc(); near_performance_metrics::actix::spawn(std::any::type_name::(), { async move { - tracing::debug!(target: "sync", %shard_id, part_id, location, "Getting an object from the external storage"); + tracing::info!(target: "sync", %shard_id, part_id, location, "Getting an object from the external storage"); let started = StaticClock::utc(); metrics::STATE_SYNC_EXTERNAL_PARTS_SCHEDULING_DELAY .with_label_values(&[&shard_id.to_string()]) @@ -734,12 +735,12 @@ impl StateSync { ); match result { Ok(response) => { - tracing::debug!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), "S3 request finished"); + tracing::info!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), "S3 request finished"); let mut lock = download_response.lock().unwrap(); *lock = Some(Ok((response.status_code(), response.bytes().to_vec()))); } Err(err) => { - tracing::debug!(target: "sync", %shard_id, part_id, location, ?err, "S3 request failed"); + tracing::info!(target: "sync", %shard_id, part_id, location, ?err, "S3 request failed"); let mut lock = download_response.lock().unwrap(); *lock = Some(Err(err.to_string())); } @@ -803,7 +804,7 @@ impl StateSync { use_colour: bool, ) -> Result { let _span = tracing::debug_span!(target: "sync", "run", sync = "StateSync").entered(); - tracing::trace!(target: "sync", %sync_hash, ?tracking_shards, "syncing state"); + tracing::debug!(target: "sync", %sync_hash, ?tracking_shards, "syncing state"); let prev_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); let now = StaticClock::utc(); @@ -914,11 +915,8 @@ impl StateSync { } /// Checks if the header is downloaded. - /// If the download is complete, then moves forward to `StateDownloadParts`, - /// otherwise retries the header request. - /// Returns `(download_timeout, run_shard_state_download)` where: - /// * `download_timeout` means that the state header request timed out (and needs to be retried). - /// * `run_shard_state_download` means that header or part download requests need to run for this shard. + /// If the download is complete, then moves forward to StateDownloadParts, otherwise retries the header request. + /// Returns (download_timeout, need_shard). fn sync_shards_download_header_status( &mut self, shard_id: ShardId, @@ -928,17 +926,17 @@ impl StateSync { now: DateTime, ) -> Result<(bool, bool), near_chain::Error> { let mut download_timeout = false; - let mut run_shard_state_download = false; + let mut need_shard = false; // StateDownloadHeader is the first step. We want to fetch the basic information about the state (its size, hash etc). if shard_sync_download.downloads[0].done { - let shard_state_header = chain.get_state_header(shard_id, sync_hash)?; + let shard_state_header = chain.get_state_header(shard_id.clone(), sync_hash)?; let state_num_parts = get_num_state_parts(shard_state_header.state_root_node().memory_usage); // If the header was downloaded successfully - move to phase 2 (downloading parts). // Create the vector with entry for each part. *shard_sync_download = ShardSyncDownload::new_download_state_parts(now, state_num_parts); - run_shard_state_download = true; + need_shard = true; } else { let prev = shard_sync_download.downloads[0].prev_update_time; let error = shard_sync_download.downloads[0].error; @@ -950,18 +948,16 @@ impl StateSync { shard_sync_download.downloads[0].prev_update_time = now; } if shard_sync_download.downloads[0].run_me.load(Ordering::SeqCst) { - run_shard_state_download = true; + need_shard = true; } } - Ok((download_timeout, run_shard_state_download)) + Ok((download_timeout, need_shard)) } /// Checks if the parts are downloaded. - /// If download of all parts is complete, then moves forward to `StateDownloadScheduling`. - /// Returns `(download_timeout, run_shard_state_download, update_sync_status)` where: - /// * `download_timeout` means that the state header request timed out (and needs to be retried). - /// * `run_shard_state_download` means that header or part download requests need to run for this shard. - /// * `update_sync_status` means that something changed in `ShardSyncDownload` and it needs to be persisted. + /// If download of all parts is complete, then moves forward to StateDownloadScheduling. + /// Otherwise, retries the failed part downloads. + /// Returns (download_timeout, need_shard). fn sync_shards_download_parts_status( &mut self, shard_id: ShardId, @@ -969,19 +965,19 @@ impl StateSync { sync_hash: CryptoHash, chain: &mut Chain, now: DateTime, - ) -> (bool, bool, bool) { + ) -> (bool, bool) { // Step 2 - download all the parts (each part is usually around 1MB). let mut download_timeout = false; - let mut run_shard_state_download = false; - let mut update_sync_status = false; + let mut need_shard = false; let mut parts_done = true; let num_parts = shard_sync_download.downloads.len(); let mut num_parts_done = 0; for (part_id, part_download) in shard_sync_download.downloads.iter_mut().enumerate() { + tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, part_download.done, part_download.error, ?part_download); if !part_download.done { // Check if a download from an external storage is finished. - update_sync_status |= check_external_storage_part_response( + check_external_storage_part_response( part_id as u64, num_parts as u64, shard_id, @@ -990,35 +986,29 @@ impl StateSync { chain, ); } + tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, part_download.done, part_download.error); if !part_download.done { parts_done = false; let prev = part_download.prev_update_time; let part_timeout = now - prev > self.timeout; // Retry parts that failed. if part_timeout || part_download.error { + metrics::STATE_SYNC_RETRY_PART + .with_label_values(&[&shard_id.to_string()]) + .inc(); download_timeout |= part_timeout; - if part_timeout || - part_download.last_target != Some(near_client_primitives::types::AccountOrPeerIdOrHash::ExternalStorage) { - // Don't immediately retry failed requests from external - // storage. Most often error is a state part not - // available. That error doesn't get fixed by retrying, - // but rather by waiting. - metrics::STATE_SYNC_RETRY_PART - .with_label_values(&[&shard_id.to_string()]) - .inc(); - part_download.run_me.store(true, Ordering::SeqCst); - part_download.error = false; - part_download.prev_update_time = now; - update_sync_status = true; - } + part_download.run_me.store(true, Ordering::SeqCst); + part_download.error = false; + part_download.prev_update_time = now; } if part_download.run_me.load(Ordering::SeqCst) { - run_shard_state_download = true; + need_shard = true; } } if part_download.done { num_parts_done += 1; } } + tracing::debug!(target: "sync", %shard_id, %sync_hash, num_parts_done, parts_done); metrics::STATE_SYNC_PARTS_DONE .with_label_values(&[&shard_id.to_string()]) .set(num_parts_done); @@ -1031,9 +1021,8 @@ impl StateSync { downloads: vec![], status: ShardSyncStatus::StateDownloadScheduling, }; - update_sync_status = true; } - (download_timeout, run_shard_state_download, update_sync_status) + (download_timeout, need_shard) } fn sync_shards_download_scheduling_status( @@ -1125,7 +1114,7 @@ impl StateSync { get_num_state_parts(shard_state_header.state_root_node().memory_usage); chain.clear_downloaded_parts(shard_id, sync_hash, state_num_parts)?; - let mut shard_sync_done = false; + let mut this_done = false; // If the shard layout is changing in this epoch - we have to apply it right now. if split_states { *shard_sync_download = ShardSyncDownload { @@ -1136,9 +1125,9 @@ impl StateSync { // If there is no layout change - we're done. *shard_sync_download = ShardSyncDownload { downloads: vec![], status: ShardSyncStatus::StateSyncDone }; - shard_sync_done = true; + this_done = true; } - Ok(shard_sync_done) + Ok(this_done) } fn sync_shards_state_split_scheduling_status( @@ -1165,7 +1154,7 @@ impl StateSync { Ok(()) } - /// Returns whether the State Sync for the given shard is complete. + /// Returns `this_done`. fn sync_shards_state_split_applying_status( &mut self, shard_id: ShardId, @@ -1174,29 +1163,27 @@ impl StateSync { chain: &mut Chain, ) -> Result { let result = self.split_state_roots.remove(&shard_id); - let mut shard_sync_done = false; + let mut this_done = false; if let Some(state_roots) = result { chain.build_state_for_split_shards_postprocessing(&sync_hash, state_roots)?; *shard_sync_download = ShardSyncDownload { downloads: vec![], status: ShardSyncStatus::StateSyncDone }; - shard_sync_done = true; + this_done = true; } - Ok(shard_sync_done) + Ok(this_done) } } -/// Verifies that one more concurrent request can be started. fn allow_request(requests_remaining: &AtomicI64) -> bool { let remaining = requests_remaining.fetch_sub(1, Ordering::SeqCst); - if remaining >= 0 { - true - } else { + if remaining <= 0 { requests_remaining.fetch_add(1, Ordering::SeqCst); false + } else { + true } } -/// Notify about a request finishing. fn finished_request(requests_remaining: &AtomicI64) { requests_remaining.fetch_add(1, Ordering::SeqCst); } @@ -1205,8 +1192,6 @@ fn finished_request(requests_remaining: &AtomicI64) { /// The response is stored on the DownloadStatus object. /// This function investigates if the response is available and updates `done` and `error` appropriately. /// If the response is successful, then also writes the state part to the DB. -/// -/// Returns whether something changed in `DownloadStatus` which means it needs to be persisted. fn check_external_storage_part_response( part_id: u64, num_parts: u64, @@ -1214,18 +1199,18 @@ fn check_external_storage_part_response( sync_hash: CryptoHash, part_download: &mut DownloadStatus, chain: &mut Chain, -) -> bool { +) { let external_storage_response = { let mut lock = part_download.response.lock().unwrap(); if let Some(response) = lock.clone() { tracing::debug!(target: "sync", %shard_id, part_id, "Got response from external storage"); // Remove the response from DownloadStatus, because - // we're going to write state parts to DB and don't need to keep - // them in `DownloadStatus`. + // we're going to write positive responses to the DB + // and retry negative responses. *lock = None; response } else { - return false; + return; } }; @@ -1244,38 +1229,32 @@ fn check_external_storage_part_response( metrics::STATE_SYNC_EXTERNAL_PARTS_DONE .with_label_values(&[&shard_id.to_string()]) .inc(); - metrics::STATE_SYNC_EXTERNAL_PARTS_SIZE_DOWNLOADED - .with_label_values(&[&shard_id.to_string()]) - .inc_by(data.len() as u64); part_download.done = true; - tracing::debug!(target: "sync", %shard_id, part_id, ?part_download, "Set state part success"); } Err(err) => { metrics::STATE_SYNC_EXTERNAL_PARTS_FAILED .with_label_values(&[&shard_id.to_string()]) .inc(); tracing::warn!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to save a state part"); - err_to_retry = - Some(near_chain::Error::Other("Failed to save a state part".to_string())); + err_to_retry = Some(Error::Other("Failed to save a state part".to_string())); } } } // Other HTTP status codes are considered errors. Ok((status_code, _)) => { - tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, status_code, "Wrong response code, expected 200"); - err_to_retry = Some(near_chain::Error::Other(format!("status_code: {}", status_code))); + err_to_retry = Some(Error::Other(format!("status_code: {}", status_code).to_string())); } // The request failed without reaching the external storage. Err(err) => { - err_to_retry = Some(near_chain::Error::Other(err)); + err_to_retry = Some(Error::Other(err.to_string())); } }; if let Some(err) = err_to_retry { tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to get a part from external storage, will retry"); part_download.error = true; + } else { } - true } /// Construct a location on the external storage. @@ -1292,10 +1271,9 @@ pub fn s3_location( ) } -/// Applies style if `use_colour` is enabled. -fn paint(s: &str, style: Style, use_style: bool) -> String { - if use_style { - style.paint(s).to_string() +fn paint(s: &str, colour: Style, use_colour: bool) -> String { + if use_colour { + colour.paint(s).to_string() } else { s.to_string() } @@ -1311,15 +1289,8 @@ fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload, use_colour: shard_sync_download.downloads[0].last_target ), ShardSyncStatus::StateDownloadParts => { - let mut num_parts_done = 0; - let mut num_parts_not_done = 0; let mut text = "".to_string(); for (i, download) in shard_sync_download.downloads.iter().enumerate() { - if download.done { - num_parts_done += 1; - continue; - } - num_parts_not_done += 1; text.push_str(&format!( "[{}: {}, {}, {:?}] ", paint(&i.to_string(), Yellow.bold(), use_colour), @@ -1329,12 +1300,10 @@ fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload, use_colour: )); } format!( - "{} [{}: is_done, requests sent, last target] {} num_parts_done={} num_parts_not_done={}", + "{} [{}: is_done, requests sent, last target] {}", paint("PARTS", Purple.bold(), use_colour), paint("part_id", Yellow.bold(), use_colour), - text, - num_parts_done, - num_parts_not_done + text ) } _ => unreachable!("timeout cannot happen when all state is downloaded"), @@ -1429,10 +1398,10 @@ mod test { let mut state_sync = StateSync::new( mock_peer_manager.clone().into(), TimeDuration::from_secs(1), - "chain_id", + "chain_id".to_string(), false, - "", - "", + "".to_string(), + "".to_string(), 100, ); let mut new_shard_sync = HashMap::new(); @@ -1484,7 +1453,6 @@ mod test { vec![0], &apply_parts_fn, &state_split_fn, - false, ) .unwrap(); diff --git a/core/chain-configs/src/client_config.rs b/core/chain-configs/src/client_config.rs index 202363eb6a6..c71ae93a895 100644 --- a/core/chain-configs/src/client_config.rs +++ b/core/chain-configs/src/client_config.rs @@ -166,10 +166,20 @@ pub struct ClientConfig { pub client_background_migration_threads: usize, /// Duration to perform background flat storage creation step. pub flat_storage_creation_period: Duration, + /// If enabled, will dump state of every epoch to external storage. + pub state_sync_dump_enabled: bool, /// S3 bucket for storing state dumps. - pub state_sync_s3_bucket: Option, + pub state_sync_s3_bucket: String, /// S3 region for storing state dumps. - pub state_sync_s3_region: Option, + pub state_sync_s3_region: String, + /// Restart dumping state of selected shards. + /// Use for troubleshooting of the state dumping process. + pub state_sync_restart_dump_for_shards: Vec, + /// Whether to enable state sync from S3. + /// If disabled will perform state sync from the peers. + pub state_sync_from_s3_enabled: bool, + /// Number of parallel in-flight requests allowed per shard. + pub state_sync_num_s3_requests_per_shard: u64, } impl ClientConfig { @@ -239,8 +249,11 @@ impl ClientConfig { enable_statistics_export: true, client_background_migration_threads: 1, flat_storage_creation_period: Duration::from_secs(1), - state_sync_s3_bucket: None, - state_sync_s3_region: None, + state_sync_dump_enabled: false, + state_sync_s3_bucket: String::new(), + state_sync_s3_region: String::new(), + state_sync_restart_dump_for_shards: vec![], + state_sync_num_s3_requests_per_shard: 10, } } } diff --git a/core/primitives/src/syncing.rs b/core/primitives/src/syncing.rs index c948fd7a895..4dc68f3b7a6 100644 --- a/core/primitives/src/syncing.rs +++ b/core/primitives/src/syncing.rs @@ -229,14 +229,29 @@ pub fn get_num_state_parts(memory_usage: u64) -> u64 { } #[derive(BorshSerialize, BorshDeserialize, Debug, Clone)] -/// Represents the state of the state machine that dumps state. +/// Represents the progress of dumps state of a shard. pub enum StateSyncDumpProgress { - AllDumped(EpochId), + /// Represents two cases: + /// * An epoch dump is complete + /// * The node is running its first epoch and there is nothing to dump. + AllDumped { + /// The dumped state corresponds to the state at the beginning of the specified epoch. + epoch_id: EpochId, + epoch_height: EpochHeight, + // Missing in case of a node running the first epoch. + num_parts: Option, + }, + /// Represents the case of an epoch being partially dumped. InProgress { + /// The dumped state corresponds to the state at the beginning of the specified epoch. epoch_id: EpochId, epoch_height: EpochHeight, + /// Block hash of the first block of the epoch. + /// The dumped state corresponds to the state before applying this block. sync_hash: CryptoHash, + /// Root of the state being dumped. state_root: StateRoot, + /// Progress made. parts_dumped: u64, num_parts: u64, }, diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index 456ec1cee17..2f61744bd48 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -337,7 +337,9 @@ pub struct Config { /// This feature is under development, do not use in production. #[serde(default, skip_serializing_if = "Option::is_none")] pub cold_store: Option, - + /// Configuration for the + #[serde(default, skip_serializing_if = "Option::is_none")] + pub split_storage: Option, // TODO(mina86): Remove those two altogether at some point. We need to be // somewhat careful though and make sure that we don’t start silently // ignoring this option without users setting corresponding store option. @@ -352,6 +354,7 @@ pub struct Config { #[serde(default, skip_serializing_if = "Option::is_none")] pub expected_shutdown: Option, /// Options for dumping state of every epoch to S3. + #[serde(skip_serializing_if = "Option::is_none")] pub state_sync: Option, } @@ -387,12 +390,56 @@ impl Default for Config { use_db_migration_snapshot: None, store: near_store::StoreConfig::default(), cold_store: None, + split_storage: None, expected_shutdown: None, state_sync: None, } } } +fn default_enable_split_storage_view_client() -> bool { + false +} + +fn default_cold_store_initial_migration_batch_size() -> usize { + 500_000_000 +} + +fn default_cold_store_initial_migration_loop_sleep_duration() -> Duration { + Duration::from_secs(30) +} + +fn default_cold_store_loop_sleep_duration() -> Duration { + Duration::from_secs(1) +} + +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] +pub struct SplitStorageConfig { + #[serde(default = "default_enable_split_storage_view_client")] + pub enable_split_storage_view_client: bool, + + #[serde(default = "default_cold_store_initial_migration_batch_size")] + pub cold_store_initial_migration_batch_size: usize, + #[serde(default = "default_cold_store_initial_migration_loop_sleep_duration")] + pub cold_store_initial_migration_loop_sleep_duration: Duration, + + #[serde(default = "default_cold_store_loop_sleep_duration")] + pub cold_store_loop_sleep_duration: Duration, +} + +impl Default for SplitStorageConfig { + fn default() -> Self { + SplitStorageConfig { + enable_split_storage_view_client: default_enable_split_storage_view_client(), + cold_store_initial_migration_batch_size: + default_cold_store_initial_migration_batch_size(), + cold_store_initial_migration_loop_sleep_duration: + default_cold_store_initial_migration_loop_sleep_duration(), + cold_store_loop_sleep_duration: default_cold_store_loop_sleep_duration(), + } + } +} + impl Config { /// load Config from config.json without panic. Do semantic validation on field values. /// If config file issues occur, a ValidationError::ConfigFileError will be returned; @@ -654,14 +701,39 @@ impl NearConfig { enable_statistics_export: config.store.enable_statistics_export, client_background_migration_threads: config.store.background_migration_threads, flat_storage_creation_period: config.store.flat_storage_creation_period, + state_sync_dump_enabled: config + .state_sync + .as_ref() + .map(|x| x.dump_enabled) + .unwrap_or(false), state_sync_s3_bucket: config .state_sync .as_ref() - .map_or(None, |x| Some(x.s3_bucket.clone())), + .map(|x| x.s3_bucket.clone()) + .unwrap_or(String::new()), state_sync_s3_region: config .state_sync .as_ref() - .map_or(None, |x| Some(x.s3_region.clone())), + .map(|x| x.s3_region.clone()) + .unwrap_or(String::new()), + state_sync_restart_dump_for_shards: config + .state_sync + .as_ref() + .map(|x| x.drop_state_of_dump.clone()) + .flatten() + .unwrap_or(vec![]), + state_sync_from_s3_enabled: config + .state_sync + .as_ref() + .map(|x| x.sync_from_s3_enabled) + .flatten() + .unwrap_or(false), + state_sync_num_s3_requests_per_shard: config + .state_sync + .as_ref() + .map(|x| x.num_s3_requests_per_shard) + .flatten() + .unwrap_or(100), }, network_config: NetworkConfig::new( config.network, @@ -1485,11 +1557,24 @@ pub fn load_test_config(seed: &str, addr: tcp::ListenerAddr, genesis: Genesis) - NearConfig::new(config, genesis, signer.into(), validator_signer).unwrap() } -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug)] +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default)] /// Options for dumping state to S3. pub struct StateSyncConfig { + /// Location of state dumps on S3. pub s3_bucket: String, + /// Region is very important on S3. pub s3_region: String, + /// Whether a node should dump state of each epoch to the external storage. + #[serde(skip_serializing_if = "Option::is_none")] + pub dump_enabled: Option, + /// Use carefully in case a node that dumps state to the external storage gets in trouble. + #[serde(skip_serializing_if = "Option::is_none")] + pub drop_state_of_dump: Option>, + /// If enabled, will download state parts from external storage and not from the peers. + #[serde(skip_serializing_if = "Option::is_none")] + pub sync_from_s3_enabled: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub num_s3_requests_per_shard: Option, } #[test] diff --git a/nearcore/src/lib.rs b/nearcore/src/lib.rs index 7ea9b4f5eed..3da5e6a2766 100644 --- a/nearcore/src/lib.rs +++ b/nearcore/src/lib.rs @@ -16,7 +16,8 @@ use near_client::{start_client, start_view_client, ClientActor, ConfigUpdater, V use near_network::PeerManagerActor; use near_primitives::block::GenesisId; use near_primitives::time; -use near_store::{DBCol, Mode, NodeStorage, StoreOpenerError, Temperature}; +use near_store::metadata::DbKind; +use near_store::{DBCol, Mode, NodeStorage, Store, StoreOpenerError}; use near_telemetry::TelemetryActor; use std::path::{Path, PathBuf}; use std::sync::Arc; @@ -154,6 +155,33 @@ fn open_storage(home_dir: &Path, near_config: &mut NearConfig) -> anyhow::Result Ok(storage) } +// Safely get the split store while checking that all conditions to use it are met. +fn get_split_store(config: &NearConfig, storage: &NodeStorage) -> anyhow::Result> { + // SplitStore should only be used on archival nodes. + if !config.config.archive { + return Ok(None); + } + + // SplitStore should only be used if cold store is configured. + if config.config.cold_store.is_none() { + return Ok(None); + } + + // SplitStore should only be used in the view client if it is enabled. + if !config.config.split_storage.as_ref().map_or(false, |c| c.enable_split_storage_view_client) { + return Ok(None); + } + + // SplitStore should only be used if the migration is finished. The + // migration to cold store is finished when the db kind of the hot store is + // changed from Archive to Hot. + if storage.get_hot_store().get_db_kind()? != Some(DbKind::Hot) { + return Ok(None); + } + + Ok(storage.get_split_store()) +} + pub struct NearNode { pub client: Addr, pub view_client: Addr, @@ -180,11 +208,17 @@ pub fn start_with_config_and_synchronization( ) -> anyhow::Result { let store = open_storage(home_dir, &mut config)?; - let runtime = Arc::new(NightshadeRuntime::from_config( - home_dir, - store.get_store(Temperature::Hot), - &config, - )); + let runtime = + Arc::new(NightshadeRuntime::from_config(home_dir, store.get_hot_store(), &config)); + + // Get the split store. If split store is some then create a new runtime for + // the view client. Otherwise just re-use the existing runtime. + let split_store = get_split_store(&config, &store)?; + let view_runtime = if let Some(split_store) = split_store { + Arc::new(NightshadeRuntime::from_config(home_dir, split_store, &config)) + } else { + runtime.clone() + }; let cold_store_loop_handle = spawn_cold_store_loop(&config, &store, runtime.clone())?; @@ -205,7 +239,7 @@ pub fn start_with_config_and_synchronization( let view_client = start_view_client( config.validator_signer.as_ref().map(|signer| signer.validator_id().clone()), chain_genesis.clone(), - runtime.clone(), + view_runtime, network_adapter.clone().into(), config.client_config.clone(), adv.clone(), @@ -229,15 +263,15 @@ pub fn start_with_config_and_synchronization( network_adapter.as_sender(), client_adapter_for_shards_manager.as_sender(), config.validator_signer.as_ref().map(|signer| signer.validator_id().clone()), - store.get_store(Temperature::Hot), + store.get_hot_store(), config.client_config.chunk_request_retry_period, ); shards_manager_adapter.bind(shards_manager_actor); let state_sync_dump_handle = spawn_state_sync_dump( &config, - &chain_genesis.clone(), - runtime.clone(), + chain_genesis, + runtime, config.network_config.node_id().public_key(), )?; @@ -334,7 +368,7 @@ pub fn recompress_storage(home_dir: &Path, opts: RecompressOpts) -> anyhow::Resu "Recompressing database"); info!("Opening database at {}", src_path.display()); - let src_store = src_opener.open_in_mode(Mode::ReadOnly)?.get_store(Temperature::Hot); + let src_store = src_opener.open_in_mode(Mode::ReadOnly)?.get_hot_store(); let final_head_height = if skip_columns.contains(&DBCol::PartialChunks) { let tip: Option = @@ -351,7 +385,7 @@ pub fn recompress_storage(home_dir: &Path, opts: RecompressOpts) -> anyhow::Resu }; info!("Creating database at {}", dst_path.display()); - let dst_store = dst_opener.open_in_mode(Mode::Create)?.get_store(Temperature::Hot); + let dst_store = dst_opener.open_in_mode(Mode::Create)?.get_hot_store(); const BATCH_SIZE_BYTES: u64 = 150_000_000; diff --git a/nearcore/src/metrics.rs b/nearcore/src/metrics.rs index 2983683b16e..9b5d99bf01f 100644 --- a/nearcore/src/metrics.rs +++ b/nearcore/src/metrics.rs @@ -14,20 +14,6 @@ pub static APPLY_CHUNK_DELAY: Lazy = Lazy::new(|| { .unwrap() }); -pub static SECONDS_PER_PETAGAS: Lazy = Lazy::new(|| { - try_create_histogram_vec( - "near_execution_seconds_per_petagas_ratio", - "Execution time per unit of gas, measured in seconds per petagas. Ignore label 'label'.", - &["shard_id"], - // Non-linear buckets with higher resolution around 1.0. - Some(vec![ - 0.0, 0.1, 0.2, 0.5, 0.7, 0.8, 0.9, 0.95, 0.97, 0.99, 1.0, 1.01, 1.03, 1.05, 1.1, 1.2, - 1.3, 1.5, 2.0, 5.0, 10.0, - ]), - ) - .unwrap() -}); - pub(crate) static CONFIG_CORRECT: Lazy = Lazy::new(|| { try_create_int_gauge( "near_config_correct", diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 44483f28c5c..458bff4e27e 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -1,52 +1,61 @@ use crate::{metrics, NearConfig, NightshadeRuntime}; +use borsh::BorshSerialize; use near_chain::types::RuntimeAdapter; -use near_chain::{Chain, ChainGenesis, DoomslugThresholdMode, Error}; +use near_chain::{Chain, ChainGenesis, ChainStoreAccess, DoomslugThresholdMode, Error}; use near_chain_configs::ClientConfig; use near_client::sync::state::StateSync; use near_crypto::PublicKey; use near_epoch_manager::EpochManagerAdapter; use near_primitives::hash::CryptoHash; use near_primitives::state_part::PartId; -use near_primitives::syncing::{get_num_state_parts, StateSyncDumpProgress}; -use near_primitives::types::{EpochId, ShardId}; +use near_primitives::syncing::{get_num_state_parts, StatePartKey, StateSyncDumpProgress}; +use near_primitives::types::{EpochHeight, EpochId, ShardId, StateRoot}; +use near_store::DBCol; use std::sync::Arc; +/// Starts one a thread per tracked shard. +/// Each started thread will be dumping state parts of a single epoch to external storage. pub fn spawn_state_sync_dump( config: &NearConfig, - chain_genesis: &ChainGenesis, + chain_genesis: ChainGenesis, runtime: Arc, node_key: &PublicKey, ) -> anyhow::Result> { - if config.client_config.state_sync_s3_bucket.is_none() - || config.client_config.state_sync_s3_region.is_none() - { + if !config.client_config.state_sync_dump_enabled { return Ok(None); } + if config.client_config.state_sync_s3_bucket.is_empty() + || config.client_config.state_sync_s3_region.is_empty() + { + panic!("Enabled dumps of state to external storage. Please specify state_sync.s3_bucket and state_sync.s3_region"); + } tracing::info!(target: "state_sync_dump", "Spawning the state sync dump loop"); // Create a connection to S3. - let s3_bucket = config.client_config.state_sync_s3_bucket.as_ref().unwrap(); - let s3_region = config.client_config.state_sync_s3_region.as_ref().unwrap(); + let s3_bucket = config.client_config.state_sync_s3_bucket.clone(); + let s3_region = config.client_config.state_sync_s3_region.clone(); + + // Credentials to establish a connection are taken from environment variables: AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY. let bucket = s3::Bucket::new( &s3_bucket, s3_region .parse::() .map_err(|err| >::into(err))?, s3::creds::Credentials::default().map_err(|err| { + tracing::error!(target: "state_sync_dump", "Failed to create a connection to S3. Did you provide environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY?"); >::into(err) })?, - ) - .map_err(|err| >::into(err))?; + ).map_err(|err| >::into(err))?; // Determine how many threads to start. - // Doesn't handle the case of changing the shard layout. + // TODO: Handle the case of changing the shard layout. let num_shards = { // Sadly, `Chain` is not `Send` and each thread needs to create its own `Chain` instance. let chain = Chain::new_for_view_client( runtime.clone(), - chain_genesis, + &chain_genesis, DoomslugThresholdMode::TwoThirds, - config.client_config.save_trie_changes, + false, )?; let epoch_id = chain.head()?.epoch_id; runtime.num_shards(&epoch_id) @@ -57,13 +66,12 @@ pub fn spawn_state_sync_dump( .map(|shard_id| { let client_config = config.client_config.clone(); let runtime = runtime.clone(); - let save_trie_changes = client_config.save_trie_changes; let chain_genesis = chain_genesis.clone(); let chain = Chain::new_for_view_client( runtime.clone(), &chain_genesis, DoomslugThresholdMode::TwoThirds, - save_trie_changes, + false, ) .unwrap(); let arbiter_handle = actix_rt::Arbiter::new().handle(); @@ -94,7 +102,7 @@ impl Drop for StateSyncDumpHandle { } impl StateSyncDumpHandle { - pub fn stop(&mut self) { + pub fn stop(&self) { let _: Vec = self.handles.iter().map(|handle| handle.stop()).collect(); } } @@ -108,11 +116,16 @@ async fn state_sync_dump( runtime: Arc, config: ClientConfig, bucket: s3::Bucket, - node_key: PublicKey, + _node_key: PublicKey, ) { tracing::info!(target: "state_sync_dump", shard_id, "Running StateSyncDump loop"); let mut interval = actix_rt::time::interval(std::time::Duration::from_secs(10)); + if config.state_sync_restart_dump_for_shards.contains(&shard_id) { + tracing::debug!(target: "state_sync_dump", shard_id, "Dropped existing progress"); + chain.store().set_state_sync_dump_progress(shard_id, None).unwrap(); + } + loop { // Avoid a busy-loop when there is nothing to do. interval.tick().await; @@ -121,19 +134,26 @@ async fn state_sync_dump( tracing::debug!(target: "state_sync_dump", shard_id, ?progress, "Running StateSyncDump loop iteration"); // The `match` returns the next state of the state machine. let next_state = match progress { - Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) => { + Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts })) => { // The latest epoch was dumped. Check if a newer epoch is available. - check_new_epoch(Some(epoch_id), shard_id, &chain, &runtime, &config) + check_new_epoch( + Some(epoch_id), + Some(epoch_height), + num_parts, + shard_id, + &chain, + &runtime, + ) } Err(Error::DBNotFoundErr(_)) | Ok(None) => { - // First invokation of this state-machine. See if at least one epoch is available for dumping. - check_new_epoch(None, shard_id, &chain, &runtime, &config) + // First invocation of this state-machine. See if at least one epoch is available for dumping. + check_new_epoch(None, None, None, shard_id, &chain, &runtime) } Err(err) => { // Something went wrong, let's retry. - tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to read the progress, delete and retry"); + tracing::warn!(target: "state_sync_dump", shard_id, ?err, "Failed to read the progress, will now delete and retry"); if let Err(err) = chain.store().set_state_sync_dump_progress(shard_id, None) { - tracing::debug!(target: "state_sync_dump", shard_id, ?err, "And failed to delete it too :("); + tracing::warn!(target: "state_sync_dump", shard_id, ?err, "and failed to delete the progress. Will later retry."); } Ok(None) } @@ -146,7 +166,7 @@ async fn state_sync_dump( num_parts, })) => { // The actual dumping of state to S3. - tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, ?sync_hash, ?state_root, parts_dumped, num_parts, "Creating parts and dumping them"); + tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, %sync_hash, %state_root, parts_dumped, num_parts, "Creating parts and dumping them"); let mut res = None; for part_id in parts_dumped..num_parts { // Dump parts sequentially synchronously. @@ -154,86 +174,50 @@ async fn state_sync_dump( let _timer = metrics::STATE_SYNC_DUMP_ITERATION_ELAPSED .with_label_values(&[&shard_id.to_string()]) .start_timer(); - let state_part = { - let _timer = metrics::STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - runtime.obtain_state_part( - shard_id, - &sync_hash, - &state_root, - PartId::new(part_id, num_parts), - ) - }; - let state_part = match state_part { + + let state_part = match get_state_part( + &runtime, + &shard_id, + &sync_hash, + &state_root, + part_id, + num_parts, + &chain, + ) { Ok(state_part) => state_part, Err(err) => { res = Some(err); break; } }; - let location = s3_location(&config.chain_id, epoch_height, shard_id, part_id); - + let location = + s3_location(&config.chain_id, epoch_height, shard_id, part_id, num_parts); + if let Err(err) = + put_state_part(&location, &state_part, &shard_id, &bucket).await { - let _timer = metrics::STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - let put = bucket - .put_object(&location, &state_part) - .await - .map_err(|err| Error::Other(err.to_string())); - if let Err(err) = put { - res = Some(err); - break; - } - - // Optional, we probably don't need this. - let put = bucket - .put_object_tagging( - &location, - &[ - ("chain_id", &config.chain_id), - ("epoch_id", &format!("{:?}", epoch_id.0)), - ("epoch_height", &epoch_height.to_string()), - ("state_root", &format!("{:?}", state_root)), - ("sync_hash", &format!("{:?}", sync_hash)), - ("node_key", &format!("{:?}", node_key)), - ], - ) - .await - .map_err(|err| Error::Other(err.to_string())); - if let Err(err) = put { - res = Some(err); - break; - } + res = Some(err); + break; } - - // Record that a part was obtained and dumped. - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, ?sync_hash, ?state_root, part_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); - let next_progress = StateSyncDumpProgress::InProgress { - epoch_id: epoch_id.clone(), + update_progress( + &shard_id, + &epoch_id, epoch_height, - sync_hash, - state_root, - parts_dumped: part_id + 1, + &sync_hash, + &state_root, + part_id, num_parts, - }; - match chain - .store() - .set_state_sync_dump_progress(shard_id, Some(next_progress.clone())) - { - Ok(_) => { - tracing::debug!(target: "state_sync_dump", shard_id, ?next_progress, "Updated dump progress"); - } - Err(err) => { - tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to update dump progress, continue"); - } - } + state_part.len(), + &chain, + ); } if let Some(err) = res { Err(err) } else { - Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) + Ok(Some(StateSyncDumpProgress::AllDumped { + epoch_id, + epoch_height, + num_parts: Some(num_parts), + })) } } }; @@ -262,6 +246,117 @@ async fn state_sync_dump( } } +async fn put_state_part( + location: &str, + state_part: &[u8], + shard_id: &ShardId, + bucket: &s3::Bucket, +) -> Result { + let _timer = metrics::STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + let put = bucket + .put_object(&location, &state_part) + .await + .map_err(|err| Error::Other(err.to_string())); + tracing::debug!(target: "state_sync_dump", shard_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); + put +} + +fn update_progress( + shard_id: &ShardId, + epoch_id: &EpochId, + epoch_height: EpochHeight, + sync_hash: &CryptoHash, + state_root: &StateRoot, + part_id: u64, + num_parts: u64, + part_len: usize, + chain: &Chain, +) { + // Record that a part was obtained and dumped. + metrics::STATE_SYNC_DUMP_SIZE_TOTAL + .with_label_values(&[&shard_id.to_string()]) + .inc_by(part_len as u64); + let next_progress = StateSyncDumpProgress::InProgress { + epoch_id: epoch_id.clone(), + epoch_height, + sync_hash: *sync_hash, + state_root: *state_root, + parts_dumped: part_id + 1, + num_parts, + }; + match chain.store().set_state_sync_dump_progress(*shard_id, Some(next_progress.clone())) { + Ok(_) => { + tracing::debug!(target: "state_sync_dump", shard_id, ?next_progress, "Updated dump progress"); + } + Err(err) => { + tracing::debug!(target: "state_sync_dump", shard_id, ?err, "Failed to update dump progress, continue"); + } + } + set_metrics(shard_id, Some(part_id + 1), Some(num_parts), Some(epoch_height)); +} + +fn set_metrics( + shard_id: &ShardId, + parts_dumped: Option, + num_parts: Option, + epoch_height: Option, +) { + if let Some(parts_dumped) = parts_dumped { + metrics::STATE_SYNC_DUMP_NUM_PARTS_DUMPED + .with_label_values(&[&shard_id.to_string()]) + .set(parts_dumped as i64); + } + if let Some(num_parts) = num_parts { + metrics::STATE_SYNC_DUMP_NUM_PARTS_TOTAL + .with_label_values(&[&shard_id.to_string()]) + .set(num_parts as i64); + } + if let Some(epoch_height) = epoch_height { + assert!( + epoch_height < 10000, + "Impossible: {:?} {:?} {:?} {:?}", + shard_id, + parts_dumped, + num_parts, + epoch_height + ); + metrics::STATE_SYNC_DUMP_EPOCH_HEIGHT + .with_label_values(&[&shard_id.to_string()]) + .set(epoch_height as i64); + } +} + +fn get_state_part( + runtime: &Arc, + shard_id: &ShardId, + sync_hash: &CryptoHash, + state_root: &StateRoot, + part_id: u64, + num_parts: u64, + chain: &Chain, +) -> Result, Error> { + let state_part = { + let _timer = metrics::STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + runtime.obtain_state_part( + *shard_id, + &sync_hash, + &state_root, + PartId::new(part_id, num_parts), + )? + }; + + // Save the part data. + let key = StatePartKey(*sync_hash, *shard_id, part_id).try_to_vec()?; + let mut store_update = chain.store().store().store_update(); + store_update.set(DBCol::StateParts, &key, &state_part); + store_update.commit()?; + Ok(state_part) +} + /// Gets basic information about the epoch to be dumped. fn start_dumping( epoch_id: EpochId, @@ -279,9 +374,10 @@ fn start_dumping( let state_root = sync_hash_block.chunks()[shard_id as usize].prev_state_root(); let state_root_node = runtime.get_state_root_node(shard_id, &sync_hash, &state_root)?; let num_parts = get_num_state_parts(state_root_node.memory_usage); - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, ?state_root, num_parts, "Initialize dumping state of Epoch"); + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, %state_root, num_parts, "Initialize dumping state of Epoch"); // Note that first the state of the state machines gets changes to // `InProgress` and it starts dumping state after a short interval. + set_metrics(&shard_id, Some(0), Some(num_parts), Some(epoch_height)); Ok(Some(StateSyncDumpProgress::InProgress { epoch_id, epoch_height, @@ -291,8 +387,8 @@ fn start_dumping( num_parts, })) } else { - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, ?sync_hash, "Shard is not tracked, skip the epoch"); - Ok(Some(StateSyncDumpProgress::AllDumped(epoch_id))) + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, "Shard is not tracked, skip the epoch"); + Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts: Some(0) })) } } @@ -300,35 +396,42 @@ fn start_dumping( /// `epoch_id` represents the last fully dumped epoch. fn check_new_epoch( epoch_id: Option, + epoch_height: Option, + num_parts: Option, shard_id: ShardId, chain: &Chain, runtime: &Arc, - config: &ClientConfig, ) -> Result, Error> { let head = chain.head()?; if Some(&head.epoch_id) == epoch_id.as_ref() { + set_metrics(&shard_id, num_parts, num_parts, epoch_height); Ok(None) } else { + // Check if the final block is now in the next epoch. tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, "Check if a new complete epoch is available"); - let mut sync_hash = head.prev_block_hash; - // Step back a few blocks to avoid dealing with forks. - for _ in 0..config.state_fetch_horizon { - sync_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); - } - let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash)?; + let hash = head.last_block_hash; + let header = chain.get_block_header(&hash)?; + let final_hash = header.last_final_block(); + let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &final_hash)?; let header = chain.get_block_header(&sync_hash)?; if Some(header.epoch_id()) == epoch_id.as_ref() { // Still in the latest dumped epoch. Do nothing. Ok(None) } else { - start_dumping(head.epoch_id.clone(), sync_hash, shard_id, &chain, runtime) + start_dumping(head.epoch_id, sync_hash, shard_id, &chain, runtime) } } } -fn s3_location(chain_id: &str, epoch_height: u64, shard_id: u64, part_id: u64) -> String { +fn s3_location( + chain_id: &str, + epoch_height: u64, + shard_id: u64, + part_id: u64, + num_parts: u64, +) -> String { format!( - "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}", - chain_id, epoch_height, shard_id, part_id + "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}_of_{:06}", + chain_id, epoch_height, shard_id, part_id, num_parts ) } diff --git a/tools/state-viewer/src/cli.rs b/tools/state-viewer/src/cli.rs index a3218119f07..fa4cafa5441 100644 --- a/tools/state-viewer/src/cli.rs +++ b/tools/state-viewer/src/cli.rs @@ -1,6 +1,8 @@ use crate::commands::*; use crate::contract_accounts::ContractAccountFilter; use crate::rocksdb_stats::get_rocksdb_stats; +use crate::state_parts::{apply_state_parts, dump_state_parts}; +use crate::{epoch_info, state_parts}; use near_chain_configs::{GenesisChangeConfig, GenesisValidationMode}; use near_primitives::account::id::AccountId; use near_primitives::hash::CryptoHash; @@ -26,6 +28,8 @@ pub enum StateViewerSubCommand { /// even if it's not included in any block on disk #[clap(alias = "apply_receipt")] ApplyReceipt(ApplyReceiptCmd), + /// Apply all or a single state part of a shard. + ApplyStateParts(ApplyStatePartsCmd), /// Apply a transaction if it occurs in some chunk we know about, /// even if it's not included in any block on disk #[clap(alias = "apply_tx")] @@ -49,6 +53,8 @@ pub enum StateViewerSubCommand { /// Generate a genesis file from the current state of the DB. #[clap(alias = "dump_state")] DumpState(DumpStateCmd), + /// Dump all or a single state part of a shard. + DumpStateParts(DumpStatePartsCmd), /// Writes state to a remote redis server. #[clap(alias = "dump_state_redis")] DumpStateRedis(DumpStateRedisCmd), @@ -70,11 +76,6 @@ pub enum StateViewerSubCommand { RocksDBStats(RocksDBStatsCmd), /// Iterates over a trie and prints the StateRecords. State, - /// Dumps or applies StateChanges. - /// Experimental tool for shard shadowing development. - StateChanges(StateChangesCmd), - /// Dump or apply state parts. - StateParts(StatePartsCmd), /// View head of the storage. #[clap(alias = "view_chain")] ViewChain(ViewChainCmd), @@ -105,8 +106,7 @@ impl StateViewerSubCommand { let storage = store_opener.open_in_mode(mode).unwrap(); let store = match temperature { Temperature::Hot => storage.get_hot_store(), - // Cold store on it's own is useless in majority of subcommands - Temperature::Cold => storage.get_split_store().unwrap(), + Temperature::Cold => storage.get_cold_store().unwrap(), }; match self { @@ -114,6 +114,7 @@ impl StateViewerSubCommand { StateViewerSubCommand::ApplyChunk(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ApplyRange(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ApplyReceipt(cmd) => cmd.run(home_dir, near_config, store), + StateViewerSubCommand::ApplyStateParts(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ApplyTx(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::Chain(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::CheckBlock => check_block_chunk_existence(near_config, store), @@ -122,6 +123,7 @@ impl StateViewerSubCommand { StateViewerSubCommand::DumpAccountStorage(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpCode(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpState(cmd) => cmd.run(home_dir, near_config, store), + StateViewerSubCommand::DumpStateParts(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpStateRedis(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpTx(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::EpochInfo(cmd) => cmd.run(home_dir, near_config, store), @@ -130,8 +132,6 @@ impl StateViewerSubCommand { StateViewerSubCommand::Replay(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::RocksDBStats(cmd) => cmd.run(store_opener.path()), StateViewerSubCommand::State => state(home_dir, near_config, store), - StateViewerSubCommand::StateChanges(cmd) => cmd.run(home_dir, near_config, store), - StateViewerSubCommand::StateParts(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ViewChain(cmd) => cmd.run(near_config, store), StateViewerSubCommand::ViewTrie(cmd) => cmd.run(store), } @@ -215,6 +215,42 @@ impl ApplyReceiptCmd { } } +#[derive(clap::Parser)] +pub struct ApplyStatePartsCmd { + /// Selects an epoch. The dump will be of the state at the beginning of this epoch. + #[clap(subcommand)] + epoch_selection: state_parts::EpochSelection, + /// Shard id. + #[clap(long)] + shard_id: ShardId, + /// State part id. Leave empty to go through every part in the shard. + #[clap(long)] + part_id: Option, + /// Where to write the state parts to. + #[clap(long)] + root_dir: Option, + /// Store state parts in an S3 bucket. + #[clap(long)] + s3_bucket: Option, + /// Store state parts in an S3 bucket. + #[clap(long)] + s3_region: Option, +} + +impl ApplyStatePartsCmd { + pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { + apply_state_parts( + self.epoch_selection, + self.shard_id, + self.part_id, + home_dir, + near_config, + store, + state_parts::Location::new(self.root_dir, (self.s3_bucket, self.s3_region)), + ); + } +} + #[derive(clap::Parser)] pub struct ApplyTxCmd { #[clap(long)] @@ -362,6 +398,49 @@ impl DumpStateCmd { } } +#[derive(clap::Parser)] +pub struct DumpStatePartsCmd { + /// Selects an epoch. The dump will be of the state at the beginning of this epoch. + #[clap(subcommand)] + epoch_selection: state_parts::EpochSelection, + /// Shard id. + #[clap(long)] + shard_id: ShardId, + /// Dump a single part id. + #[clap(long)] + part_id: Option, + /// Dump part ids starting from this part. + #[clap(long)] + part_from: Option, + /// Dump part ids up to this part (exclusive). + #[clap(long)] + part_to: Option, + /// Where to write the state parts to. + #[clap(long)] + root_dir: Option, + /// Store state parts in an S3 bucket. + #[clap(long)] + s3_bucket: Option, + /// S3 region to store state parts. + #[clap(long)] + s3_region: Option, +} + +impl DumpStatePartsCmd { + pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { + dump_state_parts( + self.epoch_selection, + self.shard_id, + self.part_from.or(self.part_id), + self.part_to.or(self.part_id.map(|x| x + 1)), + home_dir, + near_config, + store, + state_parts::Location::new(self.root_dir, (self.s3_bucket, self.s3_region)), + ); + } +} + #[derive(clap::Parser)] pub struct DumpStateRedisCmd { /// Optionally, can specify at which height to dump state. @@ -410,7 +489,7 @@ impl DumpTxCmd { #[derive(clap::Args)] pub struct EpochInfoCmd { #[clap(subcommand)] - epoch_selection: crate::epoch_info::EpochSelection, + epoch_selection: epoch_info::EpochSelection, /// Displays kickouts of the given validator and expected and missed blocks and chunks produced. #[clap(long)] validator_account_id: Option, @@ -481,18 +560,6 @@ impl RocksDBStatsCmd { } } -#[derive(clap::Parser)] -pub struct StateChangesCmd { - #[clap(subcommand)] - command: crate::state_changes::StateChangesSubCommand, -} - -impl StateChangesCmd { - pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { - self.command.run(home_dir, near_config, store) - } -} - #[derive(clap::Parser)] pub struct ViewChainCmd { #[clap(long)] @@ -572,36 +639,3 @@ impl ViewTrieCmd { } } } - -#[derive(clap::Parser)] -pub struct StatePartsCmd { - /// Shard id. - #[clap(long)] - shard_id: ShardId, - /// Location of serialized state parts. - #[clap(long)] - root_dir: Option, - /// Store state parts in an S3 bucket. - #[clap(long)] - s3_bucket: Option, - /// Store state parts in an S3 bucket. - #[clap(long)] - s3_region: Option, - /// Dump or Apply state parts. - #[clap(subcommand)] - command: crate::state_parts::StatePartsSubCommand, -} - -impl StatePartsCmd { - pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { - self.command.run( - self.shard_id, - self.root_dir, - self.s3_bucket, - self.s3_region, - home_dir, - near_config, - store, - ); - } -} diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index 27e117c1332..945e8c6490e 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -1,11 +1,12 @@ use crate::epoch_info::iterate_and_filter; +use clap::Subcommand; use near_chain::types::RuntimeAdapter; use near_chain::{ChainStore, ChainStoreAccess}; use near_epoch_manager::EpochManager; use near_primitives::epoch_manager::epoch_info::EpochInfo; use near_primitives::state_part::PartId; use near_primitives::syncing::get_num_state_parts; -use near_primitives::types::{EpochId, StateRoot}; +use near_primitives::types::EpochId; use near_primitives_core::hash::CryptoHash; use near_primitives_core::types::{BlockHeight, EpochHeight, ShardId}; use near_store::Store; @@ -18,81 +19,7 @@ use std::str::FromStr; use std::sync::Arc; use std::time::Instant; -#[derive(clap::Subcommand, Debug, Clone)] -pub(crate) enum StatePartsSubCommand { - /// Apply all or a single state part of a shard. - Apply { - /// If true, validate the state part but don't write it to the DB. - #[clap(long)] - dry_run: bool, - /// If provided, this value will be used instead of looking it up in the headers. - /// Use if those headers or blocks are not available. - #[clap(long)] - state_root: Option, - /// Choose a single part id. - /// If None - affects all state parts. - #[clap(long)] - part_id: Option, - /// Select an epoch to work on. - #[clap(subcommand)] - epoch_selection: EpochSelection, - }, - /// Dump all or a single state part of a shard. - Dump { - /// Dump part ids starting from this part. - #[clap(long)] - part_from: Option, - /// Dump part ids up to this part (exclusive). - #[clap(long)] - part_to: Option, - /// Select an epoch to work on. - #[clap(subcommand)] - epoch_selection: EpochSelection, - }, -} - -impl StatePartsSubCommand { - pub(crate) fn run( - self, - shard_id: ShardId, - root_dir: Option, - s3_bucket: Option, - s3_region: Option, - home_dir: &Path, - near_config: NearConfig, - store: Store, - ) { - match self { - StatePartsSubCommand::Apply { dry_run, state_root, part_id, epoch_selection } => { - apply_state_parts( - epoch_selection, - shard_id, - part_id, - dry_run, - state_root, - home_dir, - near_config, - store, - Location::new(root_dir, (s3_bucket, s3_region)), - ); - } - StatePartsSubCommand::Dump { part_from, part_to, epoch_selection } => { - dump_state_parts( - epoch_selection, - shard_id, - part_from, - part_to, - home_dir, - near_config, - store, - Location::new(root_dir, (s3_bucket, s3_region)), - ); - } - } - } -} - -#[derive(clap::Subcommand, Debug, Clone)] +#[derive(Subcommand, Debug, Clone)] pub(crate) enum EpochSelection { /// Current epoch. Current, @@ -107,7 +34,7 @@ pub(crate) enum EpochSelection { } impl EpochSelection { - fn to_epoch_id( + pub fn to_epoch_id( &self, store: Store, chain_store: &ChainStore, @@ -143,13 +70,13 @@ impl EpochSelection { } } -enum Location { +pub(crate) enum Location { Files(PathBuf), S3 { bucket: String, region: String }, } impl Location { - fn new( + pub(crate) fn new( root_dir: Option, s3_bucket_and_region: (Option, Option), ) -> Self { @@ -207,12 +134,10 @@ fn get_prev_hash_of_epoch( } } -fn apply_state_parts( +pub(crate) fn apply_state_parts( epoch_selection: EpochSelection, shard_id: ShardId, part_id: Option, - dry_run: bool, - maybe_state_root: Option, home_dir: &Path, near_config: NearConfig, store: Store, @@ -231,23 +156,17 @@ fn apply_state_parts( let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); + let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); + let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); - let (state_root, sync_prev_hash) = if let Some(state_root) = maybe_state_root { - (state_root, None) - } else { - let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); - let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); - - assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); - assert!( - shard_id < sync_prev_block.chunks().len() as u64, - "shard_id: {}, #shards: {}", - shard_id, - sync_prev_block.chunks().len() - ); - let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); - (state_root, Some(sync_prev_hash)) - }; + assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); + assert!( + shard_id < sync_prev_block.chunks().len() as u64, + "shard_id: {}, #shards: {}", + shard_id, + sync_prev_block.chunks().len() + ); + let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); let part_storage = get_state_part_reader( location, @@ -257,7 +176,6 @@ fn apply_state_parts( ); let num_parts = part_storage.num_parts(); - assert_ne!(num_parts, 0, "Too few num_parts: {}", num_parts); let part_ids = get_part_ids(part_id, part_id.map(|x| x + 1), num_parts); tracing::info!( target: "state-parts", @@ -274,32 +192,22 @@ fn apply_state_parts( for part_id in part_ids { let timer = Instant::now(); assert!(part_id < num_parts, "part_id: {}, num_parts: {}", part_id, num_parts); - let part = part_storage.read(part_id, num_parts); - - if dry_run { - assert!(runtime_adapter.validate_state_part( + let part = part_storage.read(part_id); + runtime_adapter + .apply_state_part( + shard_id, &state_root, PartId::new(part_id, num_parts), - &part - )); - tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Validated a state part"); - } else { - runtime_adapter - .apply_state_part( - shard_id, - &state_root, - PartId::new(part_id, num_parts), - &part, - &epoch_id, - ) - .unwrap(); - tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Applied a state part"); - } + &part, + &epoch_id, + ) + .unwrap(); + tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Applied a state part"); } tracing::info!(target: "state-parts", total_elapsed_sec = timer.elapsed().as_secs_f64(), "Applied all requested state parts"); } -fn dump_state_parts( +pub(crate) fn dump_state_parts( epoch_selection: EpochSelection, shard_id: ShardId, part_from: Option, @@ -369,7 +277,7 @@ fn dump_state_parts( PartId::new(part_id, num_parts), ) .unwrap(); - part_storage.write(&state_part, part_id, num_parts); + part_storage.write(&state_part, part_id); tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote a state part"); } tracing::info!(target: "state-parts", total_elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote all requested state parts"); @@ -383,36 +291,21 @@ fn location_prefix(chain_id: &str, epoch_height: u64, shard_id: u64) -> String { format!("chain_id={}/epoch_height={}/shard_id={}", chain_id, epoch_height, shard_id) } -fn match_filename(s: &str) -> Option { - let re = regex::Regex::new(r"^state_part_(\d{6})_of_(\d{6})$").unwrap(); - re.captures(s) -} - fn is_part_filename(s: &str) -> bool { - match_filename(s).is_some() -} - -fn get_num_parts_from_filename(s: &str) -> Option { - if let Some(captures) = match_filename(s) { - if let Some(num_parts) = captures.get(2) { - if let Ok(num_parts) = num_parts.as_str().parse::() { - return Some(num_parts); - } - } - } - None + let re = regex::Regex::new(r"^state_part_(\d{6})$").unwrap(); + re.is_match(s) } -fn part_filename(part_id: u64, num_parts: u64) -> String { - format!("state_part_{:06}_of_{:06}", part_id, num_parts) +fn part_filename(part_id: u64) -> String { + format!("state_part_{:06}", part_id) } trait StatePartWriter { - fn write(&self, state_part: &[u8], part_id: u64, num_parts: u64); + fn write(&self, state_part: &[u8], part_id: u64); } trait StatePartReader { - fn read(&self, part_id: u64, num_parts: u64) -> Vec; + fn read(&self, part_id: u64) -> Vec; fn num_parts(&self) -> u64; } @@ -469,22 +362,22 @@ impl FileSystemStorage { Self { state_parts_dir } } - fn get_location(&self, part_id: u64, num_parts: u64) -> PathBuf { - (&self.state_parts_dir).join(part_filename(part_id, num_parts)) + fn get_location(&self, part_id: u64) -> PathBuf { + (&self.state_parts_dir).join(part_filename(part_id)) } } impl StatePartWriter for FileSystemStorage { - fn write(&self, state_part: &[u8], part_id: u64, num_parts: u64) { - let filename = self.get_location(part_id, num_parts); + fn write(&self, state_part: &[u8], part_id: u64) { + let filename = self.get_location(part_id); std::fs::write(&filename, state_part).unwrap(); tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), ?filename, "Wrote a state part to disk"); } } impl StatePartReader for FileSystemStorage { - fn read(&self, part_id: u64, num_parts: u64) -> Vec { - let filename = self.get_location(part_id, num_parts); + fn read(&self, part_id: u64) -> Vec { + let filename = self.get_location(part_id); let part = std::fs::read(filename).unwrap(); part } @@ -528,22 +421,22 @@ impl S3Storage { Self { location, bucket } } - fn get_location(&self, part_id: u64, num_parts: u64) -> String { - format!("{}/{}", self.location, part_filename(part_id, num_parts)) + fn get_location(&self, part_id: u64) -> String { + format!("{}/{}", self.location, part_filename(part_id)) } } impl StatePartWriter for S3Storage { - fn write(&self, state_part: &[u8], part_id: u64, num_parts: u64) { - let location = self.get_location(part_id, num_parts); + fn write(&self, state_part: &[u8], part_id: u64) { + let location = self.get_location(part_id); self.bucket.put_object_blocking(&location, &state_part).unwrap(); tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); } } impl StatePartReader for S3Storage { - fn read(&self, part_id: u64, num_parts: u64) -> Vec { - let location = self.get_location(part_id, num_parts); + fn read(&self, part_id: u64) -> Vec { + let location = self.get_location(part_id); let response = self.bucket.get_object_blocking(location.clone()).unwrap(); tracing::info!(target: "state-parts", part_id, location, response_code = response.status_code(), "Got an object from S3"); assert_eq!(response.status_code(), 200); @@ -556,27 +449,17 @@ impl StatePartReader for S3Storage { let list: Vec = self.bucket.list_blocking(location, Some("/".to_string())).unwrap(); assert_eq!(list.len(), 1); - let mut known_num_parts = None; - let num_objects = list[0] + let num_parts = list[0] .contents .iter() .filter(|object| { let filename = Path::new(&object.key); let filename = filename.file_name().unwrap().to_str().unwrap(); tracing::debug!(target: "state-parts", object_key = ?object.key, ?filename); - if let Some(num_parts) = get_num_parts_from_filename(filename) { - if let Some(known_num_parts) = known_num_parts { - assert_eq!(known_num_parts, num_parts); - } - known_num_parts = Some(num_parts); - } is_part_filename(filename) }) .collect::>() .len(); - if let Some(known_num_parts) = known_num_parts { - assert_eq!(known_num_parts, num_objects as u64); - } - num_objects as u64 + num_parts as u64 } } From cc49285134063b3233b4f03f38699d583c64565a Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 11:29:31 +0100 Subject: [PATCH 71/88] Changelog --- CHANGELOG.md | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index cfd30d49342..f4899e654d3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,23 +6,9 @@ ### Non-protocol Changes -* State-viewer tool to dump and apply state changes from/to a range of blocks. [#8628](https://github.com/near/nearcore/pull/8628) * Experimental option to dump state of every epoch to external storage. [#8661](https://github.com/near/nearcore/pull/8661) -* Add prometheus metrics for tracked shards, block height within epoch, if is block/chunk producer. [#8728](https://github.com/near/nearcore/pull/8728) -* State sync is disabled by default [#8730](https://github.com/near/nearcore/pull/8730) -* Node can restart if State Sync gets interrupted. [#8732](https://github.com/near/nearcore/pull/8732) -* Merged two `neard view-state` commands: `apply-state-parts` and `dump-state-parts` into a single `state-parts` command. [#8739](https://github.com/near/nearcore/pull/8739) -* Add config.network.experimental.network_config_overrides to the JSON config. [#8871](https://github.com/near/nearcore/pull/8871) - -## 1.32.2 - -### Fixes -* Fix: rosetta zero balance accounts [#8833](https://github.com/near/nearcore/pull/8833) - -## 1.32.1 - -### Fixes -* Fix vulnerabilities in block outcome root validation and total supply validation [#8790](https://github.com/near/nearcore/pull/8790) +* State-viewer tool to dump and apply state changes from/to a range of blocks [#8628](https://github.com/near/nearcore/pull/8628) +* Node can sync State from S3. [#XXXX](https://github.com/near/nearcore/pull/XXXX) ## 1.32.0 From 8067addb7c1e22e1334a2dc53f0b0f939299c37b Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Mon, 20 Mar 2023 14:12:03 +0100 Subject: [PATCH 72/88] More output --- nearcore/src/runtime/mod.rs | 32 ++-- tools/state-viewer/src/state_parts.rs | 263 ++++++++++++++++++++------ 2 files changed, 225 insertions(+), 70 deletions(-) diff --git a/nearcore/src/runtime/mod.rs b/nearcore/src/runtime/mod.rs index 5fa3543c190..2c8551adaae 100644 --- a/nearcore/src/runtime/mod.rs +++ b/nearcore/src/runtime/mod.rs @@ -1270,11 +1270,17 @@ impl RuntimeAdapter for NightshadeRuntime { match Trie::validate_trie_nodes_for_part(state_root, part_id, trie_nodes) { Ok(_) => true, // Storage error should not happen - Err(_) => false, + Err(err) => { + tracing::error!(target: "state-parts", ?err, "Storage error"); + false + } } } // Deserialization error means we've got the data from malicious peer - Err(_) => false, + Err(err) => { + tracing::error!(target: "state-parts", ?err, "Deserialization error"); + false + }, } } @@ -1926,11 +1932,11 @@ mod test { * U256::from(self.runtime.genesis_config.total_supply) * U256::from(epoch_duration) / (U256::from(num_seconds_per_year) - * U256::from( - *self.runtime.genesis_config.max_inflation_rate.denom() as u128 - ) - * U256::from(num_ns_in_second))) - .as_u128(); + * U256::from( + *self.runtime.genesis_config.max_inflation_rate.denom() as u128 + ) + * U256::from(num_ns_in_second))) + .as_u128(); let per_epoch_protocol_treasury = per_epoch_total_reward * *self.runtime.genesis_config.protocol_reward_rate.numer() as u128 / *self.runtime.genesis_config.protocol_reward_rate.denom() as u128; @@ -2445,7 +2451,7 @@ mod test { block_producers[0].public_key(), 0, ) - .into()], + .into()], prev_epoch_kickout: Default::default(), epoch_start_height: 1, epoch_height: 1, @@ -2527,11 +2533,11 @@ mod test { assert!( env.runtime.cares_about_shard(Some(&validators[0]), &env.head.last_block_hash, 1, true) ^ env.runtime.cares_about_shard( - Some(&validators[1]), - &env.head.last_block_hash, - 1, - true - ) + Some(&validators[1]), + &env.head.last_block_hash, + 1, + true + ) ); assert!(env.runtime.cares_about_shard( Some(&validators[1]), diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index 945e8c6490e..6d2fcff706b 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -1,12 +1,11 @@ use crate::epoch_info::iterate_and_filter; -use clap::Subcommand; use near_chain::types::RuntimeAdapter; use near_chain::{ChainStore, ChainStoreAccess}; use near_epoch_manager::EpochManager; use near_primitives::epoch_manager::epoch_info::EpochInfo; use near_primitives::state_part::PartId; use near_primitives::syncing::get_num_state_parts; -use near_primitives::types::EpochId; +use near_primitives::types::{EpochId, StateRoot}; use near_primitives_core::hash::CryptoHash; use near_primitives_core::types::{BlockHeight, EpochHeight, ShardId}; use near_store::Store; @@ -19,7 +18,86 @@ use std::str::FromStr; use std::sync::Arc; use std::time::Instant; -#[derive(Subcommand, Debug, Clone)] +#[derive(clap::Subcommand, Debug, Clone)] +pub(crate) enum StatePartsSubCommand { + /// Apply all or a single state part of a shard. + Apply { + /// If true, validate the state part but don't write it to the DB. + #[clap(long)] + dry_run: bool, + /// If provided, this value will be used instead of looking it up in the headers. + /// Use if those headers or blocks are not available. + #[clap(long)] + state_root: Option, + /// Choose a single part id. + /// If None - affects all state parts. + #[clap(long)] + part_id: Option, + /// Select an epoch to work on. + #[clap(subcommand)] + epoch_selection: EpochSelection, + }, + /// Dump all or a single state part of a shard. + Dump { + /// Dump part ids starting from this part. + #[clap(long)] + part_from: Option, + /// Dump part ids up to this part (exclusive). + #[clap(long)] + part_to: Option, + /// Select an epoch to work on. + #[clap(subcommand)] + epoch_selection: EpochSelection, + }, +} + +impl StatePartsSubCommand { + pub(crate) fn run( + self, + shard_id: ShardId, + root_dir: Option, + s3_bucket: Option, + s3_region: Option, + home_dir: &Path, + near_config: NearConfig, + store: Store, + ) { + match self { + StatePartsSubCommand::Apply { + dry_run, + state_root, + part_id, + epoch_selection, + } => { + apply_state_parts( + epoch_selection, + shard_id, + part_id, + dry_run, + state_root, + home_dir, + near_config, + store, + Location::new(root_dir, (s3_bucket, s3_region)), + ); + } + StatePartsSubCommand::Dump { part_from, part_to, epoch_selection } => { + dump_state_parts( + epoch_selection, + shard_id, + part_from, + part_to, + home_dir, + near_config, + store, + Location::new(root_dir, (s3_bucket, s3_region)), + ); + } + } + } +} + +#[derive(clap::Subcommand, Debug, Clone)] pub(crate) enum EpochSelection { /// Current epoch. Current, @@ -34,7 +112,7 @@ pub(crate) enum EpochSelection { } impl EpochSelection { - pub fn to_epoch_id( + fn to_epoch_id( &self, store: Store, chain_store: &ChainStore, @@ -70,13 +148,13 @@ impl EpochSelection { } } -pub(crate) enum Location { +enum Location { Files(PathBuf), S3 { bucket: String, region: String }, } impl Location { - pub(crate) fn new( + fn new( root_dir: Option, s3_bucket_and_region: (Option, Option), ) -> Self { @@ -134,10 +212,12 @@ fn get_prev_hash_of_epoch( } } -pub(crate) fn apply_state_parts( +fn apply_state_parts( epoch_selection: EpochSelection, shard_id: ShardId, part_id: Option, + dry_run: bool, + maybe_state_root: Option, home_dir: &Path, near_config: NearConfig, store: Store, @@ -154,33 +234,49 @@ pub(crate) fn apply_state_parts( near_config.client_config.save_trie_changes, ); - let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); - let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); - let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); - let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); - - assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); - assert!( - shard_id < sync_prev_block.chunks().len() as u64, - "shard_id: {}, #shards: {}", - shard_id, - sync_prev_block.chunks().len() - ); - let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); + let (state_root, epoch_height, epoch_id, sync_prev_hash) = if let ( + Some(state_root), + EpochSelection::EpochHeight{epoch_height}, + ) = + (maybe_state_root, &epoch_selection) + { + (state_root, *epoch_height, None, None) + } else { + let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); + let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); + + let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); + let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); + tracing::info!( + target: "state-parts", + ?sync_prev_hash, + height = sync_prev_block.header().height(), + state_roots = ?sync_prev_block.chunks().iter().map(|chunk| chunk.prev_state_root()).collect::>()); + + assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); + assert!( + shard_id < sync_prev_block.chunks().len() as u64, + "shard_id: {}, #shards: {}", + shard_id, + sync_prev_block.chunks().len() + ); + let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); + (state_root, epoch.epoch_height(), Some(epoch_id), Some(sync_prev_hash)) + }; let part_storage = get_state_part_reader( location, &near_config.client_config.chain_id, - epoch.epoch_height(), + epoch_height, shard_id, ); let num_parts = part_storage.num_parts(); + assert_ne!(num_parts, 0, "Too few num_parts: {}", num_parts); let part_ids = get_part_ids(part_id, part_id.map(|x| x + 1), num_parts); tracing::info!( target: "state-parts", - epoch_height = epoch.epoch_height(), - epoch_id = ?epoch_id.0, + epoch_height, shard_id, num_parts, ?sync_prev_hash, @@ -192,22 +288,32 @@ pub(crate) fn apply_state_parts( for part_id in part_ids { let timer = Instant::now(); assert!(part_id < num_parts, "part_id: {}, num_parts: {}", part_id, num_parts); - let part = part_storage.read(part_id); - runtime_adapter - .apply_state_part( - shard_id, + let part = part_storage.read(part_id, num_parts); + + if dry_run { + assert!(runtime_adapter.validate_state_part( &state_root, PartId::new(part_id, num_parts), - &part, - &epoch_id, - ) - .unwrap(); - tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Applied a state part"); + &part + )); + tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Validated a state part"); + } else { + runtime_adapter + .apply_state_part( + shard_id, + &state_root, + PartId::new(part_id, num_parts), + &part, + epoch_id.as_ref().unwrap(), + ) + .unwrap(); + tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Applied a state part"); + } } tracing::info!(target: "state-parts", total_elapsed_sec = timer.elapsed().as_secs_f64(), "Applied all requested state parts"); } -pub(crate) fn dump_state_parts( +fn dump_state_parts( epoch_selection: EpochSelection, shard_id: ShardId, part_from: Option, @@ -277,7 +383,7 @@ pub(crate) fn dump_state_parts( PartId::new(part_id, num_parts), ) .unwrap(); - part_storage.write(&state_part, part_id); + part_storage.write(&state_part, part_id, num_parts); tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote a state part"); } tracing::info!(target: "state-parts", total_elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote all requested state parts"); @@ -291,21 +397,36 @@ fn location_prefix(chain_id: &str, epoch_height: u64, shard_id: u64) -> String { format!("chain_id={}/epoch_height={}/shard_id={}", chain_id, epoch_height, shard_id) } +fn match_filename(s: &str) -> Option { + let re = regex::Regex::new(r"^state_part_(\d{6})_of_(\d{6})$").unwrap(); + re.captures(s) +} + fn is_part_filename(s: &str) -> bool { - let re = regex::Regex::new(r"^state_part_(\d{6})$").unwrap(); - re.is_match(s) + match_filename(s).is_some() } -fn part_filename(part_id: u64) -> String { - format!("state_part_{:06}", part_id) +fn get_num_parts_from_filename(s: &str) -> Option { + if let Some(captures) = match_filename(s) { + if let Some(num_parts) = captures.get(2) { + if let Ok(num_parts) = num_parts.as_str().parse::() { + return Some(num_parts); + } + } + } + None +} + +fn part_filename(part_id: u64, num_parts: u64) -> String { + format!("state_part_{:06}_of_{:06}", part_id, num_parts) } trait StatePartWriter { - fn write(&self, state_part: &[u8], part_id: u64); + fn write(&self, state_part: &[u8], part_id: u64, num_parts: u64); } trait StatePartReader { - fn read(&self, part_id: u64) -> Vec; + fn read(&self, part_id: u64, num_parts: u64) -> Vec; fn num_parts(&self) -> u64; } @@ -359,40 +480,58 @@ impl FileSystemStorage { tracing::info!(target: "state-parts", ?root_dir, ?prefix, ?state_parts_dir, "Ensuring the directory exists"); std::fs::create_dir_all(&state_parts_dir).unwrap(); } + tracing::info!(target: "state-parts", ?state_parts_dir, "Initialized FileSystemStorage"); Self { state_parts_dir } } - fn get_location(&self, part_id: u64) -> PathBuf { - (&self.state_parts_dir).join(part_filename(part_id)) + fn get_location(&self, part_id: u64, num_parts: u64) -> PathBuf { + (&self.state_parts_dir).join(part_filename(part_id, num_parts)) } } impl StatePartWriter for FileSystemStorage { - fn write(&self, state_part: &[u8], part_id: u64) { - let filename = self.get_location(part_id); + fn write(&self, state_part: &[u8], part_id: u64, num_parts: u64) { + let filename = self.get_location(part_id, num_parts); std::fs::write(&filename, state_part).unwrap(); tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), ?filename, "Wrote a state part to disk"); } } impl StatePartReader for FileSystemStorage { - fn read(&self, part_id: u64) -> Vec { - let filename = self.get_location(part_id); + fn read(&self, part_id: u64, num_parts: u64) -> Vec { + let filename = self.get_location(part_id, num_parts); + tracing::debug!(target: "state-parts", part_id, num_parts, ?filename, "Reading state part file"); let part = std::fs::read(filename).unwrap(); part } fn num_parts(&self) -> u64 { let paths = std::fs::read_dir(&self.state_parts_dir).unwrap(); - let num_parts = paths + let mut known_num_parts = None; + let num_files = paths .filter(|path| { let full_path = path.as_ref().unwrap(); tracing::debug!(target: "state-parts", ?full_path); - is_part_filename(full_path.file_name().to_str().unwrap()) + let filename = full_path.file_name().to_str().unwrap().to_string(); + if let Some(num_parts) = get_num_parts_from_filename(&filename) { + if let Some(known_num_parts) = known_num_parts { + assert_eq!(known_num_parts, num_parts); + } + known_num_parts = Some(num_parts); + } + is_part_filename(&filename) }) .collect::>>() .len(); - num_parts as u64 + if known_num_parts != Some(num_files as u64) { + // This is expected when a user saves time and downloads a few parts instead of all parts. + tracing::warn!(target: "state-parts", + dir = ?self.state_parts_dir, + ?known_num_parts, + num_files, + "Filename indicates that number of files expected doesn't match the number of files available"); + } + known_num_parts.unwrap() } } @@ -421,22 +560,22 @@ impl S3Storage { Self { location, bucket } } - fn get_location(&self, part_id: u64) -> String { - format!("{}/{}", self.location, part_filename(part_id)) + fn get_location(&self, part_id: u64, num_parts: u64) -> String { + format!("{}/{}", self.location, part_filename(part_id, num_parts)) } } impl StatePartWriter for S3Storage { - fn write(&self, state_part: &[u8], part_id: u64) { - let location = self.get_location(part_id); + fn write(&self, state_part: &[u8], part_id: u64, num_parts: u64) { + let location = self.get_location(part_id, num_parts); self.bucket.put_object_blocking(&location, &state_part).unwrap(); tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), ?location, "Wrote a state part to S3"); } } impl StatePartReader for S3Storage { - fn read(&self, part_id: u64) -> Vec { - let location = self.get_location(part_id); + fn read(&self, part_id: u64, num_parts: u64) -> Vec { + let location = self.get_location(part_id, num_parts); let response = self.bucket.get_object_blocking(location.clone()).unwrap(); tracing::info!(target: "state-parts", part_id, location, response_code = response.status_code(), "Got an object from S3"); assert_eq!(response.status_code(), 200); @@ -449,17 +588,27 @@ impl StatePartReader for S3Storage { let list: Vec = self.bucket.list_blocking(location, Some("/".to_string())).unwrap(); assert_eq!(list.len(), 1); - let num_parts = list[0] + let mut known_num_parts = None; + let num_objects = list[0] .contents .iter() .filter(|object| { let filename = Path::new(&object.key); let filename = filename.file_name().unwrap().to_str().unwrap(); tracing::debug!(target: "state-parts", object_key = ?object.key, ?filename); + if let Some(num_parts) = get_num_parts_from_filename(filename) { + if let Some(known_num_parts) = known_num_parts { + assert_eq!(known_num_parts, num_parts); + } + known_num_parts = Some(num_parts); + } is_part_filename(filename) }) .collect::>() .len(); - num_parts as u64 + if let Some(known_num_parts) = known_num_parts { + assert_eq!(known_num_parts, num_objects as u64); + } + num_objects as u64 } } From 4c53eac3221be274ea18196164d2d7ea08427add Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 5 Apr 2023 12:43:29 +0200 Subject: [PATCH 73/88] Merge --- CHANGELOG.md | 25 +- chain/client/src/client_actor.rs | 372 +++++++++++++++----------- core/primitives/src/syncing.rs | 3 - nearcore/src/lib.rs | 41 +-- nearcore/src/metrics.rs | 52 +++- nearcore/src/runtime/mod.rs | 36 ++- nearcore/src/state_sync.rs | 159 +++++------ tools/state-viewer/src/state_parts.rs | 262 +++++++++--------- 8 files changed, 538 insertions(+), 412 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f4899e654d3..b9d76de3218 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,9 +6,30 @@ ### Non-protocol Changes +* Node can sync State from S3. [#8789](https://github.com/near/nearcore/pull/8789) + +## 1.33.0 + +### Protocol Changes + +### Non-protocol Changes +* State-viewer tool to dump and apply state changes from/to a range of blocks. [#8628](https://github.com/near/nearcore/pull/8628) * Experimental option to dump state of every epoch to external storage. [#8661](https://github.com/near/nearcore/pull/8661) -* State-viewer tool to dump and apply state changes from/to a range of blocks [#8628](https://github.com/near/nearcore/pull/8628) -* Node can sync State from S3. [#XXXX](https://github.com/near/nearcore/pull/XXXX) +* Add prometheus metrics for tracked shards, block height within epoch, if is block/chunk producer. [#8728](https://github.com/near/nearcore/pull/8728) +* State sync is disabled by default [#8730](https://github.com/near/nearcore/pull/8730) +* Node can restart if State Sync gets interrupted. [#8732](https://github.com/near/nearcore/pull/8732) +* Merged two `neard view-state` commands: `apply-state-parts` and `dump-state-parts` into a single `state-parts` command. [#8739](https://github.com/near/nearcore/pull/8739) +* Add config.network.experimental.network_config_overrides to the JSON config. [#8871](https://github.com/near/nearcore/pull/8871) + +## 1.32.2 + +### Fixes +* Fix: rosetta zero balance accounts [#8833](https://github.com/near/nearcore/pull/8833) + +## 1.32.1 + +### Fixes +* Fix vulnerabilities in block outcome root validation and total supply validation [#8790](https://github.com/near/nearcore/pull/8790) ## 1.32.0 diff --git a/chain/client/src/client_actor.rs b/chain/client/src/client_actor.rs index ef43582ed27..d7f3f7349a3 100644 --- a/chain/client/src/client_actor.rs +++ b/chain/client/src/client_actor.rs @@ -48,6 +48,7 @@ use near_network::types::{ use near_o11y::{handler_debug_span, OpenTelemetrySpanExt, WithSpanContext, WithSpanContextExt}; use near_performance_metrics; use near_performance_metrics_macros::perf; +use near_primitives::block::Tip; use near_primitives::block_header::ApprovalType; use near_primitives::epoch_manager::RngSeed; use near_primitives::hash::CryptoHash; @@ -66,11 +67,12 @@ use near_telemetry::TelemetryActor; use rand::seq::SliceRandom; use rand::{thread_rng, Rng}; use std::collections::HashMap; +use std::fmt; use std::sync::Arc; use std::thread; use std::time::{Duration, Instant}; use tokio::sync::broadcast; -use tracing::{error, info, warn}; +use tracing::{debug, error, info, trace, warn}; /// Multiplier on `max_block_time` to wait until deciding that chain stalled. const STATUS_WAIT_TIME_MULTIPLIER: u64 = 10; @@ -257,7 +259,7 @@ impl Actor for ClientActor { self.catchup(ctx); if let Err(err) = self.client.send_network_chain_info() { - tracing::error!(target: "client", ?err, "Failed to update network chain info"); + error!(target: "client", ?err, "Failed to update network chain info"); } } } @@ -501,7 +503,7 @@ impl Handler> for ClientActor { fn handle(&mut self, msg: WithSpanContext, ctx: &mut Context) { self.wrap(msg, ctx, "BlockApproval", |this, msg| { let BlockApproval(approval, peer_id) = msg; - tracing::debug!(target: "client", "Receive approval {:?} from peer {:?}", approval, peer_id); + debug!(target: "client", "Receive approval {:?} from peer {:?}", approval, peer_id); this.client.collect_block_approval(&approval, ApprovalType::PeerApproval(peer_id)); }) } @@ -519,7 +521,7 @@ impl Handler> for ClientActor { let hash = state_response_info.sync_hash(); let state_response = state_response_info.take_state_response(); - tracing::trace!(target: "sync", "Received state response shard_id: {} sync_hash: {:?} part(id/size): {:?}", + trace!(target: "sync", "Received state response shard_id: {} sync_hash: {:?} part(id/size): {:?}", shard_id, hash, state_response.part().as_ref().map(|(part_id, data)| (part_id, data.len())) @@ -814,13 +816,54 @@ impl Handler> for ClientActor { } } +#[derive(Debug)] +enum SyncRequirement { + SyncNeeded { peer_id: PeerId, highest_height: BlockHeight, head: Tip }, + AlreadyCaughtUp { peer_id: PeerId, highest_height: BlockHeight, head: Tip }, + NoPeers, + AdvHeaderSyncDisabled, +} + +impl SyncRequirement { + fn sync_needed(&self) -> bool { + matches!(self, Self::SyncNeeded { .. }) + } +} + +impl fmt::Display for SyncRequirement { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Self::SyncNeeded { peer_id, highest_height, head: my_head } => write!( + f, + "sync needed at #{} [{}]. highest height peer: {} at #{}", + my_head.height, + format_hash(my_head.last_block_hash), + peer_id, + highest_height + ), + Self::AlreadyCaughtUp { peer_id, highest_height, head: my_head } => write!( + f, + "synced at #{} [{}]. highest height peer: {} at #{}", + my_head.height, + format_hash(my_head.last_block_hash), + peer_id, + highest_height + ), + Self::NoPeers => write!(f, "no available peers"), + Self::AdvHeaderSyncDisabled => { + write!(f, "syncing disabled via adv_disable_header_sync") + } + } + } +} + impl ClientActor { /// Check if client Account Id should be sent and send it. /// Account Id is sent when is not current a validator but are becoming a validator soon. fn check_send_announce_account(&mut self, prev_block_hash: CryptoHash) { // If no peers, there is no one to announce to. if self.network_info.num_connected_peers == 0 { - tracing::debug!(target: "client", "No peers: skip account announce"); + debug!(target: "client", "No peers: skip account announce"); return; } @@ -840,7 +883,7 @@ impl ClientActor { } } - tracing::debug!(target: "client", "Check announce account for {}, last announce time {:?}", validator_signer.validator_id(), self.last_validator_announce_time); + debug!(target: "client", "Check announce account for {}, last announce time {:?}", validator_signer.validator_id(), self.last_validator_announce_time); // Announce AccountId if client is becoming a validator soon. let next_epoch_id = unwrap_or_return!(self @@ -850,7 +893,7 @@ impl ClientActor { // Check client is part of the futures validators if self.client.is_validator(&next_epoch_id, &prev_block_hash) { - tracing::debug!(target: "client", "Sending announce account for {}", validator_signer.validator_id()); + debug!(target: "client", "Sending announce account for {}", validator_signer.validator_id()); self.last_validator_announce_time = Some(now); let signature = validator_signer.sign_account_announce( @@ -953,7 +996,7 @@ impl ClientActor { let _span = tracing::debug_span!(target: "client", "handle_block_production").entered(); // If syncing, don't try to produce blocks. if self.client.sync_status.is_syncing() { - tracing::debug!(target:"client", sync_status=?self.client.sync_status, "Syncing - block production disabled"); + debug!(target:"client", sync_status=?self.client.sync_status, "Syncing - block production disabled"); return Ok(()); } @@ -984,9 +1027,9 @@ impl ClientActor { // We try to produce block for multiple heights (up to the highest height for which we've seen 2/3 of approvals). if latest_known.height + 1 <= self.client.doomslug.get_largest_height_crossing_threshold() { - tracing::debug!(target: "client", "Considering blocks for production between {} and {} ", latest_known.height + 1, self.client.doomslug.get_largest_height_crossing_threshold()); + debug!(target: "client", "Considering blocks for production between {} and {} ", latest_known.height + 1, self.client.doomslug.get_largest_height_crossing_threshold()); } else { - tracing::debug!(target: "client", "Cannot produce any block: not enough approvals beyond {}", latest_known.height); + debug!(target: "client", "Cannot produce any block: not enough approvals beyond {}", latest_known.height); } let me = if let Some(me) = &self.client.validator_signer { @@ -1336,7 +1379,7 @@ impl ClientActor { error!(target: "client", "Error processing sync blocks: {}", err); false } else { - tracing::debug!(target: "client", "Block headers refused by chain: {}", err); + debug!(target: "client", "Block headers refused by chain: {}", err); true } } @@ -1345,9 +1388,13 @@ impl ClientActor { /// Check whether need to (continue) sync. /// Also return higher height with known peers at that height. - fn syncing_info(&self) -> Result<(bool, u64), near_chain::Error> { + fn syncing_info(&self) -> Result { + if self.adv.disable_header_sync() { + return Ok(SyncRequirement::AdvHeaderSyncDisabled); + } + let head = self.client.chain.head()?; - let mut is_syncing = self.client.sync_status.is_syncing(); + let is_syncing = self.client.sync_status.is_syncing(); // Only consider peers whose latest block is not invalid blocks let eligible_peers: Vec<_> = self @@ -1361,39 +1408,25 @@ impl ClientActor { let peer_info = if let Some(peer_info) = eligible_peers.choose(&mut thread_rng()) { peer_info } else { - if !self.client.config.skip_sync_wait { - warn!(target: "client", "Sync: no peers available, disabling sync"); - } - return Ok((false, 0)); + return Ok(SyncRequirement::NoPeers); }; + let peer_id = peer_info.peer_info.id.clone(); + let highest_height = peer_info.highest_block_height; + if is_syncing { - if peer_info.highest_block_height <= head.height { - info!(target: "client", "Sync: synced at {} [{}], {}, highest height peer: {}", - head.height, format_hash(head.last_block_hash), - peer_info.peer_info.id, peer_info.highest_block_height, - ); - is_syncing = false; + if highest_height <= head.height { + Ok(SyncRequirement::AlreadyCaughtUp { peer_id, highest_height, head }) + } else { + Ok(SyncRequirement::SyncNeeded { peer_id, highest_height, head }) } } else { - if peer_info.highest_block_height - > head.height + self.client.config.sync_height_threshold - { - info!( - target: "client", - "Sync: height: {}, peer id/height: {}/{}, enabling sync", - head.height, - peer_info.peer_info.id, - peer_info.highest_block_height, - ); - is_syncing = true; + if highest_height > head.height + self.client.config.sync_height_threshold { + Ok(SyncRequirement::SyncNeeded { peer_id, highest_height, head }) + } else { + Ok(SyncRequirement::AlreadyCaughtUp { peer_id, highest_height, head }) } } - Ok((is_syncing, peer_info.highest_block_height)) - } - - fn needs_syncing(&self, needs_syncing: bool) -> bool { - !self.adv.disable_header_sync() && needs_syncing } fn start_flat_storage_creation(&mut self, ctx: &mut Context) { @@ -1518,8 +1551,8 @@ impl ClientActor { } fn sync_wait_period(&self) -> Duration { - if let Ok((needs_syncing, _)) = self.syncing_info() { - if !self.needs_syncing(needs_syncing) { + if let Ok(sync) = self.syncing_info() { + if !sync.sync_needed() { // If we don't need syncing - retry the sync call rarely. self.client.config.sync_check_period } else { @@ -1547,136 +1580,149 @@ impl ClientActor { })); let currently_syncing = self.client.sync_status.is_syncing(); - let (needs_syncing, highest_height) = unwrap_and_report!(self.syncing_info()); - - if !self.needs_syncing(needs_syncing) { - if currently_syncing { - tracing::debug!( - target: "client", - "{:?} transitions to no sync", - self.client.validator_signer.as_ref().map(|vs| vs.validator_id()), - ); - self.client.sync_status = SyncStatus::NoSync; - - // Initial transition out of "syncing" state. - // Announce this client's account id if their epoch is coming up. - let head = unwrap_and_report!(self.client.chain.head()); - self.check_send_announce_account(head.prev_block_hash); - } - } else { - // Run each step of syncing separately. - unwrap_and_report!(self.client.header_sync.run( - &mut self.client.sync_status, - &mut self.client.chain, - highest_height, - &self.network_info.highest_height_peers - )); - // Only body / state sync if header height is close to the latest. - let header_head = unwrap_and_report!(self.client.chain.header_head()); - - // Sync state if already running sync state or if block sync is too far. - let sync_state = match self.client.sync_status { - SyncStatus::StateSync(_, _) => true, - _ if header_head.height - >= highest_height - .saturating_sub(self.client.config.block_header_fetch_horizon) => - { - unwrap_and_report!(self.client.block_sync.run( - &mut self.client.sync_status, - &self.client.chain, - highest_height, - &self.network_info.highest_height_peers - )) - } - _ => false, - }; - if sync_state { - let (sync_hash, mut new_shard_sync, just_enter_state_sync) = - match &self.client.sync_status { - SyncStatus::StateSync(sync_hash, shard_sync) => { - (*sync_hash, shard_sync.clone(), false) - } - _ => { - let sync_hash = unwrap_and_report!(self.find_sync_hash()); - (sync_hash, HashMap::default(), true) - } - }; - - let me = self.client.validator_signer.as_ref().map(|x| x.validator_id().clone()); - let block_header = - unwrap_and_report!(self.client.chain.get_block_header(&sync_hash)); - let prev_hash = *block_header.prev_hash(); - let epoch_id = - self.client.chain.get_block_header(&sync_hash).unwrap().epoch_id().clone(); - let shards_to_sync = - (0..self.client.runtime_adapter.num_shards(&epoch_id).unwrap()) - .filter(|x| { - cares_about_shard_this_or_next_epoch( - me.as_ref(), - &prev_hash, - *x, - true, - self.client.runtime_adapter.as_ref(), - ) - }) - .collect(); - - if !self.client.config.archive && just_enter_state_sync { - unwrap_and_report!(self.client.chain.reset_data_pre_state_sync(sync_hash)); + let sync = unwrap_and_report!(self.syncing_info()); + + match sync { + SyncRequirement::AlreadyCaughtUp { .. } + | SyncRequirement::NoPeers + | SyncRequirement::AdvHeaderSyncDisabled => { + if currently_syncing { + info!(target: "client", "disabling sync: {}", &sync); + self.client.sync_status = SyncStatus::NoSync; + + // Initial transition out of "syncing" state. + // Announce this client's account id if their epoch is coming up. + let head = unwrap_and_report!(self.client.chain.head()); + self.check_send_announce_account(head.prev_block_hash); } + } - let use_colour = - matches!(self.client.config.log_summary_style, LogSummaryStyle::Colored); - match unwrap_and_report!(self.client.state_sync.run( - &me, - sync_hash, - &mut new_shard_sync, + SyncRequirement::SyncNeeded { highest_height, .. } => { + if !currently_syncing { + info!( + target: "client", + "enabling sync: {}", &sync, + ); + } + // Run each step of syncing separately. + unwrap_and_report!(self.client.header_sync.run( + &mut self.client.sync_status, &mut self.client.chain, - &self.client.runtime_adapter, - &self.network_info.highest_height_peers, - shards_to_sync, - &self.state_parts_task_scheduler, - &self.state_split_scheduler, - use_colour, - )) { - StateSyncResult::Unchanged => (), - StateSyncResult::Changed(fetch_block) => { - self.client.sync_status = SyncStatus::StateSync(sync_hash, new_shard_sync); - if fetch_block { - if let Some(peer_info) = - self.network_info.highest_height_peers.choose(&mut thread_rng()) - { - let id = peer_info.peer_info.id.clone(); - - if let Ok(header) = self.client.chain.get_block_header(&sync_hash) { - for hash in - vec![*header.prev_hash(), *header.hash()].into_iter() + highest_height, + &self.network_info.highest_height_peers + )); + // Only body / state sync if header height is close to the latest. + let header_head = unwrap_and_report!(self.client.chain.header_head()); + + // Sync state if already running sync state or if block sync is too far. + let sync_state = match self.client.sync_status { + SyncStatus::StateSync(_, _) => true, + _ if header_head.height + >= highest_height + .saturating_sub(self.client.config.block_header_fetch_horizon) => + { + unwrap_and_report!(self.client.block_sync.run( + &mut self.client.sync_status, + &self.client.chain, + highest_height, + &self.network_info.highest_height_peers + )) + } + _ => false, + }; + if sync_state { + let (sync_hash, mut new_shard_sync, just_enter_state_sync) = + match &self.client.sync_status { + SyncStatus::StateSync(sync_hash, shard_sync) => { + (*sync_hash, shard_sync.clone(), false) + } + _ => { + let sync_hash = unwrap_and_report!(self.find_sync_hash()); + (sync_hash, HashMap::default(), true) + } + }; + + let me = + self.client.validator_signer.as_ref().map(|x| x.validator_id().clone()); + let block_header = + unwrap_and_report!(self.client.chain.get_block_header(&sync_hash)); + let prev_hash = *block_header.prev_hash(); + let epoch_id = + self.client.chain.get_block_header(&sync_hash).unwrap().epoch_id().clone(); + let shards_to_sync = + (0..self.client.runtime_adapter.num_shards(&epoch_id).unwrap()) + .filter(|x| { + cares_about_shard_this_or_next_epoch( + me.as_ref(), + &prev_hash, + *x, + true, + self.client.runtime_adapter.as_ref(), + ) + }) + .collect(); + + if !self.client.config.archive && just_enter_state_sync { + unwrap_and_report!(self.client.chain.reset_data_pre_state_sync(sync_hash)); + } + + let use_colour = + matches!(self.client.config.log_summary_style, LogSummaryStyle::Colored); + match unwrap_and_report!(self.client.state_sync.run( + &me, + sync_hash, + &mut new_shard_sync, + &mut self.client.chain, + &self.client.runtime_adapter, + &self.network_info.highest_height_peers, + shards_to_sync, + &self.state_parts_task_scheduler, + &self.state_split_scheduler, + use_colour, + )) { + StateSyncResult::Unchanged => (), + StateSyncResult::Changed(fetch_block) => { + self.client.sync_status = + SyncStatus::StateSync(sync_hash, new_shard_sync); + if fetch_block { + if let Some(peer_info) = + self.network_info.highest_height_peers.choose(&mut thread_rng()) + { + let id = peer_info.peer_info.id.clone(); + + if let Ok(header) = + self.client.chain.get_block_header(&sync_hash) { - self.client.request_block(hash, id.clone()); + for hash in + vec![*header.prev_hash(), *header.hash()].into_iter() + { + self.client.request_block(hash, id.clone()); + } } } } } - } - StateSyncResult::Completed => { - info!(target: "sync", "State sync: all shards are done"); - - let mut block_processing_artifacts = BlockProcessingArtifact::default(); - - unwrap_and_report!(self.client.chain.reset_heads_post_state_sync( - &me, - sync_hash, - &mut block_processing_artifacts, - self.get_apply_chunks_done_callback(), - )); - - self.client.process_block_processing_artifact(block_processing_artifacts); - - self.client.sync_status = SyncStatus::BodySync { - start_height: 0, - current_height: 0, - highest_height: 0, - }; + StateSyncResult::Completed => { + info!(target: "sync", "State sync: all shards are done"); + + let mut block_processing_artifacts = BlockProcessingArtifact::default(); + + unwrap_and_report!(self.client.chain.reset_heads_post_state_sync( + &me, + sync_hash, + &mut block_processing_artifacts, + self.get_apply_chunks_done_callback(), + )); + + self.client + .process_block_processing_artifact(block_processing_artifacts); + + self.client.sync_status = SyncStatus::BodySync { + start_height: 0, + current_height: 0, + highest_height: 0, + }; + } } } } diff --git a/core/primitives/src/syncing.rs b/core/primitives/src/syncing.rs index 4dc68f3b7a6..a1450fceb8d 100644 --- a/core/primitives/src/syncing.rs +++ b/core/primitives/src/syncing.rs @@ -249,10 +249,7 @@ pub enum StateSyncDumpProgress { /// Block hash of the first block of the epoch. /// The dumped state corresponds to the state before applying this block. sync_hash: CryptoHash, - /// Root of the state being dumped. - state_root: StateRoot, /// Progress made. parts_dumped: u64, - num_parts: u64, }, } diff --git a/nearcore/src/lib.rs b/nearcore/src/lib.rs index 3da5e6a2766..c5276578b3c 100644 --- a/nearcore/src/lib.rs +++ b/nearcore/src/lib.rs @@ -1,7 +1,7 @@ -use crate::cold_storage::spawn_cold_store_loop; pub use crate::config::{init_configs, load_config, load_test_config, NearConfig, NEAR_BASE}; pub use crate::runtime::NightshadeRuntime; -pub use crate::shard_tracker::TrackedConfig; + +use crate::cold_storage::spawn_cold_store_loop; use crate::state_sync::{spawn_state_sync_dump, StateSyncDumpHandle}; use actix::{Actor, Addr}; use actix_rt::ArbiterHandle; @@ -17,6 +17,7 @@ use near_network::PeerManagerActor; use near_primitives::block::GenesisId; use near_primitives::time; use near_store::metadata::DbKind; +use near_store::metrics::spawn_db_metrics_loop; use near_store::{DBCol, Mode, NodeStorage, Store, StoreOpenerError}; use near_telemetry::TelemetryActor; use std::path::{Path, PathBuf}; @@ -33,7 +34,6 @@ pub mod dyn_config; mod metrics; pub mod migrations; mod runtime; -mod shard_tracker; mod state_sync; pub fn get_default_home() -> PathBuf { @@ -140,10 +140,10 @@ fn open_storage(home_dir: &Path, near_config: &mut NearConfig) -> anyhow::Result {latest_release} release" )) }, - Err(StoreOpenerError::DbVersionTooNew { got, .. }) => { + Err(StoreOpenerError::DbVersionTooNew { got, want }) => { Err(anyhow::anyhow!( - "Database version {got} is created by a newer version of \ - neard, please update neard" + "Database version {got} is higher than the expected version {want}. \ + It was likely created by newer version of neard. Please upgrade your neard." )) }, Err(StoreOpenerError::MigrationError(err)) => { @@ -206,21 +206,27 @@ pub fn start_with_config_and_synchronization( shutdown_signal: Option>, config_updater: Option, ) -> anyhow::Result { - let store = open_storage(home_dir, &mut config)?; + let storage = open_storage(home_dir, &mut config)?; + let db_metrics_arbiter = if config.client_config.enable_statistics_export { + let period = config.client_config.log_summary_period; + let db_metrics_arbiter_handle = spawn_db_metrics_loop(&storage, period)?; + Some(db_metrics_arbiter_handle) + } else { + None + }; - let runtime = - Arc::new(NightshadeRuntime::from_config(home_dir, store.get_hot_store(), &config)); + let runtime = NightshadeRuntime::from_config(home_dir, storage.get_hot_store(), &config); // Get the split store. If split store is some then create a new runtime for // the view client. Otherwise just re-use the existing runtime. - let split_store = get_split_store(&config, &store)?; + let split_store = get_split_store(&config, &storage)?; let view_runtime = if let Some(split_store) = split_store { - Arc::new(NightshadeRuntime::from_config(home_dir, split_store, &config)) + NightshadeRuntime::from_config(home_dir, split_store, &config) } else { runtime.clone() }; - let cold_store_loop_handle = spawn_cold_store_loop(&config, &store, runtime.clone())?; + let cold_store_loop_handle = spawn_cold_store_loop(&config, &storage, runtime.clone())?; let telemetry = TelemetryActor::new(config.telemetry_config.clone()).start(); let chain_genesis = ChainGenesis::new(&config.genesis); @@ -263,7 +269,7 @@ pub fn start_with_config_and_synchronization( network_adapter.as_sender(), client_adapter_for_shards_manager.as_sender(), config.validator_signer.as_ref().map(|signer| signer.validator_id().clone()), - store.get_hot_store(), + storage.get_hot_store(), config.client_config.chunk_request_retry_period, ); shards_manager_adapter.bind(shards_manager_actor); @@ -279,7 +285,7 @@ pub fn start_with_config_and_synchronization( let mut rpc_servers = Vec::new(); let network_actor = PeerManagerActor::spawn( time::Clock::real(), - store.into_inner(near_store::Temperature::Hot), + storage.into_inner(near_store::Temperature::Hot), config.network_config, Arc::new(near_client::adapter::Adapter::new(client_actor.clone(), view_client.clone())), shards_manager_adapter.as_sender(), @@ -317,11 +323,16 @@ pub fn start_with_config_and_synchronization( tracing::trace!(target: "diagnostic", key = "log", "Starting NEAR node with diagnostic activated"); + let mut arbiters = vec![client_arbiter_handle, shards_manager_arbiter_handle]; + if let Some(db_metrics_arbiter) = db_metrics_arbiter { + arbiters.push(db_metrics_arbiter); + } + Ok(NearNode { client: client_actor, view_client, rpc_servers, - arbiters: vec![client_arbiter_handle, shards_manager_arbiter_handle], + arbiters, cold_store_loop_handle, state_sync_dump_handle, }) diff --git a/nearcore/src/metrics.rs b/nearcore/src/metrics.rs index 9b5d99bf01f..19605ceeca1 100644 --- a/nearcore/src/metrics.rs +++ b/nearcore/src/metrics.rs @@ -1,6 +1,7 @@ use near_o11y::metrics::{ exponential_buckets, linear_buckets, try_create_histogram_vec, try_create_int_counter_vec, - try_create_int_gauge, HistogramVec, IntCounterVec, IntGauge, + try_create_int_gauge, try_create_int_gauge_vec, HistogramVec, IntCounterVec, IntGauge, + IntGaugeVec, }; use once_cell::sync::Lazy; @@ -49,12 +50,53 @@ pub(crate) static STATE_SYNC_DUMP_PUT_OBJECT_ELAPSED: Lazy = Lazy: ) .unwrap() }); -pub(crate) static STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED: Lazy = Lazy::new(|| { +pub(crate) static STATE_SYNC_DUMP_NUM_PARTS_TOTAL: Lazy = Lazy::new(|| { + try_create_int_gauge_vec( + "near_state_sync_dump_num_parts_total", + "Total number of parts in the epoch that being dumped", + &["shard_id"], + ) + .unwrap() +}); +pub(crate) static STATE_SYNC_DUMP_NUM_PARTS_DUMPED: Lazy = Lazy::new(|| { + try_create_int_gauge_vec( + "near_state_sync_dump_num_parts_dumped", + "Number of parts dumped in the epoch that is being dumped", + &["shard_id"], + ) + .unwrap() +}); +pub(crate) static STATE_SYNC_DUMP_SIZE_TOTAL: Lazy = Lazy::new(|| { + try_create_int_counter_vec( + "near_state_sync_dump_size_total", + "Total size of parts written to S3", + &["shard_id"], + ) + .unwrap() +}); +pub(crate) static STATE_SYNC_DUMP_EPOCH_HEIGHT: Lazy = Lazy::new(|| { + try_create_int_gauge_vec( + "near_state_sync_dump_epoch_height", + "Epoch Height of an epoch being dumped", + &["shard_id"], + ) + .unwrap() +}); +pub static STATE_SYNC_APPLY_PART_DELAY: Lazy = Lazy::new(|| { try_create_histogram_vec( - "near_state_sync_dump_obtain_part_elapsed_sec", - "Time needed to obtain a part", + "near_state_sync_apply_part_delay_sec", + "Latency of applying a state part", &["shard_id"], - Some(exponential_buckets(0.001, 1.6, 25).unwrap()), + Some(exponential_buckets(0.001, 2.0, 20).unwrap()), + ) + .unwrap() +}); +pub static STATE_SYNC_OBTAIN_PART_DELAY: Lazy = Lazy::new(|| { + try_create_histogram_vec( + "near_state_sync_obtain_part_delay_sec", + "Latency of applying a state part", + &["shard_id"], + Some(exponential_buckets(0.001, 2.0, 20).unwrap()), ) .unwrap() }); diff --git a/nearcore/src/runtime/mod.rs b/nearcore/src/runtime/mod.rs index 2c8551adaae..358387ae547 100644 --- a/nearcore/src/runtime/mod.rs +++ b/nearcore/src/runtime/mod.rs @@ -1246,6 +1246,10 @@ impl RuntimeAdapter for NightshadeRuntime { %block_hash, num_parts = part_id.total) .entered(); + let _timer = metrics::STATE_SYNC_OBTAIN_PART_DELAY + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + let epoch_id = self.get_epoch_id(block_hash)?; let shard_uid = self.get_shard_uid_from_epoch_id(shard_id, &epoch_id)?; let trie = self.tries.get_view_trie_for_shard(shard_uid, *state_root); @@ -1271,16 +1275,16 @@ impl RuntimeAdapter for NightshadeRuntime { Ok(_) => true, // Storage error should not happen Err(err) => { - tracing::error!(target: "state-parts", ?err, "Storage error"); + tracing::error!(target: "state-parts", ?err, "State part storage error"); false } } } // Deserialization error means we've got the data from malicious peer Err(err) => { - tracing::error!(target: "state-parts", ?err, "Deserialization error"); + tracing::error!(target: "state-parts", ?err, "State part deserialization error"); false - }, + } } } @@ -1377,6 +1381,10 @@ impl RuntimeAdapter for NightshadeRuntime { data: &[u8], epoch_id: &EpochId, ) -> Result<(), Error> { + let _timer = metrics::STATE_SYNC_APPLY_PART_DELAY + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + let part = BorshDeserialize::try_from_slice(data) .expect("Part was already validated earlier, so could never fail here"); let ApplyStatePartResult { trie_changes, flat_state_delta, contract_codes } = @@ -1932,11 +1940,11 @@ mod test { * U256::from(self.runtime.genesis_config.total_supply) * U256::from(epoch_duration) / (U256::from(num_seconds_per_year) - * U256::from( - *self.runtime.genesis_config.max_inflation_rate.denom() as u128 - ) - * U256::from(num_ns_in_second))) - .as_u128(); + * U256::from( + *self.runtime.genesis_config.max_inflation_rate.denom() as u128 + ) + * U256::from(num_ns_in_second))) + .as_u128(); let per_epoch_protocol_treasury = per_epoch_total_reward * *self.runtime.genesis_config.protocol_reward_rate.numer() as u128 / *self.runtime.genesis_config.protocol_reward_rate.denom() as u128; @@ -2451,7 +2459,7 @@ mod test { block_producers[0].public_key(), 0, ) - .into()], + .into()], prev_epoch_kickout: Default::default(), epoch_start_height: 1, epoch_height: 1, @@ -2533,11 +2541,11 @@ mod test { assert!( env.runtime.cares_about_shard(Some(&validators[0]), &env.head.last_block_hash, 1, true) ^ env.runtime.cares_about_shard( - Some(&validators[1]), - &env.head.last_block_hash, - 1, - true - ) + Some(&validators[1]), + &env.head.last_block_hash, + 1, + true + ) ); assert!(env.runtime.cares_about_shard( Some(&validators[1]), diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 458bff4e27e..0df51e42802 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -161,63 +161,75 @@ async fn state_sync_dump( epoch_id, epoch_height, sync_hash, - state_root, parts_dumped, - num_parts, })) => { - // The actual dumping of state to S3. - tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, %sync_hash, %state_root, parts_dumped, num_parts, "Creating parts and dumping them"); - let mut res = None; - for part_id in parts_dumped..num_parts { - // Dump parts sequentially synchronously. - // TODO: How to make it possible to dump state more effectively using multiple nodes? - let _timer = metrics::STATE_SYNC_DUMP_ITERATION_ELAPSED - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); + let state_header = chain.get_state_response_header(shard_id, sync_hash); + match state_header { + Ok(state_header) => { + let state_root = state_header.chunk_prev_state_root(); + let num_parts = + get_num_state_parts(state_header.state_root_node().memory_usage); - let state_part = match get_state_part( - &runtime, - &shard_id, - &sync_hash, - &state_root, - part_id, - num_parts, - &chain, - ) { - Ok(state_part) => state_part, - Err(err) => { - res = Some(err); - break; + let mut res = None; + // The actual dumping of state to S3. + tracing::info!(target: "state_sync_dump", shard_id, ?epoch_id, epoch_height, %sync_hash, parts_dumped, "Creating parts and dumping them"); + for part_id in parts_dumped..num_parts { + // Dump parts sequentially synchronously. + // TODO: How to make it possible to dump state more effectively using multiple nodes? + let _timer = metrics::STATE_SYNC_DUMP_ITERATION_ELAPSED + .with_label_values(&[&shard_id.to_string()]) + .start_timer(); + + let state_part = match obtain_and_store_state_part( + &runtime, + &shard_id, + &sync_hash, + &state_root, + part_id, + num_parts, + &chain, + ) { + Ok(state_part) => state_part, + Err(err) => { + res = Some(err); + break; + } + }; + let location = s3_location( + &config.chain_id, + epoch_height, + shard_id, + part_id, + num_parts, + ); + if let Err(err) = + put_state_part(&location, &state_part, &shard_id, &bucket).await + { + res = Some(err); + break; + } + update_progress( + &shard_id, + &epoch_id, + epoch_height, + &sync_hash, + part_id, + num_parts, + state_part.len(), + &chain, + ); + } + if let Some(err) = res { + Err(err) + } else { + Ok(Some(StateSyncDumpProgress::AllDumped { + epoch_id, + epoch_height, + num_parts: Some(num_parts), + })) } - }; - let location = - s3_location(&config.chain_id, epoch_height, shard_id, part_id, num_parts); - if let Err(err) = - put_state_part(&location, &state_part, &shard_id, &bucket).await - { - res = Some(err); - break; } - update_progress( - &shard_id, - &epoch_id, - epoch_height, - &sync_hash, - &state_root, - part_id, - num_parts, - state_part.len(), - &chain, - ); - } - if let Some(err) = res { - Err(err) - } else { - Ok(Some(StateSyncDumpProgress::AllDumped { - epoch_id, - epoch_height, - num_parts: Some(num_parts), - })) + Err(err) => Err(err), } } }; @@ -268,7 +280,6 @@ fn update_progress( epoch_id: &EpochId, epoch_height: EpochHeight, sync_hash: &CryptoHash, - state_root: &StateRoot, part_id: u64, num_parts: u64, part_len: usize, @@ -282,9 +293,7 @@ fn update_progress( epoch_id: epoch_id.clone(), epoch_height, sync_hash: *sync_hash, - state_root: *state_root, parts_dumped: part_id + 1, - num_parts, }; match chain.store().set_state_sync_dump_progress(*shard_id, Some(next_progress.clone())) { Ok(_) => { @@ -328,7 +337,8 @@ fn set_metrics( } } -fn get_state_part( +/// Obtains and then saves the part data. +fn obtain_and_store_state_part( runtime: &Arc, shard_id: &ShardId, sync_hash: &CryptoHash, @@ -337,19 +347,13 @@ fn get_state_part( num_parts: u64, chain: &Chain, ) -> Result, Error> { - let state_part = { - let _timer = metrics::STATE_SYNC_DUMP_OBTAIN_PART_ELAPSED - .with_label_values(&[&shard_id.to_string()]) - .start_timer(); - runtime.obtain_state_part( - *shard_id, - &sync_hash, - &state_root, - PartId::new(part_id, num_parts), - )? - }; + let state_part = runtime.obtain_state_part( + *shard_id, + &sync_hash, + &state_root, + PartId::new(part_id, num_parts), + )?; - // Save the part data. let key = StatePartKey(*sync_hash, *shard_id, part_id).try_to_vec()?; let mut store_update = chain.store().store().store_update(); store_update.set(DBCol::StateParts, &key, &state_part); @@ -367,14 +371,13 @@ fn start_dumping( ) -> Result, Error> { let epoch_info = runtime.get_epoch_info(&epoch_id)?; let epoch_height = epoch_info.epoch_height(); - let num_shards = runtime.num_shards(&epoch_id)?; - let sync_hash_block = chain.get_block(&sync_hash)?; - if runtime.cares_about_shard(None, sync_hash_block.header().prev_hash(), shard_id, false) { - assert_eq!(num_shards, sync_hash_block.chunks().len() as u64); - let state_root = sync_hash_block.chunks()[shard_id as usize].prev_state_root(); - let state_root_node = runtime.get_state_root_node(shard_id, &sync_hash, &state_root)?; - let num_parts = get_num_state_parts(state_root_node.memory_usage); - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, %state_root, num_parts, "Initialize dumping state of Epoch"); + let sync_prev_header = chain.get_block_header(&sync_hash)?; + let sync_prev_hash = sync_prev_header.hash(); + + let state_header = chain.get_state_response_header(shard_id, sync_hash)?; + let num_parts = get_num_state_parts(state_header.state_root_node().memory_usage); + if runtime.cares_about_shard(None, sync_prev_hash, shard_id, false) { + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_prev_hash, %sync_hash, "Initialize dumping state of Epoch"); // Note that first the state of the state machines gets changes to // `InProgress` and it starts dumping state after a short interval. set_metrics(&shard_id, Some(0), Some(num_parts), Some(epoch_height)); @@ -382,12 +385,10 @@ fn start_dumping( epoch_id, epoch_height, sync_hash, - state_root, parts_dumped: 0, - num_parts, })) } else { - tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_hash, "Shard is not tracked, skip the epoch"); + tracing::debug!(target: "state_sync_dump", shard_id, ?epoch_id, %sync_prev_hash, %sync_hash, "Shard is not tracked, skip the epoch"); Ok(Some(StateSyncDumpProgress::AllDumped { epoch_id, epoch_height, num_parts: Some(0) })) } } diff --git a/tools/state-viewer/src/state_parts.rs b/tools/state-viewer/src/state_parts.rs index 6d2fcff706b..fb5f0f5b0ac 100644 --- a/tools/state-viewer/src/state_parts.rs +++ b/tools/state-viewer/src/state_parts.rs @@ -1,21 +1,21 @@ use crate::epoch_info::iterate_and_filter; -use near_chain::types::RuntimeAdapter; -use near_chain::{ChainStore, ChainStoreAccess}; -use near_epoch_manager::EpochManager; +use borsh::BorshDeserialize; +use near_chain::{Chain, ChainGenesis, ChainStoreAccess, DoomslugThresholdMode}; +use near_client::sync::state::StateSync; use near_primitives::epoch_manager::epoch_info::EpochInfo; use near_primitives::state_part::PartId; +use near_primitives::state_record::StateRecord; use near_primitives::syncing::get_num_state_parts; use near_primitives::types::{EpochId, StateRoot}; use near_primitives_core::hash::CryptoHash; use near_primitives_core::types::{BlockHeight, EpochHeight, ShardId}; -use near_store::Store; +use near_store::{PartialStorage, Store, Trie}; use nearcore::{NearConfig, NightshadeRuntime}; use s3::serde_types::ListBucketResult; use std::fs::DirEntry; use std::ops::Range; use std::path::{Path, PathBuf}; use std::str::FromStr; -use std::sync::Arc; use std::time::Instant; #[derive(clap::Subcommand, Debug, Clone)] @@ -49,6 +49,12 @@ pub(crate) enum StatePartsSubCommand { #[clap(subcommand)] epoch_selection: EpochSelection, }, + /// Read State Header from the DB + ReadStateHeader { + /// Select an epoch to work on. + #[clap(subcommand)] + epoch_selection: EpochSelection, + }, } impl StatePartsSubCommand { @@ -62,21 +68,26 @@ impl StatePartsSubCommand { near_config: NearConfig, store: Store, ) { + let runtime = NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); + let chain_genesis = ChainGenesis::new(&near_config.genesis); + let mut chain = Chain::new_for_view_client( + runtime, + &chain_genesis, + DoomslugThresholdMode::TwoThirds, + false, + ) + .unwrap(); + let chain_id = &near_config.genesis.config.chain_id; match self { - StatePartsSubCommand::Apply { - dry_run, - state_root, - part_id, - epoch_selection, - } => { + StatePartsSubCommand::Apply { dry_run, state_root, part_id, epoch_selection } => { apply_state_parts( epoch_selection, shard_id, part_id, dry_run, state_root, - home_dir, - near_config, + &mut chain, + chain_id, store, Location::new(root_dir, (s3_bucket, s3_region)), ); @@ -87,12 +98,15 @@ impl StatePartsSubCommand { shard_id, part_from, part_to, - home_dir, - near_config, + &chain, + chain_id, store, Location::new(root_dir, (s3_bucket, s3_region)), ); } + StatePartsSubCommand::ReadStateHeader { epoch_selection } => { + read_state_header(epoch_selection, shard_id, &chain, store) + } } } } @@ -112,15 +126,10 @@ pub(crate) enum EpochSelection { } impl EpochSelection { - fn to_epoch_id( - &self, - store: Store, - chain_store: &ChainStore, - epoch_manager: &EpochManager, - ) -> EpochId { + fn to_epoch_id(&self, store: Store, chain: &Chain) -> EpochId { match self { EpochSelection::Current => { - epoch_manager.get_epoch_id(&chain_store.head().unwrap().last_block_hash).unwrap() + chain.runtime_adapter.get_epoch_id(&chain.head().unwrap().last_block_hash).unwrap() } EpochSelection::EpochId { epoch_id } => { EpochId(CryptoHash::from_str(&epoch_id).unwrap()) @@ -137,12 +146,12 @@ impl EpochSelection { } EpochSelection::BlockHash { block_hash } => { let block_hash = CryptoHash::from_str(&block_hash).unwrap(); - epoch_manager.get_epoch_id(&block_hash).unwrap() + chain.runtime_adapter.get_epoch_id(&block_hash).unwrap() } EpochSelection::BlockHeight { block_height } => { // Fetch an epoch containing the given block height. - let block_hash = chain_store.get_block_hash_by_height(*block_height).unwrap(); - epoch_manager.get_epoch_id(&block_hash).unwrap() + let block_hash = chain.store().get_block_hash_by_height(*block_height).unwrap(); + chain.runtime_adapter.get_epoch_id(&block_hash).unwrap() } } } @@ -177,21 +186,18 @@ impl Location { } } -/// Returns block hash of the last block of an epoch preceding the given `epoch_info`. -fn get_prev_hash_of_epoch( - epoch_info: &EpochInfo, - chain_store: &ChainStore, - epoch_manager: &EpochManager, -) -> CryptoHash { - let head = chain_store.head().unwrap(); - let mut cur_block_info = epoch_manager.get_block_info(&head.last_block_hash).unwrap(); +/// Returns block hash of some block of the given `epoch_info` epoch. +fn get_any_block_hash_of_epoch(epoch_info: &EpochInfo, chain: &Chain) -> CryptoHash { + let head = chain.store().head().unwrap(); + let mut cur_block_info = chain.runtime_adapter.get_block_info(&head.last_block_hash).unwrap(); // EpochManager doesn't have an API that maps EpochId to Blocks, and this function works // around that limitation by iterating over the epochs. // This workaround is acceptable because: // 1) Extending EpochManager's API is a major change. // 2) This use case is not critical at all. loop { - let cur_epoch_info = epoch_manager.get_epoch_info(cur_block_info.epoch_id()).unwrap(); + let cur_epoch_info = + chain.runtime_adapter.get_epoch_info(cur_block_info.epoch_id()).unwrap(); let cur_epoch_height = cur_epoch_info.epoch_height(); assert!( cur_epoch_height >= epoch_info.epoch_height(), @@ -200,12 +206,12 @@ fn get_prev_hash_of_epoch( epoch_info.epoch_height() ); let epoch_first_block_info = - epoch_manager.get_block_info(cur_block_info.epoch_first_block()).unwrap(); + chain.runtime_adapter.get_block_info(cur_block_info.epoch_first_block()).unwrap(); let prev_epoch_last_block_info = - epoch_manager.get_block_info(epoch_first_block_info.prev_hash()).unwrap(); + chain.runtime_adapter.get_block_info(epoch_first_block_info.prev_hash()).unwrap(); if cur_epoch_height == epoch_info.epoch_height() { - return *prev_epoch_last_block_info.hash(); + return *cur_block_info.hash(); } cur_block_info = prev_epoch_last_block_info; @@ -218,58 +224,30 @@ fn apply_state_parts( part_id: Option, dry_run: bool, maybe_state_root: Option, - home_dir: &Path, - near_config: NearConfig, + chain: &mut Chain, + chain_id: &str, store: Store, location: Location, ) { - let runtime_adapter: Arc = - NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); - let epoch_manager = - EpochManager::new_from_genesis_config(store.clone(), &near_config.genesis.config) - .expect("Failed to start Epoch Manager"); - let chain_store = ChainStore::new( - store.clone(), - near_config.genesis.config.genesis_height, - near_config.client_config.save_trie_changes, - ); + let (state_root, epoch_height, epoch_id, sync_hash) = + if let (Some(state_root), EpochSelection::EpochHeight { epoch_height }) = + (maybe_state_root, &epoch_selection) + { + (state_root, *epoch_height, None, None) + } else { + let epoch_id = epoch_selection.to_epoch_id(store, &chain); + let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); - let (state_root, epoch_height, epoch_id, sync_prev_hash) = if let ( - Some(state_root), - EpochSelection::EpochHeight{epoch_height}, - ) = - (maybe_state_root, &epoch_selection) - { - (state_root, *epoch_height, None, None) - } else { - let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); - let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); - - let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); - let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); - tracing::info!( - target: "state-parts", - ?sync_prev_hash, - height = sync_prev_block.header().height(), - state_roots = ?sync_prev_block.chunks().iter().map(|chunk| chunk.prev_state_root()).collect::>()); + let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); + let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); - assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); - assert!( - shard_id < sync_prev_block.chunks().len() as u64, - "shard_id: {}, #shards: {}", - shard_id, - sync_prev_block.chunks().len() - ); - let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); - (state_root, epoch.epoch_height(), Some(epoch_id), Some(sync_prev_hash)) - }; + let state_header = chain.get_state_response_header(shard_id, sync_hash).unwrap(); + let state_root = state_header.chunk_prev_state_root(); - let part_storage = get_state_part_reader( - location, - &near_config.client_config.chain_id, - epoch_height, - shard_id, - ); + (state_root, epoch.epoch_height(), Some(epoch_id), Some(sync_hash)) + }; + + let part_storage = get_state_part_reader(location, &chain_id, epoch_height, shard_id); let num_parts = part_storage.num_parts(); assert_ne!(num_parts, 0, "Too few num_parts: {}", num_parts); @@ -279,7 +257,7 @@ fn apply_state_parts( epoch_height, shard_id, num_parts, - ?sync_prev_hash, + ?sync_hash, ?part_ids, "Applying state as seen at the beginning of the specified epoch.", ); @@ -291,14 +269,23 @@ fn apply_state_parts( let part = part_storage.read(part_id, num_parts); if dry_run { - assert!(runtime_adapter.validate_state_part( + assert!(chain.runtime_adapter.validate_state_part( &state_root, PartId::new(part_id, num_parts), &part )); tracing::info!(target: "state-parts", part_id, part_length = part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Validated a state part"); } else { - runtime_adapter + chain + .set_state_part( + shard_id, + sync_hash.unwrap(), + PartId::new(part_id, num_parts), + &part, + ) + .unwrap(); + chain + .runtime_adapter .apply_state_part( shard_id, &state_root, @@ -318,39 +305,19 @@ fn dump_state_parts( shard_id: ShardId, part_from: Option, part_to: Option, - home_dir: &Path, - near_config: NearConfig, + chain: &Chain, + chain_id: &str, store: Store, location: Location, ) { - let runtime_adapter: Arc = - NightshadeRuntime::from_config(home_dir, store.clone(), &near_config); - let epoch_manager = - EpochManager::new_from_genesis_config(store.clone(), &near_config.genesis.config) - .expect("Failed to start Epoch Manager"); - let chain_store = ChainStore::new( - store.clone(), - near_config.genesis.config.genesis_height, - near_config.client_config.save_trie_changes, - ); - - let epoch_id = epoch_selection.to_epoch_id(store, &chain_store, &epoch_manager); - let epoch = epoch_manager.get_epoch_info(&epoch_id).unwrap(); - let sync_prev_hash = get_prev_hash_of_epoch(&epoch, &chain_store, &epoch_manager); - let sync_prev_block = chain_store.get_block(&sync_prev_hash).unwrap(); - - assert!(epoch_manager.is_next_block_epoch_start(&sync_prev_hash).unwrap()); - assert!( - shard_id < sync_prev_block.chunks().len() as u64, - "shard_id: {}, #shards: {}", - shard_id, - sync_prev_block.chunks().len() - ); - let state_root = sync_prev_block.chunks()[shard_id as usize].prev_state_root(); - let state_root_node = - runtime_adapter.get_state_root_node(shard_id, &sync_prev_hash, &state_root).unwrap(); - - let num_parts = get_num_state_parts(state_root_node.memory_usage); + let epoch_id = epoch_selection.to_epoch_id(store, &chain); + let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); + let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); + let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); + + let state_header = chain.compute_state_response_header(shard_id, sync_hash).unwrap(); + let state_root = state_header.chunk_prev_state_root(); + let num_parts = get_num_state_parts(state_header.state_root_node().memory_usage); let part_ids = get_part_ids(part_from, part_to, num_parts); tracing::info!( @@ -359,40 +326,73 @@ fn dump_state_parts( epoch_id = ?epoch_id.0, shard_id, num_parts, - ?sync_prev_hash, + ?sync_hash, ?part_ids, + ?state_root, "Dumping state as seen at the beginning of the specified epoch.", ); - let part_storage = get_state_part_writer( - location, - &near_config.client_config.chain_id, - epoch.epoch_height(), - shard_id, - ); + let part_storage = get_state_part_writer(location, chain_id, epoch.epoch_height(), shard_id); let timer = Instant::now(); for part_id in part_ids { let timer = Instant::now(); assert!(part_id < num_parts, "part_id: {}, num_parts: {}", part_id, num_parts); - let state_part = runtime_adapter - .obtain_state_part( - shard_id, - &sync_prev_hash, - &state_root, - PartId::new(part_id, num_parts), - ) + let state_part = chain + .runtime_adapter + .obtain_state_part(shard_id, &sync_hash, &state_root, PartId::new(part_id, num_parts)) .unwrap(); part_storage.write(&state_part, part_id, num_parts); - tracing::info!(target: "state-parts", part_id, part_length = state_part.len(), elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote a state part"); + let elapsed_sec = timer.elapsed().as_secs_f64(); + let first_state_record = get_first_state_record(&state_root, &state_part); + tracing::info!( + target: "state-parts", + part_id, + part_length = state_part.len(), + elapsed_sec, + ?first_state_record, + "Wrote a state part"); } tracing::info!(target: "state-parts", total_elapsed_sec = timer.elapsed().as_secs_f64(), "Wrote all requested state parts"); } +/// Returns the first `StateRecord` encountered while iterating over a sub-trie in the state part. +fn get_first_state_record(state_root: &StateRoot, data: &[u8]) -> Option { + let trie_nodes = BorshDeserialize::try_from_slice(data).unwrap(); + let trie = Trie::from_recorded_storage(PartialStorage { nodes: trie_nodes }, *state_root); + + for item in trie.iter().unwrap() { + if let Ok((key, value)) = item { + if let Some(sr) = StateRecord::from_raw_key_value(key, value) { + return Some(sr); + } + } + } + None +} + +/// Reads `StateHeader` stored in the DB. +fn read_state_header( + epoch_selection: EpochSelection, + shard_id: ShardId, + chain: &Chain, + store: Store, +) { + let epoch_id = epoch_selection.to_epoch_id(store, &chain); + let epoch = chain.runtime_adapter.get_epoch_info(&epoch_id).unwrap(); + + let sync_hash = get_any_block_hash_of_epoch(&epoch, &chain); + let sync_hash = StateSync::get_epoch_start_sync_hash(&chain, &sync_hash).unwrap(); + + let state_header = chain.store().get_state_header(shard_id, sync_hash); + tracing::info!(target: "state-parts", ?epoch_id, ?sync_hash, ?state_header); +} + fn get_part_ids(part_from: Option, part_to: Option, num_parts: u64) -> Range { part_from.unwrap_or(0)..part_to.unwrap_or(num_parts) } +// Needs to be in sync with `fn s3_location()`. fn location_prefix(chain_id: &str, epoch_height: u64, shard_id: u64) -> String { format!("chain_id={}/epoch_height={}/shard_id={}", chain_id, epoch_height, shard_id) } From cd34b4a797360e95be79c7fe63e0fac2fb8082a4 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 5 Apr 2023 12:45:31 +0200 Subject: [PATCH 74/88] Merge --- chain/chain/src/store.rs | 6 +- nearcore/src/config.rs | 8 +- tools/state-viewer/src/cli.rs | 144 +++++++++++++--------------------- 3 files changed, 65 insertions(+), 93 deletions(-) diff --git a/chain/chain/src/store.rs b/chain/chain/src/store.rs index cbdc3a37ff2..22ec2b5d0ea 100644 --- a/chain/chain/src/store.rs +++ b/chain/chain/src/store.rs @@ -47,7 +47,6 @@ use crate::chunks_store::ReadOnlyChunksStore; use crate::types::{Block, BlockHeader, LatestKnown}; use crate::{byzantine_assert, RuntimeWithEpochManagerAdapter}; use near_store::db::StoreStatistics; -use near_store::flat_state::{BlockInfo, ChainAccessForFlatStorage}; use std::sync::Arc; /// lru cache size @@ -2438,7 +2437,10 @@ impl<'a> ChainStoreUpdate<'a> { unreachable!(); } #[cfg(feature = "protocol_feature_flat_state")] - DBCol::FlatState | DBCol::FlatStateDeltas | DBCol::FlatStateMisc => { + DBCol::FlatState + | DBCol::FlatStateChanges + | DBCol::FlatStateDeltaMetadata + | DBCol::FlatStorageStatus => { unreachable!(); } } diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index 2f61744bd48..b6a27422ff8 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -313,8 +313,8 @@ pub struct Config { /// save_trie_changes = !archive /// save_trie_changes should be set to true iff /// - archive if false - non-archival nodes need trie changes to perform garbage collection - /// - archive is true, cold_store is configured and migration to split_storage is finished - node - /// working in split storage mode needs trie changes in order to do garbage collection on hot. + /// - archive is true and cold_store is configured - node working in split storage mode + /// needs trie changes in order to do garbage collection on hot and populate cold State column. #[serde(skip_serializing_if = "Option::is_none")] pub save_trie_changes: Option, pub log_summary_style: LogSummaryStyle, @@ -356,6 +356,9 @@ pub struct Config { /// Options for dumping state of every epoch to S3. #[serde(skip_serializing_if = "Option::is_none")] pub state_sync: Option, + /// Whether to use state sync (unreliable and corrupts the DB if fails) or do a block sync instead. + #[serde(skip_serializing_if = "is_false")] + pub state_sync_enabled: bool, } fn is_false(value: &bool) -> bool { @@ -393,6 +396,7 @@ impl Default for Config { split_storage: None, expected_shutdown: None, state_sync: None, + state_sync_enabled: false, } } } diff --git a/tools/state-viewer/src/cli.rs b/tools/state-viewer/src/cli.rs index fa4cafa5441..a3218119f07 100644 --- a/tools/state-viewer/src/cli.rs +++ b/tools/state-viewer/src/cli.rs @@ -1,8 +1,6 @@ use crate::commands::*; use crate::contract_accounts::ContractAccountFilter; use crate::rocksdb_stats::get_rocksdb_stats; -use crate::state_parts::{apply_state_parts, dump_state_parts}; -use crate::{epoch_info, state_parts}; use near_chain_configs::{GenesisChangeConfig, GenesisValidationMode}; use near_primitives::account::id::AccountId; use near_primitives::hash::CryptoHash; @@ -28,8 +26,6 @@ pub enum StateViewerSubCommand { /// even if it's not included in any block on disk #[clap(alias = "apply_receipt")] ApplyReceipt(ApplyReceiptCmd), - /// Apply all or a single state part of a shard. - ApplyStateParts(ApplyStatePartsCmd), /// Apply a transaction if it occurs in some chunk we know about, /// even if it's not included in any block on disk #[clap(alias = "apply_tx")] @@ -53,8 +49,6 @@ pub enum StateViewerSubCommand { /// Generate a genesis file from the current state of the DB. #[clap(alias = "dump_state")] DumpState(DumpStateCmd), - /// Dump all or a single state part of a shard. - DumpStateParts(DumpStatePartsCmd), /// Writes state to a remote redis server. #[clap(alias = "dump_state_redis")] DumpStateRedis(DumpStateRedisCmd), @@ -76,6 +70,11 @@ pub enum StateViewerSubCommand { RocksDBStats(RocksDBStatsCmd), /// Iterates over a trie and prints the StateRecords. State, + /// Dumps or applies StateChanges. + /// Experimental tool for shard shadowing development. + StateChanges(StateChangesCmd), + /// Dump or apply state parts. + StateParts(StatePartsCmd), /// View head of the storage. #[clap(alias = "view_chain")] ViewChain(ViewChainCmd), @@ -106,7 +105,8 @@ impl StateViewerSubCommand { let storage = store_opener.open_in_mode(mode).unwrap(); let store = match temperature { Temperature::Hot => storage.get_hot_store(), - Temperature::Cold => storage.get_cold_store().unwrap(), + // Cold store on it's own is useless in majority of subcommands + Temperature::Cold => storage.get_split_store().unwrap(), }; match self { @@ -114,7 +114,6 @@ impl StateViewerSubCommand { StateViewerSubCommand::ApplyChunk(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ApplyRange(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ApplyReceipt(cmd) => cmd.run(home_dir, near_config, store), - StateViewerSubCommand::ApplyStateParts(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ApplyTx(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::Chain(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::CheckBlock => check_block_chunk_existence(near_config, store), @@ -123,7 +122,6 @@ impl StateViewerSubCommand { StateViewerSubCommand::DumpAccountStorage(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpCode(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpState(cmd) => cmd.run(home_dir, near_config, store), - StateViewerSubCommand::DumpStateParts(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpStateRedis(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::DumpTx(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::EpochInfo(cmd) => cmd.run(home_dir, near_config, store), @@ -132,6 +130,8 @@ impl StateViewerSubCommand { StateViewerSubCommand::Replay(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::RocksDBStats(cmd) => cmd.run(store_opener.path()), StateViewerSubCommand::State => state(home_dir, near_config, store), + StateViewerSubCommand::StateChanges(cmd) => cmd.run(home_dir, near_config, store), + StateViewerSubCommand::StateParts(cmd) => cmd.run(home_dir, near_config, store), StateViewerSubCommand::ViewChain(cmd) => cmd.run(near_config, store), StateViewerSubCommand::ViewTrie(cmd) => cmd.run(store), } @@ -215,42 +215,6 @@ impl ApplyReceiptCmd { } } -#[derive(clap::Parser)] -pub struct ApplyStatePartsCmd { - /// Selects an epoch. The dump will be of the state at the beginning of this epoch. - #[clap(subcommand)] - epoch_selection: state_parts::EpochSelection, - /// Shard id. - #[clap(long)] - shard_id: ShardId, - /// State part id. Leave empty to go through every part in the shard. - #[clap(long)] - part_id: Option, - /// Where to write the state parts to. - #[clap(long)] - root_dir: Option, - /// Store state parts in an S3 bucket. - #[clap(long)] - s3_bucket: Option, - /// Store state parts in an S3 bucket. - #[clap(long)] - s3_region: Option, -} - -impl ApplyStatePartsCmd { - pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { - apply_state_parts( - self.epoch_selection, - self.shard_id, - self.part_id, - home_dir, - near_config, - store, - state_parts::Location::new(self.root_dir, (self.s3_bucket, self.s3_region)), - ); - } -} - #[derive(clap::Parser)] pub struct ApplyTxCmd { #[clap(long)] @@ -398,49 +362,6 @@ impl DumpStateCmd { } } -#[derive(clap::Parser)] -pub struct DumpStatePartsCmd { - /// Selects an epoch. The dump will be of the state at the beginning of this epoch. - #[clap(subcommand)] - epoch_selection: state_parts::EpochSelection, - /// Shard id. - #[clap(long)] - shard_id: ShardId, - /// Dump a single part id. - #[clap(long)] - part_id: Option, - /// Dump part ids starting from this part. - #[clap(long)] - part_from: Option, - /// Dump part ids up to this part (exclusive). - #[clap(long)] - part_to: Option, - /// Where to write the state parts to. - #[clap(long)] - root_dir: Option, - /// Store state parts in an S3 bucket. - #[clap(long)] - s3_bucket: Option, - /// S3 region to store state parts. - #[clap(long)] - s3_region: Option, -} - -impl DumpStatePartsCmd { - pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { - dump_state_parts( - self.epoch_selection, - self.shard_id, - self.part_from.or(self.part_id), - self.part_to.or(self.part_id.map(|x| x + 1)), - home_dir, - near_config, - store, - state_parts::Location::new(self.root_dir, (self.s3_bucket, self.s3_region)), - ); - } -} - #[derive(clap::Parser)] pub struct DumpStateRedisCmd { /// Optionally, can specify at which height to dump state. @@ -489,7 +410,7 @@ impl DumpTxCmd { #[derive(clap::Args)] pub struct EpochInfoCmd { #[clap(subcommand)] - epoch_selection: epoch_info::EpochSelection, + epoch_selection: crate::epoch_info::EpochSelection, /// Displays kickouts of the given validator and expected and missed blocks and chunks produced. #[clap(long)] validator_account_id: Option, @@ -560,6 +481,18 @@ impl RocksDBStatsCmd { } } +#[derive(clap::Parser)] +pub struct StateChangesCmd { + #[clap(subcommand)] + command: crate::state_changes::StateChangesSubCommand, +} + +impl StateChangesCmd { + pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { + self.command.run(home_dir, near_config, store) + } +} + #[derive(clap::Parser)] pub struct ViewChainCmd { #[clap(long)] @@ -639,3 +572,36 @@ impl ViewTrieCmd { } } } + +#[derive(clap::Parser)] +pub struct StatePartsCmd { + /// Shard id. + #[clap(long)] + shard_id: ShardId, + /// Location of serialized state parts. + #[clap(long)] + root_dir: Option, + /// Store state parts in an S3 bucket. + #[clap(long)] + s3_bucket: Option, + /// Store state parts in an S3 bucket. + #[clap(long)] + s3_region: Option, + /// Dump or Apply state parts. + #[clap(subcommand)] + command: crate::state_parts::StatePartsSubCommand, +} + +impl StatePartsCmd { + pub fn run(self, home_dir: &Path, near_config: NearConfig, store: Store) { + self.command.run( + self.shard_id, + self.root_dir, + self.s3_bucket, + self.s3_region, + home_dir, + near_config, + store, + ); + } +} From 8f2c9b41e18b2587b0603233a9dbd419550c84e2 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 5 Apr 2023 12:53:05 +0200 Subject: [PATCH 75/88] . --- chain/client/src/metrics.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/chain/client/src/metrics.rs b/chain/client/src/metrics.rs index 4683ca3ca40..8de327457a1 100644 --- a/chain/client/src/metrics.rs +++ b/chain/client/src/metrics.rs @@ -394,7 +394,7 @@ pub static STATE_SYNC_PARTS_TOTAL: Lazy = Lazy: pub static STATE_SYNC_DISCARD_PARTS: Lazy = Lazy::new(|| { try_create_int_counter_vec( "near_state_sync_discard_parts_total", - "Number of times all downloaded parts so far were discarded to try again", + "Number of times all downloaded parts were discarded to try again", &["shard_id"], ) .unwrap() From faa8a104ac1753aa875f78660e9e03c6e5076640 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Wed, 5 Apr 2023 13:49:53 +0200 Subject: [PATCH 76/88] Merge --- chain/client/src/metrics.rs | 10 ---------- chain/client/src/sync/state.rs | 12 +++++++----- core/chain-configs/src/client_config.rs | 9 +++++++-- nearcore/src/config.rs | 26 +++++++++++++++---------- nearcore/src/metrics.rs | 2 +- nearcore/src/state_sync.rs | 5 ++--- 6 files changed, 33 insertions(+), 31 deletions(-) diff --git a/chain/client/src/metrics.rs b/chain/client/src/metrics.rs index 8de327457a1..535d1106a16 100644 --- a/chain/client/src/metrics.rs +++ b/chain/client/src/metrics.rs @@ -441,13 +441,3 @@ pub static STATE_SYNC_EXTERNAL_PARTS_REQUEST_DELAY: Lazy = - Lazy::new(|| { - try_create_int_counter_vec( - "near_state_sync_external_parts_size_downloaded_bytes_total", - "Amount of bytes downloaded from an external storage when requesting state parts for a shard", - &["shard_id"], - ) - .unwrap() - }); diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index e93952671f3..49dd1ba29bc 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -92,7 +92,7 @@ fn make_account_or_peer_id_or_hash( From::AccountId(a) => To::AccountId(a), From::PeerId(p) => To::PeerId(p), From::Hash(h) => To::Hash(h), - From::ExternalStorage() => To::ExternalStorage(), + From::ExternalStorage => To::ExternalStorage, } } @@ -707,7 +707,7 @@ impl StateSync { } download.state_requests_count += 1; download.last_target = - Some(make_account_or_peer_id_or_hash(AccountOrPeerIdOrHash::ExternalStorage())); + Some(make_account_or_peer_id_or_hash(AccountOrPeerIdOrHash::ExternalStorage)); let location = s3_location(chain_id, epoch_height, shard_id, part_id, num_parts); let download_response = download.response.clone(); @@ -1236,17 +1236,19 @@ fn check_external_storage_part_response( .with_label_values(&[&shard_id.to_string()]) .inc(); tracing::warn!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to save a state part"); - err_to_retry = Some(Error::Other("Failed to save a state part".to_string())); + err_to_retry = + Some(near_chain::Error::Other("Failed to save a state part".to_string())); } } } // Other HTTP status codes are considered errors. Ok((status_code, _)) => { - err_to_retry = Some(Error::Other(format!("status_code: {}", status_code).to_string())); + err_to_retry = + Some(near_chain::Error::Other(format!("status_code: {}", status_code).to_string())); } // The request failed without reaching the external storage. Err(err) => { - err_to_retry = Some(Error::Other(err.to_string())); + err_to_retry = Some(near_chain::Error::Other(err.to_string())); } }; diff --git a/core/chain-configs/src/client_config.rs b/core/chain-configs/src/client_config.rs index c71ae93a895..797bb43644b 100644 --- a/core/chain-configs/src/client_config.rs +++ b/core/chain-configs/src/client_config.rs @@ -179,7 +179,10 @@ pub struct ClientConfig { /// If disabled will perform state sync from the peers. pub state_sync_from_s3_enabled: bool, /// Number of parallel in-flight requests allowed per shard. - pub state_sync_num_s3_requests_per_shard: u64, + pub state_sync_num_concurrent_s3_requests: u64, + /// Whether to use the State Sync mechanism. + /// If disabled, the node will do Block Sync instead of State Sync. + pub state_sync_enabled: bool, } impl ClientConfig { @@ -253,7 +256,9 @@ impl ClientConfig { state_sync_s3_bucket: String::new(), state_sync_s3_region: String::new(), state_sync_restart_dump_for_shards: vec![], - state_sync_num_s3_requests_per_shard: 10, + state_sync_from_s3_enabled: false, + state_sync_num_concurrent_s3_requests: 10, + state_sync_enabled: false, } } } diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index b6a27422ff8..195f85e6ce4 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -357,8 +357,8 @@ pub struct Config { #[serde(skip_serializing_if = "Option::is_none")] pub state_sync: Option, /// Whether to use state sync (unreliable and corrupts the DB if fails) or do a block sync instead. - #[serde(skip_serializing_if = "is_false")] - pub state_sync_enabled: bool, + #[serde(skip_serializing_if = "Option::is_none")] + pub state_sync_enabled: Option, } fn is_false(value: &bool) -> bool { @@ -396,7 +396,7 @@ impl Default for Config { split_storage: None, expected_shutdown: None, state_sync: None, - state_sync_enabled: false, + state_sync_enabled: None, } } } @@ -709,6 +709,7 @@ impl NearConfig { .state_sync .as_ref() .map(|x| x.dump_enabled) + .flatten() .unwrap_or(false), state_sync_s3_bucket: config .state_sync @@ -723,7 +724,7 @@ impl NearConfig { state_sync_restart_dump_for_shards: config .state_sync .as_ref() - .map(|x| x.drop_state_of_dump.clone()) + .map(|x| x.restart_dump_for_shards.clone()) .flatten() .unwrap_or(vec![]), state_sync_from_s3_enabled: config @@ -732,12 +733,13 @@ impl NearConfig { .map(|x| x.sync_from_s3_enabled) .flatten() .unwrap_or(false), - state_sync_num_s3_requests_per_shard: config + state_sync_num_concurrent_s3_requests: config .state_sync .as_ref() - .map(|x| x.num_s3_requests_per_shard) + .map(|x| x.num_concurrent_s3_requests) .flatten() .unwrap_or(100), + state_sync_enabled: config.state_sync_enabled.unwrap_or(false), }, network_config: NetworkConfig::new( config.network, @@ -1571,14 +1573,18 @@ pub struct StateSyncConfig { /// Whether a node should dump state of each epoch to the external storage. #[serde(skip_serializing_if = "Option::is_none")] pub dump_enabled: Option, - /// Use carefully in case a node that dumps state to the external storage gets in trouble. + /// Use carefully in case a node that dumps state to the external storage + /// gets in trouble. #[serde(skip_serializing_if = "Option::is_none")] - pub drop_state_of_dump: Option>, - /// If enabled, will download state parts from external storage and not from the peers. + pub restart_dump_for_shards: Option>, + /// If enabled, will download state parts from external storage and not from + /// the peers. #[serde(skip_serializing_if = "Option::is_none")] pub sync_from_s3_enabled: Option, + /// When syncing state from S3, throttle requests to this many concurrent + /// requests per shard. #[serde(skip_serializing_if = "Option::is_none")] - pub num_s3_requests_per_shard: Option, + pub num_concurrent_s3_requests: Option, } #[test] diff --git a/nearcore/src/metrics.rs b/nearcore/src/metrics.rs index 19605ceeca1..125cd923d52 100644 --- a/nearcore/src/metrics.rs +++ b/nearcore/src/metrics.rs @@ -70,7 +70,7 @@ pub(crate) static STATE_SYNC_DUMP_SIZE_TOTAL: Lazy = Lazy::new(|| try_create_int_counter_vec( "near_state_sync_dump_size_total", "Total size of parts written to S3", - &["shard_id"], + &["epoch_height", "shard_id"], ) .unwrap() }); diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index 0df51e42802..cf680c9d934 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -119,7 +119,6 @@ async fn state_sync_dump( _node_key: PublicKey, ) { tracing::info!(target: "state_sync_dump", shard_id, "Running StateSyncDump loop"); - let mut interval = actix_rt::time::interval(std::time::Duration::from_secs(10)); if config.state_sync_restart_dump_for_shards.contains(&shard_id) { tracing::debug!(target: "state_sync_dump", shard_id, "Dropped existing progress"); @@ -128,7 +127,7 @@ async fn state_sync_dump( loop { // Avoid a busy-loop when there is nothing to do. - interval.tick().await; + std::thread::sleep(std::time::Duration::from_secs(10)); let progress = chain.store().get_state_sync_dump_progress(shard_id); tracing::debug!(target: "state_sync_dump", shard_id, ?progress, "Running StateSyncDump loop iteration"); @@ -287,7 +286,7 @@ fn update_progress( ) { // Record that a part was obtained and dumped. metrics::STATE_SYNC_DUMP_SIZE_TOTAL - .with_label_values(&[&shard_id.to_string()]) + .with_label_values(&[&epoch_height.to_string(), &shard_id.to_string()]) .inc_by(part_len as u64); let next_progress = StateSyncDumpProgress::InProgress { epoch_id: epoch_id.clone(), From cfce87cf39239acf5c65a12aae3e9507b3d63ece Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 6 Apr 2023 19:46:22 +0200 Subject: [PATCH 77/88] Introduce StateSyncInner instead of StateSyncMode --- chain/client-primitives/src/types.rs | 3 +- chain/client/src/sync/state.rs | 457 ++++++++++++++------------- nearcore/src/config_validate.rs | 27 +- nearcore/src/lib.rs | 7 +- nearcore/src/state_sync.rs | 19 +- 5 files changed, 257 insertions(+), 256 deletions(-) diff --git a/chain/client-primitives/src/types.rs b/chain/client-primitives/src/types.rs index 3a104050d0b..ac9a77d2c74 100644 --- a/chain/client-primitives/src/types.rs +++ b/chain/client-primitives/src/types.rs @@ -82,6 +82,8 @@ impl DownloadStatus { } impl Clone for DownloadStatus { + /// Clones an object, but it clones the value of `run_me` instead of the + /// `Arc` that wraps that value. fn clone(&self) -> Self { DownloadStatus { start_time: self.start_time, @@ -92,7 +94,6 @@ impl Clone for DownloadStatus { done: self.done, state_requests_count: self.state_requests_count, last_target: self.last_target.clone(), - // Clones the `Arc` containing the value. response: self.response.clone(), } } diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 49dd1ba29bc..d6eb0ef4046 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -97,54 +97,48 @@ fn make_account_or_peer_id_or_hash( } /// How to retrieve the state data. -pub enum StateSyncMode { +enum StateSyncInner { /// Request both the state header and state parts from the peers. - Peers, + Peers { + /// Information about which parts were requested from which peer and when. + last_part_id_requested: HashMap<(AccountOrPeerIdOrHash, ShardId), PendingRequestStatus>, + /// Map from which part we requested to whom. + requested_target: lru::LruCache<(u64, CryptoHash), AccountOrPeerIdOrHash>, + }, /// Requests the state header from peers but gets the state parts from an /// external storage. - HeaderFromPeersAndPartsFromExternal { + PartsFromExternal { /// Chain ID. chain_id: String, /// Connection to the external storage. bucket: Arc, - /// Number state part requests allowed to be in-flight in parallel per shard. - num_s3_requests_per_shard: u64, + /// The number of requests for state parts from external storage that are + /// allowed to be started for this shard. + requests_remaining: Arc, }, } -/// Information about which parts were requested from which peer and when. -pub struct PartsRequestState { - last_part_id_requested: HashMap<(AccountOrPeerIdOrHash, ShardId), PendingRequestStatus>, - /// Map from which part we requested to whom. - requested_target: lru::LruCache<(u64, CryptoHash), AccountOrPeerIdOrHash>, -} - /// Helper to track state sync. pub struct StateSync { /// How to retrieve the state data. - mode: StateSyncMode, + inner: StateSyncInner, /// Is used for communication with the peers. network_adapter: PeerManagerAdapter, + /// When the "sync block" was requested. + /// The "sync block" is the last block of the previous epoch, i.e. `prev_hash` of the `sync_hash` block. last_time_block_requested: Option>, - /// Information about which parts were requested from which peer and when. - parts_request_state: Option, - /// Timeout (set in config - by default to 60 seconds) is used to figure out how long we should wait /// for the answer from the other node before giving up. timeout: Duration, - /// Maps shard_id to result of applying downloaded state + /// Maps shard_id to result of applying downloaded state. state_parts_apply_results: HashMap>, - /// Maps shard_id to result of splitting state for resharding + /// Maps shard_id to result of splitting state for resharding. split_state_roots: HashMap, near_chain::Error>>, - - /// The number of requests for state parts from external storage that are - /// allowed to be started for this shard. - requests_remaining: Arc, } impl StateSync { @@ -157,9 +151,7 @@ impl StateSync { s3_region: &str, num_s3_requests_per_shard: u64, ) -> Self { - let (mode, parts_request_state) = if state_sync_from_s3_enabled { - tracing::debug!(target: "sync", s3_bucket, s3_region, "Initializing S3 bucket connection."); - assert!(!s3_bucket.is_empty() && !s3_region.is_empty(), "State sync from S3 is enabled. This requires that both `s3_bucket and `s3_region` and specified and non-empty"); + let inner = if state_sync_from_s3_enabled { let mut bucket = s3::Bucket::new( s3_bucket, s3_region.parse::().unwrap(), @@ -169,33 +161,26 @@ impl StateSync { // Ensure requests finish in finite amount of time. bucket.set_request_timeout(Some(timeout)); let bucket = Arc::new(bucket); - ( - StateSyncMode::HeaderFromPeersAndPartsFromExternal { - chain_id: chain_id.to_string(), - bucket, - num_s3_requests_per_shard, - }, - None, - ) + + StateSyncInner::PartsFromExternal { + chain_id: chain_id.to_string(), + bucket, + requests_remaining: Arc::new(AtomicI64::new(num_s3_requests_per_shard as i64)), + } } else { - ( - StateSyncMode::Peers, - Some(PartsRequestState { - last_part_id_requested: Default::default(), - requested_target: lru::LruCache::new(MAX_PENDING_PART as usize), - }), - ) + StateSyncInner::Peers { + last_part_id_requested: Default::default(), + requested_target: lru::LruCache::new(MAX_PENDING_PART as usize), + } }; let timeout = Duration::from_std(timeout).unwrap(); StateSync { - mode, + inner, network_adapter, last_time_block_requested: None, - parts_request_state, timeout, state_parts_apply_results: HashMap::new(), split_state_roots: HashMap::new(), - requests_remaining: Arc::new(AtomicI64::new(num_s3_requests_per_shard as i64)), } } @@ -425,25 +410,6 @@ impl StateSync { } } - fn sent_request_part( - &mut self, - target: AccountOrPeerIdOrHash, - part_id: u64, - shard_id: ShardId, - sync_hash: CryptoHash, - ) { - // FIXME: something is wrong - the index should have a shard_id too. - let parts_request_state = self.parts_request_state.as_mut().unwrap(); - parts_request_state.requested_target.put((part_id, sync_hash), target.clone()); - parts_request_state - .last_part_id_requested - .entry((target, shard_id)) - .and_modify(|pending_request| { - pending_request.missing_parts += 1; - }) - .or_insert_with(|| PendingRequestStatus::new(self.timeout)); - } - // Function called when our node receives the network response with a part. pub fn received_requested_part( &mut self, @@ -451,19 +417,24 @@ impl StateSync { shard_id: ShardId, sync_hash: CryptoHash, ) { - let key = (part_id, sync_hash); - // Check that it came from the target that we requested it from. - let parts_request_state = self.parts_request_state.as_mut().unwrap(); - if let Some(target) = parts_request_state.requested_target.get(&key) { - if parts_request_state - .last_part_id_requested - .get_mut(&(target.clone(), shard_id)) - .map_or(false, |request| { - request.missing_parts = request.missing_parts.saturating_sub(1); - request.missing_parts == 0 - }) - { - parts_request_state.last_part_id_requested.remove(&(target.clone(), shard_id)); + match &mut self.inner { + StateSyncInner::Peers { last_part_id_requested, requested_target } => { + let key = (part_id, sync_hash); + // Check that it came from the target that we requested it from. + if let Some(target) = requested_target.get(&key) { + if last_part_id_requested.get_mut(&(target.clone(), shard_id)).map_or( + false, + |request| { + request.missing_parts = request.missing_parts.saturating_sub(1); + request.missing_parts == 0 + }, + ) { + last_part_id_requested.remove(&(target.clone(), shard_id)); + } + } + } + StateSyncInner::PartsFromExternal { .. } => { + // Do nothing. } } } @@ -480,17 +451,12 @@ impl StateSync { sync_hash: CryptoHash, highest_height_peers: &[HighestHeightPeerInfo], ) -> Result, near_chain::Error> { - // Remove candidates from pending list if request expired due to timeout - self.parts_request_state.as_mut().map(|parts_request_state| { - parts_request_state.last_part_id_requested.retain(|_, request| !request.expired()) - }); - let prev_block_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); let epoch_hash = runtime_adapter.get_epoch_id_from_prev_block(&prev_block_hash)?; let block_producers = runtime_adapter.get_epoch_block_producers_ordered(&epoch_hash, &sync_hash)?; - let peers = block_producers + let mut peers = block_producers .iter() .filter_map(|(validator_stake, _slashed)| { let account_id = validator_stake.account_id(); @@ -521,19 +487,30 @@ impl StateSync { None } })); - let result = if let Some(parts_request_state) = &self.parts_request_state { - peers - .filter(|candidate| { - // If we still have a pending request from this node - don't add another one. - !parts_request_state - .last_part_id_requested - .contains_key(&(candidate.clone(), shard_id)) - }) - .collect::>() - } else { - peers.collect::>() + Ok(self.select_peers(&mut peers, shard_id)?) + } + + fn select_peers( + &mut self, + peers: &mut dyn Iterator, + shard_id: ShardId, + ) -> Result, near_chain::Error> { + let res = match &mut self.inner { + StateSyncInner::Peers { + last_part_id_requested, + requested_target: _requested_target, + } => { + last_part_id_requested.retain(|_, request| !request.expired()); + peers + .filter(|candidate| { + // If we still have a pending request from this node - don't add another one. + !last_part_id_requested.contains_key(&(candidate.clone(), shard_id)) + }) + .collect::>() + } + StateSyncInner::PartsFromExternal { .. } => peers.collect::>(), }; - Ok(result) + Ok(res) } /// Returns new ShardSyncDownload if successful, otherwise returns given shard_sync_download @@ -628,163 +605,67 @@ impl StateSync { new_shard_sync_download: &mut ShardSyncDownload, chain: &Chain, ) { - // We'll select all the 'highest' peers + validators as candidates (excluding those that gave us timeout in the past). - // And for each one of them, we'll ask for up to 16 (MAX_STATE_PART_REQUEST) parts. - let mut possible_targets_sampler = - SamplerLimited::new(possible_targets, MAX_STATE_PART_REQUEST); - - let sync_block_header = chain.get_block_header(&sync_hash).unwrap(); - let epoch_id = sync_block_header.epoch_id(); - let epoch_info = chain.runtime_adapter.get_epoch_info(epoch_id).unwrap(); - let epoch_height = epoch_info.epoch_height(); - - let shard_state_header = chain.get_state_header(shard_id, sync_hash).unwrap(); - let state_num_parts = - get_num_state_parts(shard_state_header.state_root_node().memory_usage); - // Iterate over all parts that needs to be requested (i.e. download.run_me is true). // Parts are ordered such that its index match its part_id. - for (part_id, download) in new_shard_sync_download - .downloads - .iter_mut() - .enumerate() - .filter(|(_, download)| download.run_me.load(Ordering::SeqCst)) - { - match &self.mode { - StateSyncMode::Peers => { + match &mut self.inner { + StateSyncInner::Peers { last_part_id_requested, requested_target } => { + // We'll select all the 'highest' peers + validators as candidates (excluding those that gave us timeout in the past). + // And for each one of them, we'll ask for up to 16 (MAX_STATE_PART_REQUEST) parts. + let mut possible_targets_sampler = + SamplerLimited::new(possible_targets, MAX_STATE_PART_REQUEST); + + for (part_id, download) in parts_to_fetch(new_shard_sync_download) { // For every part that needs to be requested it is selected one // peer (target) randomly to request the part from. // IMPORTANT: here we use 'zip' with possible_target_sampler - // which is limited. So at any moment we'll not request more // than possible_targets.len() * MAX_STATE_PART_REQUEST parts. let target = possible_targets_sampler.next().unwrap(); - self.request_part_from_peers( - part_id as u64, + sent_request_part( + target.clone(), + part_id, + shard_id, + sync_hash, + last_part_id_requested, + requested_target, + self.timeout, + ); + request_part_from_peers( + part_id, target, download, shard_id, sync_hash, + &self.network_adapter, ); } - StateSyncMode::HeaderFromPeersAndPartsFromExternal { - chain_id, - bucket, - num_s3_requests_per_shard: _, - } => { - self.request_part_from_external_storage( - part_id as u64, + } + StateSyncInner::PartsFromExternal { chain_id, bucket, requests_remaining } => { + let sync_block_header = chain.get_block_header(&sync_hash).unwrap(); + let epoch_id = sync_block_header.epoch_id(); + let epoch_info = chain.runtime_adapter.get_epoch_info(epoch_id).unwrap(); + let epoch_height = epoch_info.epoch_height(); + + let shard_state_header = chain.get_state_header(shard_id, sync_hash).unwrap(); + let state_num_parts = + get_num_state_parts(shard_state_header.state_root_node().memory_usage); + + for (part_id, download) in parts_to_fetch(new_shard_sync_download) { + request_part_from_external_storage( + part_id, download, shard_id, epoch_height, state_num_parts, - &chain_id, + &chain_id.clone(), bucket.clone(), + requests_remaining.clone(), ); } } } } - /// Starts an asynchronous network request to external storage to obtain the given state part. - fn request_part_from_external_storage( - &self, - part_id: u64, - download: &mut DownloadStatus, - shard_id: ShardId, - epoch_height: EpochHeight, - num_parts: u64, - chain_id: &str, - bucket: Arc, - ) { - let requests_remaining = self.requests_remaining.clone(); - if !allow_request(&requests_remaining) { - return; - } else { - if !download.run_me.swap(false, Ordering::SeqCst) { - tracing::info!(target: "sync", %shard_id, part_id, "run_me is already false"); - return; - } - } - download.state_requests_count += 1; - download.last_target = - Some(make_account_or_peer_id_or_hash(AccountOrPeerIdOrHash::ExternalStorage)); - - let location = s3_location(chain_id, epoch_height, shard_id, part_id, num_parts); - let download_response = download.response.clone(); - let scheduled = StaticClock::utc(); - near_performance_metrics::actix::spawn(std::any::type_name::(), { - async move { - tracing::info!(target: "sync", %shard_id, part_id, location, "Getting an object from the external storage"); - let started = StaticClock::utc(); - metrics::STATE_SYNC_EXTERNAL_PARTS_SCHEDULING_DELAY - .with_label_values(&[&shard_id.to_string()]) - .observe( - started.signed_duration_since(scheduled).num_nanoseconds().unwrap_or(0) - as f64 - / 1e9, - ); - let result = bucket.get_object(location.clone()).await; - let completed = StaticClock::utc(); - finished_request(&requests_remaining); - metrics::STATE_SYNC_EXTERNAL_PARTS_REQUEST_DELAY - .with_label_values(&[&shard_id.to_string()]) - .observe( - completed.signed_duration_since(started).num_nanoseconds().unwrap_or(0) - as f64 - / 1e9, - ); - match result { - Ok(response) => { - tracing::info!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), "S3 request finished"); - let mut lock = download_response.lock().unwrap(); - *lock = Some(Ok((response.status_code(), response.bytes().to_vec()))); - } - Err(err) => { - tracing::info!(target: "sync", %shard_id, part_id, location, ?err, "S3 request failed"); - let mut lock = download_response.lock().unwrap(); - *lock = Some(Err(err.to_string())); - } - } - } - }); - } - - /// Asynchronously requests a state part from a suitable peer. - fn request_part_from_peers( - &mut self, - part_id: u64, - target: AccountOrPeerIdOrHash, - download: &mut DownloadStatus, - shard_id: ShardId, - sync_hash: CryptoHash, - ) { - self.sent_request_part(target.clone(), part_id, shard_id, sync_hash); - download.run_me.store(false, Ordering::SeqCst); - download.state_requests_count += 1; - download.last_target = Some(make_account_or_peer_id_or_hash(target.clone())); - let run_me = download.run_me.clone(); - - near_performance_metrics::actix::spawn( - std::any::type_name::(), - self.network_adapter - .send_async(PeerManagerMessageRequest::NetworkRequests( - NetworkRequests::StateRequestPart { shard_id, sync_hash, part_id, target }, - )) - .then(move |result| { - // TODO: possible optimization - in the current code, even if one of the targets it not present in the network graph - // (so we keep getting RouteNotFound) - we'll still keep trying to assign parts to it. - // Fortunately only once every 60 seconds (timeout value). - if let Ok(NetworkResponses::RouteNotFound) = - result.map(|f| f.as_network_response()) - { - // Send a StateRequestPart on the next iteration - run_me.store(true, Ordering::SeqCst); - } - future::ready(()) - }), - ); - } - /// The main 'step' function that should be called periodically to check and update the sync process. /// The current state/progress information is mostly kept within 'new_shard_sync' object. /// @@ -1174,6 +1055,132 @@ impl StateSync { } } +/// Returns parts that still need to be fetched. +fn parts_to_fetch( + new_shard_sync_download: &mut ShardSyncDownload, +) -> impl Iterator { + new_shard_sync_download + .downloads + .iter_mut() + .enumerate() + .filter(|(_, download)| download.run_me.load(Ordering::SeqCst)) + .map(|(part_id, download)| (part_id as u64, download)) +} + +/// Starts an asynchronous network request to external storage to fetch the given state part. +fn request_part_from_external_storage( + part_id: u64, + download: &mut DownloadStatus, + shard_id: ShardId, + epoch_height: EpochHeight, + num_parts: u64, + chain_id: &str, + bucket: Arc, + requests_remaining: Arc, +) { + if !allow_request(&requests_remaining) { + return; + } else { + if !download.run_me.swap(false, Ordering::SeqCst) { + tracing::info!(target: "sync", %shard_id, part_id, "run_me is already false"); + return; + } + } + download.state_requests_count += 1; + download.last_target = + Some(make_account_or_peer_id_or_hash(AccountOrPeerIdOrHash::ExternalStorage)); + + let location = s3_location(chain_id, epoch_height, shard_id, part_id, num_parts); + let download_response = download.response.clone(); + let scheduled = StaticClock::utc(); + near_performance_metrics::actix::spawn("StateSync", { + async move { + tracing::info!(target: "sync", %shard_id, part_id, location, "Getting an object from the external storage"); + let started = StaticClock::utc(); + metrics::STATE_SYNC_EXTERNAL_PARTS_SCHEDULING_DELAY + .with_label_values(&[&shard_id.to_string()]) + .observe( + started.signed_duration_since(scheduled).num_nanoseconds().unwrap_or(0) as f64 + / 1e9, + ); + let result = bucket.get_object(location.clone()).await; + let completed = StaticClock::utc(); + finished_request(&requests_remaining); + metrics::STATE_SYNC_EXTERNAL_PARTS_REQUEST_DELAY + .with_label_values(&[&shard_id.to_string()]) + .observe( + completed.signed_duration_since(started).num_nanoseconds().unwrap_or(0) as f64 + / 1e9, + ); + match result { + Ok(response) => { + tracing::info!(target: "sync", %shard_id, part_id, location, response_code = response.status_code(), num_bytes = response.bytes().len(), "S3 request finished"); + let mut lock = download_response.lock().unwrap(); + *lock = Some(Ok((response.status_code(), response.bytes().to_vec()))); + } + Err(err) => { + tracing::info!(target: "sync", %shard_id, part_id, location, ?err, "S3 request failed"); + let mut lock = download_response.lock().unwrap(); + *lock = Some(Err(err.to_string())); + } + } + } + }); +} + +/// Asynchronously requests a state part from a suitable peer. +fn request_part_from_peers( + part_id: u64, + target: AccountOrPeerIdOrHash, + download: &mut DownloadStatus, + shard_id: ShardId, + sync_hash: CryptoHash, + network_adapter: &PeerManagerAdapter, +) { + download.run_me.store(false, Ordering::SeqCst); + download.state_requests_count += 1; + download.last_target = Some(make_account_or_peer_id_or_hash(target.clone())); + let run_me = download.run_me.clone(); + + near_performance_metrics::actix::spawn( + "StateSync", + network_adapter + .send_async(PeerManagerMessageRequest::NetworkRequests( + NetworkRequests::StateRequestPart { shard_id, sync_hash, part_id, target }, + )) + .then(move |result| { + // TODO: possible optimization - in the current code, even if one of the targets it not present in the network graph + // (so we keep getting RouteNotFound) - we'll still keep trying to assign parts to it. + // Fortunately only once every 60 seconds (timeout value). + if let Ok(NetworkResponses::RouteNotFound) = result.map(|f| f.as_network_response()) + { + // Send a StateRequestPart on the next iteration + run_me.store(true, Ordering::SeqCst); + } + future::ready(()) + }), + ); +} + +fn sent_request_part( + target: AccountOrPeerIdOrHash, + part_id: u64, + shard_id: ShardId, + sync_hash: CryptoHash, + last_part_id_requested: &mut HashMap<(AccountOrPeerIdOrHash, ShardId), PendingRequestStatus>, + requested_target: &mut lru::LruCache<(u64, CryptoHash), AccountOrPeerIdOrHash>, + timeout: Duration, +) { + // FIXME: something is wrong - the index should have a shard_id too. + requested_target.put((part_id, sync_hash), target.clone()); + last_part_id_requested + .entry((target, shard_id)) + .and_modify(|pending_request| { + pending_request.missing_parts += 1; + }) + .or_insert_with(|| PendingRequestStatus::new(timeout)); +} + fn allow_request(requests_remaining: &AtomicI64) -> bool { let remaining = requests_remaining.fetch_sub(1, Ordering::SeqCst); if remaining <= 0 { diff --git a/nearcore/src/config_validate.rs b/nearcore/src/config_validate.rs index d025dd38af6..13164ea1c8b 100644 --- a/nearcore/src/config_validate.rs +++ b/nearcore/src/config_validate.rs @@ -30,14 +30,14 @@ impl<'a> ConfigValidator<'a> { fn validate_all_conditions(&mut self) { if self.config.archive == false && self.config.save_trie_changes == Some(false) { let error_message = format!("Configuration with archive = false and save_trie_changes = false is not supported because non-archival nodes must save trie changes in order to do do garbage collection."); - self.validation_errors.push_config_semantics_error(error_message) + self.validation_errors.push_config_semantics_error(error_message); } // Checking that if cold storage is configured, trie changes are definitely saved. // Unlike in the previous case, None is not a valid option here. if self.config.cold_store.is_some() && self.config.save_trie_changes != Some(true) { let error_message = format!("cold_store is configured, but save_trie_changes is {:?}. Trie changes should be saved to support cold storage.", self.config.save_trie_changes); - self.validation_errors.push_config_semantics_error(error_message) + self.validation_errors.push_config_semantics_error(error_message); } if self.config.consensus.min_block_production_delay @@ -48,7 +48,7 @@ impl<'a> ConfigValidator<'a> { self.config.consensus.min_block_production_delay, self.config.consensus.max_block_production_delay ); - self.validation_errors.push_config_semantics_error(error_message) + self.validation_errors.push_config_semantics_error(error_message); } if self.config.consensus.min_block_production_delay @@ -59,13 +59,13 @@ impl<'a> ConfigValidator<'a> { self.config.consensus.min_block_production_delay, self.config.consensus.max_block_wait_delay ); - self.validation_errors.push_config_semantics_error(error_message) + self.validation_errors.push_config_semantics_error(error_message); } if self.config.consensus.header_sync_expected_height_per_second == 0 { let error_message = format!("consensus.header_sync_expected_height_per_second should not be 0"); - self.validation_errors.push_config_semantics_error(error_message) + self.validation_errors.push_config_semantics_error(error_message); } if self.config.gc.gc_blocks_limit == 0 @@ -73,7 +73,22 @@ impl<'a> ConfigValidator<'a> { || self.config.gc.gc_num_epochs_to_keep == 0 { let error_message = format!("gc config values should all be greater than 0, but gc_blocks_limit is {:?}, gc_fork_clean_step is {}, gc_num_epochs_to_keep is {}.", self.config.gc.gc_blocks_limit, self.config.gc.gc_fork_clean_step, self.config.gc.gc_num_epochs_to_keep); - self.validation_errors.push_config_semantics_error(error_message) + self.validation_errors.push_config_semantics_error(error_message); + } + + if let Some(state_sync) = &self.config.state_sync { + if state_sync.dump_enabled.unwrap_or(false) { + if state_sync.s3_bucket.is_empty() || state_sync.s3_region.is_empty() { + let error_message = format!("'config.state_sync.s3_bucket' and 'config.state_sync.s3_region' need to be specified when 'config.state_sync.dump_enabled' is enabled."); + self.validation_errors.push_config_semantics_error(error_message); + } + } + if state_sync.sync_from_s3_enabled.unwrap_or(false) { + if state_sync.s3_bucket.is_empty() || state_sync.s3_region.is_empty() { + let error_message = format!("'config.state_sync.s3_bucket' and 'config.state_sync.s3_region' need to be specified when 'config.state_sync.sync_from_s3_enabled' is enabled."); + self.validation_errors.push_config_semantics_error(error_message); + } + } } } diff --git a/nearcore/src/lib.rs b/nearcore/src/lib.rs index c5276578b3c..b300aa0f883 100644 --- a/nearcore/src/lib.rs +++ b/nearcore/src/lib.rs @@ -274,12 +274,7 @@ pub fn start_with_config_and_synchronization( ); shards_manager_adapter.bind(shards_manager_actor); - let state_sync_dump_handle = spawn_state_sync_dump( - &config, - chain_genesis, - runtime, - config.network_config.node_id().public_key(), - )?; + let state_sync_dump_handle = spawn_state_sync_dump(&config, chain_genesis, runtime)?; #[allow(unused_mut)] let mut rpc_servers = Vec::new(); diff --git a/nearcore/src/state_sync.rs b/nearcore/src/state_sync.rs index cf680c9d934..da26df70963 100644 --- a/nearcore/src/state_sync.rs +++ b/nearcore/src/state_sync.rs @@ -3,8 +3,7 @@ use borsh::BorshSerialize; use near_chain::types::RuntimeAdapter; use near_chain::{Chain, ChainGenesis, ChainStoreAccess, DoomslugThresholdMode, Error}; use near_chain_configs::ClientConfig; -use near_client::sync::state::StateSync; -use near_crypto::PublicKey; +use near_client::sync::state::{s3_location, StateSync}; use near_epoch_manager::EpochManagerAdapter; use near_primitives::hash::CryptoHash; use near_primitives::state_part::PartId; @@ -19,7 +18,6 @@ pub fn spawn_state_sync_dump( config: &NearConfig, chain_genesis: ChainGenesis, runtime: Arc, - node_key: &PublicKey, ) -> anyhow::Result> { if !config.client_config.state_sync_dump_enabled { return Ok(None); @@ -81,7 +79,6 @@ pub fn spawn_state_sync_dump( runtime, client_config, bucket.clone(), - node_key.clone(), ))); arbiter_handle }) @@ -116,7 +113,6 @@ async fn state_sync_dump( runtime: Arc, config: ClientConfig, bucket: s3::Bucket, - _node_key: PublicKey, ) { tracing::info!(target: "state_sync_dump", shard_id, "Running StateSyncDump loop"); @@ -422,16 +418,3 @@ fn check_new_epoch( } } } - -fn s3_location( - chain_id: &str, - epoch_height: u64, - shard_id: u64, - part_id: u64, - num_parts: u64, -) -> String { - format!( - "chain_id={}/epoch_height={}/shard_id={}/state_part_{:06}_of_{:06}", - chain_id, epoch_height, shard_id, part_id, num_parts - ) -} From e71a0ea8ebc9456f2e33cd29085656c8c2b74e4e Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 6 Apr 2023 19:53:50 +0200 Subject: [PATCH 78/88] merge --- chain/client/src/sync/state.rs | 1 + nearcore/src/config.rs | 67 ++++++++++++---------------------- 2 files changed, 24 insertions(+), 44 deletions(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index d6eb0ef4046..2d45b8c6b17 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -490,6 +490,7 @@ impl StateSync { Ok(self.select_peers(&mut peers, shard_id)?) } + /// Avoids peers that already have outstanding requests for parts. fn select_peers( &mut self, peers: &mut dyn Iterator, diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index 195f85e6ce4..ba1d1fb05c5 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -1,26 +1,10 @@ -use anyhow::{anyhow, bail, Context}; -use near_primitives::static_clock::StaticClock; -use near_primitives::test_utils::create_test_signer; -use num_rational::Rational32; -use std::fs; -use std::fs::File; -use std::io::{Read, Write}; -use std::path::{Path, PathBuf}; -use std::str::FromStr; -use std::sync::Arc; -use std::time::Duration; - -use near_config_utils::{ValidationError, ValidationErrors}; - -#[cfg(test)] -use tempfile::tempdir; -use tracing::{info, warn}; - use crate::download_file::{run_download_file, FileDownloadError}; +use anyhow::{anyhow, bail, Context}; use near_chain_configs::{ get_initial_supply, ClientConfig, GCConfig, Genesis, GenesisConfig, GenesisValidationMode, LogSummaryStyle, MutableConfigValue, }; +use near_config_utils::{ValidationError, ValidationErrors}; use near_crypto::{InMemorySigner, KeyFile, KeyType, PublicKey, Signer}; #[cfg(feature = "json_rpc")] use near_jsonrpc::RpcConfig; @@ -32,9 +16,11 @@ use near_primitives::hash::CryptoHash; use near_primitives::shard_layout::account_id_to_shard_id; use near_primitives::shard_layout::ShardLayout; use near_primitives::state_record::StateRecord; +use near_primitives::static_clock::StaticClock; +use near_primitives::test_utils::create_test_signer; use near_primitives::types::{ - AccountId, AccountInfo, Balance, BlockHeight, BlockHeightDelta, EpochHeight, Gas, NumBlocks, - NumSeats, NumShards, ShardId, + AccountId, AccountInfo, Balance, BlockHeight, BlockHeightDelta, Gas, NumBlocks, NumSeats, + NumShards, ShardId, }; use near_primitives::utils::{generate_random_string, get_num_seats_per_shard}; use near_primitives::validator_signer::{InMemoryValidatorSigner, ValidatorSigner}; @@ -42,6 +28,17 @@ use near_primitives::version::PROTOCOL_VERSION; #[cfg(feature = "rosetta_rpc")] use near_rosetta_rpc::RosettaRpcConfig; use near_telemetry::TelemetryConfig; +use num_rational::Rational32; +use std::fs; +use std::fs::File; +use std::io::{Read, Write}; +use std::path::Path; +use std::str::FromStr; +use std::sync::Arc; +use std::time::Duration; +#[cfg(test)] +use tempfile::tempdir; +use tracing::{info, warn}; /// Initial balance used in tests. pub const TESTING_INIT_BALANCE: Balance = 1_000_000_000 * NEAR_BASE; @@ -67,9 +64,6 @@ pub const MAX_BLOCK_PRODUCTION_DELAY: u64 = 2_000; /// Maximum time until skipping the previous block is ms. pub const MAX_BLOCK_WAIT_DELAY: u64 = 6_000; -/// Reduce wait time for every missing block in ms. -const REDUCE_DELAY_FOR_MISSING_BLOCKS: u64 = 100; - /// Horizon at which instead of fetching block, fetch full state. const BLOCK_FETCH_HORIZON: BlockHeightDelta = 50; @@ -217,9 +211,6 @@ pub struct Consensus { pub max_block_production_delay: Duration, /// Maximum duration before skipping given height. pub max_block_wait_delay: Duration, - /// Duration to reduce the wait for each missed block by validator. - #[serde(default = "default_reduce_wait_for_missing_block")] - pub reduce_wait_for_missing_block: Duration, /// Produce empty blocks, use `false` for testing. pub produce_empty_blocks: bool, /// Horizon at which instead of fetching block, fetch full state. @@ -268,7 +259,6 @@ impl Default for Consensus { min_block_production_delay: Duration::from_millis(MIN_BLOCK_PRODUCTION_DELAY), max_block_production_delay: Duration::from_millis(MAX_BLOCK_PRODUCTION_DELAY), max_block_wait_delay: Duration::from_millis(MAX_BLOCK_WAIT_DELAY), - reduce_wait_for_missing_block: default_reduce_wait_for_missing_block(), produce_empty_blocks: true, block_fetch_horizon: BLOCK_FETCH_HORIZON, state_fetch_horizon: STATE_FETCH_HORIZON, @@ -307,6 +297,8 @@ pub struct Config { pub consensus: Consensus, pub tracked_accounts: Vec, pub tracked_shards: Vec, + #[serde(skip_serializing_if = "Option::is_none")] + pub tracked_shard_schedule: Option>>, #[serde(skip_serializing_if = "is_false")] pub archive: bool, /// If save_trie_changes is not set it will get inferred from the `archive` field as follows: @@ -340,17 +332,8 @@ pub struct Config { /// Configuration for the #[serde(default, skip_serializing_if = "Option::is_none")] pub split_storage: Option, - // TODO(mina86): Remove those two altogether at some point. We need to be - // somewhat careful though and make sure that we don’t start silently - // ignoring this option without users setting corresponding store option. - // For the time being, we’re failing inside of create_db_checkpoint if this - // option is set. - /// Deprecated; use `store.migration_snapshot` instead. - #[serde(default, skip_serializing_if = "Option::is_none")] - pub use_db_migration_snapshot: Option, - /// Deprecated; use `store.migration_snapshot` instead. - #[serde(skip_serializing_if = "Option::is_none")] - pub db_migration_snapshot_path: Option, + /// The node will stop after the head exceeds this height. + /// The node usually stops within several seconds after reaching the target height. #[serde(default, skip_serializing_if = "Option::is_none")] pub expected_shutdown: Option, /// Options for dumping state of every epoch to S3. @@ -380,6 +363,7 @@ impl Default for Config { consensus: Consensus::default(), tracked_accounts: vec![], tracked_shards: vec![], + tracked_shard_schedule: None, archive: false, save_trie_changes: None, log_summary_style: LogSummaryStyle::Colored, @@ -389,8 +373,6 @@ impl Default for Config { view_client_throttle_period: default_view_client_throttle_period(), trie_viewer_state_size_limit: default_trie_viewer_state_size_limit(), max_gas_burnt_view: None, - db_migration_snapshot_path: None, - use_db_migration_snapshot: None, store: near_store::StoreConfig::default(), cold_store: None, split_storage: None, @@ -571,7 +553,6 @@ impl Genesis { avg_hidden_validator_seats_per_shard: vec![0; num_validator_seats_per_shard.len()], dynamic_resharding: false, protocol_upgrade_stake_threshold: PROTOCOL_UPGRADE_STAKE_THRESHOLD, - protocol_upgrade_num_epochs: PROTOCOL_UPGRADE_NUM_EPOCHS, epoch_length: FAST_EPOCH_LENGTH, gas_limit: INITIAL_GAS_LIMIT, gas_price_adjustment_rate: GAS_PRICE_ADJUSTMENT_RATE, @@ -665,7 +646,6 @@ impl NearConfig { min_block_production_delay: config.consensus.min_block_production_delay, max_block_production_delay: config.consensus.max_block_production_delay, max_block_wait_delay: config.consensus.max_block_wait_delay, - reduce_wait_for_missing_block: config.consensus.reduce_wait_for_missing_block, skip_sync_wait: config.network.skip_sync_wait, sync_check_period: config.consensus.sync_check_period, sync_step_period: config.consensus.sync_step_period, @@ -682,7 +662,6 @@ impl NearConfig { produce_empty_blocks: config.consensus.produce_empty_blocks, epoch_length: genesis.config.epoch_length, num_block_producer_seats: genesis.config.num_block_producer_seats, - announce_account_horizon: genesis.config.epoch_length / 2, ttl_account_id_router: config.network.ttl_account_id_router, // TODO(1047): this should be adjusted depending on the speed of sync of state. block_fetch_horizon: config.consensus.block_fetch_horizon, @@ -693,6 +672,7 @@ impl NearConfig { doosmslug_step_period: config.consensus.doomslug_step_period, tracked_accounts: config.tracked_accounts, tracked_shards: config.tracked_shards, + tracked_shard_schedule: config.tracked_shard_schedule.unwrap_or(vec![]), archive: config.archive, save_trie_changes: config.save_trie_changes.unwrap_or(!config.archive), log_summary_style: config.log_summary_style, @@ -1178,7 +1158,6 @@ pub fn init_configs( avg_hidden_validator_seats_per_shard: (0..num_shards).map(|_| 0).collect(), dynamic_resharding: false, protocol_upgrade_stake_threshold: PROTOCOL_UPGRADE_STAKE_THRESHOLD, - protocol_upgrade_num_epochs: PROTOCOL_UPGRADE_NUM_EPOCHS, epoch_length: if fast { FAST_EPOCH_LENGTH } else { EXPECTED_EPOCH_LENGTH }, gas_limit: INITIAL_GAS_LIMIT, gas_price_adjustment_rate: GAS_PRICE_ADJUSTMENT_RATE, From b01050fdd8a49d0a86b5b49903fd0e452ce71ce0 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 6 Apr 2023 19:56:14 +0200 Subject: [PATCH 79/88] merge --- nearcore/src/config.rs | 8 -------- 1 file changed, 8 deletions(-) diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index ba1d1fb05c5..6c8db4cadde 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -117,9 +117,6 @@ pub const NUM_BLOCK_PRODUCER_SEATS: NumSeats = 50; /// The minimum stake required for staking is last seat price divided by this number. pub const MINIMUM_STAKE_DIVISOR: u64 = 10; -/// Number of epochs before protocol upgrade. -pub const PROTOCOL_UPGRADE_NUM_EPOCHS: EpochHeight = 2; - pub const CONFIG_FILENAME: &str = "config.json"; pub const GENESIS_CONFIG_FILENAME: &str = "genesis.json"; pub const NODE_KEY_FILE: &str = "node_key.json"; @@ -146,11 +143,6 @@ pub const MAX_INFLATION_RATE: Rational32 = Rational32::new_raw(1, 20); /// Protocol upgrade stake threshold. pub const PROTOCOL_UPGRADE_STAKE_THRESHOLD: Rational32 = Rational32::new_raw(4, 5); -/// Serde default only supports functions without parameters. -fn default_reduce_wait_for_missing_block() -> Duration { - Duration::from_millis(REDUCE_DELAY_FOR_MISSING_BLOCKS) -} - fn default_header_sync_initial_timeout() -> Duration { Duration::from_secs(10) } From b61749a4a1905d85a6c45d929587a3ccf8108636 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 11 Apr 2023 11:13:42 +0200 Subject: [PATCH 80/88] Re-fixed. Persist `new_shard_sync` when S3 download completes. --- chain/client/src/client.rs | 2 +- chain/client/src/metrics.rs | 10 ++ chain/client/src/sync/state.rs | 166 +++++++++++++++++++------------- docs/misc/state_sync_from_s3.md | 12 +-- 4 files changed, 115 insertions(+), 75 deletions(-) diff --git a/chain/client/src/client.rs b/chain/client/src/client.rs index 38838c3dbfd..c2bd2cd7565 100644 --- a/chain/client/src/client.rs +++ b/chain/client/src/client.rs @@ -2084,7 +2084,7 @@ impl Client { assert_eq!(sync_hash, state_sync_info.epoch_tail_hash); let network_adapter1 = self.network_adapter.clone(); - let new_shard_sync = { + let = { let prev_hash = *self.chain.get_block(&sync_hash)?.header().prev_hash(); let need_to_split_states = self.runtime_adapter.will_shard_layout_change_next_epoch(&prev_hash)?; diff --git a/chain/client/src/metrics.rs b/chain/client/src/metrics.rs index 535d1106a16..8de327457a1 100644 --- a/chain/client/src/metrics.rs +++ b/chain/client/src/metrics.rs @@ -441,3 +441,13 @@ pub static STATE_SYNC_EXTERNAL_PARTS_REQUEST_DELAY: Lazy = + Lazy::new(|| { + try_create_int_counter_vec( + "near_state_sync_external_parts_size_downloaded_bytes_total", + "Amount of bytes downloaded from an external storage when requesting state parts for a shard", + &["shard_id"], + ) + .unwrap() + }); diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 2d45b8c6b17..243969f4153 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -199,7 +199,7 @@ impl StateSync { target: "sync", %prev_hash, timeout_sec = self.timeout.num_seconds(), - "State sync: block request timed"); + "State sync: block request timed out"); (true, false) } else { (false, false) @@ -250,36 +250,40 @@ impl StateSync { for shard_id in tracking_shards { let mut download_timeout = false; - let mut need_shard = false; + let mut run_shard_state_download = false; let shard_sync_download = new_shard_sync.entry(shard_id).or_insert_with(|| { - need_shard = true; + run_shard_state_download = true; update_sync_status = true; ShardSyncDownload::new_download_state_header(now) }); let old_status = shard_sync_download.status.clone(); - let mut this_done = false; + let mut shard_sync_done = false; metrics::STATE_SYNC_STAGE .with_label_values(&[&shard_id.to_string()]) .set(shard_sync_download.status.repr() as i64); match &shard_sync_download.status { ShardSyncStatus::StateDownloadHeader => { - (download_timeout, need_shard) = self.sync_shards_download_header_status( - shard_id, - shard_sync_download, - sync_hash, - chain, - now, - )?; + (download_timeout, run_shard_state_download) = self + .sync_shards_download_header_status( + shard_id, + shard_sync_download, + sync_hash, + chain, + now, + )?; } ShardSyncStatus::StateDownloadParts => { - (download_timeout, need_shard) = self.sync_shards_download_parts_status( + let res = self.sync_shards_download_parts_status( shard_id, shard_sync_download, sync_hash, chain, now, ); + download_timeout = res.0; + run_shard_state_download = res.1; + update_sync_status |= res.2; } ShardSyncStatus::StateDownloadScheduling => { self.sync_shards_download_scheduling_status( @@ -301,7 +305,7 @@ impl StateSync { )?; } ShardSyncStatus::StateDownloadComplete => { - this_done = self.sync_shards_download_complete_status( + shard_sync_done = self.sync_shards_download_complete_status( split_states, shard_id, shard_sync_download, @@ -322,7 +326,7 @@ impl StateSync { } ShardSyncStatus::StateSplitApplying(_status) => { debug_assert!(split_states); - this_done = self.sync_shards_state_split_applying_status( + shard_sync_done = self.sync_shards_state_split_applying_status( shard_id, shard_sync_download, sync_hash, @@ -330,10 +334,10 @@ impl StateSync { )?; } ShardSyncStatus::StateSyncDone => { - this_done = true; + shard_sync_done = true; } } - all_done &= this_done; + all_done &= shard_sync_done; if download_timeout { tracing::warn!( @@ -341,7 +345,7 @@ impl StateSync { %shard_id, timeout_sec = self.timeout.num_seconds(), "State sync didn't download the state, sending StateRequest again"); - tracing::info!( + tracing::debug!( target: "sync", %shard_id, %sync_hash, @@ -351,7 +355,7 @@ impl StateSync { } // Execute syncing for shard `shard_id` - if need_shard { + if run_shard_state_download { update_sync_status = true; self.request_shard( me, @@ -686,7 +690,7 @@ impl StateSync { use_colour: bool, ) -> Result { let _span = tracing::debug_span!(target: "sync", "run", sync = "StateSync").entered(); - tracing::debug!(target: "sync", %sync_hash, ?tracking_shards, "syncing state"); + tracing::trace!(target: "sync", %sync_hash, ?tracking_shards, "syncing state"); let prev_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); let now = StaticClock::utc(); @@ -797,8 +801,11 @@ impl StateSync { } /// Checks if the header is downloaded. - /// If the download is complete, then moves forward to StateDownloadParts, otherwise retries the header request. - /// Returns (download_timeout, need_shard). + /// If the download is complete, then moves forward to `StateDownloadParts`, + /// otherwise retries the header request. + /// Returns `(download_timeout, run_shard_state_download)` where: + /// * `download_timeout` means that the state header request timed out (and needs to be retried). + /// * `run_shard_state_download` means that header or part download requests need to run for this shard. fn sync_shards_download_header_status( &mut self, shard_id: ShardId, @@ -808,17 +815,17 @@ impl StateSync { now: DateTime, ) -> Result<(bool, bool), near_chain::Error> { let mut download_timeout = false; - let mut need_shard = false; + let mut run_shard_state_download = false; // StateDownloadHeader is the first step. We want to fetch the basic information about the state (its size, hash etc). if shard_sync_download.downloads[0].done { - let shard_state_header = chain.get_state_header(shard_id.clone(), sync_hash)?; + let shard_state_header = chain.get_state_header(shard_id, sync_hash)?; let state_num_parts = get_num_state_parts(shard_state_header.state_root_node().memory_usage); // If the header was downloaded successfully - move to phase 2 (downloading parts). // Create the vector with entry for each part. *shard_sync_download = ShardSyncDownload::new_download_state_parts(now, state_num_parts); - need_shard = true; + run_shard_state_download = true; } else { let prev = shard_sync_download.downloads[0].prev_update_time; let error = shard_sync_download.downloads[0].error; @@ -830,16 +837,18 @@ impl StateSync { shard_sync_download.downloads[0].prev_update_time = now; } if shard_sync_download.downloads[0].run_me.load(Ordering::SeqCst) { - need_shard = true; + run_shard_state_download = true; } } - Ok((download_timeout, need_shard)) + Ok((download_timeout, run_shard_state_download)) } /// Checks if the parts are downloaded. - /// If download of all parts is complete, then moves forward to StateDownloadScheduling. - /// Otherwise, retries the failed part downloads. - /// Returns (download_timeout, need_shard). + /// If download of all parts is complete, then moves forward to `StateDownloadScheduling`. + /// Returns `(download_timeout, run_shard_state_download, update_sync_status)` where: + /// * `download_timeout` means that the state header request timed out (and needs to be retried). + /// * `run_shard_state_download` means that header or part download requests need to run for this shard. + /// * `update_sync_status` means that something changed in `ShardSyncDownload` and it needs to be persisted. fn sync_shards_download_parts_status( &mut self, shard_id: ShardId, @@ -847,19 +856,19 @@ impl StateSync { sync_hash: CryptoHash, chain: &mut Chain, now: DateTime, - ) -> (bool, bool) { + ) -> (bool, bool, bool) { // Step 2 - download all the parts (each part is usually around 1MB). let mut download_timeout = false; - let mut need_shard = false; + let mut run_shard_state_download = false; + let mut update_sync_status = false; let mut parts_done = true; let num_parts = shard_sync_download.downloads.len(); let mut num_parts_done = 0; for (part_id, part_download) in shard_sync_download.downloads.iter_mut().enumerate() { - tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, part_download.done, part_download.error, ?part_download); if !part_download.done { // Check if a download from an external storage is finished. - check_external_storage_part_response( + update_sync_status |= check_external_storage_part_response( part_id as u64, num_parts as u64, shard_id, @@ -868,29 +877,35 @@ impl StateSync { chain, ); } - tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, part_download.done, part_download.error); if !part_download.done { parts_done = false; let prev = part_download.prev_update_time; let part_timeout = now - prev > self.timeout; // Retry parts that failed. if part_timeout || part_download.error { - metrics::STATE_SYNC_RETRY_PART - .with_label_values(&[&shard_id.to_string()]) - .inc(); download_timeout |= part_timeout; - part_download.run_me.store(true, Ordering::SeqCst); - part_download.error = false; - part_download.prev_update_time = now; + if part_timeout || + part_download.last_target != Some(near_client_primitives::types::AccountOrPeerIdOrHash::ExternalStorage) { + // Don't immediately retry failed requests from external + // storage. Most often error is a state part not + // available. That error doesn't get fixed by retrying, + // but rather by waiting. + metrics::STATE_SYNC_RETRY_PART + .with_label_values(&[&shard_id.to_string()]) + .inc(); + part_download.run_me.store(true, Ordering::SeqCst); + part_download.error = false; + part_download.prev_update_time = now; + update_sync_status = true; + } } if part_download.run_me.load(Ordering::SeqCst) { - need_shard = true; + run_shard_state_download = true; } } if part_download.done { num_parts_done += 1; } } - tracing::debug!(target: "sync", %shard_id, %sync_hash, num_parts_done, parts_done); metrics::STATE_SYNC_PARTS_DONE .with_label_values(&[&shard_id.to_string()]) .set(num_parts_done); @@ -903,8 +918,9 @@ impl StateSync { downloads: vec![], status: ShardSyncStatus::StateDownloadScheduling, }; + update_sync_status = true; } - (download_timeout, need_shard) + (download_timeout, run_shard_state_download, update_sync_status) } fn sync_shards_download_scheduling_status( @@ -996,7 +1012,7 @@ impl StateSync { get_num_state_parts(shard_state_header.state_root_node().memory_usage); chain.clear_downloaded_parts(shard_id, sync_hash, state_num_parts)?; - let mut this_done = false; + let mut shard_sync_done = false; // If the shard layout is changing in this epoch - we have to apply it right now. if split_states { *shard_sync_download = ShardSyncDownload { @@ -1007,9 +1023,9 @@ impl StateSync { // If there is no layout change - we're done. *shard_sync_download = ShardSyncDownload { downloads: vec![], status: ShardSyncStatus::StateSyncDone }; - this_done = true; + shard_sync_done = true; } - Ok(this_done) + Ok(shard_sync_done) } fn sync_shards_state_split_scheduling_status( @@ -1036,7 +1052,7 @@ impl StateSync { Ok(()) } - /// Returns `this_done`. + /// Returns whether the State Sync for the given shard is complete. fn sync_shards_state_split_applying_status( &mut self, shard_id: ShardId, @@ -1045,14 +1061,14 @@ impl StateSync { chain: &mut Chain, ) -> Result { let result = self.split_state_roots.remove(&shard_id); - let mut this_done = false; + let mut shard_sync_done = false; if let Some(state_roots) = result { chain.build_state_for_split_shards_postprocessing(&sync_hash, state_roots)?; *shard_sync_download = ShardSyncDownload { downloads: vec![], status: ShardSyncStatus::StateSyncDone }; - this_done = true; + shard_sync_done = true; } - Ok(this_done) + Ok(shard_sync_done) } } @@ -1182,6 +1198,7 @@ fn sent_request_part( .or_insert_with(|| PendingRequestStatus::new(timeout)); } +/// Verifies that one more concurrent request can be started. fn allow_request(requests_remaining: &AtomicI64) -> bool { let remaining = requests_remaining.fetch_sub(1, Ordering::SeqCst); if remaining <= 0 { @@ -1200,6 +1217,8 @@ fn finished_request(requests_remaining: &AtomicI64) { /// The response is stored on the DownloadStatus object. /// This function investigates if the response is available and updates `done` and `error` appropriately. /// If the response is successful, then also writes the state part to the DB. +/// +/// Returns whether something changed in `DownloadStatus` which means it needs to be persisted. fn check_external_storage_part_response( part_id: u64, num_parts: u64, @@ -1207,18 +1226,18 @@ fn check_external_storage_part_response( sync_hash: CryptoHash, part_download: &mut DownloadStatus, chain: &mut Chain, -) { +) -> bool { let external_storage_response = { let mut lock = part_download.response.lock().unwrap(); if let Some(response) = lock.clone() { tracing::debug!(target: "sync", %shard_id, part_id, "Got response from external storage"); // Remove the response from DownloadStatus, because - // we're going to write positive responses to the DB - // and retry negative responses. + // we're going to write state parts to DB and don't need to keep + // them in `DownloadStatus`. *lock = None; response } else { - return; + return false; } }; @@ -1237,7 +1256,11 @@ fn check_external_storage_part_response( metrics::STATE_SYNC_EXTERNAL_PARTS_DONE .with_label_values(&[&shard_id.to_string()]) .inc(); + metrics::STATE_SYNC_EXTERNAL_PARTS_SIZE_DOWNLOADED + .with_label_values(&[&shard_id.to_string()]) + .inc_by(data.len() as u64); part_download.done = true; + tracing::debug!(target: "sync", %shard_id, part_id, ?part_download, "Set state part success"); } Err(err) => { metrics::STATE_SYNC_EXTERNAL_PARTS_FAILED @@ -1251,20 +1274,20 @@ fn check_external_storage_part_response( } // Other HTTP status codes are considered errors. Ok((status_code, _)) => { - err_to_retry = - Some(near_chain::Error::Other(format!("status_code: {}", status_code).to_string())); + tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, status_code, "Wrong response code, expected 200"); + err_to_retry = Some(near_chain::Error::Other(format!("status_code: {}", status_code))); } // The request failed without reaching the external storage. Err(err) => { - err_to_retry = Some(near_chain::Error::Other(err.to_string())); + err_to_retry = Some(near_chain::Error::Other(err)); } }; if let Some(err) = err_to_retry { tracing::debug!(target: "sync", %shard_id, %sync_hash, part_id, ?err, "Failed to get a part from external storage, will retry"); part_download.error = true; - } else { } + true } /// Construct a location on the external storage. @@ -1281,9 +1304,10 @@ pub fn s3_location( ) } -fn paint(s: &str, colour: Style, use_colour: bool) -> String { - if use_colour { - colour.paint(s).to_string() +/// Applies style if `use_colour` is enabled. +fn paint(s: &str, style: Style, use_style: bool) -> String { + if use_style { + style.paint(s).to_string() } else { s.to_string() } @@ -1299,8 +1323,15 @@ fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload, use_colour: shard_sync_download.downloads[0].last_target ), ShardSyncStatus::StateDownloadParts => { + let mut num_parts_done = 0; + let mut num_parts_not_done = 0; let mut text = "".to_string(); for (i, download) in shard_sync_download.downloads.iter().enumerate() { + if download.done { + num_parts_done += 1; + continue; + } + num_parts_not_done += 1; text.push_str(&format!( "[{}: {}, {}, {:?}] ", paint(&i.to_string(), Yellow.bold(), use_colour), @@ -1310,10 +1341,12 @@ fn format_shard_sync_phase(shard_sync_download: &ShardSyncDownload, use_colour: )); } format!( - "{} [{}: is_done, requests sent, last target] {}", + "{} [{}: is_done, requests sent, last target] {} num_parts_done={} num_parts_not_done={}", paint("PARTS", Purple.bold(), use_colour), paint("part_id", Yellow.bold(), use_colour), - text + text, + num_parts_done, + num_parts_not_done ) } _ => unreachable!("timeout cannot happen when all state is downloaded"), @@ -1408,10 +1441,10 @@ mod test { let mut state_sync = StateSync::new( mock_peer_manager.clone().into(), TimeDuration::from_secs(1), - "chain_id".to_string(), + "chain_id", false, - "".to_string(), - "".to_string(), + "", + "", 100, ); let mut new_shard_sync = HashMap::new(); @@ -1463,6 +1496,7 @@ mod test { vec![0], &apply_parts_fn, &state_split_fn, + false, ) .unwrap(); diff --git a/docs/misc/state_sync_from_s3.md b/docs/misc/state_sync_from_s3.md index ff1929b981b..22eb70d6d4e 100644 --- a/docs/misc/state_sync_from_s3.md +++ b/docs/misc/state_sync_from_s3.md @@ -8,12 +8,10 @@ details) doesn't allow the nodes to reliably perform state sync for testnet or mainnet. That's why a new solution for state sync is being designed. -The experimental code is likely going to be a part of solution to greatly -improve both reliability and speed of state sync. - -The new solution will probably involve making the state available on external -storage, making downloading the state both low latency and reliable process, -thanks to the robust infrastructure of external storage such as S3. +This is a short-term solution that is needed to let nodes sync and let chunk +only producers to switch tracked shards. +The experimental code is will not be kept for long and will be replaced with a +decentralized solution. ## How-to @@ -40,8 +38,6 @@ AWS_ACCESS_KEY_ID="MY_ACCESS_KEY" AWS_SECRET_ACCESS_KEY="MY_AWS_SECRET_ACCESS_KE ## Implementation Details -The experimental option replaces how a node fetches state parts. -The legacy implementation asks peer nodes to create and share a state part over network. The new implementation expects to find state parts as files on an S3 storage. The sync mechanism proceeds to download state parts mostly-sequentially from S3. From 594cafa49c77099cd5e524a0a4174fb60ca671c3 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 11 Apr 2023 11:17:49 +0200 Subject: [PATCH 81/88] . --- chain/client/src/client.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/chain/client/src/client.rs b/chain/client/src/client.rs index c2bd2cd7565..38838c3dbfd 100644 --- a/chain/client/src/client.rs +++ b/chain/client/src/client.rs @@ -2084,7 +2084,7 @@ impl Client { assert_eq!(sync_hash, state_sync_info.epoch_tail_hash); let network_adapter1 = self.network_adapter.clone(); - let = { + let new_shard_sync = { let prev_hash = *self.chain.get_block(&sync_hash)?.header().prev_hash(); let need_to_split_states = self.runtime_adapter.will_shard_layout_change_next_epoch(&prev_hash)?; From a9debc5421943f82f6118bf9e2d9f032dba141f2 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 11 Apr 2023 15:15:48 +0200 Subject: [PATCH 82/88] Remove unneeded enum value and simplify target sampling. --- chain/client/src/sync/state.rs | 34 ++++++++++--------- chain/network/src/network_protocol/mod.rs | 1 - .../src/peer_manager/peer_manager_actor.rs | 1 - 3 files changed, 18 insertions(+), 18 deletions(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 243969f4153..44123357e12 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -92,7 +92,6 @@ fn make_account_or_peer_id_or_hash( From::AccountId(a) => To::AccountId(a), From::PeerId(p) => To::PeerId(p), From::Hash(h) => To::Hash(h), - From::ExternalStorage => To::ExternalStorage, } } @@ -152,10 +151,12 @@ impl StateSync { num_s3_requests_per_shard: u64, ) -> Self { let inner = if state_sync_from_s3_enabled { + // `unwrap()` here is fine, because the config validation has already + // ensured those fields are present. let mut bucket = s3::Bucket::new( s3_bucket, s3_region.parse::().unwrap(), - s3::creds::Credentials::default().unwrap(), + s3::creds::Credentials::anonymous().unwrap(), ) .unwrap(); // Ensure requests finish in finite amount of time. @@ -460,7 +461,7 @@ impl StateSync { let block_producers = runtime_adapter.get_epoch_block_producers_ordered(&epoch_hash, &sync_hash)?; - let mut peers = block_producers + let peers = block_producers .iter() .filter_map(|(validator_stake, _slashed)| { let account_id = validator_stake.account_id(); @@ -491,13 +492,13 @@ impl StateSync { None } })); - Ok(self.select_peers(&mut peers, shard_id)?) + Ok(self.select_peers(peers.collect(), shard_id)?) } /// Avoids peers that already have outstanding requests for parts. fn select_peers( &mut self, - peers: &mut dyn Iterator, + peers: Vec, shard_id: ShardId, ) -> Result, near_chain::Error> { let res = match &mut self.inner { @@ -507,13 +508,14 @@ impl StateSync { } => { last_part_id_requested.retain(|_, request| !request.expired()); peers + .into_iter() .filter(|candidate| { // If we still have a pending request from this node - don't add another one. !last_part_id_requested.contains_key(&(candidate.clone(), shard_id)) }) .collect::>() } - StateSyncInner::PartsFromExternal { .. } => peers.collect::>(), + StateSyncInner::PartsFromExternal { .. } => peers, }; Ok(res) } @@ -616,16 +618,17 @@ impl StateSync { StateSyncInner::Peers { last_part_id_requested, requested_target } => { // We'll select all the 'highest' peers + validators as candidates (excluding those that gave us timeout in the past). // And for each one of them, we'll ask for up to 16 (MAX_STATE_PART_REQUEST) parts. - let mut possible_targets_sampler = + let possible_targets_sampler = SamplerLimited::new(possible_targets, MAX_STATE_PART_REQUEST); - for (part_id, download) in parts_to_fetch(new_shard_sync_download) { - // For every part that needs to be requested it is selected one - // peer (target) randomly to request the part from. - // IMPORTANT: here we use 'zip' with possible_target_sampler - - // which is limited. So at any moment we'll not request more - // than possible_targets.len() * MAX_STATE_PART_REQUEST parts. - let target = possible_targets_sampler.next().unwrap(); + // For every part that needs to be requested it is selected one + // peer (target) randomly to request the part from. + // IMPORTANT: here we use 'zip' with possible_target_sampler - + // which is limited. So at any moment we'll not request more + // than possible_targets.len() * MAX_STATE_PART_REQUEST parts. + for ((part_id, download), target) in + parts_to_fetch(new_shard_sync_download).zip(possible_targets_sampler) + { sent_request_part( target.clone(), part_id, @@ -1104,8 +1107,7 @@ fn request_part_from_external_storage( } } download.state_requests_count += 1; - download.last_target = - Some(make_account_or_peer_id_or_hash(AccountOrPeerIdOrHash::ExternalStorage)); + download.last_target = None; let location = s3_location(chain_id, epoch_height, shard_id, part_id, num_parts); let download_response = download.response.clone(); diff --git a/chain/network/src/network_protocol/mod.rs b/chain/network/src/network_protocol/mod.rs index 9b3bb8502ec..93fb8fa1ba8 100644 --- a/chain/network/src/network_protocol/mod.rs +++ b/chain/network/src/network_protocol/mod.rs @@ -795,7 +795,6 @@ pub enum AccountOrPeerIdOrHash { AccountId(AccountId), PeerId(PeerId), Hash(CryptoHash), - ExternalStorage, } pub(crate) struct RawRoutedMessage { diff --git a/chain/network/src/peer_manager/peer_manager_actor.rs b/chain/network/src/peer_manager/peer_manager_actor.rs index c1fd90cead9..ed68932f308 100644 --- a/chain/network/src/peer_manager/peer_manager_actor.rs +++ b/chain/network/src/peer_manager/peer_manager_actor.rs @@ -658,7 +658,6 @@ impl PeerManagerActor { } AccountOrPeerIdOrHash::PeerId(it) => PeerIdOrHash::PeerId(it.clone()), AccountOrPeerIdOrHash::Hash(it) => PeerIdOrHash::Hash(*it), - AccountOrPeerIdOrHash::ExternalStorage => unreachable!(), }; self.state.send_message_to_peer( From 00f70d468c8870fe12da0dba1553db73e824a4ea Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Tue, 11 Apr 2023 15:55:15 +0200 Subject: [PATCH 83/88] Remove `unwrap()`s when creating an S3 bucket. --- chain/client/src/sync/state.rs | 35 +++++++++++++++++++++------------- 1 file changed, 22 insertions(+), 13 deletions(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 44123357e12..924afa065ee 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -151,21 +151,13 @@ impl StateSync { num_s3_requests_per_shard: u64, ) -> Self { let inner = if state_sync_from_s3_enabled { - // `unwrap()` here is fine, because the config validation has already - // ensured those fields are present. - let mut bucket = s3::Bucket::new( - s3_bucket, - s3_region.parse::().unwrap(), - s3::creds::Credentials::anonymous().unwrap(), - ) - .unwrap(); - // Ensure requests finish in finite amount of time. - bucket.set_request_timeout(Some(timeout)); - let bucket = Arc::new(bucket); - + let bucket = create_bucket(s3_bucket, s3_region, timeout); + if let Err(err) = bucket { + panic!("Failed to create an S3 bucket: {}", err); + } StateSyncInner::PartsFromExternal { chain_id: chain_id.to_string(), - bucket, + bucket: Arc::new(bucket.unwrap()), requests_remaining: Arc::new(AtomicI64::new(num_s3_requests_per_shard as i64)), } } else { @@ -1075,6 +1067,23 @@ impl StateSync { } } +fn create_bucket( + bucket: &str, + region: &str, + timeout: TimeDuration, +) -> Result { + let mut bucket = s3::Bucket::new( + bucket, + region.parse::().map_err(|err| near_chain::Error::Other(err.to_string()))?, + s3::creds::Credentials::anonymous() + .map_err(|err| near_chain::Error::Other(err.to_string()))?, + ) + .map_err(|err| near_chain::Error::Other(err.to_string()))?; + // Ensure requests finish in finite amount of time. + bucket.set_request_timeout(Some(timeout)); + Ok(bucket) +} + /// Returns parts that still need to be fetched. fn parts_to_fetch( new_shard_sync_download: &mut ShardSyncDownload, From 08550021bb06db4738a36d82d42a491ea407cc03 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 13 Apr 2023 15:12:39 +0200 Subject: [PATCH 84/88] Merge --- chain/client/src/client.rs | 6 ++++-- chain/client/src/client_actor.rs | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/chain/client/src/client.rs b/chain/client/src/client.rs index 38838c3dbfd..31996891d6a 100644 --- a/chain/client/src/client.rs +++ b/chain/client/src/client.rs @@ -32,6 +32,7 @@ use near_chunks::logic::{ use near_chunks::ShardsManager; use near_client_primitives::debug::ChunkProduction; use near_client_primitives::types::{Error, ShardSyncDownload, ShardSyncStatus}; +use near_epoch_manager::shard_tracker::ShardTracker; use near_network::types::{AccountKeys, ChainInfo, PeerManagerMessageRequest, SetChainInfo}; use near_network::types::{ HighestHeightPeerInfo, NetworkRequests, PeerManagerAdapter, ReasonForBan, @@ -359,7 +360,7 @@ impl Client { block.header().prev_hash(), shard_id, false, - self.runtime_adapter.as_ref(), + &self.shard_tracker, ) { self.sharded_tx_pool.reintroduce_transactions( shard_id, @@ -1596,7 +1597,8 @@ impl Client { &encoded_chunk, merkle_paths.clone(), Some(&validator_id), - self.runtime_adapter.as_ref(), + self.runtime_adapter.epoch_manager_adapter(), + &self.shard_tracker, )?; persist_chunk(partial_chunk.clone(), Some(shard_chunk), self.chain.mut_store())?; self.on_chunk_header_ready_for_inclusion(encoded_chunk.cloned_header(), validator_id); diff --git a/chain/client/src/client_actor.rs b/chain/client/src/client_actor.rs index d7f3f7349a3..6240c621c6e 100644 --- a/chain/client/src/client_actor.rs +++ b/chain/client/src/client_actor.rs @@ -1657,7 +1657,7 @@ impl ClientActor { &prev_hash, *x, true, - self.client.runtime_adapter.as_ref(), + &self.client.shard_tracker, ) }) .collect(); From b09200cf51699b9497b89312404ea1a3e2d4fc5d Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 13 Apr 2023 15:23:49 +0200 Subject: [PATCH 85/88] Merge --- chain/client/src/client.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/chain/client/src/client.rs b/chain/client/src/client.rs index 31996891d6a..e8885cac80c 100644 --- a/chain/client/src/client.rs +++ b/chain/client/src/client.rs @@ -103,6 +103,7 @@ pub struct Client { pub chain: Chain, pub doomslug: Doomslug, pub runtime_adapter: Arc, + pub shard_tracker: ShardTracker, pub shards_manager_adapter: Sender, pub sharded_tx_pool: ShardedTransactionPool, prev_block_to_chunk_headers_ready_for_inclusion: LruCache< @@ -284,6 +285,7 @@ impl Client { sync_status, chain, doomslug, + shard_tracker: runtime_adapter.shard_tracker(), runtime_adapter, shards_manager_adapter, sharded_tx_pool, @@ -336,7 +338,7 @@ impl Client { block.header().prev_hash(), shard_id, true, - self.runtime_adapter.as_ref(), + &self.shard_tracker, ) { self.sharded_tx_pool.remove_transactions( shard_id, From 27423e5649868ce24be97c949574690b0bcda12b Mon Sep 17 00:00:00 2001 From: Ekleog-NEAR <96595974+Ekleog-NEAR@users.noreply.github.com> Date: Thu, 13 Apr 2023 05:05:31 +0200 Subject: [PATCH 86/88] Merge --- chain/client/src/sync/state.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 924afa065ee..5db95e58aa7 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -99,7 +99,7 @@ fn make_account_or_peer_id_or_hash( enum StateSyncInner { /// Request both the state header and state parts from the peers. Peers { - /// Information about which parts were requested from which peer and when. + /// Which parts were requested from which peer and when. last_part_id_requested: HashMap<(AccountOrPeerIdOrHash, ShardId), PendingRequestStatus>, /// Map from which part we requested to whom. requested_target: lru::LruCache<(u64, CryptoHash), AccountOrPeerIdOrHash>, From 2a7454f4bbe2ec32e4e2a56d44fc12bed9a0b75d Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Thu, 13 Apr 2023 19:21:14 +0200 Subject: [PATCH 87/88] re-push --- chain/client/src/sync/state.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 5db95e58aa7..7fa75d62a1c 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -451,9 +451,8 @@ impl StateSync { let prev_block_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); let epoch_hash = runtime_adapter.get_epoch_id_from_prev_block(&prev_block_hash)?; - let block_producers = - runtime_adapter.get_epoch_block_producers_ordered(&epoch_hash, &sync_hash)?; - let peers = block_producers + let peers = runtime_adapter + .get_epoch_block_producers_ordered(&epoch_hash, &sync_hash)? .iter() .filter_map(|(validator_stake, _slashed)| { let account_id = validator_stake.account_id(); From b29d2b3aec61058806d60fe3058d2b33806aa811 Mon Sep 17 00:00:00 2001 From: Nikolay Kurtov Date: Fri, 14 Apr 2023 10:46:07 +0200 Subject: [PATCH 88/88] fix --- chain/client/src/sync/state.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/chain/client/src/sync/state.rs b/chain/client/src/sync/state.rs index 7fa75d62a1c..5db95e58aa7 100644 --- a/chain/client/src/sync/state.rs +++ b/chain/client/src/sync/state.rs @@ -451,8 +451,9 @@ impl StateSync { let prev_block_hash = *chain.get_block_header(&sync_hash)?.prev_hash(); let epoch_hash = runtime_adapter.get_epoch_id_from_prev_block(&prev_block_hash)?; - let peers = runtime_adapter - .get_epoch_block_producers_ordered(&epoch_hash, &sync_hash)? + let block_producers = + runtime_adapter.get_epoch_block_producers_ordered(&epoch_hash, &sync_hash)?; + let peers = block_producers .iter() .filter_map(|(validator_stake, _slashed)| { let account_id = validator_stake.account_id();