From 9778963331b128cc321b403bfbe7b558282e7ee0 Mon Sep 17 00:00:00 2001 From: cheme Date: Fri, 7 Aug 2020 21:43:47 +0200 Subject: [PATCH 01/17] synch babe headers need with cht pruning --- bin/node-template/node/src/service.rs | 2 +- bin/node/cli/src/service.rs | 4 +- client/api/src/backend.rs | 145 ++++++++++++++++ client/consensus/babe/rpc/src/lib.rs | 1 + client/consensus/babe/src/lib.rs | 29 ++++ client/consensus/babe/src/tests.rs | 1 + client/consensus/epochs/src/lib.rs | 12 +- client/db/src/light.rs | 227 ++++++++++++++++++++++---- client/service/src/builder.rs | 7 +- utils/fork-tree/src/lib.rs | 14 ++ 10 files changed, 403 insertions(+), 39 deletions(-) diff --git a/bin/node-template/node/src/service.rs b/bin/node-template/node/src/service.rs index 0de17103b05a4..037d1c9e9bcfc 100644 --- a/bin/node-template/node/src/service.rs +++ b/bin/node-template/node/src/service.rs @@ -205,7 +205,7 @@ pub fn new_full(config: Configuration) -> Result { /// Builds a new service for a light client. pub fn new_light(config: Configuration) -> Result { - let (client, backend, keystore, mut task_manager, on_demand) = + let (client, backend, keystore, mut task_manager, on_demand, _shared_pruning_requirements) = sc_service::new_light_parts::(&config)?; let transaction_pool = Arc::new(sc_transaction_pool::BasicPool::new_light( diff --git a/bin/node/cli/src/service.rs b/bin/node/cli/src/service.rs index a47869ed83232..c6869e829cd80 100644 --- a/bin/node/cli/src/service.rs +++ b/bin/node/cli/src/service.rs @@ -80,6 +80,7 @@ pub fn new_partial(config: &Configuration) -> Result Result<( Arc::Hash>>, Arc>> ), ServiceError> { - let (client, backend, keystore, mut task_manager, on_demand) = + let (client, backend, keystore, mut task_manager, on_demand, shared_pruning_requirements) = sc_service::new_light_parts::(&config)?; let select_chain = sc_consensus::LongestChain::new(backend.clone()); @@ -364,6 +365,7 @@ pub fn new_light_base(config: Configuration) -> Result<( sc_consensus_babe::Config::get_or_compute(&*client)?, grandpa_block_import, client.clone(), + Some(&shared_pruning_requirements), )?; let inherent_data_providers = sp_inherents::InherentDataProviders::new(); diff --git a/client/api/src/backend.rs b/client/api/src/backend.rs index efc5ca4ee8ca0..d8020a5108be1 100644 --- a/client/api/src/backend.rs +++ b/client/api/src/backend.rs @@ -536,3 +536,148 @@ pub fn changes_tries_state_at_block<'a, Block: BlockT>( None => Ok(None), } } + +#[derive(Clone)] +/// Pruning requirement to share between multiple client component. +/// +/// This allows pruning related synchronisation. For instance in light +/// client we need to synchronize header pruning from CHT (every N blocks) +/// with the pruning from consensus used (babe for instance require that +/// its epoch headers are not pruned which works as long as the slot length +/// is less than the CHT pruning window. +/// Each compenent register at a given index (call are done by this order). +/// +/// Note that this struct could be split in two different struct (provider without +/// component and Component), depending on future usage of this shared info. +pub struct SharedPruningRequirements { + shared: Arc>, usize)>>, + component: Option, +} + +impl Default for SharedPruningRequirements { + fn default() -> Self { + SharedPruningRequirements { + shared: Arc::new(RwLock::new((Vec::new(), 0))), + component: None, + } + } +} + +impl SharedPruningRequirements { + /// Add a following requirement to apply. + /// Returns the shared requirement to use from this component. + pub fn next_instance( + &self, + ) -> SharedPruningRequirements { + let req = ComponentPruningRequirements::default(); + let index = { + let mut shared = self.shared.write(); + let index = shared.0.len(); + shared.0.push(req); + shared.1 += 1; + if shared.1 == usize::max_value() { + shared.1 = 1; + // marking all as changed is the safest + for component in shared.0.iter_mut() { + component.last_modified_check = 0; + } + } + index + }; + let mut result = self.clone(); + result.component = Some(index); + result + } + + /// Check if some content was modified after last check. + pub fn modification_check(&self) -> bool { + if let Some(index) = self.component { + let modified = { + let read = self.shared.read(); + if read.0[index].last_modified_check < read.1 { + Some(read.1.clone()) + } else { + None + } + }; + if let Some(modified) = modified { + self.shared.write().0[index].last_modified_check = modified; + true + } else { + false + } + } else { + false + } + } + + /// Resolve a finalized block headers to keep. + pub fn finalized_headers_needed(&self) -> PruningLimit> { + let mut result = PruningLimit::None; + for req in self.shared.read().0.iter() { + match &req.requires_finalized_header_up_to { + PruningLimit::Locked => return PruningLimit::Locked, + PruningLimit::None => (), + PruningLimit::Some(n) => { + if let PruningLimit::Some(p) = result { + if &p < n { + result = PruningLimit::Some(p); + continue; + } + } + result = PruningLimit::Some(n.clone()); + }, + } + } + result + } + + /// Set new requirement on finalized headers. + /// Returns false if we do not have a handle on the shared requirement. + pub fn set_finalized_headers_needed(&self, limit: PruningLimit>) -> bool { + if let Some(index) = self.component { + let mut shared = self.shared.write(); + if shared.0[index].requires_finalized_header_up_to != limit { + shared.0[index].requires_finalized_header_up_to = limit; + shared.1 += 1; + if shared.1 == usize::max_value() { + shared.1 = 1; + // marking all as changed is the safest + for component in shared.0.iter_mut() { + component.last_modified_check = 0; + } + } + } + true + } else { + false + } + } +} + +/// Individual pruning requirement for any substrate component. +struct ComponentPruningRequirements { + requires_finalized_header_up_to: PruningLimit>, + last_modified_check: usize, +} + +impl Default for ComponentPruningRequirements { + fn default() -> Self { + ComponentPruningRequirements { + requires_finalized_header_up_to: PruningLimit::None, + last_modified_check: 0, + } + } +} + +#[derive(Eq, PartialEq)] +/// Define a block number limit to apply. +pub enum PruningLimit { + /// Ignore. + None, + /// The component require at least this number + /// of unpruned elements. + Some(N), + /// We lock all pruning. + Locked, +} diff --git a/client/consensus/babe/rpc/src/lib.rs b/client/consensus/babe/rpc/src/lib.rs index 652f4f00baac2..b876208f72c82 100644 --- a/client/consensus/babe/rpc/src/lib.rs +++ b/client/consensus/babe/rpc/src/lib.rs @@ -264,6 +264,7 @@ mod tests { config.clone(), client.clone(), client.clone(), + None, ).expect("can initialize block-import"); let epoch_changes = link.epoch_changes().clone(); diff --git a/client/consensus/babe/src/lib.rs b/client/consensus/babe/src/lib.rs index 951d1467b4983..1efa7af47d009 100644 --- a/client/consensus/babe/src/lib.rs +++ b/client/consensus/babe/src/lib.rs @@ -104,6 +104,7 @@ use sp_consensus::import_queue::{Verifier, BasicQueue, DefaultImportQueue, Cache use sc_client_api::{ backend::AuxStore, BlockchainEvents, ProvideUncles, + SharedPruningRequirements, PruningLimit, }; use sp_block_builder::BlockBuilder as BlockBuilderApi; use futures::channel::mpsc::{channel, Sender, Receiver}; @@ -1063,6 +1064,8 @@ pub struct BabeBlockImport { client: Arc, epoch_changes: SharedEpochChanges, config: Config, + previous_needed_height: Option>, + shared_pruning_requirements: Option>, } impl Clone for BabeBlockImport { @@ -1072,6 +1075,8 @@ impl Clone for BabeBlockImport BabeBlockImport { epoch_changes: SharedEpochChanges, block_import: I, config: Config, + shared_pruning_requirements: Option<&SharedPruningRequirements>, ) -> Self { + let shared_pruning_requirements = shared_pruning_requirements.map(|shared| { + let req = shared.next_instance(); + assert!(req.set_finalized_headers_needed(PruningLimit::Locked)); + req + }); BabeBlockImport { client, inner: block_import, epoch_changes, config, + previous_needed_height: Some(Zero::zero()), + shared_pruning_requirements, } } } @@ -1286,6 +1299,20 @@ impl BlockImport for BabeBlockImport( config: Config, wrapped_block_import: I, client: Arc, + shared_pruning_requirements: Option<&SharedPruningRequirements>, ) -> ClientResult<(BabeBlockImport, BabeLink)> where Client: AuxStore + HeaderBackend + HeaderMetadata, { @@ -1408,6 +1436,7 @@ pub fn block_import( epoch_changes, wrapped_block_import, config, + shared_pruning_requirements, ); Ok((import, link)) diff --git a/client/consensus/babe/src/tests.rs b/client/consensus/babe/src/tests.rs index 958d7845edbc6..b9cd51fb97179 100644 --- a/client/consensus/babe/src/tests.rs +++ b/client/consensus/babe/src/tests.rs @@ -278,6 +278,7 @@ impl TestNetFactory for BabeTestNet { config, client.clone(), client.clone(), + None, ).expect("can initialize block-import"); let block_import = PanickingBlockImport(block_import); diff --git a/client/consensus/epochs/src/lib.rs b/client/consensus/epochs/src/lib.rs index acb07dd668a3c..962d103319305 100644 --- a/client/consensus/epochs/src/lib.rs +++ b/client/consensus/epochs/src/lib.rs @@ -18,7 +18,7 @@ pub mod migration; -use std::{sync::Arc, ops::Add, collections::BTreeMap, borrow::{Borrow, BorrowMut}}; +use std::{sync::Arc, ops::{Add, Sub}, collections::BTreeMap, borrow::{Borrow, BorrowMut}}; use parking_lot::Mutex; use codec::{Encode, Decode}; use fork_tree::ForkTree; @@ -322,7 +322,7 @@ impl Default for EpochChanges where impl EpochChanges where Hash: PartialEq + Ord + AsRef<[u8]> + AsMut<[u8]> + Copy, - Number: Ord + One + Zero + Add + Copy, + Number: Ord + One + Zero + Sub + Add + Copy, { /// Create a new epoch change. pub fn new() -> Self { @@ -640,6 +640,14 @@ impl EpochChanges where pub fn tree(&self) -> &ForkTree> { &self.inner } + + /// Get the minimal block height where we need to be able + /// to do `is_descendent_of` queries. + pub fn needed_parent_relation(&self) -> Option { + // TODO consider using tree accessor and remove this function + self.inner.lowest_node_number() + .map(|number| number - One::one()) + } } /// Type alias to produce the epoch-changes tree from a block type. diff --git a/client/db/src/light.rs b/client/db/src/light.rs index 139ecf3b22c72..c5ff722874573 100644 --- a/client/db/src/light.rs +++ b/client/db/src/light.rs @@ -21,13 +21,14 @@ use std::{sync::Arc, collections::HashMap}; use std::convert::TryInto; use parking_lot::RwLock; +use std::collections::VecDeque; use sc_client_api::{ cht, backend::{AuxStore, NewBlockState}, UsageInfo, blockchain::{ BlockStatus, Cache as BlockchainCache, Info as BlockchainInfo, }, - Storage + Storage, SharedPruningRequirements,PruningLimit, }; use sp_blockchain::{ CachedHeaderMetadata, HeaderMetadata, HeaderMetadataCache, @@ -42,7 +43,7 @@ use sp_runtime::traits::{Block as BlockT, Header as HeaderT, Zero, One, NumberFo use crate::cache::{DbCacheSync, DbCache, ComplexBlockId, EntryType as CacheEntryType}; use crate::utils::{self, meta_keys, DatabaseType, Meta, read_db, block_id_to_lookup_key, read_meta}; use crate::{DatabaseSettings, FrozenForDuration, DbHash}; -use log::{trace, warn, debug}; +use log::{warn, debug}; pub(crate) mod columns { pub const META: u32 = crate::utils::COLUMN_META; @@ -58,6 +59,9 @@ const HEADER_CHT_PREFIX: u8 = 0; /// Prefix for changes tries roots CHT. const CHANGES_TRIE_CHT_PREFIX: u8 = 1; +/// Aux key to store pending pruning CHT ranges. +const AUX_PENDING_CHT_RANGES: &'static[u8] = b"pending_cht_pruning_ranges"; + /// Light blockchain storage. Stores most recent headers + CHTs for older headers. /// Locks order: meta, cache. pub struct LightStorage { @@ -65,6 +69,8 @@ pub struct LightStorage { meta: RwLock, Block::Hash>>, cache: Arc>, header_metadata_cache: Arc>, + shared_pruning_requirements: SharedPruningRequirements, + pending_cht_pruning: RwLock, NumberFor)>>, #[cfg(not(target_os = "unknown"))] io_stats: FrozenForDuration, @@ -72,19 +78,26 @@ pub struct LightStorage { impl LightStorage { /// Create new storage with given settings. - pub fn new(config: DatabaseSettings) -> ClientResult { + pub fn new( + config: DatabaseSettings, + shared_pruning_requirements: SharedPruningRequirements, + ) -> ClientResult { let db = crate::utils::open_database::(&config, DatabaseType::Light)?; - Self::from_kvdb(db as Arc<_>) + Self::from_kvdb(db as Arc<_>, shared_pruning_requirements) } /// Create new memory-backed `LightStorage` for tests. #[cfg(any(test, feature = "test-helpers"))] pub fn new_test() -> Self { let db = Arc::new(sp_database::MemDb::default()); - Self::from_kvdb(db as Arc<_>).expect("failed to create test-db") + let shared_pruning_requirements = SharedPruningRequirements::default(); + Self::from_kvdb(db as Arc<_>, shared_pruning_requirements).expect("failed to create test-db") } - fn from_kvdb(db: Arc>) -> ClientResult { + fn from_kvdb( + db: Arc>, + shared_pruning_requirements: SharedPruningRequirements, + ) -> ClientResult { let meta = read_meta::(&*db, columns::HEADER)?; let header_metadata_cache = Arc::new(HeaderMetadataCache::default()); let cache = DbCache::new( @@ -97,6 +110,9 @@ impl LightStorage { ComplexBlockId::new(meta.finalized_hash, meta.finalized_number), ); + let pending_cht_pruning_ranges = db.get(columns::AUX, AUX_PENDING_CHT_RANGES) + .and_then(|encoded| Decode::decode(&mut encoded.as_slice()).ok()) + .unwrap_or_default(); Ok(LightStorage { db, meta: RwLock::new(meta), @@ -104,6 +120,8 @@ impl LightStorage { header_metadata_cache, #[cfg(not(target_os = "unknown"))] io_stats: FrozenForDuration::new(std::time::Duration::from_secs(1)), + shared_pruning_requirements, + pending_cht_pruning: RwLock::new(pending_cht_pruning_ranges), }) } @@ -335,26 +353,144 @@ impl LightStorage { ); } - // prune headers that are replaced with CHT - let mut prune_block = new_cht_start; let new_cht_end = cht::end_number(cht::size(), new_cht_number); - trace!(target: "db", "Replacing blocks [{}..{}] with CHT#{}", - new_cht_start, new_cht_end, new_cht_number); - - while prune_block <= new_cht_end { - if let Some(hash) = self.hash(prune_block)? { - let lookup_key = block_id_to_lookup_key::(&*self.db, columns::KEY_LOOKUP, BlockId::Number(prune_block))? - .expect("retrieved hash for `prune_block` right above. therefore retrieving lookup key must succeed. q.e.d."); - utils::remove_key_mappings( - transaction, - columns::KEY_LOOKUP, - prune_block, - hash - )?; - transaction.remove(columns::HEADER, &lookup_key); + self.try_prune_pending(transaction, false)?; + self.prune_range(new_cht_start, new_cht_end, transaction)?; + } else { + self.try_prune_pending(transaction, true)?; + } + + Ok(()) + } + + /// Prune headers that are replaced with CHT. + /// Note that end index is inclusive (as start). + fn prune_range( + &self, + start: NumberFor, + end: NumberFor, + transaction: &mut Transaction, + ) -> ClientResult<()> { + if let Some((start, end)) = match self.shared_pruning_requirements.finalized_headers_needed() { + PruningLimit::None => { + Some((start, end)) + }, + PruningLimit::Some(limit) => { + if limit > end { + Some((start, end)) + } else if limit > start { + self.add_prune_range_pending(limit, end, transaction)?; + Some((start, limit - One::one())) + } else { + self.add_prune_range_pending(start, end, transaction)?; + None } - prune_block += One::one(); + }, + PruningLimit::Locked => { + self.add_prune_range_pending(start, end, transaction)?; + None + }, + } { + self.prune_range_unchecked(start, end, transaction)?; + } + Ok(()) + } + + fn prune_range_unchecked( + &self, + start: NumberFor, + end: NumberFor, + transaction: &mut Transaction, + ) -> ClientResult<()> { + let mut prune_block = start; + while prune_block <= end { + if let Some(hash) = self.hash(prune_block)? { + let lookup_key = block_id_to_lookup_key::(&*self.db, columns::KEY_LOOKUP, BlockId::Number(prune_block))? + .expect("retrieved hash for `prune_block` right above. therefore retrieving lookup key must succeed. q.e.d."); + utils::remove_key_mappings( + transaction, + columns::KEY_LOOKUP, + prune_block, + hash + )?; + transaction.remove(columns::HEADER, &lookup_key); } + prune_block += One::one(); + } + Ok(()) + } + + /// Stack some pending pruning range. + fn add_prune_range_pending( + &self, + start: NumberFor, + end: NumberFor, + transaction: &mut Transaction, + ) -> ClientResult<()> { + // Note that we do not handle the error case for in memory data + // as with other field of light storage. + self.pending_cht_pruning.write().push_back((start, end)); + + let encoded_pending = self.pending_cht_pruning.read().encode(); + transaction.set_from_vec(columns::AUX, AUX_PENDING_CHT_RANGES, encoded_pending); + + Ok(()) + } + + #[cfg(test)] + fn try_prune_pending_test(&self, do_check: bool) -> ClientResult<()> { + let mut transaction = Transaction::new(); + self.try_prune_pending(&mut transaction, do_check)?; + self.db.commit(transaction)?; + Ok(()) + } + + /// Try pruning pending. + fn try_prune_pending( + &self, + transaction: &mut Transaction, + do_check: bool, + ) -> ClientResult<()> { + let mut changed = false; + let mut to_prune = VecDeque::new(); + // TODO bool condition + let pass = if do_check { + self.shared_pruning_requirements.modification_check() + } else { + true + }; + if !self.pending_cht_pruning.read().is_empty() + && pass { + match self.shared_pruning_requirements.finalized_headers_needed() { + PruningLimit::Locked => (), + PruningLimit::None => { + to_prune = std::mem::replace(&mut *self.pending_cht_pruning.write(), VecDeque::new()); + }, + PruningLimit::Some(limit) => { + let mut shared = self.pending_cht_pruning.write(); + while let Some(range) = shared.pop_front() { + if limit > range.1 { + to_prune.push_back(range); + } else if limit > range.0 { + to_prune.push_back((range.0, limit - One::one())); + shared.push_front((limit, range.1)); + break; + } else { + shared.push_front((range.0, range.1)); + break; + } + } + }, + } + } + for range in to_prune { + changed = true; + self.prune_range_unchecked(range.0, range.1, transaction)?; + } + + if changed { + let encoded_pending = self.pending_cht_pruning.read().encode(); + transaction.set_from_vec(columns::AUX, AUX_PENDING_CHT_RANGES, encoded_pending); } Ok(()) @@ -738,7 +874,7 @@ pub(crate) mod tests { #[test] fn import_header_works() { let raw_db = Arc::new(sp_database::MemDb::default()); - let db = LightStorage::from_kvdb(raw_db.clone()).unwrap(); + let db = LightStorage::from_kvdb(raw_db.clone(), Default::default()).unwrap(); let genesis_hash = insert_block(&db, HashMap::new(), || default_header(&Default::default(), 0)); assert_eq!(raw_db.count(columns::HEADER), 1); @@ -751,11 +887,15 @@ pub(crate) mod tests { #[test] fn finalized_ancient_headers_are_replaced_with_cht() { - fn insert_headers Header>(header_producer: F) -> - (Arc>, LightStorage) - { + fn insert_headers Header>( + header_producer: F, + limit: PruningLimit, + ) -> (Arc>, LightStorage, SharedPruningRequirements) { let raw_db = Arc::new(sp_database::MemDb::default()); - let db = LightStorage::from_kvdb(raw_db.clone()).unwrap(); + let shared_pruning_requirements = SharedPruningRequirements::default(); + let req = shared_pruning_requirements.next_instance(); + req.set_finalized_headers_needed(limit); + let db = LightStorage::from_kvdb(raw_db.clone(), req.clone()).unwrap(); let cht_size: u64 = cht::size(); let ucht_size: usize = cht_size as _; @@ -796,11 +936,11 @@ pub(crate) mod tests { db.finalize_header(BlockId::Number(i as _)).unwrap(); } db.finalize_header(BlockId::Hash(prev_hash)).unwrap(); - (raw_db, db) + (raw_db, db, req) } // when headers are created without changes tries roots - let (raw_db, db) = insert_headers(default_header); + let (raw_db, db, _pruning_limit) = insert_headers(default_header, PruningLimit::None); let cht_size: u64 = cht::size(); assert_eq!(raw_db.count(columns::HEADER), (1 + cht_size + 1) as usize); assert_eq!(raw_db.count(columns::KEY_LOOKUP), (2 * (1 + cht_size + 1)) as usize); @@ -811,8 +951,28 @@ pub(crate) mod tests { assert!(db.changes_trie_cht_root(cht_size, cht_size / 2).is_err()); assert!(db.changes_trie_cht_root(cht_size, cht_size + cht_size / 2).unwrap().is_none()); + // try locked pruning + let (raw_db, db, pruning_limit) = insert_headers(default_header, PruningLimit::Locked); + let cht_size: u64 = cht::size(); + assert_eq!(raw_db.count(columns::HEADER), (2 + 2 * cht_size) as usize); + assert_eq!(raw_db.count(columns::KEY_LOOKUP), (2 * (2 + 2 * cht_size)) as usize); + assert!((0..cht_size as _).all(|i| db.header(BlockId::Number(1 + i)).unwrap().is_some())); + + assert!(pruning_limit.set_finalized_headers_needed(PruningLimit::Some(1000))); + db.try_prune_pending_test(true).unwrap(); + assert_eq!(raw_db.count(columns::HEADER), (2 + 2 * cht_size) as usize - 999); + assert_eq!(raw_db.count(columns::KEY_LOOKUP), (2 * (2 + 2 * cht_size - 999)) as usize); + assert!((1..999 as _).all(|i| db.header(BlockId::Number(1 + i)).unwrap().is_none())); + assert!((999..cht_size as _).all(|i| db.header(BlockId::Number(1 + i)).unwrap().is_some())); + + assert!(pruning_limit.set_finalized_headers_needed(PruningLimit::None)); + db.try_prune_pending_test(true).unwrap(); + assert_eq!(raw_db.count(columns::HEADER), (1 + cht_size + 1) as usize); + assert_eq!(raw_db.count(columns::KEY_LOOKUP), (2 * (1 + cht_size + 1)) as usize); + assert!((0..cht_size as _).all(|i| db.header(BlockId::Number(1 + i)).unwrap().is_none())); + // when headers are created with changes tries roots - let (raw_db, db) = insert_headers(header_with_changes_trie); + let (raw_db, db, _pruning_limit) = insert_headers(header_with_changes_trie, PruningLimit::None); assert_eq!(raw_db.count(columns::HEADER), (1 + cht_size + 1) as usize); assert_eq!(raw_db.count(columns::CHT), 2); assert!((0..cht_size as _).all(|i| db.header(BlockId::Number(1 + i)).unwrap().is_none())); @@ -1138,7 +1298,7 @@ pub(crate) mod tests { assert_eq!(db.header(BlockId::Hash(hash0)).unwrap().unwrap().hash(), hash0); let db = db.db; - let db = LightStorage::from_kvdb(db).unwrap(); + let db = LightStorage::from_kvdb(db, Default::default()).unwrap(); assert_eq!(db.info().best_hash, hash0); assert_eq!(db.header(BlockId::Hash::(hash0)).unwrap().unwrap().hash(), hash0); } @@ -1199,7 +1359,8 @@ pub(crate) mod tests { }; // restart && check that after restart value is read from the cache - let db = LightStorage::::from_kvdb(storage as Arc<_>).expect("failed to create test-db"); + let db = LightStorage::::from_kvdb(storage as Arc<_>, Default::default()) + .expect("failed to create test-db"); assert_eq!( db.cache().get_at(b"test", &BlockId::Number(0)).unwrap(), Some(((0, genesis_hash.unwrap()), None, vec![42])), diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index 4c7c1f57ee04b..31649373d3be1 100644 --- a/client/service/src/builder.rs +++ b/client/service/src/builder.rs @@ -26,6 +26,7 @@ use crate::{ }; use sc_client_api::{ light::RemoteBlockchain, ForkBlocks, BadBlocks, UsageProvider, ExecutorProvider, + SharedPruningRequirements, }; use sp_utils::mpsc::{tracing_unbounded, TracingUnboundedSender}; use sc_chain_spec::get_extension; @@ -167,6 +168,7 @@ type TLightParts = ( Arc>, TaskManager, Arc>, + SharedPruningRequirements, ); /// Light client backend type with a specific hash type. @@ -271,6 +273,7 @@ pub fn new_light_parts( TBl: BlockT, TExecDisp: NativeExecutionDispatch + 'static, { + let shared_pruning_requirements = SharedPruningRequirements::default(); let task_manager = { let registry = config.prometheus_config.as_ref().map(|cfg| &cfg.registry); @@ -299,7 +302,7 @@ pub fn new_light_parts( pruning: config.pruning.clone(), source: config.database.clone(), }; - sc_client_db::light::LightStorage::new(db_settings)? + sc_client_db::light::LightStorage::new(db_settings, shared_pruning_requirements.clone())? }; let light_blockchain = sc_light::new_light_blockchain(db_storage); let fetch_checker = Arc::new( @@ -319,7 +322,7 @@ pub fn new_light_parts( config.prometheus_config.as_ref().map(|config| config.registry.clone()), )?); - Ok((client, backend, keystore, task_manager, on_demand)) + Ok((client, backend, keystore, task_manager, on_demand, shared_pruning_requirements)) } /// Create an instance of db-backed client. diff --git a/utils/fork-tree/src/lib.rs b/utils/fork-tree/src/lib.rs index 1d01c53417649..33533182c58a4 100644 --- a/utils/fork-tree/src/lib.rs +++ b/utils/fork-tree/src/lib.rs @@ -166,6 +166,20 @@ impl ForkTree where Ok(RemovedIterator { stack: removed }) } + + /// Return the lowest root number. + pub fn lowest_node_number(&self) -> Option { + let mut result = None; + for root in self.roots.iter() { + let replace = result.as_ref().map(|lowest| &root.number < lowest) + .unwrap_or(true); + if replace { + result = Some(root.number.clone()); + } + } + + result + } } impl ForkTree where From 8fe1bea99fd5ae9f251b9c3aef9fd1966d0d4563 Mon Sep 17 00:00:00 2001 From: cheme Date: Mon, 17 Aug 2020 09:49:22 +0200 Subject: [PATCH 02/17] Put derive under comment blocks. --- client/api/src/backend.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/client/api/src/backend.rs b/client/api/src/backend.rs index d8020a5108be1..5fbb5981c7dc2 100644 --- a/client/api/src/backend.rs +++ b/client/api/src/backend.rs @@ -537,7 +537,6 @@ pub fn changes_tries_state_at_block<'a, Block: BlockT>( } } -#[derive(Clone)] /// Pruning requirement to share between multiple client component. /// /// This allows pruning related synchronisation. For instance in light @@ -549,6 +548,7 @@ pub fn changes_tries_state_at_block<'a, Block: BlockT>( /// /// Note that this struct could be split in two different struct (provider without /// component and Component), depending on future usage of this shared info. +#[derive(Clone)] pub struct SharedPruningRequirements { shared: Arc>, usize)>>, component: Option, @@ -670,8 +670,8 @@ impl Default for ComponentPruningRequirements { } } -#[derive(Eq, PartialEq)] /// Define a block number limit to apply. +#[derive(Eq, PartialEq)] pub enum PruningLimit { /// Ignore. None, From cfea1f12cf2eb8f6dcb521043ecb2f01d1683689 Mon Sep 17 00:00:00 2001 From: cheme Date: Mon, 17 Aug 2020 11:04:39 +0200 Subject: [PATCH 03/17] Safer api (no more optional component, option could be remove from babe too but would need to use traits). Removed checked optimization. --- client/api/src/backend.rs | 120 +++++++++++------------------ client/consensus/babe/src/lib.rs | 18 +++-- client/consensus/epochs/src/lib.rs | 8 -- client/db/src/light.rs | 44 +++++------ client/service/src/builder.rs | 6 +- 5 files changed, 78 insertions(+), 118 deletions(-) diff --git a/client/api/src/backend.rs b/client/api/src/backend.rs index 5fbb5981c7dc2..84fdf562a1914 100644 --- a/client/api/src/backend.rs +++ b/client/api/src/backend.rs @@ -537,6 +537,12 @@ pub fn changes_tries_state_at_block<'a, Block: BlockT>( } } +/// Shared pruning requirement read only access and instantiation. +#[derive(Clone)] +pub struct SharedPruningRequirementsSource { + shared: Arc>, usize)>>, +} + /// Pruning requirement to share between multiple client component. /// /// This allows pruning related synchronisation. For instance in light @@ -551,19 +557,40 @@ pub fn changes_tries_state_at_block<'a, Block: BlockT>( #[derive(Clone)] pub struct SharedPruningRequirements { shared: Arc>, usize)>>, - component: Option, + component: usize, } -impl Default for SharedPruningRequirements { +impl Default for SharedPruningRequirementsSource { fn default() -> Self { - SharedPruningRequirements { + SharedPruningRequirementsSource { shared: Arc::new(RwLock::new((Vec::new(), 0))), - component: None, } } } -impl SharedPruningRequirements { +fn shared_finalized_headers_needed( + shared: &Arc>, usize)>>, +) -> PruningLimit> { + let mut result = PruningLimit::None; + for req in shared.read().0.iter() { + match &req.requires_finalized_header_up_to { + PruningLimit::Locked => return PruningLimit::Locked, + PruningLimit::None => (), + PruningLimit::Some(n) => { + if let PruningLimit::Some(p) = result { + if &p < n { + result = PruningLimit::Some(p); + continue; + } + } + result = PruningLimit::Some(n.clone()); + }, + } + } + result +} + +impl SharedPruningRequirementsSource { /// Add a following requirement to apply. /// Returns the shared requirement to use from this component. pub fn next_instance( @@ -575,97 +602,44 @@ impl SharedPruningRequirements { let index = shared.0.len(); shared.0.push(req); shared.1 += 1; - if shared.1 == usize::max_value() { - shared.1 = 1; - // marking all as changed is the safest - for component in shared.0.iter_mut() { - component.last_modified_check = 0; - } - } index }; - let mut result = self.clone(); - result.component = Some(index); - result + SharedPruningRequirements { + shared: self.shared.clone(), + component: index, + } } - /// Check if some content was modified after last check. - pub fn modification_check(&self) -> bool { - if let Some(index) = self.component { - let modified = { - let read = self.shared.read(); - if read.0[index].last_modified_check < read.1 { - Some(read.1.clone()) - } else { - None - } - }; - if let Some(modified) = modified { - self.shared.write().0[index].last_modified_check = modified; - true - } else { - false - } - } else { - false - } + /// Resolve a finalized block headers to keep. + pub fn finalized_headers_needed(&self) -> PruningLimit> { + shared_finalized_headers_needed(&self.shared) } - +} + +impl SharedPruningRequirements { /// Resolve a finalized block headers to keep. pub fn finalized_headers_needed(&self) -> PruningLimit> { - let mut result = PruningLimit::None; - for req in self.shared.read().0.iter() { - match &req.requires_finalized_header_up_to { - PruningLimit::Locked => return PruningLimit::Locked, - PruningLimit::None => (), - PruningLimit::Some(n) => { - if let PruningLimit::Some(p) = result { - if &p < n { - result = PruningLimit::Some(p); - continue; - } - } - result = PruningLimit::Some(n.clone()); - }, - } - } - result + shared_finalized_headers_needed(&self.shared) } /// Set new requirement on finalized headers. /// Returns false if we do not have a handle on the shared requirement. - pub fn set_finalized_headers_needed(&self, limit: PruningLimit>) -> bool { - if let Some(index) = self.component { - let mut shared = self.shared.write(); - if shared.0[index].requires_finalized_header_up_to != limit { - shared.0[index].requires_finalized_header_up_to = limit; - shared.1 += 1; - if shared.1 == usize::max_value() { - shared.1 = 1; - // marking all as changed is the safest - for component in shared.0.iter_mut() { - component.last_modified_check = 0; - } - } - } - true - } else { - false - } + pub fn set_finalized_headers_needed(&self, limit: PruningLimit>) { + let index = self.component; + let mut shared = self.shared.write(); + shared.0[index].requires_finalized_header_up_to = limit; } } /// Individual pruning requirement for any substrate component. struct ComponentPruningRequirements { requires_finalized_header_up_to: PruningLimit>, - last_modified_check: usize, } impl Default for ComponentPruningRequirements { fn default() -> Self { ComponentPruningRequirements { requires_finalized_header_up_to: PruningLimit::None, - last_modified_check: 0, } } } diff --git a/client/consensus/babe/src/lib.rs b/client/consensus/babe/src/lib.rs index 1efa7af47d009..f042c16ad9596 100644 --- a/client/consensus/babe/src/lib.rs +++ b/client/consensus/babe/src/lib.rs @@ -86,7 +86,7 @@ use sp_core::{crypto::Public, traits::BareCryptoStore}; use sp_application_crypto::AppKey; use sp_runtime::{ generic::{BlockId, OpaqueDigestItemId}, Justification, - traits::{Block as BlockT, Header, DigestItemFor, Zero}, + traits::{Block as BlockT, Header, DigestItemFor, Zero, One}, }; use sp_api::{ProvideRuntimeApi, NumberFor}; use sc_keystore::KeyStorePtr; @@ -104,7 +104,7 @@ use sp_consensus::import_queue::{Verifier, BasicQueue, DefaultImportQueue, Cache use sc_client_api::{ backend::AuxStore, BlockchainEvents, ProvideUncles, - SharedPruningRequirements, PruningLimit, + SharedPruningRequirements, SharedPruningRequirementsSource, PruningLimit, }; use sp_block_builder::BlockBuilder as BlockBuilderApi; use futures::channel::mpsc::{channel, Sender, Receiver}; @@ -1087,11 +1087,11 @@ impl BabeBlockImport { epoch_changes: SharedEpochChanges, block_import: I, config: Config, - shared_pruning_requirements: Option<&SharedPruningRequirements>, + shared_pruning_requirements: Option<&SharedPruningRequirementsSource>, ) -> Self { let shared_pruning_requirements = shared_pruning_requirements.map(|shared| { let req = shared.next_instance(); - assert!(req.set_finalized_headers_needed(PruningLimit::Locked)); + req.set_finalized_headers_needed(PruningLimit::Locked); req }); BabeBlockImport { @@ -1302,14 +1302,16 @@ impl BlockImport for BabeBlockImport( config: Config, wrapped_block_import: I, client: Arc, - shared_pruning_requirements: Option<&SharedPruningRequirements>, + shared_pruning_requirements: Option<&SharedPruningRequirementsSource>, ) -> ClientResult<(BabeBlockImport, BabeLink)> where Client: AuxStore + HeaderBackend + HeaderMetadata, { diff --git a/client/consensus/epochs/src/lib.rs b/client/consensus/epochs/src/lib.rs index 962d103319305..6bb981b2998b6 100644 --- a/client/consensus/epochs/src/lib.rs +++ b/client/consensus/epochs/src/lib.rs @@ -640,14 +640,6 @@ impl EpochChanges where pub fn tree(&self) -> &ForkTree> { &self.inner } - - /// Get the minimal block height where we need to be able - /// to do `is_descendent_of` queries. - pub fn needed_parent_relation(&self) -> Option { - // TODO consider using tree accessor and remove this function - self.inner.lowest_node_number() - .map(|number| number - One::one()) - } } /// Type alias to produce the epoch-changes tree from a block type. diff --git a/client/db/src/light.rs b/client/db/src/light.rs index c5ff722874573..5fc86f2e86d6e 100644 --- a/client/db/src/light.rs +++ b/client/db/src/light.rs @@ -28,7 +28,7 @@ use sc_client_api::{ blockchain::{ BlockStatus, Cache as BlockchainCache, Info as BlockchainInfo, }, - Storage, SharedPruningRequirements,PruningLimit, + Storage, SharedPruningRequirementsSource, PruningLimit, }; use sp_blockchain::{ CachedHeaderMetadata, HeaderMetadata, HeaderMetadataCache, @@ -69,7 +69,7 @@ pub struct LightStorage { meta: RwLock, Block::Hash>>, cache: Arc>, header_metadata_cache: Arc>, - shared_pruning_requirements: SharedPruningRequirements, + shared_pruning_requirements: SharedPruningRequirementsSource, pending_cht_pruning: RwLock, NumberFor)>>, #[cfg(not(target_os = "unknown"))] @@ -80,7 +80,7 @@ impl LightStorage { /// Create new storage with given settings. pub fn new( config: DatabaseSettings, - shared_pruning_requirements: SharedPruningRequirements, + shared_pruning_requirements: SharedPruningRequirementsSource, ) -> ClientResult { let db = crate::utils::open_database::(&config, DatabaseType::Light)?; Self::from_kvdb(db as Arc<_>, shared_pruning_requirements) @@ -90,13 +90,13 @@ impl LightStorage { #[cfg(any(test, feature = "test-helpers"))] pub fn new_test() -> Self { let db = Arc::new(sp_database::MemDb::default()); - let shared_pruning_requirements = SharedPruningRequirements::default(); + let shared_pruning_requirements = SharedPruningRequirementsSource::default(); Self::from_kvdb(db as Arc<_>, shared_pruning_requirements).expect("failed to create test-db") } fn from_kvdb( db: Arc>, - shared_pruning_requirements: SharedPruningRequirements, + shared_pruning_requirements: SharedPruningRequirementsSource, ) -> ClientResult { let meta = read_meta::(&*db, columns::HEADER)?; let header_metadata_cache = Arc::new(HeaderMetadataCache::default()); @@ -354,10 +354,10 @@ impl LightStorage { } let new_cht_end = cht::end_number(cht::size(), new_cht_number); - self.try_prune_pending(transaction, false)?; + self.try_prune_pending(transaction)?; self.prune_range(new_cht_start, new_cht_end, transaction)?; } else { - self.try_prune_pending(transaction, true)?; + self.try_prune_pending(transaction)?; } Ok(()) @@ -438,9 +438,9 @@ impl LightStorage { } #[cfg(test)] - fn try_prune_pending_test(&self, do_check: bool) -> ClientResult<()> { + fn try_prune_pending_test(&self) -> ClientResult<()> { let mut transaction = Transaction::new(); - self.try_prune_pending(&mut transaction, do_check)?; + self.try_prune_pending(&mut transaction)?; self.db.commit(transaction)?; Ok(()) } @@ -449,18 +449,10 @@ impl LightStorage { fn try_prune_pending( &self, transaction: &mut Transaction, - do_check: bool, ) -> ClientResult<()> { let mut changed = false; let mut to_prune = VecDeque::new(); - // TODO bool condition - let pass = if do_check { - self.shared_pruning_requirements.modification_check() - } else { - true - }; - if !self.pending_cht_pruning.read().is_empty() - && pass { + if !self.pending_cht_pruning.read().is_empty() { match self.shared_pruning_requirements.finalized_headers_needed() { PruningLimit::Locked => (), PruningLimit::None => { @@ -757,7 +749,7 @@ fn cht_key>(cht_type: u8, block: N) -> ClientResult<[u8; 5]> { #[cfg(test)] pub(crate) mod tests { - use sc_client_api::cht; + use sc_client_api::{cht, SharedPruningRequirements}; use sp_core::ChangesTrieConfiguration; use sp_runtime::generic::{DigestItem, ChangesTrieSignal}; use sp_runtime::testing::{H256 as Hash, Header, Block as RawBlock, ExtrinsicWrapper}; @@ -892,10 +884,10 @@ pub(crate) mod tests { limit: PruningLimit, ) -> (Arc>, LightStorage, SharedPruningRequirements) { let raw_db = Arc::new(sp_database::MemDb::default()); - let shared_pruning_requirements = SharedPruningRequirements::default(); - let req = shared_pruning_requirements.next_instance(); + let shared_source = SharedPruningRequirementsSource::default(); + let req = shared_source.next_instance(); req.set_finalized_headers_needed(limit); - let db = LightStorage::from_kvdb(raw_db.clone(), req.clone()).unwrap(); + let db = LightStorage::from_kvdb(raw_db.clone(), shared_source).unwrap(); let cht_size: u64 = cht::size(); let ucht_size: usize = cht_size as _; @@ -958,15 +950,15 @@ pub(crate) mod tests { assert_eq!(raw_db.count(columns::KEY_LOOKUP), (2 * (2 + 2 * cht_size)) as usize); assert!((0..cht_size as _).all(|i| db.header(BlockId::Number(1 + i)).unwrap().is_some())); - assert!(pruning_limit.set_finalized_headers_needed(PruningLimit::Some(1000))); - db.try_prune_pending_test(true).unwrap(); + pruning_limit.set_finalized_headers_needed(PruningLimit::Some(1000)); + db.try_prune_pending_test().unwrap(); assert_eq!(raw_db.count(columns::HEADER), (2 + 2 * cht_size) as usize - 999); assert_eq!(raw_db.count(columns::KEY_LOOKUP), (2 * (2 + 2 * cht_size - 999)) as usize); assert!((1..999 as _).all(|i| db.header(BlockId::Number(1 + i)).unwrap().is_none())); assert!((999..cht_size as _).all(|i| db.header(BlockId::Number(1 + i)).unwrap().is_some())); - assert!(pruning_limit.set_finalized_headers_needed(PruningLimit::None)); - db.try_prune_pending_test(true).unwrap(); + pruning_limit.set_finalized_headers_needed(PruningLimit::None); + db.try_prune_pending_test().unwrap(); assert_eq!(raw_db.count(columns::HEADER), (1 + cht_size + 1) as usize); assert_eq!(raw_db.count(columns::KEY_LOOKUP), (2 * (1 + cht_size + 1)) as usize); assert!((0..cht_size as _).all(|i| db.header(BlockId::Number(1 + i)).unwrap().is_none())); diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index 82efd187461ae..e64867e0ba7d0 100644 --- a/client/service/src/builder.rs +++ b/client/service/src/builder.rs @@ -26,7 +26,7 @@ use crate::{ }; use sc_client_api::{ light::RemoteBlockchain, ForkBlocks, BadBlocks, UsageProvider, ExecutorProvider, - SharedPruningRequirements, + SharedPruningRequirementsSource, }; use sp_utils::mpsc::{tracing_unbounded, TracingUnboundedSender}; use sc_chain_spec::get_extension; @@ -168,7 +168,7 @@ type TLightParts = ( Arc>, TaskManager, Arc>, - SharedPruningRequirements, + SharedPruningRequirementsSource, ); /// Light client backend type with a specific hash type. @@ -273,7 +273,7 @@ pub fn new_light_parts( TBl: BlockT, TExecDisp: NativeExecutionDispatch + 'static, { - let shared_pruning_requirements = SharedPruningRequirements::default(); + let shared_pruning_requirements = SharedPruningRequirementsSource::default(); let task_manager = { let registry = config.prometheus_config.as_ref().map(|cfg| &cfg.registry); From 4ad7e182f7647ce7e4dcbac0c2e5ed5761268380 Mon Sep 17 00:00:00 2001 From: cheme Date: Mon, 17 Aug 2020 12:39:57 +0200 Subject: [PATCH 04/17] unneeded change --- client/consensus/epochs/src/lib.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/client/consensus/epochs/src/lib.rs b/client/consensus/epochs/src/lib.rs index 6bb981b2998b6..acb07dd668a3c 100644 --- a/client/consensus/epochs/src/lib.rs +++ b/client/consensus/epochs/src/lib.rs @@ -18,7 +18,7 @@ pub mod migration; -use std::{sync::Arc, ops::{Add, Sub}, collections::BTreeMap, borrow::{Borrow, BorrowMut}}; +use std::{sync::Arc, ops::Add, collections::BTreeMap, borrow::{Borrow, BorrowMut}}; use parking_lot::Mutex; use codec::{Encode, Decode}; use fork_tree::ForkTree; @@ -322,7 +322,7 @@ impl Default for EpochChanges where impl EpochChanges where Hash: PartialEq + Ord + AsRef<[u8]> + AsMut<[u8]> + Copy, - Number: Ord + One + Zero + Sub + Add + Copy, + Number: Ord + One + Zero + Add + Copy, { /// Create a new epoch change. pub fn new() -> Self { From c3b6ef039c0491ac872101ea418cb3086bc0cc68 Mon Sep 17 00:00:00 2001 From: cheme Date: Mon, 17 Aug 2020 14:44:17 +0200 Subject: [PATCH 05/17] switch to array of range to single range. --- client/api/src/backend.rs | 4 ++-- client/db/src/light.rs | 45 +++++++++++++++++---------------------- 2 files changed, 22 insertions(+), 27 deletions(-) diff --git a/client/api/src/backend.rs b/client/api/src/backend.rs index 84fdf562a1914..1268942f53ada 100644 --- a/client/api/src/backend.rs +++ b/client/api/src/backend.rs @@ -649,8 +649,8 @@ impl Default for ComponentPruningRequirements { pub enum PruningLimit { /// Ignore. None, - /// The component require at least this number - /// of unpruned elements. + /// Only block with a number lower than N + /// can be pruned. Some(N), /// We lock all pruning. Locked, diff --git a/client/db/src/light.rs b/client/db/src/light.rs index 5fc86f2e86d6e..25c33d30a9730 100644 --- a/client/db/src/light.rs +++ b/client/db/src/light.rs @@ -21,7 +21,6 @@ use std::{sync::Arc, collections::HashMap}; use std::convert::TryInto; use parking_lot::RwLock; -use std::collections::VecDeque; use sc_client_api::{ cht, backend::{AuxStore, NewBlockState}, UsageInfo, @@ -70,7 +69,7 @@ pub struct LightStorage { cache: Arc>, header_metadata_cache: Arc>, shared_pruning_requirements: SharedPruningRequirementsSource, - pending_cht_pruning: RwLock, NumberFor)>>, + pending_cht_pruning: RwLock, NumberFor)>>, #[cfg(not(target_os = "unknown"))] io_stats: FrozenForDuration, @@ -429,9 +428,15 @@ impl LightStorage { ) -> ClientResult<()> { // Note that we do not handle the error case for in memory data // as with other field of light storage. - self.pending_cht_pruning.write().push_back((start, end)); - - let encoded_pending = self.pending_cht_pruning.read().encode(); + let mut pending_cht_pruning = self.pending_cht_pruning.write(); + let start = pending_cht_pruning.as_ref() + .map(|(old_start, old_end)| { + debug_assert!(old_end == &(start - One::one())); + old_start.clone() + }) + .unwrap_or(start); + *pending_cht_pruning = Some((start, end)); + let encoded_pending = pending_cht_pruning.encode(); transaction.set_from_vec(columns::AUX, AUX_PENDING_CHT_RANGES, encoded_pending); Ok(()) @@ -450,37 +455,27 @@ impl LightStorage { &self, transaction: &mut Transaction, ) -> ClientResult<()> { - let mut changed = false; - let mut to_prune = VecDeque::new(); - if !self.pending_cht_pruning.read().is_empty() { + let mut to_prune = None; + if !self.pending_cht_pruning.read().is_none() { match self.shared_pruning_requirements.finalized_headers_needed() { PruningLimit::Locked => (), PruningLimit::None => { - to_prune = std::mem::replace(&mut *self.pending_cht_pruning.write(), VecDeque::new()); + to_prune = std::mem::replace(&mut *self.pending_cht_pruning.write(), None); }, PruningLimit::Some(limit) => { - let mut shared = self.pending_cht_pruning.write(); - while let Some(range) = shared.pop_front() { - if limit > range.1 { - to_prune.push_back(range); - } else if limit > range.0 { - to_prune.push_back((range.0, limit - One::one())); - shared.push_front((limit, range.1)); - break; - } else { - shared.push_front((range.0, range.1)); - break; + if let Some((start, end)) = self.pending_cht_pruning.write().as_mut() { + if &limit > end { + to_prune = self.pending_cht_pruning.write().take(); + } else if &limit > start { + let start = std::mem::replace(start, limit); + to_prune = Some((start, limit - One::one())); } } }, } } - for range in to_prune { - changed = true; + if let Some(range) = to_prune { self.prune_range_unchecked(range.0, range.1, transaction)?; - } - - if changed { let encoded_pending = self.pending_cht_pruning.read().encode(); transaction.set_from_vec(columns::AUX, AUX_PENDING_CHT_RANGES, encoded_pending); } From bf4027404b38ad96926718336d0b82880990ba0c Mon Sep 17 00:00:00 2001 From: cheme Date: Mon, 17 Aug 2020 15:12:17 +0200 Subject: [PATCH 06/17] Only log error on pruning failure in babe. --- client/consensus/babe/src/lib.rs | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/client/consensus/babe/src/lib.rs b/client/consensus/babe/src/lib.rs index f042c16ad9596..4f0ee855b8d34 100644 --- a/client/consensus/babe/src/lib.rs +++ b/client/consensus/babe/src/lib.rs @@ -111,7 +111,7 @@ use futures::channel::mpsc::{channel, Sender, Receiver}; use retain_mut::RetainMut; use futures::prelude::*; -use log::{debug, info, log, trace, warn}; +use log::{debug, info, log, trace, warn, error}; use prometheus_endpoint::Registry; use sc_consensus_slots::{ SlotWorker, SlotInfo, SlotCompatible, StorageChanges, CheckedHeader, check_equivocation, @@ -1400,7 +1400,10 @@ fn prune_finalized( &info.finalized_hash, info.finalized_number, finalized_slot, - ).map_err(|e| ConsensusError::ClientImport(format!("{:?}", e)))?; + ) + .map_err(|e| error!(target: "babe", "Failed pruning epoch changes tree {:?}", e)) + // Continue runing node on pruning failure. + .unwrap_or(()); Ok(()) } From 0a8f0bf3a750fcd8befe6e915c736bb07285cb2b Mon Sep 17 00:00:00 2001 From: cheme Date: Mon, 17 Aug 2020 15:13:16 +0200 Subject: [PATCH 07/17] Revert "Only log error on pruning failure in babe." This reverts commit bf4027404b38ad96926718336d0b82880990ba0c. Allowing failure on a PR that try to avoid failure seems awkward. --- client/consensus/babe/src/lib.rs | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/client/consensus/babe/src/lib.rs b/client/consensus/babe/src/lib.rs index 4f0ee855b8d34..f042c16ad9596 100644 --- a/client/consensus/babe/src/lib.rs +++ b/client/consensus/babe/src/lib.rs @@ -111,7 +111,7 @@ use futures::channel::mpsc::{channel, Sender, Receiver}; use retain_mut::RetainMut; use futures::prelude::*; -use log::{debug, info, log, trace, warn, error}; +use log::{debug, info, log, trace, warn}; use prometheus_endpoint::Registry; use sc_consensus_slots::{ SlotWorker, SlotInfo, SlotCompatible, StorageChanges, CheckedHeader, check_equivocation, @@ -1400,10 +1400,7 @@ fn prune_finalized( &info.finalized_hash, info.finalized_number, finalized_slot, - ) - .map_err(|e| error!(target: "babe", "Failed pruning epoch changes tree {:?}", e)) - // Continue runing node on pruning failure. - .unwrap_or(()); + ).map_err(|e| ConsensusError::ClientImport(format!("{:?}", e)))?; Ok(()) } From fcd14109a00ef9815f28c6de3a5e24681d0b5e86 Mon Sep 17 00:00:00 2001 From: cheme Date: Tue, 18 Aug 2020 15:33:26 +0200 Subject: [PATCH 08/17] alternative implementation, HeaderLookup trait need to be redefine to something less technical. --- bin/node-template/node/src/service.rs | 10 +- bin/node/cli/src/service.rs | 8 +- client/api/src/backend.rs | 133 ++----------- client/api/src/in_mem.rs | 21 ++ client/api/src/light.rs | 5 +- client/consensus/babe/src/lib.rs | 85 ++++---- client/consensus/epochs/src/lib.rs | 37 ++++ client/db/src/lib.rs | 24 +++ client/db/src/light.rs | 251 +++++++----------------- client/db/src/utils.rs | 13 +- client/light/src/backend.rs | 12 +- client/service/src/builder.rs | 12 +- client/service/src/client/client.rs | 35 ++++ client/service/test/src/client/light.rs | 12 ++ utils/fork-tree/src/lib.rs | 188 +++++++++++++++++- 15 files changed, 486 insertions(+), 360 deletions(-) diff --git a/bin/node-template/node/src/service.rs b/bin/node-template/node/src/service.rs index 4cb7cf25e9dfb..0b69c6fc86d63 100644 --- a/bin/node-template/node/src/service.rs +++ b/bin/node-template/node/src/service.rs @@ -2,7 +2,7 @@ use std::sync::Arc; use std::time::Duration; -use sc_client_api::{ExecutorProvider, RemoteBackend}; +use sc_client_api::{ExecutorProvider, RemoteBackend, SharedPruningRequirements}; use node_template_runtime::{self, opaque::Block, RuntimeApi}; use sc_service::{error::Error as ServiceError, Configuration, TaskManager}; use sp_inherents::InherentDataProviders; @@ -221,8 +221,12 @@ pub fn new_full(config: Configuration) -> Result { /// Builds a new service for a light client. pub fn new_light(config: Configuration) -> Result { - let (client, backend, keystore, mut task_manager, on_demand, _shared_pruning_requirements) = - sc_service::new_light_parts::(&config)?; + let shared_pruning_requirements = SharedPruningRequirements::default(); + let (client, backend, keystore, mut task_manager, on_demand) = + sc_service::new_light_parts::( + &config, + &shared_pruning_requirements, + )?; let transaction_pool = Arc::new(sc_transaction_pool::BasicPool::new_light( config.transaction_pool.clone(), diff --git a/bin/node/cli/src/service.rs b/bin/node/cli/src/service.rs index c57187f2ebb85..777f3174684c8 100644 --- a/bin/node/cli/src/service.rs +++ b/bin/node/cli/src/service.rs @@ -33,7 +33,7 @@ use sp_inherents::InherentDataProviders; use sc_network::{Event, NetworkService}; use sp_runtime::traits::Block as BlockT; use futures::prelude::*; -use sc_client_api::{ExecutorProvider, RemoteBackend}; +use sc_client_api::{ExecutorProvider, RemoteBackend, SharedPruningRequirements}; use sp_core::traits::BareCryptoStorePtr; use node_executor::Executor; @@ -346,8 +346,10 @@ pub fn new_light_base(config: Configuration) -> Result<( Arc::Hash>>, Arc>> ), ServiceError> { - let (client, backend, keystore, mut task_manager, on_demand, shared_pruning_requirements) = - sc_service::new_light_parts::(&config)?; + let mut shared_pruning_requirements = SharedPruningRequirements::default(); + sc_consensus_babe::light_pruning_requirements(&mut shared_pruning_requirements); + let (client, backend, keystore, mut task_manager, on_demand) = + sc_service::new_light_parts::(&config, &shared_pruning_requirements)?; let select_chain = sc_consensus::LongestChain::new(backend.clone()); diff --git a/client/api/src/backend.rs b/client/api/src/backend.rs index 1268942f53ada..b308bd2f9fb44 100644 --- a/client/api/src/backend.rs +++ b/client/api/src/backend.rs @@ -270,6 +270,15 @@ pub trait AuxStore { fn get_aux(&self, key: &[u8]) -> sp_blockchain::Result>>; } +/// Provides access to an header lookup database. +pub trait HeaderLookupStore { + /// Is the header lookup define for a given number. + fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result; + + /// Cleanup header mapping. + fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()>; +} + /// An `Iterator` that iterates keys in a given block under a prefix. pub struct KeyIterator<'a, State, Block> { state: State, @@ -395,7 +404,7 @@ pub trait StorageProvider> { /// /// The same applies for live `BlockImportOperation`s: while an import operation building on a /// parent `P` is alive, the state for `P` should not be pruned. -pub trait Backend: AuxStore + Send + Sync { +pub trait Backend: AuxStore + HeaderLookupStore + Send + Sync { /// Associated block insertion operation type. type BlockImportOperation: BlockImportOperation; /// Associated blockchain backend type. @@ -537,121 +546,11 @@ pub fn changes_tries_state_at_block<'a, Block: BlockT>( } } -/// Shared pruning requirement read only access and instantiation. -#[derive(Clone)] -pub struct SharedPruningRequirementsSource { - shared: Arc>, usize)>>, -} - /// Pruning requirement to share between multiple client component. -/// -/// This allows pruning related synchronisation. For instance in light -/// client we need to synchronize header pruning from CHT (every N blocks) -/// with the pruning from consensus used (babe for instance require that -/// its epoch headers are not pruned which works as long as the slot length -/// is less than the CHT pruning window. -/// Each compenent register at a given index (call are done by this order). -/// -/// Note that this struct could be split in two different struct (provider without -/// component and Component), depending on future usage of this shared info. -#[derive(Clone)] -pub struct SharedPruningRequirements { - shared: Arc>, usize)>>, - component: usize, -} - -impl Default for SharedPruningRequirementsSource { - fn default() -> Self { - SharedPruningRequirementsSource { - shared: Arc::new(RwLock::new((Vec::new(), 0))), - } - } -} - -fn shared_finalized_headers_needed( - shared: &Arc>, usize)>>, -) -> PruningLimit> { - let mut result = PruningLimit::None; - for req in shared.read().0.iter() { - match &req.requires_finalized_header_up_to { - PruningLimit::Locked => return PruningLimit::Locked, - PruningLimit::None => (), - PruningLimit::Some(n) => { - if let PruningLimit::Some(p) = result { - if &p < n { - result = PruningLimit::Some(p); - continue; - } - } - result = PruningLimit::Some(n.clone()); - }, - } - } - result -} - -impl SharedPruningRequirementsSource { - /// Add a following requirement to apply. - /// Returns the shared requirement to use from this component. - pub fn next_instance( - &self, - ) -> SharedPruningRequirements { - let req = ComponentPruningRequirements::default(); - let index = { - let mut shared = self.shared.write(); - let index = shared.0.len(); - shared.0.push(req); - shared.1 += 1; - index - }; - SharedPruningRequirements { - shared: self.shared.clone(), - component: index, - } - } - - /// Resolve a finalized block headers to keep. - pub fn finalized_headers_needed(&self) -> PruningLimit> { - shared_finalized_headers_needed(&self.shared) - } -} - -impl SharedPruningRequirements { - /// Resolve a finalized block headers to keep. - pub fn finalized_headers_needed(&self) -> PruningLimit> { - shared_finalized_headers_needed(&self.shared) - } - - /// Set new requirement on finalized headers. - /// Returns false if we do not have a handle on the shared requirement. - pub fn set_finalized_headers_needed(&self, limit: PruningLimit>) { - let index = self.component; - let mut shared = self.shared.write(); - shared.0[index].requires_finalized_header_up_to = limit; - } -} - -/// Individual pruning requirement for any substrate component. -struct ComponentPruningRequirements { - requires_finalized_header_up_to: PruningLimit>, -} - -impl Default for ComponentPruningRequirements { - fn default() -> Self { - ComponentPruningRequirements { - requires_finalized_header_up_to: PruningLimit::None, - } - } -} - -/// Define a block number limit to apply. -#[derive(Eq, PartialEq)] -pub enum PruningLimit { - /// Ignore. - None, - /// Only block with a number lower than N - /// can be pruned. - Some(N), - /// We lock all pruning. - Locked, +#[derive(Clone, Default)] +pub struct SharedPruningRequirements { + /// Some component like babe will need to resolve canonical branch, + /// for it we keep some key lookup mapping when pruning CHT in light + /// client. + pub need_mapping_for_light_pruning: bool, } diff --git a/client/api/src/in_mem.rs b/client/api/src/in_mem.rs index 306c3c2b2f10c..62bc5329805e2 100644 --- a/client/api/src/in_mem.rs +++ b/client/api/src/in_mem.rs @@ -417,6 +417,17 @@ impl backend::AuxStore for Blockchain { } } +impl backend::HeaderLookupStore for Blockchain { + fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result { + Ok(self.storage.read().hashes.get(number).is_some()) + } + + fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + self.storage.write().hashes.remove(number); + Ok(()) + } +} + impl light::Storage for Blockchain where Block::Hash: From<[u8; 32]>, @@ -618,6 +629,16 @@ impl backend::AuxStore for Backend where Block::Hash: Ord } } +impl backend::HeaderLookupStore for Backend { + fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result { + self.blockchain.is_lookup_define_for_number(number) + } + + fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + self.blockchain.clean_up_number_lookup(number) + } +} + impl backend::Backend for Backend where Block::Hash: Ord { type BlockImportOperation = BlockImportOperation; type Blockchain = Blockchain; diff --git a/client/api/src/light.rs b/client/api/src/light.rs index b359c1149eea6..a9c84b8ab14b3 100644 --- a/client/api/src/light.rs +++ b/client/api/src/light.rs @@ -32,7 +32,7 @@ use sp_blockchain::{ HeaderMetadata, well_known_cache_keys, HeaderBackend, Cache as BlockchainCache, Error as ClientError, Result as ClientResult, }; -use crate::{backend::{AuxStore, NewBlockState}, UsageInfo}; +use crate::{backend::{AuxStore, NewBlockState, HeaderLookupStore}, UsageInfo}; /// Remote call request. #[derive(Clone, Debug, PartialEq, Eq, Hash)] @@ -232,7 +232,8 @@ pub trait FetchChecker: Send + Sync { /// Light client blockchain storage. -pub trait Storage: AuxStore + HeaderBackend + HeaderMetadata { +pub trait Storage: AuxStore + HeaderBackend + HeaderMetadata + + HeaderLookupStore { /// Store new header. Should refuse to revert any finalized blocks. /// /// Takes new authorities, the leaf state of the new block, and diff --git a/client/consensus/babe/src/lib.rs b/client/consensus/babe/src/lib.rs index f042c16ad9596..f2ad0f4e764bd 100644 --- a/client/consensus/babe/src/lib.rs +++ b/client/consensus/babe/src/lib.rs @@ -86,7 +86,7 @@ use sp_core::{crypto::Public, traits::BareCryptoStore}; use sp_application_crypto::AppKey; use sp_runtime::{ generic::{BlockId, OpaqueDigestItemId}, Justification, - traits::{Block as BlockT, Header, DigestItemFor, Zero, One}, + traits::{Block as BlockT, Header, DigestItemFor, Zero}, }; use sp_api::{ProvideRuntimeApi, NumberFor}; use sc_keystore::KeyStorePtr; @@ -104,7 +104,7 @@ use sp_consensus::import_queue::{Verifier, BasicQueue, DefaultImportQueue, Cache use sc_client_api::{ backend::AuxStore, BlockchainEvents, ProvideUncles, - SharedPruningRequirements, SharedPruningRequirementsSource, PruningLimit, + SharedPruningRequirements, HeaderLookupStore, }; use sp_block_builder::BlockBuilder as BlockBuilderApi; use futures::channel::mpsc::{channel, Sender, Receiver}; @@ -1064,8 +1064,7 @@ pub struct BabeBlockImport { client: Arc, epoch_changes: SharedEpochChanges, config: Config, - previous_needed_height: Option>, - shared_pruning_requirements: Option>, + light_pruning: bool, } impl Clone for BabeBlockImport { @@ -1075,8 +1074,7 @@ impl Clone for BabeBlockImport BabeBlockImport { epoch_changes: SharedEpochChanges, block_import: I, config: Config, - shared_pruning_requirements: Option<&SharedPruningRequirementsSource>, + shared_pruning_requirements: Option<&SharedPruningRequirements>, ) -> Self { - let shared_pruning_requirements = shared_pruning_requirements.map(|shared| { - let req = shared.next_instance(); - req.set_finalized_headers_needed(PruningLimit::Locked); - req - }); + let light_pruning = shared_pruning_requirements.is_some(); BabeBlockImport { client, inner: block_import, epoch_changes, config, - previous_needed_height: Some(Zero::zero()), - shared_pruning_requirements, + light_pruning, } } } @@ -1110,6 +1103,7 @@ impl BlockImport for BabeBlockImport> + Send + Sync, Inner::Error: Into, Client: HeaderBackend + HeaderMetadata + + HeaderLookupStore + AuxStore + ProvideRuntimeApi + ProvideCache + Send + Sync, Client::Api: BabeApi + ApiExt, { @@ -1274,6 +1268,7 @@ impl BlockImport for BabeBlockImport BlockImport for BabeBlockImport BlockImport for BabeBlockImport( client: Arc, epoch_changes: &mut EpochChangesFor, + prune_light: bool, ) -> Result<(), ConsensusError> where Block: BlockT, - Client: HeaderBackend + HeaderMetadata, + Client: HeaderBackend + + HeaderMetadata + + HeaderLookupStore, { let info = client.info(); @@ -1395,16 +1377,39 @@ fn prune_finalized( .slot_number() }; - epoch_changes.prune_finalized( - descendent_query(&*client), - &info.finalized_hash, - info.finalized_number, - finalized_slot, - ).map_err(|e| ConsensusError::ClientImport(format!("{:?}", e)))?; + if prune_light { + let is_canonical = |_hash: &Block::Hash, number: &NumberFor, finalized_number: &NumberFor| { + Ok(number <= finalized_number && client.is_lookup_define_for_number(number)?) + }; + let clean_up = |number: &NumberFor| { + client.clean_up_number_lookup(number)?; + Ok(()) + }; + + epoch_changes.prune_finalized_light::( + &info.finalized_hash, + info.finalized_number, + finalized_slot, + is_canonical, + clean_up, + ).map_err(|e| ConsensusError::ClientImport(format!("{:?}", e)))?; + } else { + epoch_changes.prune_finalized( + descendent_query(&*client), + &info.finalized_hash, + info.finalized_number, + finalized_slot, + ).map_err(|e| ConsensusError::ClientImport(format!("{:?}", e)))?; + } Ok(()) } +/// Initialize babe pruning requirements. +pub fn light_pruning_requirements(shared_pruning_requirements: &mut SharedPruningRequirements) { + shared_pruning_requirements.need_mapping_for_light_pruning = true; +} + /// Produce a BABE block-import object to be used later on in the construction of /// an import-queue. /// @@ -1414,9 +1419,10 @@ pub fn block_import( config: Config, wrapped_block_import: I, client: Arc, - shared_pruning_requirements: Option<&SharedPruningRequirementsSource>, + shared_pruning_requirements: Option<&SharedPruningRequirements>, ) -> ClientResult<(BabeBlockImport, BabeLink)> where - Client: AuxStore + HeaderBackend + HeaderMetadata, + Client: AuxStore + HeaderBackend + HeaderMetadata + + HeaderLookupStore, { let epoch_changes = aux_schema::load_epoch_changes::(&*client, &config)?; let link = BabeLink { @@ -1431,6 +1437,7 @@ pub fn block_import( prune_finalized( client.clone(), &mut epoch_changes.lock(), + shared_pruning_requirements.is_some(), )?; let import = BabeBlockImport::new( diff --git a/client/consensus/epochs/src/lib.rs b/client/consensus/epochs/src/lib.rs index acb07dd668a3c..3d1295b8433f2 100644 --- a/client/consensus/epochs/src/lib.rs +++ b/client/consensus/epochs/src/lib.rs @@ -421,6 +421,43 @@ impl EpochChanges where Ok(()) } + /// Prune out finalized epochs, except for the ancestor of the finalized + /// block. The given slot should be the slot number at which the finalized + /// block was authored. + pub fn prune_finalized_light( + &mut self, + hash: &Hash, + number: Number, + slot: E::SlotNumber, + is_canonical: F, + clean_up: C, + ) -> Result<(), fork_tree::Error> + where Error: std::error::Error, + F: Fn(&Hash, &Number, &Number) -> Result, + C: Fn(&Number) -> Result<(), Error>, + { + + let predicate = |epoch: &PersistedEpochHeader| match *epoch { + PersistedEpochHeader::Genesis(_, ref epoch_1) => + slot >= epoch_1.end_slot, + PersistedEpochHeader::Regular(ref epoch_n) => + slot >= epoch_n.end_slot, + }; + let removed = self.inner.prune_light( + hash, + &number, + &is_canonical, + &predicate, + &clean_up, + )?; + + for (hash, number, _) in removed { + self.epochs.remove(&(hash, number)); + } + + Ok(()) + } + /// Get a reference to an epoch with given identifier. pub fn epoch(&self, id: &EpochIdentifier) -> Option<&E> { self.epochs.get(&(id.hash, id.number)) diff --git a/client/db/src/lib.rs b/client/db/src/lib.rs index d854c80bf3535..19db0750e37f7 100644 --- a/client/db/src/lib.rs +++ b/client/db/src/lib.rs @@ -1367,6 +1367,30 @@ impl sc_client_api::backend::AuxStore for Backend where Block: Blo } } + +impl sc_client_api::backend::HeaderLookupStore for Backend + where Block: BlockT +{ + fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result { + utils::block_id_to_lookup_key::(&*self.storage.db, columns::KEY_LOOKUP, BlockId::Number(number.clone())) + .map(|r| r.is_some()) + } + + fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + // TODO pass transaction as parameter? + let mut transaction = Transaction::new(); + utils::remove_number_to_key_mapping( + &mut transaction, + columns::KEY_LOOKUP, + number.clone(), + )?; + + self.storage.db.commit(transaction)?; + + Ok(()) + } +} + impl sc_client_api::backend::Backend for Backend { type BlockImportOperation = BlockImportOperation; type Blockchain = BlockchainDb; diff --git a/client/db/src/light.rs b/client/db/src/light.rs index 25c33d30a9730..0835e246a2238 100644 --- a/client/db/src/light.rs +++ b/client/db/src/light.rs @@ -23,11 +23,11 @@ use std::convert::TryInto; use parking_lot::RwLock; use sc_client_api::{ - cht, backend::{AuxStore, NewBlockState}, UsageInfo, + cht, backend::{AuxStore, NewBlockState, HeaderLookupStore}, UsageInfo, blockchain::{ BlockStatus, Cache as BlockchainCache, Info as BlockchainInfo, }, - Storage, SharedPruningRequirementsSource, PruningLimit, + Storage, SharedPruningRequirements, }; use sp_blockchain::{ CachedHeaderMetadata, HeaderMetadata, HeaderMetadataCache, @@ -42,7 +42,7 @@ use sp_runtime::traits::{Block as BlockT, Header as HeaderT, Zero, One, NumberFo use crate::cache::{DbCacheSync, DbCache, ComplexBlockId, EntryType as CacheEntryType}; use crate::utils::{self, meta_keys, DatabaseType, Meta, read_db, block_id_to_lookup_key, read_meta}; use crate::{DatabaseSettings, FrozenForDuration, DbHash}; -use log::{warn, debug}; +use log::{trace, warn, debug}; pub(crate) mod columns { pub const META: u32 = crate::utils::COLUMN_META; @@ -58,9 +58,6 @@ const HEADER_CHT_PREFIX: u8 = 0; /// Prefix for changes tries roots CHT. const CHANGES_TRIE_CHT_PREFIX: u8 = 1; -/// Aux key to store pending pruning CHT ranges. -const AUX_PENDING_CHT_RANGES: &'static[u8] = b"pending_cht_pruning_ranges"; - /// Light blockchain storage. Stores most recent headers + CHTs for older headers. /// Locks order: meta, cache. pub struct LightStorage { @@ -68,8 +65,7 @@ pub struct LightStorage { meta: RwLock, Block::Hash>>, cache: Arc>, header_metadata_cache: Arc>, - shared_pruning_requirements: SharedPruningRequirementsSource, - pending_cht_pruning: RwLock, NumberFor)>>, + keep_canonical_mapping: bool, #[cfg(not(target_os = "unknown"))] io_stats: FrozenForDuration, @@ -79,7 +75,7 @@ impl LightStorage { /// Create new storage with given settings. pub fn new( config: DatabaseSettings, - shared_pruning_requirements: SharedPruningRequirementsSource, + shared_pruning_requirements: &SharedPruningRequirements, ) -> ClientResult { let db = crate::utils::open_database::(&config, DatabaseType::Light)?; Self::from_kvdb(db as Arc<_>, shared_pruning_requirements) @@ -89,13 +85,13 @@ impl LightStorage { #[cfg(any(test, feature = "test-helpers"))] pub fn new_test() -> Self { let db = Arc::new(sp_database::MemDb::default()); - let shared_pruning_requirements = SharedPruningRequirementsSource::default(); - Self::from_kvdb(db as Arc<_>, shared_pruning_requirements).expect("failed to create test-db") + let shared_pruning_requirements = SharedPruningRequirements::default(); + Self::from_kvdb(db as Arc<_>, &shared_pruning_requirements).expect("failed to create test-db") } fn from_kvdb( db: Arc>, - shared_pruning_requirements: SharedPruningRequirementsSource, + shared_pruning_requirements: &SharedPruningRequirements, ) -> ClientResult { let meta = read_meta::(&*db, columns::HEADER)?; let header_metadata_cache = Arc::new(HeaderMetadataCache::default()); @@ -109,9 +105,7 @@ impl LightStorage { ComplexBlockId::new(meta.finalized_hash, meta.finalized_number), ); - let pending_cht_pruning_ranges = db.get(columns::AUX, AUX_PENDING_CHT_RANGES) - .and_then(|encoded| Decode::decode(&mut encoded.as_slice()).ok()) - .unwrap_or_default(); + let keep_canonical_mapping = shared_pruning_requirements.need_mapping_for_light_pruning; Ok(LightStorage { db, meta: RwLock::new(meta), @@ -119,8 +113,7 @@ impl LightStorage { header_metadata_cache, #[cfg(not(target_os = "unknown"))] io_stats: FrozenForDuration::new(std::time::Duration::from_secs(1)), - shared_pruning_requirements, - pending_cht_pruning: RwLock::new(pending_cht_pruning_ranges), + keep_canonical_mapping, }) } @@ -270,7 +263,7 @@ impl LightStorage { utils::remove_number_to_key_mapping( transaction, columns::KEY_LOOKUP, - retracted.number + retracted.number, )?; } @@ -352,133 +345,37 @@ impl LightStorage { ); } + // prune headers that are replaced with CHT + let mut prune_block = new_cht_start; let new_cht_end = cht::end_number(cht::size(), new_cht_number); - self.try_prune_pending(transaction)?; - self.prune_range(new_cht_start, new_cht_end, transaction)?; - } else { - self.try_prune_pending(transaction)?; - } - - Ok(()) - } - - /// Prune headers that are replaced with CHT. - /// Note that end index is inclusive (as start). - fn prune_range( - &self, - start: NumberFor, - end: NumberFor, - transaction: &mut Transaction, - ) -> ClientResult<()> { - if let Some((start, end)) = match self.shared_pruning_requirements.finalized_headers_needed() { - PruningLimit::None => { - Some((start, end)) - }, - PruningLimit::Some(limit) => { - if limit > end { - Some((start, end)) - } else if limit > start { - self.add_prune_range_pending(limit, end, transaction)?; - Some((start, limit - One::one())) - } else { - self.add_prune_range_pending(start, end, transaction)?; - None - } - }, - PruningLimit::Locked => { - self.add_prune_range_pending(start, end, transaction)?; - None - }, - } { - self.prune_range_unchecked(start, end, transaction)?; - } - Ok(()) - } - - fn prune_range_unchecked( - &self, - start: NumberFor, - end: NumberFor, - transaction: &mut Transaction, - ) -> ClientResult<()> { - let mut prune_block = start; - while prune_block <= end { - if let Some(hash) = self.hash(prune_block)? { - let lookup_key = block_id_to_lookup_key::(&*self.db, columns::KEY_LOOKUP, BlockId::Number(prune_block))? - .expect("retrieved hash for `prune_block` right above. therefore retrieving lookup key must succeed. q.e.d."); - utils::remove_key_mappings( - transaction, - columns::KEY_LOOKUP, - prune_block, - hash - )?; - transaction.remove(columns::HEADER, &lookup_key); - } - prune_block += One::one(); - } - Ok(()) - } - - /// Stack some pending pruning range. - fn add_prune_range_pending( - &self, - start: NumberFor, - end: NumberFor, - transaction: &mut Transaction, - ) -> ClientResult<()> { - // Note that we do not handle the error case for in memory data - // as with other field of light storage. - let mut pending_cht_pruning = self.pending_cht_pruning.write(); - let start = pending_cht_pruning.as_ref() - .map(|(old_start, old_end)| { - debug_assert!(old_end == &(start - One::one())); - old_start.clone() - }) - .unwrap_or(start); - *pending_cht_pruning = Some((start, end)); - let encoded_pending = pending_cht_pruning.encode(); - transaction.set_from_vec(columns::AUX, AUX_PENDING_CHT_RANGES, encoded_pending); - - Ok(()) - } - - #[cfg(test)] - fn try_prune_pending_test(&self) -> ClientResult<()> { - let mut transaction = Transaction::new(); - self.try_prune_pending(&mut transaction)?; - self.db.commit(transaction)?; - Ok(()) - } - - /// Try pruning pending. - fn try_prune_pending( - &self, - transaction: &mut Transaction, - ) -> ClientResult<()> { - let mut to_prune = None; - if !self.pending_cht_pruning.read().is_none() { - match self.shared_pruning_requirements.finalized_headers_needed() { - PruningLimit::Locked => (), - PruningLimit::None => { - to_prune = std::mem::replace(&mut *self.pending_cht_pruning.write(), None); - }, - PruningLimit::Some(limit) => { - if let Some((start, end)) = self.pending_cht_pruning.write().as_mut() { - if &limit > end { - to_prune = self.pending_cht_pruning.write().take(); - } else if &limit > start { - let start = std::mem::replace(start, limit); - to_prune = Some((start, limit - One::one())); - } + trace!(target: "db", "Replacing blocks [{}..{}] with CHT#{}", + new_cht_start, new_cht_end, new_cht_number); + + while prune_block <= new_cht_end { + if let Some(hash) = self.hash(prune_block)? { + let lookup_key = block_id_to_lookup_key::(&*self.db, columns::KEY_LOOKUP, BlockId::Number(prune_block))? + .expect("retrieved hash for `prune_block` right above. therefore retrieving lookup key must succeed. q.e.d."); + + if self.keep_canonical_mapping { + utils::remove_hash_to_key_mapping( + transaction, + columns::KEY_LOOKUP, + hash + )?; + + } else { + utils::remove_key_mappings( + transaction, + columns::KEY_LOOKUP, + prune_block, + hash + )?; } - }, + transaction.remove(columns::HEADER, &lookup_key); + } + prune_block += One::one(); } } - if let Some(range) = to_prune { - self.prune_range_unchecked(range.0, range.1, transaction)?; - let encoded_pending = self.pending_cht_pruning.read().encode(); - transaction.set_from_vec(columns::AUX, AUX_PENDING_CHT_RANGES, encoded_pending); - } Ok(()) } @@ -535,6 +432,31 @@ impl AuxStore for LightStorage } } +impl HeaderLookupStore for LightStorage + where Block: BlockT, +{ + + fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result { + utils::block_id_to_lookup_key::(&*self.db, columns::KEY_LOOKUP, BlockId::Number(number.clone())) + .map(|r| r.is_some()) + } + + fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + // TODO pass transaction as parameter? + let mut transaction = Transaction::new(); + utils::remove_number_to_key_mapping( + &mut transaction, + columns::KEY_LOOKUP, + number.clone(), + )?; + + self.db.commit(transaction)?; + + Ok(()) + } +} + + impl Storage for LightStorage where Block: BlockT, { @@ -744,7 +666,7 @@ fn cht_key>(cht_type: u8, block: N) -> ClientResult<[u8; 5]> { #[cfg(test)] pub(crate) mod tests { - use sc_client_api::{cht, SharedPruningRequirements}; + use sc_client_api::cht; use sp_core::ChangesTrieConfiguration; use sp_runtime::generic::{DigestItem, ChangesTrieSignal}; use sp_runtime::testing::{H256 as Hash, Header, Block as RawBlock, ExtrinsicWrapper}; @@ -861,7 +783,7 @@ pub(crate) mod tests { #[test] fn import_header_works() { let raw_db = Arc::new(sp_database::MemDb::default()); - let db = LightStorage::from_kvdb(raw_db.clone(), Default::default()).unwrap(); + let db = LightStorage::from_kvdb(raw_db.clone(), &Default::default()).unwrap(); let genesis_hash = insert_block(&db, HashMap::new(), || default_header(&Default::default(), 0)); assert_eq!(raw_db.count(columns::HEADER), 1); @@ -874,15 +796,11 @@ pub(crate) mod tests { #[test] fn finalized_ancient_headers_are_replaced_with_cht() { - fn insert_headers Header>( - header_producer: F, - limit: PruningLimit, - ) -> (Arc>, LightStorage, SharedPruningRequirements) { + fn insert_headers Header>(header_producer: F) -> + (Arc>, LightStorage) + { let raw_db = Arc::new(sp_database::MemDb::default()); - let shared_source = SharedPruningRequirementsSource::default(); - let req = shared_source.next_instance(); - req.set_finalized_headers_needed(limit); - let db = LightStorage::from_kvdb(raw_db.clone(), shared_source).unwrap(); + let db = LightStorage::from_kvdb(raw_db.clone(), &Default::default()).unwrap(); let cht_size: u64 = cht::size(); let ucht_size: usize = cht_size as _; @@ -923,11 +841,11 @@ pub(crate) mod tests { db.finalize_header(BlockId::Number(i as _)).unwrap(); } db.finalize_header(BlockId::Hash(prev_hash)).unwrap(); - (raw_db, db, req) + (raw_db, db) } // when headers are created without changes tries roots - let (raw_db, db, _pruning_limit) = insert_headers(default_header, PruningLimit::None); + let (raw_db, db) = insert_headers(default_header); let cht_size: u64 = cht::size(); assert_eq!(raw_db.count(columns::HEADER), (1 + cht_size + 1) as usize); assert_eq!(raw_db.count(columns::KEY_LOOKUP), (2 * (1 + cht_size + 1)) as usize); @@ -938,28 +856,8 @@ pub(crate) mod tests { assert!(db.changes_trie_cht_root(cht_size, cht_size / 2).is_err()); assert!(db.changes_trie_cht_root(cht_size, cht_size + cht_size / 2).unwrap().is_none()); - // try locked pruning - let (raw_db, db, pruning_limit) = insert_headers(default_header, PruningLimit::Locked); - let cht_size: u64 = cht::size(); - assert_eq!(raw_db.count(columns::HEADER), (2 + 2 * cht_size) as usize); - assert_eq!(raw_db.count(columns::KEY_LOOKUP), (2 * (2 + 2 * cht_size)) as usize); - assert!((0..cht_size as _).all(|i| db.header(BlockId::Number(1 + i)).unwrap().is_some())); - - pruning_limit.set_finalized_headers_needed(PruningLimit::Some(1000)); - db.try_prune_pending_test().unwrap(); - assert_eq!(raw_db.count(columns::HEADER), (2 + 2 * cht_size) as usize - 999); - assert_eq!(raw_db.count(columns::KEY_LOOKUP), (2 * (2 + 2 * cht_size - 999)) as usize); - assert!((1..999 as _).all(|i| db.header(BlockId::Number(1 + i)).unwrap().is_none())); - assert!((999..cht_size as _).all(|i| db.header(BlockId::Number(1 + i)).unwrap().is_some())); - - pruning_limit.set_finalized_headers_needed(PruningLimit::None); - db.try_prune_pending_test().unwrap(); - assert_eq!(raw_db.count(columns::HEADER), (1 + cht_size + 1) as usize); - assert_eq!(raw_db.count(columns::KEY_LOOKUP), (2 * (1 + cht_size + 1)) as usize); - assert!((0..cht_size as _).all(|i| db.header(BlockId::Number(1 + i)).unwrap().is_none())); - // when headers are created with changes tries roots - let (raw_db, db, _pruning_limit) = insert_headers(header_with_changes_trie, PruningLimit::None); + let (raw_db, db) = insert_headers(header_with_changes_trie); assert_eq!(raw_db.count(columns::HEADER), (1 + cht_size + 1) as usize); assert_eq!(raw_db.count(columns::CHT), 2); assert!((0..cht_size as _).all(|i| db.header(BlockId::Number(1 + i)).unwrap().is_none())); @@ -1285,7 +1183,7 @@ pub(crate) mod tests { assert_eq!(db.header(BlockId::Hash(hash0)).unwrap().unwrap().hash(), hash0); let db = db.db; - let db = LightStorage::from_kvdb(db, Default::default()).unwrap(); + let db = LightStorage::from_kvdb(db, &Default::default()).unwrap(); assert_eq!(db.info().best_hash, hash0); assert_eq!(db.header(BlockId::Hash::(hash0)).unwrap().unwrap().hash(), hash0); } @@ -1346,8 +1244,7 @@ pub(crate) mod tests { }; // restart && check that after restart value is read from the cache - let db = LightStorage::::from_kvdb(storage as Arc<_>, Default::default()) - .expect("failed to create test-db"); + let db = LightStorage::::from_kvdb(storage as Arc<_>, &Default::default()).expect("failed to create test-db"); assert_eq!( db.cache().get_at(b"test", &BlockId::Number(0)).unwrap(), Some(((0, genesis_hash.unwrap()), None, vec![42])), diff --git a/client/db/src/utils.rs b/client/db/src/utils.rs index 168ab9bbb71f6..3fd4e340de11b 100644 --- a/client/db/src/utils.rs +++ b/client/db/src/utils.rs @@ -144,6 +144,17 @@ pub fn remove_number_to_key_mapping>( Ok(()) } +/// Delete hash to hash mapping in DB transaction. +pub fn remove_hash_to_key_mapping>( + transaction: &mut Transaction, + key_lookup_col: u32, + hash: H, +) -> sp_blockchain::Result<()> { + transaction.remove(key_lookup_col, hash.as_ref()); + Ok(()) +} + + /// Remove key mappings. pub fn remove_key_mappings, H: AsRef<[u8]>>( transaction: &mut Transaction, @@ -152,7 +163,7 @@ pub fn remove_key_mappings, H: AsRef<[u8]>>( hash: H, ) -> sp_blockchain::Result<()> { remove_number_to_key_mapping(transaction, key_lookup_col, number)?; - transaction.remove(key_lookup_col, hash.as_ref()); + remove_hash_to_key_mapping(transaction, key_lookup_col, hash)?; Ok(()) } diff --git a/client/light/src/backend.rs b/client/light/src/backend.rs index be7953e528bd8..e980a620f49c0 100644 --- a/client/light/src/backend.rs +++ b/client/light/src/backend.rs @@ -38,7 +38,7 @@ use sp_blockchain::{Error as ClientError, Result as ClientResult}; use sc_client_api::{ backend::{ AuxStore, Backend as ClientBackend, BlockImportOperation, RemoteBackend, NewBlockState, - PrunableStateChangesTrieStorage, + PrunableStateChangesTrieStorage, HeaderLookupStore, }, blockchain::{ HeaderBackend as BlockchainHeaderBackend, well_known_cache_keys, @@ -113,6 +113,16 @@ impl AuxStore for Backend { } } +impl, H: Hasher> HeaderLookupStore for Backend { + fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result { + self.blockchain.storage().is_lookup_define_for_number(number) + } + + fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + self.blockchain.storage().clean_up_number_lookup(number) + } +} + impl ClientBackend for Backend> where Block: BlockT, diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index e64867e0ba7d0..766708027c094 100644 --- a/client/service/src/builder.rs +++ b/client/service/src/builder.rs @@ -26,7 +26,7 @@ use crate::{ }; use sc_client_api::{ light::RemoteBlockchain, ForkBlocks, BadBlocks, UsageProvider, ExecutorProvider, - SharedPruningRequirementsSource, + SharedPruningRequirements, }; use sp_utils::mpsc::{tracing_unbounded, TracingUnboundedSender}; use sc_chain_spec::get_extension; @@ -168,7 +168,6 @@ type TLightParts = ( Arc>, TaskManager, Arc>, - SharedPruningRequirementsSource, ); /// Light client backend type with a specific hash type. @@ -268,13 +267,12 @@ pub fn new_full_parts( /// Create the initial parts of a light node. pub fn new_light_parts( - config: &Configuration + config: &Configuration, + shared_pruning_requirements: &SharedPruningRequirements, ) -> Result, Error> where TBl: BlockT, TExecDisp: NativeExecutionDispatch + 'static, { - let shared_pruning_requirements = SharedPruningRequirementsSource::default(); - let task_manager = { let registry = config.prometheus_config.as_ref().map(|cfg| &cfg.registry); TaskManager::new(config.task_executor.clone(), registry)? @@ -302,7 +300,7 @@ pub fn new_light_parts( pruning: config.pruning.clone(), source: config.database.clone(), }; - sc_client_db::light::LightStorage::new(db_settings, shared_pruning_requirements.clone())? + sc_client_db::light::LightStorage::new(db_settings, shared_pruning_requirements)? }; let light_blockchain = sc_light::new_light_blockchain(db_storage); let fetch_checker = Arc::new( @@ -322,7 +320,7 @@ pub fn new_light_parts( config.prometheus_config.as_ref().map(|config| config.registry.clone()), )?); - Ok((client, backend, keystore, task_manager, on_demand, shared_pruning_requirements)) + Ok((client, backend, keystore, task_manager, on_demand)) } /// Create an instance of db-backed client. diff --git a/client/service/src/client/client.rs b/client/service/src/client/client.rs index d0859f4ee0392..5fe12323d0cb8 100644 --- a/client/service/src/client/client.rs +++ b/client/service/src/client/client.rs @@ -1988,6 +1988,41 @@ impl backend::AuxStore for &Client } } +impl sc_client_api::HeaderLookupStore for Client + where + B: backend::Backend, + E: CallExecutor, + Block: BlockT, + Self: ProvideRuntimeApi, + >::Api: CoreApi, +{ + fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result { + sc_client_api::HeaderLookupStore::is_lookup_define_for_number(&*self.backend, number) + } + + fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + sc_client_api::HeaderLookupStore::clean_up_number_lookup(&*self.backend, number) + } +} + +impl sc_client_api::HeaderLookupStore for &Client + where + B: backend::Backend, + E: CallExecutor, + Block: BlockT, + Client: ProvideRuntimeApi, + as ProvideRuntimeApi>::Api: CoreApi, +{ + fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result { + (**self).is_lookup_define_for_number(number) + } + + fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + (**self).clean_up_number_lookup(number) + } +} + + impl sp_consensus::block_validation::Chain for Client where BE: backend::Backend, E: CallExecutor, diff --git a/client/service/test/src/client/light.rs b/client/service/test/src/client/light.rs index ffc84ad47b8f3..c328f6415bd76 100644 --- a/client/service/test/src/client/light.rs +++ b/client/service/test/src/client/light.rs @@ -46,6 +46,7 @@ use sc_client_api::{ AuxStore, Storage, CallExecutor, cht, ExecutionStrategy, StorageProof, BlockImportOperation, RemoteCallRequest, StorageProvider, ChangesProof, RemoteBodyRequest, RemoteReadRequest, RemoteChangesRequest, FetchChecker, RemoteReadChildRequest, RemoteHeaderRequest, BlockBackend, + HeaderLookupStore, }; use sp_externalities::Extensions; use sc_block_builder::BlockBuilderProvider; @@ -141,6 +142,17 @@ impl AuxStore for DummyStorage { } } +impl HeaderLookupStore for DummyStorage { + fn is_lookup_define_for_number(&self, _number: &NumberFor) -> sp_blockchain::Result { + Err(ClientError::Backend("Test error".into())) + } + + fn clean_up_number_lookup(&self, _number: &NumberFor) -> sp_blockchain::Result<()> { + Err(ClientError::Backend("Test error".into())) + } +} + + impl Storage for DummyStorage { fn import_header( &self, diff --git a/utils/fork-tree/src/lib.rs b/utils/fork-tree/src/lib.rs index 33533182c58a4..fdbd40c244323 100644 --- a/utils/fork-tree/src/lib.rs +++ b/utils/fork-tree/src/lib.rs @@ -167,18 +167,89 @@ impl ForkTree where Ok(RemovedIterator { stack: removed }) } - /// Return the lowest root number. - pub fn lowest_node_number(&self) -> Option { - let mut result = None; - for root in self.roots.iter() { - let replace = result.as_ref().map(|lowest| &root.number < lowest) - .unwrap_or(true); - if replace { - result = Some(root.number.clone()); + /// Prune the tree, removing all non-canonical nodes. + /// We remove roots that are non-canonical. + /// The given function `is_canonical` should return `true` if the given block + /// (hash and number input, and last finalized nubmer) is canonical, this only apply for block less than + /// the finalized one. + /// It apply recursively a clean up call back on branch. + /// + /// Returns all pruned node data. + pub fn prune_light( + &mut self, + hash: &H, + number: &N, + is_canonical: &F, + predicate: &P, + clean_up: &C, + ) -> Result, Error> + where E: std::error::Error, + F: Fn(&H, &N, &N) -> Result, + P: Fn(&V) -> bool, + C: Fn(&N) -> Result<(), E>, + { + let new_root_index = self.find_node_index_where_light( + hash, + number, + is_canonical, + predicate, + )?; + + let removed = if let Some(mut root_index) = new_root_index { + let mut old_roots = std::mem::take(&mut self.roots); + + let mut root = None; + let mut cur_children = Some(&mut old_roots); + + while let Some(cur_index) = root_index.pop() { + if let Some(children) = cur_children.take() { + if root_index.is_empty() { + root = Some(children.remove(cur_index)); + } else { + cur_children = Some(&mut children[cur_index].children); + } + } } + + let mut root = root + .expect("find_node_index_where will return array with at least one index; \ + this results in at least one item in removed; qed"); + + let mut removed = old_roots; + + // we found the deepest ancestor of the finalized block, so we prune + // out any children that don't include the finalized block. + let root_children = std::mem::take(&mut root.children); + let mut is_first = true; + + for child in root_children { + if is_first && + (child.number == *number && child.hash == *hash || + child.number < *number && is_canonical(&child.hash, &child.number, number).unwrap_or(false)) + { + root.children.push(child); + // assuming that the tree is well formed only one child should pass this requirement + // due to ancestry restrictions (i.e. they must be different forks). + is_first = false; + } else { + removed.push(child); + } + } + + self.roots = vec![root]; + + removed + } else { + Vec::new() + }; + + self.rebalance(); + + for node in removed.iter() { + node.clean_up(clean_up)?; } - - result + + Ok(RemovedIterator { stack: removed }) } } @@ -372,6 +443,33 @@ impl ForkTree where Ok(None) } + /// Same as [`find_node_index_where`](ForkTree::find_node_where), but with a predicate + /// on canonical nature of block. + pub fn find_node_index_where_light( + &self, + hash: &H, + number: &N, + is_canonical: &F, + predicate: &P, + ) -> Result>, Error> where + E: std::error::Error, + F: Fn(&H, &N, &N) -> Result, + P: Fn(&V) -> bool, + { + // search for node starting from all roots + for (index, root) in self.roots.iter().enumerate() { + let node = root.find_node_index_where_light(hash, number, is_canonical, predicate)?; + + // found the node, early exit + if let FindOutcome::Found(mut node) = node { + node.push(index); + return Ok(Some(node)); + } + } + + Ok(None) + } + /// Finalize a root in the tree and return it, return `None` in case no root /// with the given hash exists. All other roots are pruned, and the children /// of the finalized node become the new roots. @@ -813,6 +911,64 @@ mod node_implementation { Ok(FindOutcome::Failure(is_descendent_of)) } + /// Same as [`find_node_index_where`](Node::find_node_where), but with + /// a direct check on block being canonical. + pub fn find_node_index_where_light( + &self, + hash: &H, + number: &N, + is_canonical: &F, + predicate: &P, + ) -> Result>, Error> + where E: std::error::Error, + F: Fn(&H, &N, &N) -> Result, + P: Fn(&V) -> bool, + { + // stop searching this branch + if *number < self.number { + return Ok(FindOutcome::Failure(false)); + } + + let mut known_canonical = false; + + // continue depth-first search through all children + for (i, node) in self.children.iter().enumerate() { + // found node, early exit + match node.find_node_index_where_light( + hash, + number, + is_canonical, + predicate, + )? { + FindOutcome::Abort => return Ok(FindOutcome::Abort), + FindOutcome::Found(mut x) => { + x.push(i); + return Ok(FindOutcome::Found(x)) + }, + FindOutcome::Failure(true) => { + // if the block was a descendent of this child, + // then it cannot be a descendent of any others, + // so we don't search them. + known_canonical = true; + break; + }, + FindOutcome::Failure(false) => {}, + } + } + + let is_canonical = known_canonical || is_canonical(&self.hash, &self.number, number)?; + if is_canonical { + // if the predicate passes we return the node + if predicate(&self.data) { + return Ok(FindOutcome::Found(Vec::new())); + } + } + + // otherwise, tell our ancestor that we failed, and whether + // the block was a descendent. + Ok(FindOutcome::Failure(is_canonical)) + } + /// Find a node in the tree that is the deepest ancestor of the given /// block hash which also passes the given predicate, backtracking /// when the predicate fails. @@ -876,6 +1032,18 @@ mod node_implementation { }, } } + + /// Clean up the key mapping associated with removed nodes. + pub fn clean_up(&self, do_clean: &F) -> Result<(), Error> + where E: std::error::Error, + F: Fn(&N) -> Result<(), E>, + { + do_clean(&self.number)?; + for child in self.children.iter() { + child.clean_up(do_clean)?; + } + Ok(()) + } } } From e9255a24901ec040cb79ecdadc2e05a7b2d956a2 Mon Sep 17 00:00:00 2001 From: cheme Date: Tue, 18 Aug 2020 16:04:16 +0200 Subject: [PATCH 09/17] temporary code --- Cargo.lock | 1 + bin/node/testing/Cargo.toml | 2 +- client/db/Cargo.toml | 5 +- client/db/src/upgrade.rs | 135 +++++++++++++++++++++++++++++++++++- 4 files changed, 138 insertions(+), 5 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ec5af8aca4ecf..ddfb4d55ac606 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6257,6 +6257,7 @@ dependencies = [ "parity-util-mem 0.7.0", "parking_lot 0.10.2", "quickcheck", + "rocksdb", "sc-client-api", "sc-executor", "sc-state-db", diff --git a/bin/node/testing/Cargo.toml b/bin/node/testing/Cargo.toml index a61a344cceed1..cf3aab7f2ed14 100644 --- a/bin/node/testing/Cargo.toml +++ b/bin/node/testing/Cargo.toml @@ -15,7 +15,7 @@ targets = ["x86_64-unknown-linux-gnu"] [dependencies] pallet-balances = { version = "2.0.0-rc5", path = "../../../frame/balances" } sc-service = { version = "0.8.0-rc5", features = ["test-helpers", "db"], path = "../../../client/service" } -sc-client-db = { version = "0.8.0-rc5", path = "../../../client/db/", features = ["kvdb-rocksdb", "parity-db"] } +sc-client-db = { version = "0.8.0-rc5", path = "../../../client/db/", features = ["parity-db"] } sc-client-api = { version = "2.0.0-rc5", path = "../../../client/api/" } codec = { package = "parity-scale-codec", version = "1.3.4" } pallet-contracts = { version = "2.0.0-rc5", path = "../../../frame/contracts" } diff --git a/client/db/Cargo.toml b/client/db/Cargo.toml index 50e14fcaae602..cb7f0ebf27932 100644 --- a/client/db/Cargo.toml +++ b/client/db/Cargo.toml @@ -15,7 +15,7 @@ targets = ["x86_64-unknown-linux-gnu"] parking_lot = "0.10.0" log = "0.4.8" kvdb = "0.7.0" -kvdb-rocksdb = { version = "0.9", optional = true } +kvdb-rocksdb = { version = "0.9" } kvdb-memorydb = "0.7.0" linked-hash-map = "0.5.2" hash-db = "0.15.2" @@ -37,6 +37,7 @@ sp-database = { version = "2.0.0-rc5", path = "../../primitives/database" } parity-db = { version = "0.1.2", optional = true } prometheus-endpoint = { package = "substrate-prometheus-endpoint", version = "0.8.0-rc5", path = "../../utils/prometheus" } +rocksdb = { version = "0.14", features = ["snappy"], default-features = false } # TODO remove, just to get collection size [dev-dependencies] sp-keyring = { version = "2.0.0-rc5", path = "../../primitives/keyring" } substrate-test-runtime-client = { version = "2.0.0-rc5", path = "../../test-utils/runtime/client" } @@ -48,6 +49,6 @@ tempfile = "3" [features] default = [] test-helpers = [] -with-kvdb-rocksdb = ["kvdb-rocksdb"] +with-kvdb-rocksdb = [] with-parity-db = ["parity-db"] with-subdb = [] diff --git a/client/db/src/upgrade.rs b/client/db/src/upgrade.rs index 95592d071f777..0f237dfc49b07 100644 --- a/client/db/src/upgrade.rs +++ b/client/db/src/upgrade.rs @@ -19,9 +19,18 @@ use std::fs; use std::io::{Read, Write, ErrorKind}; use std::path::{Path, PathBuf}; +use log::warn; +use std::marker::PhantomData; +use std::time::{Duration, Instant}; -use sp_runtime::traits::Block as BlockT; +use sp_runtime::traits::{Block as BlockT, HashFor, NumberFor, Header as HeaderT}; use crate::utils::DatabaseType; +use crate::{StateDb, PruningMode, StateMetaDb}; +use codec::{Decode, Encode}; +use kvdb::KeyValueDB; +use std::io; + +use std::sync::Arc; /// Version file name. const VERSION_FILE_NAME: &'static str = "db_version"; @@ -30,13 +39,16 @@ const VERSION_FILE_NAME: &'static str = "db_version"; const CURRENT_VERSION: u32 = 1; /// Upgrade database to current version. -pub fn upgrade_db(db_path: &Path, _db_type: DatabaseType) -> sp_blockchain::Result<()> { +pub fn upgrade_db(db_path: &Path, db_type: DatabaseType) -> sp_blockchain::Result<()> { let is_empty = db_path.read_dir().map_or(true, |mut d| d.next().is_none()); if !is_empty { let db_version = current_version(db_path)?; match db_version { 0 => Err(sp_blockchain::Error::Backend(format!("Unsupported database version: {}", db_version)))?, 1 => (), + 42 => { + test_thing::(db_path, db_type)?; + }, _ => Err(sp_blockchain::Error::Backend(format!("Future database version: {}", db_version)))?, } } @@ -44,6 +56,125 @@ pub fn upgrade_db(db_path: &Path, _db_type: DatabaseType) -> sp_b update_version(db_path) } +/// Database backed tree management for a rocksdb database. +pub struct RocksdbStorage(Arc); + +impl RocksdbStorage { +/* pub fn resolve_collection(c: &'static [u8]) -> Option { + if c.len() != 4 { + return None; + } + let index = Self::resolve_collection_inner(c); + if index < crate::utils::NUM_COLUMNS { + return Some(index); + } + None + } + const fn resolve_collection_inner(c: &'static [u8]) -> u32 { + let mut buf = [0u8; 4]; + buf[0] = c[0]; + buf[1] = c[1]; + buf[2] = c[2]; + buf[3] = c[3]; + u32::from_le_bytes(buf) + } + + fn write(&mut self, c: &'static [u8], k: &[u8], v: &[u8]) { + Self::resolve_collection(c).map(|c| { + let mut tx = self.0.transaction(); + tx.put(c, k, v); + self.0.write(tx) + .expect("Unsupported serialize error") + }); + } + + fn remove(&mut self, c: &'static [u8], k: &[u8]) { + Self::resolve_collection(c).map(|c| { + let mut tx = self.0.transaction(); + tx.delete(c, k); + self.0.write(tx) + .expect("Unsupported serialize error") + }); + } + + fn clear(&mut self, c: &'static [u8]) { + Self::resolve_collection(c).map(|c| { + let mut tx = self.0.transaction(); + tx.delete_prefix(c, &[]); + self.0.write(tx) + .expect("Unsupported serialize error") + }); + } + + fn read(&self, c: &'static [u8], k: &[u8]) -> Option> { + Self::resolve_collection(c).and_then(|c| { + self.0.get(c, k) + .expect("Unsupported readdb error") + }) + } + fn iter<'a>(&'a self, c: &'static [u8]) -> SerializeDBIter<'a> { + let iter = Self::resolve_collection(c).map(|c| { + self.0.iter(c).map(|(k, v)| (Vec::::from(k), Vec::::from(v))) + }).into_iter().flat_map(|i| i); + + Box::new(iter) + } +*/ + fn iter<'a>(&'a self, c: u32) -> SerializeDBIter<'a> { + let iter = self.0.iter(c).map(|(k, v)| (Vec::::from(k), Vec::::from(v))); + + Box::new(iter) + } + +/* + fn contains_collection(collection: &'static [u8]) -> bool { + Self::resolve_collection(collection).is_some() + }*/ +} +type SerializeDBIter<'a> = Box, Vec)> + 'a>; + +/// Hacky migrate to trigger action on db. +/// Here drop historied state content. +fn test_thing(db_path: &Path, db_type: DatabaseType) -> sp_blockchain::Result<()> { + + let mut db_config = kvdb_rocksdb::DatabaseConfig::with_columns(crate::utils::NUM_COLUMNS); + { + let option = rocksdb::Options::default(); + let cfs = rocksdb::DB::list_cf(&option, db_path).unwrap(); + let db = rocksdb::DB::open_cf(&option, db_path, cfs.clone()).unwrap(); + let mut i = 0; + for cf in cfs { + + if let Some(col) = db.cf_handle(&cf) { + println!("{:?}, {:?}", cf, db.property_int_value_cf(col, "rocksdb.estimate-table-readers-mem")); + println!("{:?}, {:?}", cf, db.property_int_value_cf(col, "rocksdb.size-all-mem-tables")); + println!("{:?}, {:?}", cf, db.property_int_value_cf(col, "rocksdb.cur-size-all-mem-tables")); + } + } + + } + + let mut i = 0; + { + let path = db_path.to_str() + .ok_or_else(|| sp_blockchain::Error::Backend("Invalid database path".into()))?; + while i < 6 { + let db_read = Arc::new(kvdb_rocksdb::Database::open(&db_config, path) + .map_err(|err| sp_blockchain::Error::Backend(format!("{}", err)))?); + + let db_r = RocksdbStorage(db_read.clone()); + let iter_kv = db_r.iter(i); + println!("{:?}, nb_iter {:?}", i, iter_kv.count()); + i += 1; + + } + } + + + + Ok(()) +} + /// Reads current database version from the file at given path. /// If the file does not exist returns 0. From 34712ab50e39c635a8f047d36f976273451ed49a Mon Sep 17 00:00:00 2001 From: cheme Date: Tue, 18 Aug 2020 16:04:47 +0200 Subject: [PATCH 10/17] Revert "temporary code" This reverts commit e9255a24901ec040cb79ecdadc2e05a7b2d956a2. --- Cargo.lock | 1 - bin/node/testing/Cargo.toml | 2 +- client/db/Cargo.toml | 5 +- client/db/src/upgrade.rs | 135 +----------------------------------- 4 files changed, 5 insertions(+), 138 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ddfb4d55ac606..ec5af8aca4ecf 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6257,7 +6257,6 @@ dependencies = [ "parity-util-mem 0.7.0", "parking_lot 0.10.2", "quickcheck", - "rocksdb", "sc-client-api", "sc-executor", "sc-state-db", diff --git a/bin/node/testing/Cargo.toml b/bin/node/testing/Cargo.toml index cf3aab7f2ed14..a61a344cceed1 100644 --- a/bin/node/testing/Cargo.toml +++ b/bin/node/testing/Cargo.toml @@ -15,7 +15,7 @@ targets = ["x86_64-unknown-linux-gnu"] [dependencies] pallet-balances = { version = "2.0.0-rc5", path = "../../../frame/balances" } sc-service = { version = "0.8.0-rc5", features = ["test-helpers", "db"], path = "../../../client/service" } -sc-client-db = { version = "0.8.0-rc5", path = "../../../client/db/", features = ["parity-db"] } +sc-client-db = { version = "0.8.0-rc5", path = "../../../client/db/", features = ["kvdb-rocksdb", "parity-db"] } sc-client-api = { version = "2.0.0-rc5", path = "../../../client/api/" } codec = { package = "parity-scale-codec", version = "1.3.4" } pallet-contracts = { version = "2.0.0-rc5", path = "../../../frame/contracts" } diff --git a/client/db/Cargo.toml b/client/db/Cargo.toml index cb7f0ebf27932..50e14fcaae602 100644 --- a/client/db/Cargo.toml +++ b/client/db/Cargo.toml @@ -15,7 +15,7 @@ targets = ["x86_64-unknown-linux-gnu"] parking_lot = "0.10.0" log = "0.4.8" kvdb = "0.7.0" -kvdb-rocksdb = { version = "0.9" } +kvdb-rocksdb = { version = "0.9", optional = true } kvdb-memorydb = "0.7.0" linked-hash-map = "0.5.2" hash-db = "0.15.2" @@ -37,7 +37,6 @@ sp-database = { version = "2.0.0-rc5", path = "../../primitives/database" } parity-db = { version = "0.1.2", optional = true } prometheus-endpoint = { package = "substrate-prometheus-endpoint", version = "0.8.0-rc5", path = "../../utils/prometheus" } -rocksdb = { version = "0.14", features = ["snappy"], default-features = false } # TODO remove, just to get collection size [dev-dependencies] sp-keyring = { version = "2.0.0-rc5", path = "../../primitives/keyring" } substrate-test-runtime-client = { version = "2.0.0-rc5", path = "../../test-utils/runtime/client" } @@ -49,6 +48,6 @@ tempfile = "3" [features] default = [] test-helpers = [] -with-kvdb-rocksdb = [] +with-kvdb-rocksdb = ["kvdb-rocksdb"] with-parity-db = ["parity-db"] with-subdb = [] diff --git a/client/db/src/upgrade.rs b/client/db/src/upgrade.rs index 0f237dfc49b07..95592d071f777 100644 --- a/client/db/src/upgrade.rs +++ b/client/db/src/upgrade.rs @@ -19,18 +19,9 @@ use std::fs; use std::io::{Read, Write, ErrorKind}; use std::path::{Path, PathBuf}; -use log::warn; -use std::marker::PhantomData; -use std::time::{Duration, Instant}; -use sp_runtime::traits::{Block as BlockT, HashFor, NumberFor, Header as HeaderT}; +use sp_runtime::traits::Block as BlockT; use crate::utils::DatabaseType; -use crate::{StateDb, PruningMode, StateMetaDb}; -use codec::{Decode, Encode}; -use kvdb::KeyValueDB; -use std::io; - -use std::sync::Arc; /// Version file name. const VERSION_FILE_NAME: &'static str = "db_version"; @@ -39,16 +30,13 @@ const VERSION_FILE_NAME: &'static str = "db_version"; const CURRENT_VERSION: u32 = 1; /// Upgrade database to current version. -pub fn upgrade_db(db_path: &Path, db_type: DatabaseType) -> sp_blockchain::Result<()> { +pub fn upgrade_db(db_path: &Path, _db_type: DatabaseType) -> sp_blockchain::Result<()> { let is_empty = db_path.read_dir().map_or(true, |mut d| d.next().is_none()); if !is_empty { let db_version = current_version(db_path)?; match db_version { 0 => Err(sp_blockchain::Error::Backend(format!("Unsupported database version: {}", db_version)))?, 1 => (), - 42 => { - test_thing::(db_path, db_type)?; - }, _ => Err(sp_blockchain::Error::Backend(format!("Future database version: {}", db_version)))?, } } @@ -56,125 +44,6 @@ pub fn upgrade_db(db_path: &Path, db_type: DatabaseType) -> sp_bl update_version(db_path) } -/// Database backed tree management for a rocksdb database. -pub struct RocksdbStorage(Arc); - -impl RocksdbStorage { -/* pub fn resolve_collection(c: &'static [u8]) -> Option { - if c.len() != 4 { - return None; - } - let index = Self::resolve_collection_inner(c); - if index < crate::utils::NUM_COLUMNS { - return Some(index); - } - None - } - const fn resolve_collection_inner(c: &'static [u8]) -> u32 { - let mut buf = [0u8; 4]; - buf[0] = c[0]; - buf[1] = c[1]; - buf[2] = c[2]; - buf[3] = c[3]; - u32::from_le_bytes(buf) - } - - fn write(&mut self, c: &'static [u8], k: &[u8], v: &[u8]) { - Self::resolve_collection(c).map(|c| { - let mut tx = self.0.transaction(); - tx.put(c, k, v); - self.0.write(tx) - .expect("Unsupported serialize error") - }); - } - - fn remove(&mut self, c: &'static [u8], k: &[u8]) { - Self::resolve_collection(c).map(|c| { - let mut tx = self.0.transaction(); - tx.delete(c, k); - self.0.write(tx) - .expect("Unsupported serialize error") - }); - } - - fn clear(&mut self, c: &'static [u8]) { - Self::resolve_collection(c).map(|c| { - let mut tx = self.0.transaction(); - tx.delete_prefix(c, &[]); - self.0.write(tx) - .expect("Unsupported serialize error") - }); - } - - fn read(&self, c: &'static [u8], k: &[u8]) -> Option> { - Self::resolve_collection(c).and_then(|c| { - self.0.get(c, k) - .expect("Unsupported readdb error") - }) - } - fn iter<'a>(&'a self, c: &'static [u8]) -> SerializeDBIter<'a> { - let iter = Self::resolve_collection(c).map(|c| { - self.0.iter(c).map(|(k, v)| (Vec::::from(k), Vec::::from(v))) - }).into_iter().flat_map(|i| i); - - Box::new(iter) - } -*/ - fn iter<'a>(&'a self, c: u32) -> SerializeDBIter<'a> { - let iter = self.0.iter(c).map(|(k, v)| (Vec::::from(k), Vec::::from(v))); - - Box::new(iter) - } - -/* - fn contains_collection(collection: &'static [u8]) -> bool { - Self::resolve_collection(collection).is_some() - }*/ -} -type SerializeDBIter<'a> = Box, Vec)> + 'a>; - -/// Hacky migrate to trigger action on db. -/// Here drop historied state content. -fn test_thing(db_path: &Path, db_type: DatabaseType) -> sp_blockchain::Result<()> { - - let mut db_config = kvdb_rocksdb::DatabaseConfig::with_columns(crate::utils::NUM_COLUMNS); - { - let option = rocksdb::Options::default(); - let cfs = rocksdb::DB::list_cf(&option, db_path).unwrap(); - let db = rocksdb::DB::open_cf(&option, db_path, cfs.clone()).unwrap(); - let mut i = 0; - for cf in cfs { - - if let Some(col) = db.cf_handle(&cf) { - println!("{:?}, {:?}", cf, db.property_int_value_cf(col, "rocksdb.estimate-table-readers-mem")); - println!("{:?}, {:?}", cf, db.property_int_value_cf(col, "rocksdb.size-all-mem-tables")); - println!("{:?}, {:?}", cf, db.property_int_value_cf(col, "rocksdb.cur-size-all-mem-tables")); - } - } - - } - - let mut i = 0; - { - let path = db_path.to_str() - .ok_or_else(|| sp_blockchain::Error::Backend("Invalid database path".into()))?; - while i < 6 { - let db_read = Arc::new(kvdb_rocksdb::Database::open(&db_config, path) - .map_err(|err| sp_blockchain::Error::Backend(format!("{}", err)))?); - - let db_r = RocksdbStorage(db_read.clone()); - let iter_kv = db_r.iter(i); - println!("{:?}, nb_iter {:?}", i, iter_kv.count()); - i += 1; - - } - } - - - - Ok(()) -} - /// Reads current database version from the file at given path. /// If the file does not exist returns 0. From cbad9b0e1ca19e4b1d06f39b2babcca06c0df8bb Mon Sep 17 00:00:00 2001 From: cheme Date: Wed, 19 Aug 2020 11:44:51 +0200 Subject: [PATCH 11/17] Add missing check --- client/api/src/backend.rs | 2 +- client/api/src/in_mem.rs | 8 ++++---- client/consensus/babe/src/lib.rs | 4 ++-- client/db/src/lib.rs | 14 +++++++++++--- client/db/src/light.rs | 10 +++++++--- client/db/src/utils.rs | 9 +++++++++ client/light/src/backend.rs | 4 ++-- client/service/src/client/client.rs | 8 ++++---- client/service/test/src/client/light.rs | 8 ++++++-- 9 files changed, 46 insertions(+), 21 deletions(-) diff --git a/client/api/src/backend.rs b/client/api/src/backend.rs index b308bd2f9fb44..85b7330482451 100644 --- a/client/api/src/backend.rs +++ b/client/api/src/backend.rs @@ -273,7 +273,7 @@ pub trait AuxStore { /// Provides access to an header lookup database. pub trait HeaderLookupStore { /// Is the header lookup define for a given number. - fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result; + fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result; /// Cleanup header mapping. fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()>; diff --git a/client/api/src/in_mem.rs b/client/api/src/in_mem.rs index 62bc5329805e2..e6d865faa0cf3 100644 --- a/client/api/src/in_mem.rs +++ b/client/api/src/in_mem.rs @@ -418,8 +418,8 @@ impl backend::AuxStore for Blockchain { } impl backend::HeaderLookupStore for Blockchain { - fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result { - Ok(self.storage.read().hashes.get(number).is_some()) + fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + Ok(self.storage.read().hashes.get(number) == Some(hash)) } fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { @@ -630,8 +630,8 @@ impl backend::AuxStore for Backend where Block::Hash: Ord } impl backend::HeaderLookupStore for Backend { - fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result { - self.blockchain.is_lookup_define_for_number(number) + fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + self.blockchain.is_lookup_define_for_number(number, hash) } fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { diff --git a/client/consensus/babe/src/lib.rs b/client/consensus/babe/src/lib.rs index efade7f6987c8..6d4bf5a7d15c6 100644 --- a/client/consensus/babe/src/lib.rs +++ b/client/consensus/babe/src/lib.rs @@ -1392,8 +1392,8 @@ fn prune_finalized( }; if prune_light { - let is_canonical = |_hash: &Block::Hash, number: &NumberFor, finalized_number: &NumberFor| { - Ok(number <= finalized_number && client.is_lookup_define_for_number(number)?) + let is_canonical = |hash: &Block::Hash, number: &NumberFor, finalized_number: &NumberFor| { + Ok(number <= finalized_number && client.is_lookup_define_for_number(number, hash)?) }; let clean_up = |number: &NumberFor| { client.clean_up_number_lookup(number)?; diff --git a/client/db/src/lib.rs b/client/db/src/lib.rs index 19db0750e37f7..2161125e627b0 100644 --- a/client/db/src/lib.rs +++ b/client/db/src/lib.rs @@ -1371,9 +1371,17 @@ impl sc_client_api::backend::AuxStore for Backend where Block: Blo impl sc_client_api::backend::HeaderLookupStore for Backend where Block: BlockT { - fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result { - utils::block_id_to_lookup_key::(&*self.storage.db, columns::KEY_LOOKUP, BlockId::Number(number.clone())) - .map(|r| r.is_some()) + fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + let lookup_key = utils::block_id_to_lookup_key::( + &*self.storage.db, + columns::KEY_LOOKUP, + BlockId::Number(number.clone()), + )?; + Ok(if let Some(lookup_key) = lookup_key { + utils::lookup_key_to_hash(lookup_key.as_ref())? == hash.as_ref() + } else { + false + }) } fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { diff --git a/client/db/src/light.rs b/client/db/src/light.rs index 0835e246a2238..051a6471fb921 100644 --- a/client/db/src/light.rs +++ b/client/db/src/light.rs @@ -436,9 +436,13 @@ impl HeaderLookupStore for LightStorage where Block: BlockT, { - fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result { - utils::block_id_to_lookup_key::(&*self.db, columns::KEY_LOOKUP, BlockId::Number(number.clone())) - .map(|r| r.is_some()) + fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + let lookup_key = utils::block_id_to_lookup_key::(&*self.db, columns::KEY_LOOKUP, BlockId::Number(number.clone()))?; + Ok(if let Some(lookup_key) = lookup_key { + utils::lookup_key_to_hash(lookup_key.as_ref())? == hash.as_ref() + } else { + false + }) } fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { diff --git a/client/db/src/utils.rs b/client/db/src/utils.rs index 3fd4e340de11b..2ed0fc9a7c750 100644 --- a/client/db/src/utils.rs +++ b/client/db/src/utils.rs @@ -134,6 +134,15 @@ pub fn lookup_key_to_number(key: &[u8]) -> sp_blockchain::Result where | (key[3] as u32)).map(Into::into) } +/// Convert block lookup key to its hash. +pub fn lookup_key_to_hash(key: &[u8]) -> sp_blockchain::Result<&[u8]> +{ + if key.len() < 4 { + return Err(sp_blockchain::Error::Backend("Invalid block key".into())); + } + Ok(&key[4..]) +} + /// Delete number to hash mapping in DB transaction. pub fn remove_number_to_key_mapping>( transaction: &mut Transaction, diff --git a/client/light/src/backend.rs b/client/light/src/backend.rs index e980a620f49c0..0f182219f0e71 100644 --- a/client/light/src/backend.rs +++ b/client/light/src/backend.rs @@ -114,8 +114,8 @@ impl AuxStore for Backend { } impl, H: Hasher> HeaderLookupStore for Backend { - fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result { - self.blockchain.storage().is_lookup_define_for_number(number) + fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + self.blockchain.storage().is_lookup_define_for_number(number, hash) } fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { diff --git a/client/service/src/client/client.rs b/client/service/src/client/client.rs index 5fe12323d0cb8..b0ad6cc4e47bd 100644 --- a/client/service/src/client/client.rs +++ b/client/service/src/client/client.rs @@ -1996,8 +1996,8 @@ impl sc_client_api::HeaderLookupStore for Client, >::Api: CoreApi, { - fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result { - sc_client_api::HeaderLookupStore::is_lookup_define_for_number(&*self.backend, number) + fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + sc_client_api::HeaderLookupStore::is_lookup_define_for_number(&*self.backend, number, hash) } fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { @@ -2013,8 +2013,8 @@ impl sc_client_api::HeaderLookupStore for &Client: ProvideRuntimeApi, as ProvideRuntimeApi>::Api: CoreApi, { - fn is_lookup_define_for_number(&self, number: &NumberFor) -> sp_blockchain::Result { - (**self).is_lookup_define_for_number(number) + fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + (**self).is_lookup_define_for_number(number, hash) } fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { diff --git a/client/service/test/src/client/light.rs b/client/service/test/src/client/light.rs index c328f6415bd76..185935fad04ee 100644 --- a/client/service/test/src/client/light.rs +++ b/client/service/test/src/client/light.rs @@ -29,7 +29,7 @@ use sc_light::{ use std::sync::Arc; use sp_runtime::{ traits::{BlakeTwo256, HashFor, NumberFor}, - generic::BlockId, traits::{Block as _, Header as HeaderT}, Digest, + generic::BlockId, traits::{Block as BlockT, Header as HeaderT}, Digest, }; use std::collections::HashMap; use parking_lot::Mutex; @@ -143,7 +143,11 @@ impl AuxStore for DummyStorage { } impl HeaderLookupStore for DummyStorage { - fn is_lookup_define_for_number(&self, _number: &NumberFor) -> sp_blockchain::Result { + fn is_lookup_define_for_number( + &self, + _number: &NumberFor, + _hash: &::Hash, + ) -> sp_blockchain::Result { Err(ClientError::Backend("Test error".into())) } From 8701952d1c4db40a586a022c03dd6960099d20f3 Mon Sep 17 00:00:00 2001 From: cheme Date: Tue, 18 Aug 2020 16:04:16 +0200 Subject: [PATCH 12/17] temporary code --- Cargo.lock | 1 + bin/node/testing/Cargo.toml | 2 +- client/db/Cargo.toml | 5 +- client/db/src/upgrade.rs | 135 +++++++++++++++++++++++++++++++++++- 4 files changed, 138 insertions(+), 5 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ec5af8aca4ecf..ddfb4d55ac606 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6257,6 +6257,7 @@ dependencies = [ "parity-util-mem 0.7.0", "parking_lot 0.10.2", "quickcheck", + "rocksdb", "sc-client-api", "sc-executor", "sc-state-db", diff --git a/bin/node/testing/Cargo.toml b/bin/node/testing/Cargo.toml index a61a344cceed1..cf3aab7f2ed14 100644 --- a/bin/node/testing/Cargo.toml +++ b/bin/node/testing/Cargo.toml @@ -15,7 +15,7 @@ targets = ["x86_64-unknown-linux-gnu"] [dependencies] pallet-balances = { version = "2.0.0-rc5", path = "../../../frame/balances" } sc-service = { version = "0.8.0-rc5", features = ["test-helpers", "db"], path = "../../../client/service" } -sc-client-db = { version = "0.8.0-rc5", path = "../../../client/db/", features = ["kvdb-rocksdb", "parity-db"] } +sc-client-db = { version = "0.8.0-rc5", path = "../../../client/db/", features = ["parity-db"] } sc-client-api = { version = "2.0.0-rc5", path = "../../../client/api/" } codec = { package = "parity-scale-codec", version = "1.3.4" } pallet-contracts = { version = "2.0.0-rc5", path = "../../../frame/contracts" } diff --git a/client/db/Cargo.toml b/client/db/Cargo.toml index 50e14fcaae602..cb7f0ebf27932 100644 --- a/client/db/Cargo.toml +++ b/client/db/Cargo.toml @@ -15,7 +15,7 @@ targets = ["x86_64-unknown-linux-gnu"] parking_lot = "0.10.0" log = "0.4.8" kvdb = "0.7.0" -kvdb-rocksdb = { version = "0.9", optional = true } +kvdb-rocksdb = { version = "0.9" } kvdb-memorydb = "0.7.0" linked-hash-map = "0.5.2" hash-db = "0.15.2" @@ -37,6 +37,7 @@ sp-database = { version = "2.0.0-rc5", path = "../../primitives/database" } parity-db = { version = "0.1.2", optional = true } prometheus-endpoint = { package = "substrate-prometheus-endpoint", version = "0.8.0-rc5", path = "../../utils/prometheus" } +rocksdb = { version = "0.14", features = ["snappy"], default-features = false } # TODO remove, just to get collection size [dev-dependencies] sp-keyring = { version = "2.0.0-rc5", path = "../../primitives/keyring" } substrate-test-runtime-client = { version = "2.0.0-rc5", path = "../../test-utils/runtime/client" } @@ -48,6 +49,6 @@ tempfile = "3" [features] default = [] test-helpers = [] -with-kvdb-rocksdb = ["kvdb-rocksdb"] +with-kvdb-rocksdb = [] with-parity-db = ["parity-db"] with-subdb = [] diff --git a/client/db/src/upgrade.rs b/client/db/src/upgrade.rs index 95592d071f777..0f237dfc49b07 100644 --- a/client/db/src/upgrade.rs +++ b/client/db/src/upgrade.rs @@ -19,9 +19,18 @@ use std::fs; use std::io::{Read, Write, ErrorKind}; use std::path::{Path, PathBuf}; +use log::warn; +use std::marker::PhantomData; +use std::time::{Duration, Instant}; -use sp_runtime::traits::Block as BlockT; +use sp_runtime::traits::{Block as BlockT, HashFor, NumberFor, Header as HeaderT}; use crate::utils::DatabaseType; +use crate::{StateDb, PruningMode, StateMetaDb}; +use codec::{Decode, Encode}; +use kvdb::KeyValueDB; +use std::io; + +use std::sync::Arc; /// Version file name. const VERSION_FILE_NAME: &'static str = "db_version"; @@ -30,13 +39,16 @@ const VERSION_FILE_NAME: &'static str = "db_version"; const CURRENT_VERSION: u32 = 1; /// Upgrade database to current version. -pub fn upgrade_db(db_path: &Path, _db_type: DatabaseType) -> sp_blockchain::Result<()> { +pub fn upgrade_db(db_path: &Path, db_type: DatabaseType) -> sp_blockchain::Result<()> { let is_empty = db_path.read_dir().map_or(true, |mut d| d.next().is_none()); if !is_empty { let db_version = current_version(db_path)?; match db_version { 0 => Err(sp_blockchain::Error::Backend(format!("Unsupported database version: {}", db_version)))?, 1 => (), + 42 => { + test_thing::(db_path, db_type)?; + }, _ => Err(sp_blockchain::Error::Backend(format!("Future database version: {}", db_version)))?, } } @@ -44,6 +56,125 @@ pub fn upgrade_db(db_path: &Path, _db_type: DatabaseType) -> sp_b update_version(db_path) } +/// Database backed tree management for a rocksdb database. +pub struct RocksdbStorage(Arc); + +impl RocksdbStorage { +/* pub fn resolve_collection(c: &'static [u8]) -> Option { + if c.len() != 4 { + return None; + } + let index = Self::resolve_collection_inner(c); + if index < crate::utils::NUM_COLUMNS { + return Some(index); + } + None + } + const fn resolve_collection_inner(c: &'static [u8]) -> u32 { + let mut buf = [0u8; 4]; + buf[0] = c[0]; + buf[1] = c[1]; + buf[2] = c[2]; + buf[3] = c[3]; + u32::from_le_bytes(buf) + } + + fn write(&mut self, c: &'static [u8], k: &[u8], v: &[u8]) { + Self::resolve_collection(c).map(|c| { + let mut tx = self.0.transaction(); + tx.put(c, k, v); + self.0.write(tx) + .expect("Unsupported serialize error") + }); + } + + fn remove(&mut self, c: &'static [u8], k: &[u8]) { + Self::resolve_collection(c).map(|c| { + let mut tx = self.0.transaction(); + tx.delete(c, k); + self.0.write(tx) + .expect("Unsupported serialize error") + }); + } + + fn clear(&mut self, c: &'static [u8]) { + Self::resolve_collection(c).map(|c| { + let mut tx = self.0.transaction(); + tx.delete_prefix(c, &[]); + self.0.write(tx) + .expect("Unsupported serialize error") + }); + } + + fn read(&self, c: &'static [u8], k: &[u8]) -> Option> { + Self::resolve_collection(c).and_then(|c| { + self.0.get(c, k) + .expect("Unsupported readdb error") + }) + } + fn iter<'a>(&'a self, c: &'static [u8]) -> SerializeDBIter<'a> { + let iter = Self::resolve_collection(c).map(|c| { + self.0.iter(c).map(|(k, v)| (Vec::::from(k), Vec::::from(v))) + }).into_iter().flat_map(|i| i); + + Box::new(iter) + } +*/ + fn iter<'a>(&'a self, c: u32) -> SerializeDBIter<'a> { + let iter = self.0.iter(c).map(|(k, v)| (Vec::::from(k), Vec::::from(v))); + + Box::new(iter) + } + +/* + fn contains_collection(collection: &'static [u8]) -> bool { + Self::resolve_collection(collection).is_some() + }*/ +} +type SerializeDBIter<'a> = Box, Vec)> + 'a>; + +/// Hacky migrate to trigger action on db. +/// Here drop historied state content. +fn test_thing(db_path: &Path, db_type: DatabaseType) -> sp_blockchain::Result<()> { + + let mut db_config = kvdb_rocksdb::DatabaseConfig::with_columns(crate::utils::NUM_COLUMNS); + { + let option = rocksdb::Options::default(); + let cfs = rocksdb::DB::list_cf(&option, db_path).unwrap(); + let db = rocksdb::DB::open_cf(&option, db_path, cfs.clone()).unwrap(); + let mut i = 0; + for cf in cfs { + + if let Some(col) = db.cf_handle(&cf) { + println!("{:?}, {:?}", cf, db.property_int_value_cf(col, "rocksdb.estimate-table-readers-mem")); + println!("{:?}, {:?}", cf, db.property_int_value_cf(col, "rocksdb.size-all-mem-tables")); + println!("{:?}, {:?}", cf, db.property_int_value_cf(col, "rocksdb.cur-size-all-mem-tables")); + } + } + + } + + let mut i = 0; + { + let path = db_path.to_str() + .ok_or_else(|| sp_blockchain::Error::Backend("Invalid database path".into()))?; + while i < 6 { + let db_read = Arc::new(kvdb_rocksdb::Database::open(&db_config, path) + .map_err(|err| sp_blockchain::Error::Backend(format!("{}", err)))?); + + let db_r = RocksdbStorage(db_read.clone()); + let iter_kv = db_r.iter(i); + println!("{:?}, nb_iter {:?}", i, iter_kv.count()); + i += 1; + + } + } + + + + Ok(()) +} + /// Reads current database version from the file at given path. /// If the file does not exist returns 0. From 689fb6e243a1329e1386ede2fe261033174b1414 Mon Sep 17 00:00:00 2001 From: cheme Date: Wed, 19 Aug 2020 11:46:19 +0200 Subject: [PATCH 13/17] Revert "temporary code" This reverts commit e9255a24901ec040cb79ecdadc2e05a7b2d956a2. --- Cargo.lock | 1 - bin/node/testing/Cargo.toml | 2 +- client/db/Cargo.toml | 5 +- client/db/src/upgrade.rs | 135 +----------------------------------- 4 files changed, 5 insertions(+), 138 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ddfb4d55ac606..ec5af8aca4ecf 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6257,7 +6257,6 @@ dependencies = [ "parity-util-mem 0.7.0", "parking_lot 0.10.2", "quickcheck", - "rocksdb", "sc-client-api", "sc-executor", "sc-state-db", diff --git a/bin/node/testing/Cargo.toml b/bin/node/testing/Cargo.toml index cf3aab7f2ed14..a61a344cceed1 100644 --- a/bin/node/testing/Cargo.toml +++ b/bin/node/testing/Cargo.toml @@ -15,7 +15,7 @@ targets = ["x86_64-unknown-linux-gnu"] [dependencies] pallet-balances = { version = "2.0.0-rc5", path = "../../../frame/balances" } sc-service = { version = "0.8.0-rc5", features = ["test-helpers", "db"], path = "../../../client/service" } -sc-client-db = { version = "0.8.0-rc5", path = "../../../client/db/", features = ["parity-db"] } +sc-client-db = { version = "0.8.0-rc5", path = "../../../client/db/", features = ["kvdb-rocksdb", "parity-db"] } sc-client-api = { version = "2.0.0-rc5", path = "../../../client/api/" } codec = { package = "parity-scale-codec", version = "1.3.4" } pallet-contracts = { version = "2.0.0-rc5", path = "../../../frame/contracts" } diff --git a/client/db/Cargo.toml b/client/db/Cargo.toml index cb7f0ebf27932..50e14fcaae602 100644 --- a/client/db/Cargo.toml +++ b/client/db/Cargo.toml @@ -15,7 +15,7 @@ targets = ["x86_64-unknown-linux-gnu"] parking_lot = "0.10.0" log = "0.4.8" kvdb = "0.7.0" -kvdb-rocksdb = { version = "0.9" } +kvdb-rocksdb = { version = "0.9", optional = true } kvdb-memorydb = "0.7.0" linked-hash-map = "0.5.2" hash-db = "0.15.2" @@ -37,7 +37,6 @@ sp-database = { version = "2.0.0-rc5", path = "../../primitives/database" } parity-db = { version = "0.1.2", optional = true } prometheus-endpoint = { package = "substrate-prometheus-endpoint", version = "0.8.0-rc5", path = "../../utils/prometheus" } -rocksdb = { version = "0.14", features = ["snappy"], default-features = false } # TODO remove, just to get collection size [dev-dependencies] sp-keyring = { version = "2.0.0-rc5", path = "../../primitives/keyring" } substrate-test-runtime-client = { version = "2.0.0-rc5", path = "../../test-utils/runtime/client" } @@ -49,6 +48,6 @@ tempfile = "3" [features] default = [] test-helpers = [] -with-kvdb-rocksdb = [] +with-kvdb-rocksdb = ["kvdb-rocksdb"] with-parity-db = ["parity-db"] with-subdb = [] diff --git a/client/db/src/upgrade.rs b/client/db/src/upgrade.rs index 0f237dfc49b07..95592d071f777 100644 --- a/client/db/src/upgrade.rs +++ b/client/db/src/upgrade.rs @@ -19,18 +19,9 @@ use std::fs; use std::io::{Read, Write, ErrorKind}; use std::path::{Path, PathBuf}; -use log::warn; -use std::marker::PhantomData; -use std::time::{Duration, Instant}; -use sp_runtime::traits::{Block as BlockT, HashFor, NumberFor, Header as HeaderT}; +use sp_runtime::traits::Block as BlockT; use crate::utils::DatabaseType; -use crate::{StateDb, PruningMode, StateMetaDb}; -use codec::{Decode, Encode}; -use kvdb::KeyValueDB; -use std::io; - -use std::sync::Arc; /// Version file name. const VERSION_FILE_NAME: &'static str = "db_version"; @@ -39,16 +30,13 @@ const VERSION_FILE_NAME: &'static str = "db_version"; const CURRENT_VERSION: u32 = 1; /// Upgrade database to current version. -pub fn upgrade_db(db_path: &Path, db_type: DatabaseType) -> sp_blockchain::Result<()> { +pub fn upgrade_db(db_path: &Path, _db_type: DatabaseType) -> sp_blockchain::Result<()> { let is_empty = db_path.read_dir().map_or(true, |mut d| d.next().is_none()); if !is_empty { let db_version = current_version(db_path)?; match db_version { 0 => Err(sp_blockchain::Error::Backend(format!("Unsupported database version: {}", db_version)))?, 1 => (), - 42 => { - test_thing::(db_path, db_type)?; - }, _ => Err(sp_blockchain::Error::Backend(format!("Future database version: {}", db_version)))?, } } @@ -56,125 +44,6 @@ pub fn upgrade_db(db_path: &Path, db_type: DatabaseType) -> sp_bl update_version(db_path) } -/// Database backed tree management for a rocksdb database. -pub struct RocksdbStorage(Arc); - -impl RocksdbStorage { -/* pub fn resolve_collection(c: &'static [u8]) -> Option { - if c.len() != 4 { - return None; - } - let index = Self::resolve_collection_inner(c); - if index < crate::utils::NUM_COLUMNS { - return Some(index); - } - None - } - const fn resolve_collection_inner(c: &'static [u8]) -> u32 { - let mut buf = [0u8; 4]; - buf[0] = c[0]; - buf[1] = c[1]; - buf[2] = c[2]; - buf[3] = c[3]; - u32::from_le_bytes(buf) - } - - fn write(&mut self, c: &'static [u8], k: &[u8], v: &[u8]) { - Self::resolve_collection(c).map(|c| { - let mut tx = self.0.transaction(); - tx.put(c, k, v); - self.0.write(tx) - .expect("Unsupported serialize error") - }); - } - - fn remove(&mut self, c: &'static [u8], k: &[u8]) { - Self::resolve_collection(c).map(|c| { - let mut tx = self.0.transaction(); - tx.delete(c, k); - self.0.write(tx) - .expect("Unsupported serialize error") - }); - } - - fn clear(&mut self, c: &'static [u8]) { - Self::resolve_collection(c).map(|c| { - let mut tx = self.0.transaction(); - tx.delete_prefix(c, &[]); - self.0.write(tx) - .expect("Unsupported serialize error") - }); - } - - fn read(&self, c: &'static [u8], k: &[u8]) -> Option> { - Self::resolve_collection(c).and_then(|c| { - self.0.get(c, k) - .expect("Unsupported readdb error") - }) - } - fn iter<'a>(&'a self, c: &'static [u8]) -> SerializeDBIter<'a> { - let iter = Self::resolve_collection(c).map(|c| { - self.0.iter(c).map(|(k, v)| (Vec::::from(k), Vec::::from(v))) - }).into_iter().flat_map(|i| i); - - Box::new(iter) - } -*/ - fn iter<'a>(&'a self, c: u32) -> SerializeDBIter<'a> { - let iter = self.0.iter(c).map(|(k, v)| (Vec::::from(k), Vec::::from(v))); - - Box::new(iter) - } - -/* - fn contains_collection(collection: &'static [u8]) -> bool { - Self::resolve_collection(collection).is_some() - }*/ -} -type SerializeDBIter<'a> = Box, Vec)> + 'a>; - -/// Hacky migrate to trigger action on db. -/// Here drop historied state content. -fn test_thing(db_path: &Path, db_type: DatabaseType) -> sp_blockchain::Result<()> { - - let mut db_config = kvdb_rocksdb::DatabaseConfig::with_columns(crate::utils::NUM_COLUMNS); - { - let option = rocksdb::Options::default(); - let cfs = rocksdb::DB::list_cf(&option, db_path).unwrap(); - let db = rocksdb::DB::open_cf(&option, db_path, cfs.clone()).unwrap(); - let mut i = 0; - for cf in cfs { - - if let Some(col) = db.cf_handle(&cf) { - println!("{:?}, {:?}", cf, db.property_int_value_cf(col, "rocksdb.estimate-table-readers-mem")); - println!("{:?}, {:?}", cf, db.property_int_value_cf(col, "rocksdb.size-all-mem-tables")); - println!("{:?}, {:?}", cf, db.property_int_value_cf(col, "rocksdb.cur-size-all-mem-tables")); - } - } - - } - - let mut i = 0; - { - let path = db_path.to_str() - .ok_or_else(|| sp_blockchain::Error::Backend("Invalid database path".into()))?; - while i < 6 { - let db_read = Arc::new(kvdb_rocksdb::Database::open(&db_config, path) - .map_err(|err| sp_blockchain::Error::Backend(format!("{}", err)))?); - - let db_r = RocksdbStorage(db_read.clone()); - let iter_kv = db_r.iter(i); - println!("{:?}, nb_iter {:?}", i, iter_kv.count()); - i += 1; - - } - } - - - - Ok(()) -} - /// Reads current database version from the file at given path. /// If the file does not exist returns 0. From 2e0d12264359e51df0b14ec1e68899d0fa554442 Mon Sep 17 00:00:00 2001 From: cheme Date: Wed, 19 Aug 2020 12:18:35 +0200 Subject: [PATCH 14/17] Move HeaderLookupBackend methods to HeaderBackend --- client/api/src/backend.rs | 11 +--- client/api/src/in_mem.rs | 30 +++------- client/api/src/light.rs | 5 +- .../authority-discovery/src/worker/tests.rs | 15 +++++ client/consensus/babe/src/lib.rs | 5 +- client/db/src/lib.rs | 59 +++++++++---------- client/db/src/light.rs | 54 ++++++++--------- client/light/src/backend.rs | 12 +--- client/light/src/blockchain.rs | 8 +++ client/service/src/client/client.rs | 51 +++++----------- client/service/test/src/client/light.rs | 28 ++++----- primitives/blockchain/src/backend.rs | 6 ++ 12 files changed, 122 insertions(+), 162 deletions(-) diff --git a/client/api/src/backend.rs b/client/api/src/backend.rs index 85b7330482451..eabc37e53c142 100644 --- a/client/api/src/backend.rs +++ b/client/api/src/backend.rs @@ -270,15 +270,6 @@ pub trait AuxStore { fn get_aux(&self, key: &[u8]) -> sp_blockchain::Result>>; } -/// Provides access to an header lookup database. -pub trait HeaderLookupStore { - /// Is the header lookup define for a given number. - fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result; - - /// Cleanup header mapping. - fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()>; -} - /// An `Iterator` that iterates keys in a given block under a prefix. pub struct KeyIterator<'a, State, Block> { state: State, @@ -404,7 +395,7 @@ pub trait StorageProvider> { /// /// The same applies for live `BlockImportOperation`s: while an import operation building on a /// parent `P` is alive, the state for `P` should not be pruned. -pub trait Backend: AuxStore + HeaderLookupStore + Send + Sync { +pub trait Backend: AuxStore + Send + Sync { /// Associated block insertion operation type. type BlockImportOperation: BlockImportOperation; /// Associated blockchain backend type. diff --git a/client/api/src/in_mem.rs b/client/api/src/in_mem.rs index e6d865faa0cf3..9e4200610f5c1 100644 --- a/client/api/src/in_mem.rs +++ b/client/api/src/in_mem.rs @@ -339,6 +339,15 @@ impl HeaderBackend for Blockchain { fn hash(&self, number: <::Header as HeaderT>::Number) -> sp_blockchain::Result> { Ok(self.id(BlockId::Number(number))) } + + fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + Ok(self.storage.read().hashes.get(number) == Some(hash)) + } + + fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + self.storage.write().hashes.remove(number); + Ok(()) + } } impl HeaderMetadata for Blockchain { @@ -417,17 +426,6 @@ impl backend::AuxStore for Blockchain { } } -impl backend::HeaderLookupStore for Blockchain { - fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { - Ok(self.storage.read().hashes.get(number) == Some(hash)) - } - - fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { - self.storage.write().hashes.remove(number); - Ok(()) - } -} - impl light::Storage for Blockchain where Block::Hash: From<[u8; 32]>, @@ -629,16 +627,6 @@ impl backend::AuxStore for Backend where Block::Hash: Ord } } -impl backend::HeaderLookupStore for Backend { - fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { - self.blockchain.is_lookup_define_for_number(number, hash) - } - - fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { - self.blockchain.clean_up_number_lookup(number) - } -} - impl backend::Backend for Backend where Block::Hash: Ord { type BlockImportOperation = BlockImportOperation; type Blockchain = Blockchain; diff --git a/client/api/src/light.rs b/client/api/src/light.rs index a9c84b8ab14b3..b359c1149eea6 100644 --- a/client/api/src/light.rs +++ b/client/api/src/light.rs @@ -32,7 +32,7 @@ use sp_blockchain::{ HeaderMetadata, well_known_cache_keys, HeaderBackend, Cache as BlockchainCache, Error as ClientError, Result as ClientResult, }; -use crate::{backend::{AuxStore, NewBlockState, HeaderLookupStore}, UsageInfo}; +use crate::{backend::{AuxStore, NewBlockState}, UsageInfo}; /// Remote call request. #[derive(Clone, Debug, PartialEq, Eq, Hash)] @@ -232,8 +232,7 @@ pub trait FetchChecker: Send + Sync { /// Light client blockchain storage. -pub trait Storage: AuxStore + HeaderBackend + HeaderMetadata - + HeaderLookupStore { +pub trait Storage: AuxStore + HeaderBackend + HeaderMetadata { /// Store new header. Should refuse to revert any finalized blocks. /// /// Takes new authorities, the leaf state of the new block, and diff --git a/client/authority-discovery/src/worker/tests.rs b/client/authority-discovery/src/worker/tests.rs index 68aadca7a7f30..20f0d321818e4 100644 --- a/client/authority-discovery/src/worker/tests.rs +++ b/client/authority-discovery/src/worker/tests.rs @@ -150,6 +150,21 @@ impl HeaderBackend for TestApi { ) -> std::result::Result, sp_blockchain::Error> { Ok(None) } + + fn is_lookup_define_for_number( + &self, + _number: &NumberFor, + _hash: &Block::Hash, + ) -> std::result::Result { + Ok(false) + } + + fn clean_up_number_lookup( + &self, + number: &NumberFor, + ) -> std::result::Result<(), sp_blockchain::Error> { + Ok(()) + } } pub(crate) struct RuntimeApi { diff --git a/client/consensus/babe/src/lib.rs b/client/consensus/babe/src/lib.rs index 6d4bf5a7d15c6..0b34cf5c4cffa 100644 --- a/client/consensus/babe/src/lib.rs +++ b/client/consensus/babe/src/lib.rs @@ -104,7 +104,7 @@ use sp_consensus::import_queue::{Verifier, BasicQueue, DefaultImportQueue, Cache use sc_client_api::{ backend::AuxStore, BlockchainEvents, ProvideUncles, - SharedPruningRequirements, HeaderLookupStore, + SharedPruningRequirements, }; use sp_block_builder::BlockBuilder as BlockBuilderApi; use futures::channel::mpsc::{channel, Sender, Receiver}; @@ -1117,7 +1117,6 @@ impl BlockImport for BabeBlockImport> + Send + Sync, Inner::Error: Into, Client: HeaderBackend + HeaderMetadata - + HeaderLookupStore + AuxStore + ProvideRuntimeApi + ProvideCache + Send + Sync, Client::Api: BabeApi + ApiExt, { @@ -1375,7 +1374,6 @@ fn prune_finalized( Block: BlockT, Client: HeaderBackend + HeaderMetadata - + HeaderLookupStore, { let info = client.info(); @@ -1436,7 +1434,6 @@ pub fn block_import( shared_pruning_requirements: Option<&SharedPruningRequirements>, ) -> ClientResult<(BabeBlockImport, BabeLink)> where Client: AuxStore + HeaderBackend + HeaderMetadata - + HeaderLookupStore, { let epoch_changes = aux_schema::load_epoch_changes::(&*client, &config)?; let link = BabeLink { diff --git a/client/db/src/lib.rs b/client/db/src/lib.rs index 2161125e627b0..e870737b3983f 100644 --- a/client/db/src/lib.rs +++ b/client/db/src/lib.rs @@ -450,6 +450,33 @@ impl sc_client_api::blockchain::HeaderBackend for Blockcha None => Ok(None), }) } + + fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + let lookup_key = utils::block_id_to_lookup_key::( + &*self.db, + columns::KEY_LOOKUP, + BlockId::Number(number.clone()), + )?; + Ok(if let Some(lookup_key) = lookup_key { + utils::lookup_key_to_hash(lookup_key.as_ref())? == hash.as_ref() + } else { + false + }) + } + + fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + // TODO pass transaction as parameter? + let mut transaction = Transaction::new(); + utils::remove_number_to_key_mapping( + &mut transaction, + columns::KEY_LOOKUP, + number.clone(), + )?; + + self.db.commit(transaction)?; + + Ok(()) + } } impl sc_client_api::blockchain::Backend for BlockchainDb { @@ -1367,38 +1394,6 @@ impl sc_client_api::backend::AuxStore for Backend where Block: Blo } } - -impl sc_client_api::backend::HeaderLookupStore for Backend - where Block: BlockT -{ - fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { - let lookup_key = utils::block_id_to_lookup_key::( - &*self.storage.db, - columns::KEY_LOOKUP, - BlockId::Number(number.clone()), - )?; - Ok(if let Some(lookup_key) = lookup_key { - utils::lookup_key_to_hash(lookup_key.as_ref())? == hash.as_ref() - } else { - false - }) - } - - fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { - // TODO pass transaction as parameter? - let mut transaction = Transaction::new(); - utils::remove_number_to_key_mapping( - &mut transaction, - columns::KEY_LOOKUP, - number.clone(), - )?; - - self.storage.db.commit(transaction)?; - - Ok(()) - } -} - impl sc_client_api::backend::Backend for Backend { type BlockImportOperation = BlockImportOperation; type Blockchain = BlockchainDb; diff --git a/client/db/src/light.rs b/client/db/src/light.rs index 051a6471fb921..14f8d2e9c47c6 100644 --- a/client/db/src/light.rs +++ b/client/db/src/light.rs @@ -23,7 +23,7 @@ use std::convert::TryInto; use parking_lot::RwLock; use sc_client_api::{ - cht, backend::{AuxStore, NewBlockState, HeaderLookupStore}, UsageInfo, + cht, backend::{AuxStore, NewBlockState}, UsageInfo, blockchain::{ BlockStatus, Cache as BlockchainCache, Info as BlockchainInfo, }, @@ -196,6 +196,29 @@ impl BlockchainHeaderBackend for LightStorage fn hash(&self, number: NumberFor) -> ClientResult> { Ok(self.header(BlockId::Number(number))?.map(|header| header.hash().clone())) } + + fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + let lookup_key = utils::block_id_to_lookup_key::(&*self.db, columns::KEY_LOOKUP, BlockId::Number(number.clone()))?; + Ok(if let Some(lookup_key) = lookup_key { + utils::lookup_key_to_hash(lookup_key.as_ref())? == hash.as_ref() + } else { + false + }) + } + + fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + // TODO pass transaction as parameter? + let mut transaction = Transaction::new(); + utils::remove_number_to_key_mapping( + &mut transaction, + columns::KEY_LOOKUP, + number.clone(), + )?; + + self.db.commit(transaction)?; + + Ok(()) + } } impl HeaderMetadata for LightStorage { @@ -432,35 +455,6 @@ impl AuxStore for LightStorage } } -impl HeaderLookupStore for LightStorage - where Block: BlockT, -{ - - fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { - let lookup_key = utils::block_id_to_lookup_key::(&*self.db, columns::KEY_LOOKUP, BlockId::Number(number.clone()))?; - Ok(if let Some(lookup_key) = lookup_key { - utils::lookup_key_to_hash(lookup_key.as_ref())? == hash.as_ref() - } else { - false - }) - } - - fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { - // TODO pass transaction as parameter? - let mut transaction = Transaction::new(); - utils::remove_number_to_key_mapping( - &mut transaction, - columns::KEY_LOOKUP, - number.clone(), - )?; - - self.db.commit(transaction)?; - - Ok(()) - } -} - - impl Storage for LightStorage where Block: BlockT, { diff --git a/client/light/src/backend.rs b/client/light/src/backend.rs index 0f182219f0e71..be7953e528bd8 100644 --- a/client/light/src/backend.rs +++ b/client/light/src/backend.rs @@ -38,7 +38,7 @@ use sp_blockchain::{Error as ClientError, Result as ClientResult}; use sc_client_api::{ backend::{ AuxStore, Backend as ClientBackend, BlockImportOperation, RemoteBackend, NewBlockState, - PrunableStateChangesTrieStorage, HeaderLookupStore, + PrunableStateChangesTrieStorage, }, blockchain::{ HeaderBackend as BlockchainHeaderBackend, well_known_cache_keys, @@ -113,16 +113,6 @@ impl AuxStore for Backend { } } -impl, H: Hasher> HeaderLookupStore for Backend { - fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { - self.blockchain.storage().is_lookup_define_for_number(number, hash) - } - - fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { - self.blockchain.storage().clean_up_number_lookup(number) - } -} - impl ClientBackend for Backend> where Block: BlockT, diff --git a/client/light/src/blockchain.rs b/client/light/src/blockchain.rs index 9d557db887d29..57746dd41558c 100644 --- a/client/light/src/blockchain.rs +++ b/client/light/src/blockchain.rs @@ -86,6 +86,14 @@ impl BlockchainHeaderBackend for Blockchain where Block: Blo fn hash(&self, number: <::Header as HeaderT>::Number) -> ClientResult> { self.storage.hash(number) } + + fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> ClientResult { + self.storage.is_lookup_define_for_number(number, hash) + } + + fn clean_up_number_lookup(&self, number: &NumberFor) -> ClientResult<()> { + self.storage.clean_up_number_lookup(number) + } } impl HeaderMetadata for Blockchain where Block: BlockT, S: Storage { diff --git a/client/service/src/client/client.rs b/client/service/src/client/client.rs index b0ad6cc4e47bd..bfd9629b35b90 100644 --- a/client/service/src/client/client.rs +++ b/client/service/src/client/client.rs @@ -1548,6 +1548,14 @@ impl ChainHeaderBackend for Client wher fn hash(&self, number: NumberFor) -> sp_blockchain::Result> { self.backend.blockchain().hash(number) } + + fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + self.backend.blockchain().is_lookup_define_for_number(number, hash) + } + + fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + self.backend.blockchain().clean_up_number_lookup(number) + } } impl sp_runtime::traits::BlockIdTo for Client where @@ -1592,6 +1600,14 @@ impl ChainHeaderBackend for &Client whe fn hash(&self, number: NumberFor) -> sp_blockchain::Result> { (**self).hash(number) } + + fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + (**self).is_lookup_define_for_number(number, hash) + } + + fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + (**self).clean_up_number_lookup(number) + } } impl ProvideCache for Client where @@ -1988,41 +2004,6 @@ impl backend::AuxStore for &Client } } -impl sc_client_api::HeaderLookupStore for Client - where - B: backend::Backend, - E: CallExecutor, - Block: BlockT, - Self: ProvideRuntimeApi, - >::Api: CoreApi, -{ - fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { - sc_client_api::HeaderLookupStore::is_lookup_define_for_number(&*self.backend, number, hash) - } - - fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { - sc_client_api::HeaderLookupStore::clean_up_number_lookup(&*self.backend, number) - } -} - -impl sc_client_api::HeaderLookupStore for &Client - where - B: backend::Backend, - E: CallExecutor, - Block: BlockT, - Client: ProvideRuntimeApi, - as ProvideRuntimeApi>::Api: CoreApi, -{ - fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { - (**self).is_lookup_define_for_number(number, hash) - } - - fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { - (**self).clean_up_number_lookup(number) - } -} - - impl sp_consensus::block_validation::Chain for Client where BE: backend::Backend, E: CallExecutor, diff --git a/client/service/test/src/client/light.rs b/client/service/test/src/client/light.rs index 185935fad04ee..9fe1abe7b2712 100644 --- a/client/service/test/src/client/light.rs +++ b/client/service/test/src/client/light.rs @@ -46,7 +46,6 @@ use sc_client_api::{ AuxStore, Storage, CallExecutor, cht, ExecutionStrategy, StorageProof, BlockImportOperation, RemoteCallRequest, StorageProvider, ChangesProof, RemoteBodyRequest, RemoteReadRequest, RemoteChangesRequest, FetchChecker, RemoteReadChildRequest, RemoteHeaderRequest, BlockBackend, - HeaderLookupStore, }; use sp_externalities::Extensions; use sc_block_builder::BlockBuilderProvider; @@ -110,6 +109,18 @@ impl sp_blockchain::HeaderBackend for DummyStorage { Err(ClientError::Backend("Test error".into())) } } + + fn is_lookup_define_for_number( + &self, + _number: &NumberFor, + _hash: &::Hash, + ) -> sp_blockchain::Result { + Err(ClientError::Backend("Test error".into())) + } + + fn clean_up_number_lookup(&self, _number: &NumberFor) -> sp_blockchain::Result<()> { + Err(ClientError::Backend("Test error".into())) + } } impl sp_blockchain::HeaderMetadata for DummyStorage { @@ -142,21 +153,6 @@ impl AuxStore for DummyStorage { } } -impl HeaderLookupStore for DummyStorage { - fn is_lookup_define_for_number( - &self, - _number: &NumberFor, - _hash: &::Hash, - ) -> sp_blockchain::Result { - Err(ClientError::Backend("Test error".into())) - } - - fn clean_up_number_lookup(&self, _number: &NumberFor) -> sp_blockchain::Result<()> { - Err(ClientError::Backend("Test error".into())) - } -} - - impl Storage for DummyStorage { fn import_header( &self, diff --git a/primitives/blockchain/src/backend.rs b/primitives/blockchain/src/backend.rs index 1328dfb5752fc..488f671e6541b 100644 --- a/primitives/blockchain/src/backend.rs +++ b/primitives/blockchain/src/backend.rs @@ -78,6 +78,12 @@ pub trait HeaderBackend: Send + Sync { Error::UnknownBlock(format!("Expect block hash from id: {}", id)) )) } + + /// Was this header stored a canonical (partially pruned). TODO pruned_header_previously_canonical + fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> Result; + + /// Cleanup partially pruned header. TODO pruned_header_clean_up + fn clean_up_number_lookup(&self, number: &NumberFor) -> Result<()>; } /// Blockchain database backend. Does not perform any validation. From ec15ed96213a8091b76dbaaaed00fb465c61673f Mon Sep 17 00:00:00 2001 From: cheme Date: Wed, 19 Aug 2020 12:25:23 +0200 Subject: [PATCH 15/17] Rename methods. --- client/api/src/in_mem.rs | 4 ++-- client/authority-discovery/src/worker/tests.rs | 6 +++--- client/consensus/babe/src/lib.rs | 4 ++-- client/db/src/lib.rs | 5 ++--- client/db/src/light.rs | 5 ++--- client/light/src/blockchain.rs | 8 ++++---- client/service/src/client/client.rs | 16 ++++++++-------- client/service/test/src/client/light.rs | 4 ++-- primitives/blockchain/src/backend.rs | 8 ++++---- 9 files changed, 29 insertions(+), 31 deletions(-) diff --git a/client/api/src/in_mem.rs b/client/api/src/in_mem.rs index 9e4200610f5c1..f8ddbb99b9c10 100644 --- a/client/api/src/in_mem.rs +++ b/client/api/src/in_mem.rs @@ -340,11 +340,11 @@ impl HeaderBackend for Blockchain { Ok(self.id(BlockId::Number(number))) } - fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + fn pruned_header_was_canonical(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { Ok(self.storage.read().hashes.get(number) == Some(hash)) } - fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + fn pruned_header_clean_up(&self, number: &NumberFor) -> sp_blockchain::Result<()> { self.storage.write().hashes.remove(number); Ok(()) } diff --git a/client/authority-discovery/src/worker/tests.rs b/client/authority-discovery/src/worker/tests.rs index 20f0d321818e4..6574d4e4134c5 100644 --- a/client/authority-discovery/src/worker/tests.rs +++ b/client/authority-discovery/src/worker/tests.rs @@ -151,7 +151,7 @@ impl HeaderBackend for TestApi { Ok(None) } - fn is_lookup_define_for_number( + fn pruned_header_was_canonical( &self, _number: &NumberFor, _hash: &Block::Hash, @@ -159,9 +159,9 @@ impl HeaderBackend for TestApi { Ok(false) } - fn clean_up_number_lookup( + fn pruned_header_clean_up( &self, - number: &NumberFor, + _number: &NumberFor, ) -> std::result::Result<(), sp_blockchain::Error> { Ok(()) } diff --git a/client/consensus/babe/src/lib.rs b/client/consensus/babe/src/lib.rs index 0b34cf5c4cffa..e31280269b524 100644 --- a/client/consensus/babe/src/lib.rs +++ b/client/consensus/babe/src/lib.rs @@ -1391,10 +1391,10 @@ fn prune_finalized( if prune_light { let is_canonical = |hash: &Block::Hash, number: &NumberFor, finalized_number: &NumberFor| { - Ok(number <= finalized_number && client.is_lookup_define_for_number(number, hash)?) + Ok(number <= finalized_number && client.pruned_header_was_canonical(number, hash)?) }; let clean_up = |number: &NumberFor| { - client.clean_up_number_lookup(number)?; + client.pruned_header_clean_up(number)?; Ok(()) }; diff --git a/client/db/src/lib.rs b/client/db/src/lib.rs index e870737b3983f..555e5ae1260cd 100644 --- a/client/db/src/lib.rs +++ b/client/db/src/lib.rs @@ -451,7 +451,7 @@ impl sc_client_api::blockchain::HeaderBackend for Blockcha }) } - fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + fn pruned_header_was_canonical(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { let lookup_key = utils::block_id_to_lookup_key::( &*self.db, columns::KEY_LOOKUP, @@ -464,8 +464,7 @@ impl sc_client_api::blockchain::HeaderBackend for Blockcha }) } - fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { - // TODO pass transaction as parameter? + fn pruned_header_clean_up(&self, number: &NumberFor) -> sp_blockchain::Result<()> { let mut transaction = Transaction::new(); utils::remove_number_to_key_mapping( &mut transaction, diff --git a/client/db/src/light.rs b/client/db/src/light.rs index 14f8d2e9c47c6..e5e65f67e7347 100644 --- a/client/db/src/light.rs +++ b/client/db/src/light.rs @@ -197,7 +197,7 @@ impl BlockchainHeaderBackend for LightStorage Ok(self.header(BlockId::Number(number))?.map(|header| header.hash().clone())) } - fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + fn pruned_header_was_canonical(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { let lookup_key = utils::block_id_to_lookup_key::(&*self.db, columns::KEY_LOOKUP, BlockId::Number(number.clone()))?; Ok(if let Some(lookup_key) = lookup_key { utils::lookup_key_to_hash(lookup_key.as_ref())? == hash.as_ref() @@ -206,8 +206,7 @@ impl BlockchainHeaderBackend for LightStorage }) } - fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { - // TODO pass transaction as parameter? + fn pruned_header_clean_up(&self, number: &NumberFor) -> sp_blockchain::Result<()> { let mut transaction = Transaction::new(); utils::remove_number_to_key_mapping( &mut transaction, diff --git a/client/light/src/blockchain.rs b/client/light/src/blockchain.rs index 57746dd41558c..132f6a03ad555 100644 --- a/client/light/src/blockchain.rs +++ b/client/light/src/blockchain.rs @@ -87,12 +87,12 @@ impl BlockchainHeaderBackend for Blockchain where Block: Blo self.storage.hash(number) } - fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> ClientResult { - self.storage.is_lookup_define_for_number(number, hash) + fn pruned_header_was_canonical(&self, number: &NumberFor, hash: &Block::Hash) -> ClientResult { + self.storage.pruned_header_was_canonical(number, hash) } - fn clean_up_number_lookup(&self, number: &NumberFor) -> ClientResult<()> { - self.storage.clean_up_number_lookup(number) + fn pruned_header_clean_up(&self, number: &NumberFor) -> ClientResult<()> { + self.storage.pruned_header_clean_up(number) } } diff --git a/client/service/src/client/client.rs b/client/service/src/client/client.rs index bfd9629b35b90..55ee757ddcd0f 100644 --- a/client/service/src/client/client.rs +++ b/client/service/src/client/client.rs @@ -1549,12 +1549,12 @@ impl ChainHeaderBackend for Client wher self.backend.blockchain().hash(number) } - fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { - self.backend.blockchain().is_lookup_define_for_number(number, hash) + fn pruned_header_was_canonical(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + self.backend.blockchain().pruned_header_was_canonical(number, hash) } - fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { - self.backend.blockchain().clean_up_number_lookup(number) + fn pruned_header_clean_up(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + self.backend.blockchain().pruned_header_clean_up(number) } } @@ -1601,12 +1601,12 @@ impl ChainHeaderBackend for &Client whe (**self).hash(number) } - fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { - (**self).is_lookup_define_for_number(number, hash) + fn pruned_header_was_canonical(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + (**self).pruned_header_was_canonical(number, hash) } - fn clean_up_number_lookup(&self, number: &NumberFor) -> sp_blockchain::Result<()> { - (**self).clean_up_number_lookup(number) + fn pruned_header_clean_up(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + (**self).pruned_header_clean_up(number) } } diff --git a/client/service/test/src/client/light.rs b/client/service/test/src/client/light.rs index 9fe1abe7b2712..2b0b825a58d69 100644 --- a/client/service/test/src/client/light.rs +++ b/client/service/test/src/client/light.rs @@ -110,7 +110,7 @@ impl sp_blockchain::HeaderBackend for DummyStorage { } } - fn is_lookup_define_for_number( + fn pruned_header_was_canonical( &self, _number: &NumberFor, _hash: &::Hash, @@ -118,7 +118,7 @@ impl sp_blockchain::HeaderBackend for DummyStorage { Err(ClientError::Backend("Test error".into())) } - fn clean_up_number_lookup(&self, _number: &NumberFor) -> sp_blockchain::Result<()> { + fn pruned_header_clean_up(&self, _number: &NumberFor) -> sp_blockchain::Result<()> { Err(ClientError::Backend("Test error".into())) } } diff --git a/primitives/blockchain/src/backend.rs b/primitives/blockchain/src/backend.rs index 488f671e6541b..ed6b3bb62be98 100644 --- a/primitives/blockchain/src/backend.rs +++ b/primitives/blockchain/src/backend.rs @@ -79,11 +79,11 @@ pub trait HeaderBackend: Send + Sync { )) } - /// Was this header stored a canonical (partially pruned). TODO pruned_header_previously_canonical - fn is_lookup_define_for_number(&self, number: &NumberFor, hash: &Block::Hash) -> Result; + /// Was this header stored a canonical (partially pruned). + fn pruned_header_was_canonical(&self, number: &NumberFor, hash: &Block::Hash) -> Result; - /// Cleanup partially pruned header. TODO pruned_header_clean_up - fn clean_up_number_lookup(&self, number: &NumberFor) -> Result<()>; + /// Cleanup partially pruned header. + fn pruned_header_clean_up(&self, number: &NumberFor) -> Result<()>; } /// Blockchain database backend. Does not perform any validation. From 07cf6dd870839bb7d5017589beb4747b796932a1 Mon Sep 17 00:00:00 2001 From: cheme Date: Wed, 19 Aug 2020 12:47:09 +0200 Subject: [PATCH 16/17] Factor duplicated code. --- client/consensus/epochs/src/lib.rs | 2 +- utils/fork-tree/src/lib.rs | 146 +++++++++-------------------- 2 files changed, 47 insertions(+), 101 deletions(-) diff --git a/client/consensus/epochs/src/lib.rs b/client/consensus/epochs/src/lib.rs index 3d1295b8433f2..73a78a706d364 100644 --- a/client/consensus/epochs/src/lib.rs +++ b/client/consensus/epochs/src/lib.rs @@ -443,7 +443,7 @@ impl EpochChanges where PersistedEpochHeader::Regular(ref epoch_n) => slot >= epoch_n.end_slot, }; - let removed = self.inner.prune_light( + let removed = self.inner.prune_non_cannonical( hash, &number, &is_canonical, diff --git a/utils/fork-tree/src/lib.rs b/utils/fork-tree/src/lib.rs index fdbd40c244323..6aee0e857ae85 100644 --- a/utils/fork-tree/src/lib.rs +++ b/utils/fork-tree/src/lib.rs @@ -114,6 +114,28 @@ impl ForkTree where predicate, )?; + let is_canonical = |self_hash: &H, _number: &N, _final_number: &N| { + is_descendent_of(self_hash, hash) + }; + + self.prune_inner( + new_root_index, + hash, + number, + &is_canonical, + ) + } + + fn prune_inner( + &mut self, + new_root_index: Option>, + hash: &H, + number: &N, + is_canonical: &F, + ) -> Result, Error> + where E: std::error::Error, + F: Fn(&H, &N, &N) -> Result, + { let removed = if let Some(mut root_index) = new_root_index { let mut old_roots = std::mem::take(&mut self.roots); @@ -144,7 +166,7 @@ impl ForkTree where for child in root_children { if is_first && (child.number == *number && child.hash == *hash || - child.number < *number && is_descendent_of(&child.hash, hash).unwrap_or(false)) + child.number < *number && is_canonical(&child.hash, &child.number, number).unwrap_or(false)) { root.children.push(child); // assuming that the tree is well formed only one child should pass this requirement @@ -175,7 +197,7 @@ impl ForkTree where /// It apply recursively a clean up call back on branch. /// /// Returns all pruned node data. - pub fn prune_light( + pub fn prune_non_cannonical( &mut self, hash: &H, number: &N, @@ -188,68 +210,25 @@ impl ForkTree where P: Fn(&V) -> bool, C: Fn(&N) -> Result<(), E>, { - let new_root_index = self.find_node_index_where_light( + let new_root_index = self.find_node_index_where_canonical( hash, number, is_canonical, predicate, )?; - let removed = if let Some(mut root_index) = new_root_index { - let mut old_roots = std::mem::take(&mut self.roots); - - let mut root = None; - let mut cur_children = Some(&mut old_roots); - - while let Some(cur_index) = root_index.pop() { - if let Some(children) = cur_children.take() { - if root_index.is_empty() { - root = Some(children.remove(cur_index)); - } else { - cur_children = Some(&mut children[cur_index].children); - } - } - } - - let mut root = root - .expect("find_node_index_where will return array with at least one index; \ - this results in at least one item in removed; qed"); - - let mut removed = old_roots; - - // we found the deepest ancestor of the finalized block, so we prune - // out any children that don't include the finalized block. - let root_children = std::mem::take(&mut root.children); - let mut is_first = true; - - for child in root_children { - if is_first && - (child.number == *number && child.hash == *hash || - child.number < *number && is_canonical(&child.hash, &child.number, number).unwrap_or(false)) - { - root.children.push(child); - // assuming that the tree is well formed only one child should pass this requirement - // due to ancestry restrictions (i.e. they must be different forks). - is_first = false; - } else { - removed.push(child); - } - } - - self.roots = vec![root]; - - removed - } else { - Vec::new() - }; - - self.rebalance(); + let result = self.prune_inner( + new_root_index, + hash, + number, + is_canonical, + )?; - for node in removed.iter() { + for node in result.stack.iter() { node.clean_up(clean_up)?; } - Ok(RemovedIterator { stack: removed }) + Ok(result) } } @@ -445,7 +424,7 @@ impl ForkTree where /// Same as [`find_node_index_where`](ForkTree::find_node_where), but with a predicate /// on canonical nature of block. - pub fn find_node_index_where_light( + pub fn find_node_index_where_canonical( &self, hash: &H, number: &N, @@ -458,7 +437,7 @@ impl ForkTree where { // search for node starting from all roots for (index, root) in self.roots.iter().enumerate() { - let node = root.find_node_index_where_light(hash, number, is_canonical, predicate)?; + let node = root.find_node_index_where_canonical(hash, number, is_canonical, predicate)?; // found the node, early exit if let FindOutcome::Found(mut node) = node { @@ -867,53 +846,20 @@ mod node_implementation { F: Fn(&H, &H) -> Result, P: Fn(&V) -> bool, { - // stop searching this branch - if *number < self.number { - return Ok(FindOutcome::Failure(false)); - } - - let mut known_descendent_of = false; - - // continue depth-first search through all children - for (i, node) in self.children.iter().enumerate() { - // found node, early exit - match node.find_node_index_where(hash, number, is_descendent_of, predicate)? { - FindOutcome::Abort => return Ok(FindOutcome::Abort), - FindOutcome::Found(mut x) => { - x.push(i); - return Ok(FindOutcome::Found(x)) - }, - FindOutcome::Failure(true) => { - // if the block was a descendent of this child, - // then it cannot be a descendent of any others, - // so we don't search them. - known_descendent_of = true; - break; - }, - FindOutcome::Failure(false) => {}, - } - } - - // node not found in any of the descendents, if the node we're - // searching for is a descendent of this node then we will stop the - // search here, since there aren't any more children and we found - // the correct node so we don't want to backtrack. - let is_descendent_of = known_descendent_of || is_descendent_of(&self.hash, hash)?; - if is_descendent_of { - // if the predicate passes we return the node - if predicate(&self.data) { - return Ok(FindOutcome::Found(Vec::new())); - } - } - - // otherwise, tell our ancestor that we failed, and whether - // the block was a descendent. - Ok(FindOutcome::Failure(is_descendent_of)) + let is_canonical = |self_hash: &H, _number: &N, _final_number: &N| { + is_descendent_of(self_hash, hash) + }; + self.find_node_index_where_canonical( + hash, + number, + &is_canonical, + predicate, + ) } /// Same as [`find_node_index_where`](Node::find_node_where), but with /// a direct check on block being canonical. - pub fn find_node_index_where_light( + pub fn find_node_index_where_canonical( &self, hash: &H, number: &N, @@ -934,7 +880,7 @@ mod node_implementation { // continue depth-first search through all children for (i, node) in self.children.iter().enumerate() { // found node, early exit - match node.find_node_index_where_light( + match node.find_node_index_where_canonical( hash, number, is_canonical, From 079eecda5ecd157048efa19af82b0c27fdfdbe3a Mon Sep 17 00:00:00 2001 From: cheme Date: Wed, 9 Sep 2020 11:45:35 +0200 Subject: [PATCH 17/17] Break some long lines. --- bin/node/cli/src/service.rs | 12 ++++++++++-- client/api/src/in_mem.rs | 6 +++++- client/consensus/babe/src/lib.rs | 6 +++++- client/db/src/lib.rs | 6 +++++- client/db/src/light.rs | 6 +++++- client/light/src/blockchain.rs | 6 +++++- client/service/src/client/client.rs | 12 ++++++++++-- primitives/blockchain/src/backend.rs | 6 +++++- utils/fork-tree/src/lib.rs | 4 ++-- 9 files changed, 52 insertions(+), 12 deletions(-) diff --git a/bin/node/cli/src/service.rs b/bin/node/cli/src/service.rs index a176f9cace083..d20ad5acab92a 100644 --- a/bin/node/cli/src/service.rs +++ b/bin/node/cli/src/service.rs @@ -349,8 +349,16 @@ pub fn new_light_base(config: Configuration) -> Result<( ), ServiceError> { let mut shared_pruning_requirements = SharedPruningRequirements::default(); sc_consensus_babe::light_pruning_requirements(&mut shared_pruning_requirements); - let (client, backend, keystore, mut task_manager, on_demand) = - sc_service::new_light_parts::(&config, &shared_pruning_requirements)?; + let ( + client, + backend, + keystore, + mut task_manager, + on_demand, + ) = sc_service::new_light_parts::( + &config, + &shared_pruning_requirements, + )?; let select_chain = sc_consensus::LongestChain::new(backend.clone()); diff --git a/client/api/src/in_mem.rs b/client/api/src/in_mem.rs index c25a9d8ffd47c..b3a41e3557432 100644 --- a/client/api/src/in_mem.rs +++ b/client/api/src/in_mem.rs @@ -340,7 +340,11 @@ impl HeaderBackend for Blockchain { Ok(self.id(BlockId::Number(number))) } - fn pruned_header_was_canonical(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + fn pruned_header_was_canonical( + &self, + number: &NumberFor, + hash: &Block::Hash, + ) -> sp_blockchain::Result { Ok(self.storage.read().hashes.get(number) == Some(hash)) } diff --git a/client/consensus/babe/src/lib.rs b/client/consensus/babe/src/lib.rs index 66d1bbb712959..f01a1f0810449 100644 --- a/client/consensus/babe/src/lib.rs +++ b/client/consensus/babe/src/lib.rs @@ -1393,7 +1393,11 @@ fn prune_finalized( }; if prune_light { - let is_canonical = |hash: &Block::Hash, number: &NumberFor, finalized_number: &NumberFor| { + let is_canonical = | + hash: &Block::Hash, + number: &NumberFor, + finalized_number: &NumberFor, + | { Ok(number <= finalized_number && client.pruned_header_was_canonical(number, hash)?) }; let clean_up = |number: &NumberFor| { diff --git a/client/db/src/lib.rs b/client/db/src/lib.rs index 531c8cebd5a31..635e2e9d1589c 100644 --- a/client/db/src/lib.rs +++ b/client/db/src/lib.rs @@ -459,7 +459,11 @@ impl sc_client_api::blockchain::HeaderBackend for Blockcha }) } - fn pruned_header_was_canonical(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + fn pruned_header_was_canonical( + &self, + number: &NumberFor, + hash: &Block::Hash, + ) -> sp_blockchain::Result { let lookup_key = utils::block_id_to_lookup_key::( &*self.db, columns::KEY_LOOKUP, diff --git a/client/db/src/light.rs b/client/db/src/light.rs index d3bbe96009d05..f0891bc08adc6 100644 --- a/client/db/src/light.rs +++ b/client/db/src/light.rs @@ -198,7 +198,11 @@ impl BlockchainHeaderBackend for LightStorage } fn pruned_header_was_canonical(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { - let lookup_key = utils::block_id_to_lookup_key::(&*self.db, columns::KEY_LOOKUP, BlockId::Number(number.clone()))?; + let lookup_key = utils::block_id_to_lookup_key::( + &*self.db, + columns::KEY_LOOKUP, + BlockId::Number(number.clone()), + )?; Ok(if let Some(lookup_key) = lookup_key { utils::lookup_key_to_hash(lookup_key.as_ref())? == hash.as_ref() } else { diff --git a/client/light/src/blockchain.rs b/client/light/src/blockchain.rs index e8f57c2c050e6..992a52ff59791 100644 --- a/client/light/src/blockchain.rs +++ b/client/light/src/blockchain.rs @@ -87,7 +87,11 @@ impl BlockchainHeaderBackend for Blockchain where Block: Blo self.storage.hash(number) } - fn pruned_header_was_canonical(&self, number: &NumberFor, hash: &Block::Hash) -> ClientResult { + fn pruned_header_was_canonical( + &self, + number: &NumberFor, + hash: &Block::Hash, + ) -> ClientResult { self.storage.pruned_header_was_canonical(number, hash) } diff --git a/client/service/src/client/client.rs b/client/service/src/client/client.rs index 55ee757ddcd0f..ba36b1f4e88f4 100644 --- a/client/service/src/client/client.rs +++ b/client/service/src/client/client.rs @@ -1549,7 +1549,11 @@ impl ChainHeaderBackend for Client wher self.backend.blockchain().hash(number) } - fn pruned_header_was_canonical(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + fn pruned_header_was_canonical( + &self, + number: &NumberFor, + hash: &Block::Hash, + ) -> sp_blockchain::Result { self.backend.blockchain().pruned_header_was_canonical(number, hash) } @@ -1601,7 +1605,11 @@ impl ChainHeaderBackend for &Client whe (**self).hash(number) } - fn pruned_header_was_canonical(&self, number: &NumberFor, hash: &Block::Hash) -> sp_blockchain::Result { + fn pruned_header_was_canonical( + &self, + number: &NumberFor, + hash: &Block::Hash, + ) -> sp_blockchain::Result { (**self).pruned_header_was_canonical(number, hash) } diff --git a/primitives/blockchain/src/backend.rs b/primitives/blockchain/src/backend.rs index ed6b3bb62be98..2856836fe093a 100644 --- a/primitives/blockchain/src/backend.rs +++ b/primitives/blockchain/src/backend.rs @@ -80,7 +80,11 @@ pub trait HeaderBackend: Send + Sync { } /// Was this header stored a canonical (partially pruned). - fn pruned_header_was_canonical(&self, number: &NumberFor, hash: &Block::Hash) -> Result; + fn pruned_header_was_canonical( + &self, + number: &NumberFor, + hash: &Block::Hash, + ) -> Result; /// Cleanup partially pruned header. fn pruned_header_clean_up(&self, number: &NumberFor) -> Result<()>; diff --git a/utils/fork-tree/src/lib.rs b/utils/fork-tree/src/lib.rs index 6aee0e857ae85..fa9eaecbf4e7a 100644 --- a/utils/fork-tree/src/lib.rs +++ b/utils/fork-tree/src/lib.rs @@ -192,8 +192,8 @@ impl ForkTree where /// Prune the tree, removing all non-canonical nodes. /// We remove roots that are non-canonical. /// The given function `is_canonical` should return `true` if the given block - /// (hash and number input, and last finalized nubmer) is canonical, this only apply for block less than - /// the finalized one. + /// (hash and number input, and last finalized nubmer) is canonical. + /// This only apply for block less than the finalized one. /// It apply recursively a clean up call back on branch. /// /// Returns all pruned node data.