diff --git a/bin/node-template/node/src/service.rs b/bin/node-template/node/src/service.rs index 3de31dc61ab51..76ed61061e4ec 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; @@ -228,8 +228,12 @@ pub fn new_full(config: Configuration) -> Result { /// Builds a new service for a light client. pub fn new_light(config: Configuration) -> Result { + let shared_pruning_requirements = SharedPruningRequirements::default(); let (client, backend, keystore, mut task_manager, on_demand) = - sc_service::new_light_parts::(&config)?; + 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 b15ace6181a8f..e48f2445b7c84 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; @@ -86,6 +86,7 @@ pub fn new_partial(config: &Configuration) -> Result Result<( Arc::Hash>>, Arc>> ), ServiceError> { - let (client, backend, keystore, mut task_manager, on_demand) = - 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()); @@ -384,6 +395,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 47fec977f5e82..9ff49f393475f 100644 --- a/client/api/src/backend.rs +++ b/client/api/src/backend.rs @@ -537,6 +537,15 @@ pub fn changes_tries_state_at_block<'a, Block: BlockT>( } } +/// Pruning requirement to share between multiple client component. +#[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, +} + /// Provide CHT roots. These are stored on a light client and generated dynamically on a full /// client. pub trait ProvideChtRoots { diff --git a/client/api/src/in_mem.rs b/client/api/src/in_mem.rs index ded030fb8046f..b3a41e3557432 100644 --- a/client/api/src/in_mem.rs +++ b/client/api/src/in_mem.rs @@ -339,6 +339,19 @@ impl HeaderBackend for Blockchain { fn hash(&self, number: <::Header as HeaderT>::Number) -> sp_blockchain::Result> { Ok(self.id(BlockId::Number(number))) } + + fn pruned_header_was_canonical( + &self, + number: &NumberFor, + hash: &Block::Hash, + ) -> sp_blockchain::Result { + Ok(self.storage.read().hashes.get(number) == Some(hash)) + } + + fn pruned_header_clean_up(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + self.storage.write().hashes.remove(number); + Ok(()) + } } impl HeaderMetadata for Blockchain { diff --git a/client/authority-discovery/src/worker/tests.rs b/client/authority-discovery/src/worker/tests.rs index f7b7dc41fee40..36af5ae9e5264 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 pruned_header_was_canonical( + &self, + _number: &NumberFor, + _hash: &Block::Hash, + ) -> std::result::Result { + Ok(false) + } + + fn pruned_header_clean_up( + &self, + _number: &NumberFor, + ) -> std::result::Result<(), sp_blockchain::Error> { + Ok(()) + } } pub(crate) struct RuntimeApi { 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 95f1653d8646d..f01a1f0810449 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, }; use sp_block_builder::BlockBuilder as BlockBuilderApi; use futures::channel::mpsc::{channel, Sender, Receiver}; @@ -1080,6 +1081,7 @@ pub struct BabeBlockImport { client: Arc, epoch_changes: SharedEpochChanges, config: Config, + light_pruning: bool, } impl Clone for BabeBlockImport { @@ -1089,6 +1091,7 @@ impl Clone for BabeBlockImport BabeBlockImport { epoch_changes: SharedEpochChanges, block_import: I, config: Config, + shared_pruning_requirements: Option<&SharedPruningRequirements>, ) -> Self { + let light_pruning = shared_pruning_requirements.is_some(); BabeBlockImport { client, inner: block_import, epoch_changes, config, + light_pruning, } } } @@ -1278,6 +1284,7 @@ impl 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 { let info = client.info(); @@ -1383,16 +1392,43 @@ 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.pruned_header_was_canonical(number, hash)?) + }; + let clean_up = |number: &NumberFor| { + client.pruned_header_clean_up(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. /// @@ -1402,8 +1438,9 @@ pub fn block_import( config: Config, wrapped_block_import: I, client: Arc, + shared_pruning_requirements: Option<&SharedPruningRequirements>, ) -> ClientResult<(BabeBlockImport, BabeLink)> where - Client: AuxStore + HeaderBackend + HeaderMetadata, + Client: AuxStore + HeaderBackend + HeaderMetadata { let epoch_changes = aux_schema::load_epoch_changes::(&*client, &config)?; let link = BabeLink { @@ -1418,6 +1455,7 @@ pub fn block_import( prune_finalized( client.clone(), &mut epoch_changes.lock(), + shared_pruning_requirements.is_some(), )?; let import = BabeBlockImport::new( @@ -1425,6 +1463,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 87876be8ae456..6ee967f7d9b8a 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..73a78a706d364 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_non_cannonical( + 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 8196a750557a8..72371f4350104 100644 --- a/client/db/src/lib.rs +++ b/client/db/src/lib.rs @@ -448,6 +448,36 @@ impl sc_client_api::blockchain::HeaderBackend for Blockcha None => Ok(None), }) } + + 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() + } else { + false + }) + } + + fn pruned_header_clean_up(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + 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 { diff --git a/client/db/src/light.rs b/client/db/src/light.rs index acfb6217ce9e0..f0891bc08adc6 100644 --- a/client/db/src/light.rs +++ b/client/db/src/light.rs @@ -27,7 +27,7 @@ use sc_client_api::{ blockchain::{ BlockStatus, Cache as BlockchainCache, Info as BlockchainInfo, }, - Storage, + Storage, SharedPruningRequirements, }; use sp_blockchain::{ CachedHeaderMetadata, HeaderMetadata, HeaderMetadataCache, @@ -65,6 +65,7 @@ pub struct LightStorage { meta: RwLock, Block::Hash>>, cache: Arc>, header_metadata_cache: Arc>, + keep_canonical_mapping: bool, #[cfg(not(target_os = "unknown"))] io_stats: FrozenForDuration, @@ -72,19 +73,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 +105,7 @@ impl LightStorage { ComplexBlockId::new(meta.finalized_hash, meta.finalized_number), ); + let keep_canonical_mapping = shared_pruning_requirements.need_mapping_for_light_pruning; Ok(LightStorage { db, meta: RwLock::new(meta), @@ -104,6 +113,7 @@ impl LightStorage { header_metadata_cache, #[cfg(not(target_os = "unknown"))] io_stats: FrozenForDuration::new(std::time::Duration::from_secs(1)), + keep_canonical_mapping, }) } @@ -186,6 +196,32 @@ impl BlockchainHeaderBackend for LightStorage fn hash(&self, number: NumberFor) -> ClientResult> { Ok(self.header(BlockId::Number(number))?.map(|header| header.hash().clone())) } + + 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() + } else { + false + }) + } + + fn pruned_header_clean_up(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + 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 { @@ -253,7 +289,7 @@ impl LightStorage { utils::remove_number_to_key_mapping( transaction, columns::KEY_LOOKUP, - retracted.number + retracted.number, )?; } @@ -345,12 +381,22 @@ impl LightStorage { 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 - )?; + + 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(); @@ -742,7 +788,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); @@ -759,7 +805,7 @@ pub(crate) mod tests { (Arc>, LightStorage) { 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 cht_size: u64 = cht::size(); let ucht_size: usize = cht_size as _; @@ -1142,7 +1188,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); } @@ -1203,7 +1249,7 @@ 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/db/src/utils.rs b/client/db/src/utils.rs index e999469c18ff0..94be3337288d9 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, @@ -144,6 +153,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 +172,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/blockchain.rs b/client/light/src/blockchain.rs index 3b5753f2849d5..992a52ff59791 100644 --- a/client/light/src/blockchain.rs +++ b/client/light/src/blockchain.rs @@ -86,6 +86,18 @@ impl BlockchainHeaderBackend for Blockchain where Block: Blo fn hash(&self, number: <::Header as HeaderT>::Number) -> ClientResult> { self.storage.hash(number) } + + fn pruned_header_was_canonical( + &self, + number: &NumberFor, + hash: &Block::Hash, + ) -> ClientResult { + self.storage.pruned_header_was_canonical(number, hash) + } + + fn pruned_header_clean_up(&self, number: &NumberFor) -> ClientResult<()> { + self.storage.pruned_header_clean_up(number) + } } impl HeaderMetadata for Blockchain where Block: BlockT, S: Storage { diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index 410198af26da3..c882963e3a497 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; @@ -278,12 +279,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 task_manager = { let registry = config.prometheus_config.as_ref().map(|cfg| &cfg.registry); TaskManager::new(config.task_executor.clone(), registry)? @@ -311,7 +312,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)? }; let light_blockchain = sc_light::new_light_blockchain(db_storage); let fetch_checker = Arc::new( diff --git a/client/service/src/client/client.rs b/client/service/src/client/client.rs index fd76084988dbc..d1173dfd3e3f3 100644 --- a/client/service/src/client/client.rs +++ b/client/service/src/client/client.rs @@ -1549,6 +1549,18 @@ impl ChainHeaderBackend for Client wher fn hash(&self, number: NumberFor) -> sp_blockchain::Result> { self.backend.blockchain().hash(number) } + + fn pruned_header_was_canonical( + &self, + number: &NumberFor, + hash: &Block::Hash, + ) -> sp_blockchain::Result { + self.backend.blockchain().pruned_header_was_canonical(number, hash) + } + + fn pruned_header_clean_up(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + self.backend.blockchain().pruned_header_clean_up(number) + } } impl sp_runtime::traits::BlockIdTo for Client where @@ -1593,6 +1605,18 @@ impl ChainHeaderBackend for &Client whe fn hash(&self, number: NumberFor) -> sp_blockchain::Result> { (**self).hash(number) } + + fn pruned_header_was_canonical( + &self, + number: &NumberFor, + hash: &Block::Hash, + ) -> sp_blockchain::Result { + (**self).pruned_header_was_canonical(number, hash) + } + + fn pruned_header_clean_up(&self, number: &NumberFor) -> sp_blockchain::Result<()> { + (**self).pruned_header_clean_up(number) + } } impl ProvideCache for Client where diff --git a/client/service/test/src/client/light.rs b/client/service/test/src/client/light.rs index f38aef008e11c..38270da02ec0d 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; @@ -109,6 +109,18 @@ impl sp_blockchain::HeaderBackend for DummyStorage { Err(ClientError::Backend("Test error".into())) } } + + fn pruned_header_was_canonical( + &self, + _number: &NumberFor, + _hash: &::Hash, + ) -> sp_blockchain::Result { + Err(ClientError::Backend("Test error".into())) + } + + fn pruned_header_clean_up(&self, _number: &NumberFor) -> sp_blockchain::Result<()> { + Err(ClientError::Backend("Test error".into())) + } } impl sp_blockchain::HeaderMetadata for DummyStorage { diff --git a/primitives/blockchain/src/backend.rs b/primitives/blockchain/src/backend.rs index 1328dfb5752fc..2856836fe093a 100644 --- a/primitives/blockchain/src/backend.rs +++ b/primitives/blockchain/src/backend.rs @@ -78,6 +78,16 @@ pub trait HeaderBackend: Send + Sync { Error::UnknownBlock(format!("Expect block hash from id: {}", id)) )) } + + /// Was this header stored a canonical (partially pruned). + 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<()>; } /// Blockchain database backend. Does not perform any validation. diff --git a/utils/fork-tree/src/lib.rs b/utils/fork-tree/src/lib.rs index 1d01c53417649..fa9eaecbf4e7a 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 @@ -166,6 +188,48 @@ impl ForkTree where Ok(RemovedIterator { stack: removed }) } + + /// 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_non_cannonical( + &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_canonical( + hash, + number, + is_canonical, + predicate, + )?; + + let result = self.prune_inner( + new_root_index, + hash, + number, + is_canonical, + )?; + + for node in result.stack.iter() { + node.clean_up(clean_up)?; + } + + Ok(result) + } } impl ForkTree where @@ -358,6 +422,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_canonical( + &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_canonical(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. @@ -754,18 +845,47 @@ mod node_implementation { where E: std::error::Error, F: Fn(&H, &H) -> Result, P: Fn(&V) -> bool, + { + 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_canonical( + &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_descendent_of = 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(hash, number, is_descendent_of, predicate)? { + match node.find_node_index_where_canonical( + hash, + number, + is_canonical, + predicate, + )? { FindOutcome::Abort => return Ok(FindOutcome::Abort), FindOutcome::Found(mut x) => { x.push(i); @@ -775,19 +895,15 @@ mod node_implementation { // 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; + known_canonical = 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 { + 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())); @@ -796,7 +912,7 @@ mod node_implementation { // otherwise, tell our ancestor that we failed, and whether // the block was a descendent. - Ok(FindOutcome::Failure(is_descendent_of)) + Ok(FindOutcome::Failure(is_canonical)) } /// Find a node in the tree that is the deepest ancestor of the given @@ -862,6 +978,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(()) + } } }