diff --git a/Cargo.lock b/Cargo.lock index a7fdd2b4c9161..7371e1c2bff1b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1478,6 +1478,7 @@ dependencies = [ "sc-executor", "sc-service", "sp-runtime", + "sp-state-machine", "structopt", ] @@ -5638,6 +5639,8 @@ dependencies = [ "sp-core", "sp-runtime", "sp-state-machine", + "sp-trie", + "substrate-test-runtime-client", ] [[package]] @@ -6941,6 +6944,7 @@ dependencies = [ "sp-api", "sp-blockchain", "sp-consensus", + "sp-core", "sp-runtime", "sp-state-machine", "sp-version", diff --git a/bin/node/executor/benches/bench.rs b/bin/node/executor/benches/bench.rs index 034c7c6759e11..c411bec851c0d 100644 --- a/bin/node/executor/benches/bench.rs +++ b/bin/node/executor/benches/bench.rs @@ -25,8 +25,8 @@ use node_runtime::constants::currency::*; use node_testing::keyring::*; use sp_core::{Blake2Hasher, NativeOrEncoded, NeverNativeValue}; use sp_core::storage::well_known_keys; -use sp_core::traits::CodeExecutor; -use frame_support::Hashable; +use sp_core::traits::{CodeExecutor, RuntimeCode}; +use frame_support::Hashable; use sp_state_machine::TestExternalities as CoreTestExternalities; use sc_executor::{NativeExecutor, RuntimeInfo, WasmExecutionMethod, Externalities}; @@ -89,9 +89,12 @@ fn construct_block( digest: Default::default(), }; + let runtime_code = RuntimeCode::from_externalities(ext).expect("`ext` provides `:code`"); + // execute the block to get the real header. executor.call::<_, NeverNativeValue, fn() -> _>( ext, + &runtime_code, "Core_initialize_block", &header.encode(), true, @@ -101,6 +104,7 @@ fn construct_block( for i in extrinsics.iter() { executor.call::<_, NeverNativeValue, fn() -> _>( ext, + &runtime_code, "BlockBuilder_apply_extrinsic", &i.encode(), true, @@ -110,6 +114,7 @@ fn construct_block( let header = match executor.call::<_, NeverNativeValue, fn() -> _>( ext, + &runtime_code, "BlockBuilder_finalize_block", &[0u8;0], true, @@ -165,7 +170,9 @@ fn bench_execute_block(c: &mut Criterion) { // Get the runtime version to initialize the runtimes cache. { let mut test_ext = new_test_ext(&genesis_config); - executor.runtime_version(&mut test_ext.ext()); + let runtime_code = RuntimeCode::from_externalities(&test_ext.ext()) + .expect("`test_ext` provides `:code`"); + executor.runtime_version(&mut test_ext.ext(), &runtime_code).unwrap(); } let blocks = test_blocks(&genesis_config, &executor); @@ -173,9 +180,12 @@ fn bench_execute_block(c: &mut Criterion) { b.iter_batched_ref( || new_test_ext(&genesis_config), |test_ext| { + let runtime_code = RuntimeCode::from_externalities(&test_ext.ext()) + .expect("`test_ext` provides `:code`"); for block in blocks.iter() { executor.call::<_, NeverNativeValue, fn() -> _>( &mut test_ext.ext(), + &runtime_code, "Core_execute_block", &block.0, use_native, diff --git a/bin/node/executor/tests/common.rs b/bin/node/executor/tests/common.rs index 090d2ee5d4ace..4b83825722283 100644 --- a/bin/node/executor/tests/common.rs +++ b/bin/node/executor/tests/common.rs @@ -18,8 +18,7 @@ use codec::{Encode, Decode}; use frame_support::Hashable; use sp_state_machine::TestExternalities as CoreTestExternalities; use sp_core::{ - Blake2Hasher, NeverNativeValue, NativeOrEncoded, - traits::CodeExecutor, + Blake2Hasher, NeverNativeValue, NativeOrEncoded, traits::{CodeExecutor, RuntimeCode}, }; use sp_runtime::{ApplyExtrinsicResult, traits::Header as HeaderT}; use sc_executor::{NativeExecutor, WasmExecutionMethod}; @@ -74,8 +73,11 @@ pub fn executor_call< native_call: Option, ) -> (Result>, bool) { let mut t = t.ext(); + let runtime_code = RuntimeCode::from_externalities(&t) + .expect("Code should be part of the externalities"); executor().call::<_, R, NC>( &mut t, + &runtime_code, method, data, use_native, diff --git a/client/basic-authorship/src/basic_authorship.rs b/client/basic-authorship/src/basic_authorship.rs index 231d0255919b4..a816a5498f853 100644 --- a/client/basic-authorship/src/basic_authorship.rs +++ b/client/basic-authorship/src/basic_authorship.rs @@ -231,7 +231,8 @@ impl ProposerInner debug!("[{:?}] Pushed to the block.", pending_tx_hash); } Err(sp_blockchain::Error::ApplyExtrinsicFailed(sp_blockchain::ApplyExtrinsicFailed::Validity(e))) - if e.exhausted_resources() => { + if e.exhausted_resources() => + { if is_first { debug!("[{:?}] Invalid transaction: FullBlock on empty block", pending_tx_hash); unqueue_invalid.push(pending_tx_hash); diff --git a/client/block-builder/Cargo.toml b/client/block-builder/Cargo.toml index 745669c033e38..dd4ebcb07f889 100644 --- a/client/block-builder/Cargo.toml +++ b/client/block-builder/Cargo.toml @@ -19,3 +19,7 @@ sp-core = { version = "2.0.0-alpha.2", path = "../../primitives/core" } sp-block-builder = { version = "2.0.0-alpha.2", path = "../../primitives/block-builder" } sc-client-api = { version = "2.0.0-alpha.2", path = "../api" } codec = { package = "parity-scale-codec", version = "1.2.0", features = ["derive"] } + +[dev-dependencies] +substrate-test-runtime-client = { path = "../../test-utils/runtime/client" } +sp-trie = { version = "2.0.0-alpha.2", path = "../../primitives/trie" } diff --git a/client/block-builder/src/lib.rs b/client/block-builder/src/lib.rs index 4c14f3716f210..c9b2efbea9ab4 100644 --- a/client/block-builder/src/lib.rs +++ b/client/block-builder/src/lib.rs @@ -152,12 +152,20 @@ where /// Push onto the block's list of extrinsics. /// - /// This will treat incoming extrinsic `xt` as trusted and skip signature check (for signed transactions). - pub fn push_trusted(&mut self, xt: ::Extrinsic) -> Result<(), ApiErrorFor> { + /// This will treat incoming extrinsic `xt` as trusted and skip signature check + /// (for signed transactions). + pub fn push_trusted( + &mut self, + xt: ::Extrinsic, + ) -> Result<(), ApiErrorFor> { self.push_internal(xt, true) } - fn push_internal(&mut self, xt: ::Extrinsic, skip_signature: bool) -> Result<(), ApiErrorFor> { + fn push_internal( + &mut self, + xt: ::Extrinsic, + skip_signature: bool, + ) -> Result<(), ApiErrorFor> { let block_id = &self.block_id; let extrinsics = &mut self.extrinsics; @@ -175,7 +183,7 @@ where ExecutionContext::BlockConstruction, xt.clone(), )? - } else { + } else { api.apply_extrinsic_with_context( block_id, ExecutionContext::BlockConstruction, @@ -241,3 +249,41 @@ where }) } } + +#[cfg(test)] +mod tests { + use super::*; + use sp_blockchain::HeaderBackend; + use sp_core::Blake2Hasher; + use sp_state_machine::Backend; + use substrate_test_runtime_client::{DefaultTestClientBuilderExt, TestClientBuilderExt}; + + #[test] + fn block_building_storage_proof_does_not_include_runtime_by_default() { + let builder = substrate_test_runtime_client::TestClientBuilder::new(); + let backend = builder.backend(); + let client = builder.build(); + + let block = BlockBuilder::new( + &client, + client.info().best_hash, + client.info().best_number, + RecordProof::Yes, + Default::default(), + &*backend, + ).unwrap().build().unwrap(); + + let proof = block.proof.expect("Proof is build on request"); + + let backend = sp_state_machine::create_proof_check_backend::( + block.storage_changes.transaction_storage_root, + proof, + ).unwrap(); + + assert!( + backend.storage(&sp_core::storage::well_known_keys::CODE) + .unwrap_err() + .contains("Database missing expected key"), + ); + } +} diff --git a/client/executor/src/lib.rs b/client/executor/src/lib.rs index af53ed91838cc..f8761caa98394 100644 --- a/client/executor/src/lib.rs +++ b/client/executor/src/lib.rs @@ -110,8 +110,12 @@ pub trait RuntimeInfo { /// Native runtime information. fn native_version(&self) -> &NativeVersion; - /// Extract RuntimeVersion of given :code block - fn runtime_version (&self, ext: &mut E) -> error::Result; + /// Extract [`RuntimeVersion`](sp_version::RuntimeVersion) of the given `runtime_code`. + fn runtime_version( + &self, + ext: &mut E, + runtime_code: &sp_core::traits::RuntimeCode, + ) -> error::Result; } #[cfg(test)] diff --git a/client/executor/src/native_executor.rs b/client/executor/src/native_executor.rs index 1364b753dbeeb..5a41b6597171c 100644 --- a/client/executor/src/native_executor.rs +++ b/client/executor/src/native_executor.rs @@ -20,7 +20,7 @@ use crate::{ }; use sp_version::{NativeVersion, RuntimeVersion}; use codec::{Decode, Encode}; -use sp_core::{NativeOrEncoded, traits::{CodeExecutor, Externalities}}; +use sp_core::{NativeOrEncoded, traits::{CodeExecutor, Externalities, RuntimeCode}}; use log::trace; use std::{result, cell::RefCell, panic::{UnwindSafe, AssertUnwindSafe}, sync::Arc}; use sp_wasm_interface::{HostFunctions, Function}; @@ -130,6 +130,7 @@ impl NativeExecutor { fn with_runtime( &self, ext: &mut E, + runtime_code: &RuntimeCode, f: impl for<'a> FnOnce( AssertUnwindSafe<&'a mut (dyn WasmRuntime + 'static)>, &'a RuntimeVersion, @@ -138,8 +139,9 @@ impl NativeExecutor { ) -> Result where E: Externalities { RUNTIMES_CACHE.with(|cache| { let mut cache = cache.borrow_mut(); - let (runtime, version, code_hash) = cache.fetch_runtime( + let (runtime, version) = cache.fetch_runtime( ext, + runtime_code, self.fallback_method, self.default_heap_pages, &*self.host_functions, @@ -151,7 +153,7 @@ impl NativeExecutor { match f(runtime, version, ext) { Ok(res) => res, Err(e) => { - cache.invalidate_runtime(self.fallback_method, code_hash); + cache.invalidate_runtime(self.fallback_method, runtime_code.hash.clone()); Err(e) } } @@ -179,8 +181,9 @@ impl RuntimeInfo for NativeExecutor { fn runtime_version( &self, ext: &mut E, + runtime_code: &RuntimeCode, ) -> Result { - self.with_runtime(ext, |_runtime, version, _ext| Ok(Ok(version.clone()))) + self.with_runtime(ext, runtime_code, |_runtime, version, _ext| Ok(Ok(version.clone()))) } } @@ -195,13 +198,14 @@ impl CodeExecutor for NativeExecutor { >( &self, ext: &mut E, + runtime_code: &RuntimeCode, method: &str, data: &[u8], use_native: bool, native_call: Option, - ) -> (Result>, bool){ + ) -> (Result>, bool) { let mut used_native = false; - let result = self.with_runtime(ext, |mut runtime, onchain_version, mut ext| { + let result = self.with_runtime(ext, runtime_code, |mut runtime, onchain_version, mut ext| { match ( use_native, onchain_version.can_call_with(&self.native_version.runtime_version), diff --git a/client/executor/src/wasm_runtime.rs b/client/executor/src/wasm_runtime.rs index 9d54246ee0763..ab7b219dbf867 100644 --- a/client/executor/src/wasm_runtime.rs +++ b/client/executor/src/wasm_runtime.rs @@ -22,7 +22,7 @@ use crate::error::{Error, WasmError}; use log::{trace, warn}; use codec::Decode; -use sp_core::{storage::well_known_keys, traits::Externalities}; +use sp_core::traits::{Externalities, RuntimeCode}; use sp_version::RuntimeVersion; use std::{collections::hash_map::{Entry, HashMap}, panic::AssertUnwindSafe}; use sc_executor_common::wasm_runtime::WasmRuntime; @@ -86,8 +86,9 @@ impl RuntimesCache { /// /// # Parameters /// - /// `ext` - Externalities to use for the runtime. This is used for setting - /// up an initial runtime instance. + /// `ext` - Externalities to use for the getting the runtime's version call. + /// + /// `runtime_code` - The runtime wasm code used setup the runtime. /// /// `default_heap_pages` - Number of 64KB pages to allocate for Wasm execution. /// @@ -95,8 +96,8 @@ impl RuntimesCache { /// /// # Return value /// - /// If no error occurred a tuple `(&mut WasmRuntime, H256)` is - /// returned. `H256` is the hash of the runtime code. + /// If no error occurred a tuple `(&mut WasmRuntime, RuntimeVerion)` is + /// returned. /// /// In case of failure one of two errors can be returned: /// @@ -107,20 +108,14 @@ impl RuntimesCache { pub fn fetch_runtime( &mut self, ext: &mut E, + runtime_code: &RuntimeCode, wasm_method: WasmExecutionMethod, default_heap_pages: u64, host_functions: &[&'static dyn Function], - ) -> Result<(&mut (dyn WasmRuntime + 'static), &RuntimeVersion, Vec), Error> { - let code_hash = ext - .original_storage_hash(well_known_keys::CODE) - .ok_or(Error::InvalidCode("`CODE` not found in storage.".into()))?; - - let heap_pages = ext - .storage(well_known_keys::HEAP_PAGES) - .and_then(|pages| u64::decode(&mut &pages[..]).ok()) - .unwrap_or(default_heap_pages); + ) -> Result<(&mut (dyn WasmRuntime + 'static), &RuntimeVersion), Error> { + let heap_pages = runtime_code.heap_pages.unwrap_or(default_heap_pages); - let result = match self.instances.entry((wasm_method, code_hash.clone())) { + let result = match self.instances.entry((wasm_method, runtime_code.hash.clone())) { Entry::Occupied(o) => { let result = o.into_mut(); if let Ok(ref mut cached_runtime) = result { @@ -142,6 +137,7 @@ impl RuntimesCache { *result = create_versioned_wasm_runtime( ext, wasm_method, + runtime_code, heap_pages, host_functions.into(), ); @@ -157,6 +153,7 @@ impl RuntimesCache { let result = create_versioned_wasm_runtime( ext, wasm_method, + runtime_code, heap_pages, host_functions.into(), ); @@ -168,7 +165,7 @@ impl RuntimesCache { }; result.as_mut() - .map(|entry| (entry.runtime.as_mut(), &entry.version, code_hash)) + .map(|entry| (entry.runtime.as_mut(), &entry.version)) .map_err(|ref e| Error::InvalidCode(format!("{:?}", e))) } @@ -209,13 +206,17 @@ pub fn create_wasm_runtime_with_code( fn create_versioned_wasm_runtime( ext: &mut E, wasm_method: WasmExecutionMethod, + runtime_code: &RuntimeCode, heap_pages: u64, host_functions: Vec<&'static dyn Function>, ) -> Result { - let code = ext - .original_storage(well_known_keys::CODE) - .ok_or(WasmError::CodeNotFound)?; - let mut runtime = create_wasm_runtime_with_code(wasm_method, heap_pages, &code, host_functions, false)?; + let mut runtime = create_wasm_runtime_with_code( + wasm_method, + heap_pages, + &runtime_code.code, + host_functions, + false, + )?; // Call to determine runtime version. let version_result = { diff --git a/client/network/src/chain.rs b/client/network/src/chain.rs index b991a0e65208c..e85b5af79b95a 100644 --- a/client/network/src/chain.rs +++ b/client/network/src/chain.rs @@ -62,7 +62,12 @@ pub trait Client: Send + Sync { ) -> Result; /// Get method execution proof. - fn execution_proof(&self, block: &Block::Hash, method: &str, data: &[u8]) -> Result<(Vec, StorageProof), Error>; + fn execution_proof( + &self, + block: &Block::Hash, + method: &str, + data: &[u8], + ) -> Result<(Vec, StorageProof), Error>; /// Get key changes proof. fn key_changes_proof( @@ -152,11 +157,7 @@ impl Client for SubstrateClient where method: &str, data: &[u8], ) -> Result<(Vec, StorageProof), Error> { - (self as &SubstrateClient).execution_proof( - &BlockId::Hash(block.clone()), - method, - data, - ) + SubstrateClient::execution_proof(self, &BlockId::Hash(block.clone()), method, data) } fn key_changes_proof( @@ -168,7 +169,14 @@ impl Client for SubstrateClient where storage_key: Option<&StorageKey>, key: &StorageKey, ) -> Result, Error> { - (self as &SubstrateClient).key_changes_proof(first, last, min, max, storage_key, key) + (self as &SubstrateClient).key_changes_proof( + first, + last, + min, + max, + storage_key, + key, + ) } fn is_descendent_of(&self, base: &Block::Hash, block: &Block::Hash) -> Result { diff --git a/client/network/src/protocol/light_client_handler.rs b/client/network/src/protocol/light_client_handler.rs index b531f3515a6ec..e4996b7c8b565 100644 --- a/client/network/src/protocol/light_client_handler.rs +++ b/client/network/src/protocol/light_client_handler.rs @@ -254,13 +254,12 @@ where B: Block, { /// Construct a new light client handler. - pub fn new - ( cfg: Config - , chain: Arc> - , checker: Arc> - , peerset: sc_peerset::PeersetHandle - ) -> Self - { + pub fn new( + cfg: Config, + chain: Arc>, + checker: Arc>, + peerset: sc_peerset::PeersetHandle, + ) -> Self { LightClientHandler { config: cfg, chain, @@ -425,7 +424,8 @@ where log::trace!("remote call request from {} ({} at {:?})", peer, request.method, - request.block); + request.block, + ); let block = Decode::decode(&mut request.block.as_ref())?; @@ -436,7 +436,8 @@ where peer, request.method, request.block, - e); + e, + ); StorageProof::empty() } }; diff --git a/client/network/src/protocol/light_dispatch.rs b/client/network/src/protocol/light_dispatch.rs index aff220b6e03a6..f3da2c2a96c81 100644 --- a/client/network/src/protocol/light_dispatch.rs +++ b/client/network/src/protocol/light_dispatch.rs @@ -750,7 +750,11 @@ pub mod tests { } } - fn check_execution_proof(&self, _: &RemoteCallRequest, _: StorageProof) -> ClientResult> { + fn check_execution_proof( + &self, + _: &RemoteCallRequest, + _: StorageProof, + ) -> ClientResult> { match self.ok { true => Ok(vec![42]), false => Err(ClientError::Backend("Test error".into())), diff --git a/client/src/call_executor.rs b/client/src/call_executor.rs index 18ad5b113e983..659e01523997b 100644 --- a/client/src/call_executor.rs +++ b/client/src/call_executor.rs @@ -25,7 +25,7 @@ use sp_state_machine::{ }; use sc_executor::{RuntimeVersion, RuntimeInfo, NativeVersion}; use sp_externalities::Extensions; -use sp_core::{NativeOrEncoded, NeverNativeValue, traits::CodeExecutor}; +use sp_core::{NativeOrEncoded, NeverNativeValue, traits::{CodeExecutor, RuntimeCode}}; use sp_api::{ProofRecorder, InitializeBlock, StorageTransactionCache}; use sc_client_api::{backend, call_executor::CallExecutor}; @@ -90,6 +90,7 @@ where method, call_data, extensions.unwrap_or_default(), + &sp_state_machine::backend::get_runtime_code(&state)?, ).execute_using_consensus_failure_handler::<_, NeverNativeValue, fn() -> _>( strategy.get_manager(), None, @@ -140,6 +141,8 @@ where // make sure to destroy state before exiting this function let mut state = self.backend.state_at(*at)?; + let runtime_code = sp_state_machine::backend::get_runtime_code(&state)?; + let result = match recorder { Some(recorder) => state.as_trie_backend() .ok_or_else(|| @@ -160,6 +163,7 @@ where method, call_data, extensions.unwrap_or_default(), + &runtime_code, ) // TODO: https://github.com/paritytech/substrate/issues/4455 // .with_storage_transaction_cache(storage_transaction_cache.as_mut().map(|c| &mut **c)) @@ -173,6 +177,7 @@ where method, call_data, extensions.unwrap_or_default(), + &runtime_code, ) .with_storage_transaction_cache(storage_transaction_cache.as_mut().map(|c| &mut **c)) .execute_using_consensus_failure_handler(execution_manager, native_call) @@ -197,7 +202,8 @@ where changes_trie_state, None, ); - let version = self.executor.runtime_version(&mut ext); + let wasm_code = RuntimeCode::from_externalities(&ext).map_err(|e| e.to_string().into()); + let version = wasm_code.and_then(|c| self.executor.runtime_version(&mut ext, &c)); { let _lock = self.backend.get_import_lock().read(); self.backend.destroy_state(state)?; @@ -218,6 +224,7 @@ where &self.executor, method, call_data, + &sp_state_machine::backend::get_runtime_code(trie_state)?, ) .map_err(Into::into) } diff --git a/client/src/client.rs b/client/src/client.rs index d461a17ded75f..68970cbb2ac0f 100644 --- a/client/src/client.rs +++ b/client/src/client.rs @@ -41,8 +41,7 @@ use sp_runtime::{ use sp_state_machine::{ DBValue, Backend as StateBackend, ChangesTrieAnchorBlockId, prove_read, prove_child_read, ChangesTrieRootsStorage, ChangesTrieStorage, - ChangesTrieConfigurationRange, key_changes, key_changes_proof, StorageProof, - merge_storage_proofs, + ChangesTrieConfigurationRange, key_changes, key_changes_proof, }; use sc_executor::{RuntimeVersion, RuntimeInfo}; use sp_consensus::{ @@ -55,6 +54,7 @@ use sp_blockchain::{self as blockchain, well_known_cache_keys::Id as CacheKeyId, HeaderMetadata, CachedHeaderMetadata, }; +use sp_trie::StorageProof; use sp_api::{ CallApiAt, ConstructRuntimeApi, Core as CoreApi, ApiExt, ApiRef, ProvideRuntimeApi, @@ -482,9 +482,19 @@ impl Client where method: &str, call_data: &[u8] ) -> sp_blockchain::Result<(Vec, StorageProof)> { + // Make sure we include the `:code` and `:heap_pages` in the execution proof to be + // backwards compatible. + // + // TODO: Remove when solved: https://github.com/paritytech/substrate/issues/5047 + let code_proof = self.read_proof( + id, + &[well_known_keys::CODE.to_vec(), well_known_keys::HEAP_PAGES.to_vec()], + )?; + let state = self.state_at(id)?; let header = self.prepare_environment_block(id)?; prove_execution(state, header, &self.executor, method, call_data) + .map(|p| (p.0, StorageProof::merge(vec![p.1, code_proof]))) } /// Reads given header and generates CHT-based header proof. @@ -769,7 +779,7 @@ impl Client where Ok(()) }, ())?; - Ok(merge_storage_proofs(proofs)) + Ok(StorageProof::merge(proofs)) } /// Generates CHT-based proof for roots of changes tries at given blocks (that are part of single CHT). diff --git a/client/src/genesis.rs b/client/src/genesis.rs index fccdd71817e1f..006ed00c436c3 100644 --- a/client/src/genesis.rs +++ b/client/src/genesis.rs @@ -89,6 +89,8 @@ mod tests { }; let hash = header.hash(); let mut overlay = OverlayedChanges::default(); + let runtime_code = sp_state_machine::backend::get_runtime_code(&backend) + .expect("Code is part of the backend"); StateMachine::new( backend, @@ -98,6 +100,7 @@ mod tests { "Core_initialize_block", &header.encode(), Default::default(), + &runtime_code, ).execute( ExecutionStrategy::NativeElseWasm, ).unwrap(); @@ -111,6 +114,7 @@ mod tests { "BlockBuilder_apply_extrinsic", &tx.encode(), Default::default(), + &runtime_code, ).execute( ExecutionStrategy::NativeElseWasm, ).unwrap(); @@ -124,6 +128,7 @@ mod tests { "BlockBuilder_finalize_block", &[], Default::default(), + &runtime_code, ).execute( ExecutionStrategy::NativeElseWasm, ).unwrap(); @@ -161,6 +166,8 @@ mod tests { let backend = InMemoryBackend::from(storage); let (b1data, _b1hash) = block1(genesis_hash, &backend); + let runtime_code = sp_state_machine::backend::get_runtime_code(&backend) + .expect("Code is part of the backend"); let mut overlay = OverlayedChanges::default(); let _ = StateMachine::new( @@ -171,6 +178,7 @@ mod tests { "Core_execute_block", &b1data, Default::default(), + &runtime_code, ).execute( ExecutionStrategy::NativeElseWasm, ).unwrap(); @@ -189,6 +197,8 @@ mod tests { let backend = InMemoryBackend::from(storage); let (b1data, _b1hash) = block1(genesis_hash, &backend); + let runtime_code = sp_state_machine::backend::get_runtime_code(&backend) + .expect("Code is part of the backend"); let mut overlay = OverlayedChanges::default(); let _ = StateMachine::new( @@ -199,6 +209,7 @@ mod tests { "Core_execute_block", &b1data, Default::default(), + &runtime_code, ).execute( ExecutionStrategy::AlwaysWasm, ).unwrap(); @@ -217,6 +228,8 @@ mod tests { let backend = InMemoryBackend::from(storage); let (b1data, _b1hash) = block1(genesis_hash, &backend); + let runtime_code = sp_state_machine::backend::get_runtime_code(&backend) + .expect("Code is part of the backend"); let mut overlay = OverlayedChanges::default(); let r = StateMachine::new( @@ -227,6 +240,7 @@ mod tests { "Core_execute_block", &b1data, Default::default(), + &runtime_code, ).execute( ExecutionStrategy::NativeElseWasm, ); diff --git a/client/src/light/backend.rs b/client/src/light/backend.rs index ad9f43587e4cd..48434bb41d175 100644 --- a/client/src/light/backend.rs +++ b/client/src/light/backend.rs @@ -172,9 +172,9 @@ impl ClientBackend for Backend> match maybe_val { Some(val) => self.blockchain.storage().insert_aux( &[(&key[..], &val[..])], - ::std::iter::empty(), + std::iter::empty(), )?, - None => self.blockchain.storage().insert_aux(::std::iter::empty(), &[&key[..]])?, + None => self.blockchain.storage().insert_aux(std::iter::empty(), &[&key[..]])?, } } } diff --git a/client/src/light/call_executor.rs b/client/src/light/call_executor.rs index 01a93c78219bc..a440f5ad6795a 100644 --- a/client/src/light/call_executor.rs +++ b/client/src/light/call_executor.rs @@ -29,7 +29,6 @@ use sp_externalities::Extensions; use sp_state_machine::{ self, Backend as StateBackend, OverlayedChanges, ExecutionStrategy, create_proof_check_backend, execution_proof_check_on_trie_backend, ExecutionManager, StorageProof, - merge_storage_proofs, }; use hash_db::Hasher; @@ -206,7 +205,7 @@ pub fn prove_execution( method, call_data, )?; - let total_proof = merge_storage_proofs(vec![init_proof, exec_proof]); + let total_proof = StorageProof::merge(vec![init_proof, exec_proof]); Ok((result, total_proof)) } @@ -259,12 +258,17 @@ fn check_execution_proof_with_make_header( &trie_backend, &mut changes, executor, "Core_initialize_block", &next_header.encode(), + &runtime_code, )?; // execute method @@ -274,7 +278,9 @@ fn check_execution_proof_with_make_header> LightDataChecker { H::Out: Ord + codec::Codec, { // all the checks are sharing the same storage - let storage = create_proof_check_backend_storage(remote_roots_proof); + let storage = remote_roots_proof.into_memory_db(); // remote_roots.keys() are sorted => we can use this to group changes tries roots // that are belongs to the same CHT @@ -187,7 +187,8 @@ impl> LightDataChecker { local_cht_root, block, remote_changes_trie_root, - &proving_backend)?; + &proving_backend, + )?; // and return the storage to use in following checks storage = proving_backend.into_storage(); @@ -270,7 +271,7 @@ impl FetchChecker for LightDataChecker body: Vec ) -> ClientResult> { // TODO: #2621 - let extrinsics_root = HashFor::::ordered_trie_root( + let extrinsics_root = HashFor::::ordered_trie_root( body.iter().map(Encode::encode).collect(), ); if *request.header.extrinsics_root() == extrinsics_root { @@ -294,7 +295,7 @@ struct RootsStorage<'a, Number: AtLeast32Bit, Hash: 'a> { impl<'a, H, Number, Hash> ChangesTrieRootsStorage for RootsStorage<'a, Number, Hash> where H: Hasher, - Number: ::std::fmt::Display + ::std::hash::Hash + Clone + AtLeast32Bit + Encode + Decode + Send + Sync + 'static, + Number: std::fmt::Display + std::hash::Hash + Clone + AtLeast32Bit + Encode + Decode + Send + Sync + 'static, Hash: 'a + Send + Sync + Clone + AsRef<[u8]>, { fn build_anchor( diff --git a/frame/system/benches/bench.rs b/frame/system/benches/bench.rs index cfcaa6f64ac59..90a4ad1d34de8 100644 --- a/frame/system/benches/bench.rs +++ b/frame/system/benches/bench.rs @@ -18,7 +18,7 @@ use criterion::{Criterion, criterion_group, criterion_main, black_box}; use frame_system as system; use frame_support::{decl_module, decl_event, impl_outer_origin, impl_outer_event, weights::Weight}; use sp_core::H256; -use sp_runtime::{Perbill, PerThing, traits::{BlakeTwo256, IdentityLookup}, testing::Header}; +use sp_runtime::{Perbill, traits::{BlakeTwo256, IdentityLookup}, testing::Header}; mod module { use super::*; diff --git a/primitives/api/test/Cargo.toml b/primitives/api/test/Cargo.toml index 6d2207c178a22..3b41e28cf3b2b 100644 --- a/primitives/api/test/Cargo.toml +++ b/primitives/api/test/Cargo.toml @@ -24,6 +24,7 @@ rustversion = "1.0.0" [dev-dependencies] criterion = "0.3.0" substrate-test-runtime-client = { version = "2.0.0-dev", path = "../../../test-utils/runtime/client" } +sp-core = { version = "2.0.0-alpha.1", path = "../../core" } [[bench]] name = "bench" diff --git a/primitives/api/test/tests/runtime_calls.rs b/primitives/api/test/tests/runtime_calls.rs index 6a3af7469cc0e..64c20473d1c1b 100644 --- a/primitives/api/test/tests/runtime_calls.rs +++ b/primitives/api/test/tests/runtime_calls.rs @@ -164,6 +164,12 @@ fn record_proof_works() { let block_id = BlockId::Number(client.chain_info().best_number); let storage_root = longest_chain.best_chain().unwrap().state_root().clone(); + let runtime_code = sp_core::traits::RuntimeCode { + code: client.code_at(&block_id).unwrap(), + hash: vec![1], + heap_pages: None, + }; + let transaction = Transfer { amount: 1000, nonce: 0, @@ -192,5 +198,6 @@ fn record_proof_works() { &executor, "Core_execute_block", &block.encode(), + &runtime_code, ).expect("Executes block while using the proof backend"); } diff --git a/primitives/core/src/traits.rs b/primitives/core/src/traits.rs index bd02d39fb55a5..297430cfa6392 100644 --- a/primitives/core/src/traits.rs +++ b/primitives/core/src/traits.rs @@ -98,6 +98,7 @@ pub trait CodeExecutor: Sized + Send + Sync + CallInWasm + Clone + 'static { >( &self, ext: &mut E, + runtime_code: &RuntimeCode, method: &str, data: &[u8], use_native: bool, @@ -105,6 +106,70 @@ pub trait CodeExecutor: Sized + Send + Sync + CallInWasm + Clone + 'static { ) -> (Result, Self::Error>, bool); } + +/// The Wasm code of a Substrate runtime. +#[derive(Debug, Clone, codec::Encode, codec::Decode)] +pub struct RuntimeCode { + /// The actual Wasm code as binary blob. + pub code: Vec, + /// The optional heap pages this `code` should be executed with. + /// + /// If `None` are given, the default value of the executor will be used. + pub heap_pages: Option, + /// The SCALE encoded hash of `code`. + /// + /// The hashing algorithm isn't that important, as long as all runtime + /// code instances use the same. + pub hash: Vec, +} + +impl PartialEq for RuntimeCode { + fn eq(&self, other: &Self) -> bool { + self.hash == other.hash + } +} + +impl RuntimeCode { + /// Create an `RuntimeCode` instance from the given `Externalities`. + /// + /// Extracts the code and the heap pages using the well known keys. + /// + /// Returns an error if the code could not be found. + pub fn from_externalities(ext: &dyn Externalities) -> Result { + let code = ext.storage(sp_storage::well_known_keys::CODE).ok_or(CodeNotFound)?; + let hash = ext.storage_hash(sp_storage::well_known_keys::CODE).ok_or(CodeNotFound)?; + let heap_pages = ext.storage(sp_storage::well_known_keys::HEAP_PAGES) + .and_then(|hp| codec::Decode::decode(&mut &hp[..]).ok()); + + Ok(Self { + code, + hash, + heap_pages, + }) + } + + /// Create an empty instance. + /// + /// This is only useful for tests that don't want to execute any code. + pub fn empty() -> Self { + Self { + code: Vec::new(), + hash: Vec::new(), + heap_pages: None, + } + } +} + +/// Could not find the `:code` in the externalities while initializing the [`RuntimeCode`]. +#[derive(Debug)] +pub struct CodeNotFound; + +impl std::fmt::Display for CodeNotFound { + fn fmt(&self, f: &mut std::fmt::Formatter) -> Result<(), std::fmt::Error> { + write!(f, "the storage entry `:code` doesn't have any code") + } +} + /// Something that can call a method in a WASM blob. pub trait CallInWasm: Send + Sync { /// Call the given `method` in the given `wasm_blob` using `call_data` (SCALE encoded arguments) diff --git a/primitives/runtime/src/generic/block.rs b/primitives/runtime/src/generic/block.rs index a46396dce08f4..fb07d6c215d81 100644 --- a/primitives/runtime/src/generic/block.rs +++ b/primitives/runtime/src/generic/block.rs @@ -25,7 +25,10 @@ use serde::{Deserialize, Serialize}; use sp_std::prelude::*; use sp_core::RuntimeDebug; use crate::codec::{Codec, Encode, Decode}; -use crate::traits::{self, Member, Block as BlockT, Header as HeaderT, MaybeSerialize, MaybeMallocSizeOf}; +use crate::traits::{ + self, Member, Block as BlockT, Header as HeaderT, MaybeSerialize, MaybeMallocSizeOf, + NumberFor, +}; use crate::Justification; /// Something to identify a block. @@ -35,9 +38,9 @@ use crate::Justification; #[cfg_attr(feature = "std", serde(deny_unknown_fields))] pub enum BlockId { /// Identify by block header hash. - Hash(<::Header as HeaderT>::Hash), + Hash(Block::Hash), /// Identify by block number. - Number(<::Header as HeaderT>::Number), + Number(NumberFor), } impl BlockId { @@ -47,7 +50,7 @@ impl BlockId { } /// Create a block ID from a number. - pub fn number(number: ::Number) -> Self { + pub fn number(number: NumberFor) -> Self { BlockId::Number(number) } } diff --git a/primitives/state-machine/src/backend.rs b/primitives/state-machine/src/backend.rs index ca6612a5e92dd..9db41718ee09c 100644 --- a/primitives/state-machine/src/backend.rs +++ b/primitives/state-machine/src/backend.rs @@ -18,9 +18,9 @@ use log::warn; use hash_db::Hasher; -use codec::Encode; +use codec::{Decode, Encode}; -use sp_core::storage::{ChildInfo, OwnedChildInfo}; +use sp_core::{traits::RuntimeCode, storage::{ChildInfo, OwnedChildInfo, well_known_keys}}; use sp_trie::{TrieMut, MemoryDB, trie_types::TrieDBMut}; use crate::{ @@ -359,3 +359,22 @@ pub(crate) fn insert_into_memory_db(mdb: &mut MemoryDB, input: I) -> Op Some(root) } + +/// Get the runtime code from the given `backend`. +/// +/// Returns an error if the `:code` could not be found. +pub fn get_runtime_code>(backend: &B) -> Result + where H::Out: Encode, +{ + let code = backend.storage(well_known_keys::CODE) + .ok() + .flatten() + .ok_or("`:code` not found")?; + let hash = H::hash(&code).encode(); + let heap_pages = backend.storage(well_known_keys::HEAP_PAGES) + .ok() + .flatten() + .and_then(|d| Decode::decode(&mut &d[..]).ok()); + + Ok(RuntimeCode { code, hash, heap_pages }) +} diff --git a/primitives/state-machine/src/lib.rs b/primitives/state-machine/src/lib.rs index bb2bb2c52c86f..2c6245c70d579 100644 --- a/primitives/state-machine/src/lib.rs +++ b/primitives/state-machine/src/lib.rs @@ -23,8 +23,8 @@ use log::{warn, trace}; use hash_db::Hasher; use codec::{Decode, Encode, Codec}; use sp_core::{ - storage::ChildInfo, NativeOrEncoded, NeverNativeValue, - traits::{CodeExecutor, CallInWasmExt}, hexdisplay::HexDisplay, + storage::ChildInfo, NativeOrEncoded, NeverNativeValue, hexdisplay::HexDisplay, + traits::{CodeExecutor, CallInWasmExt, RuntimeCode}, }; use overlayed_changes::OverlayedChangeSet; use sp_externalities::Extensions; @@ -42,7 +42,7 @@ mod trie_backend; mod trie_backend_essence; mod stats; -pub use sp_trie::{trie_types::{Layout, TrieDBMut}, TrieMut, DBValue, MemoryDB}; +pub use sp_trie::{trie_types::{Layout, TrieDBMut}, StorageProof, TrieMut, DBValue, MemoryDB}; pub use testing::TestExternalities; pub use basic::BasicExternalities; pub use ext::Ext; @@ -67,8 +67,7 @@ pub use overlayed_changes::{ StorageCollection, ChildStorageCollection, }; pub use proving_backend::{ - create_proof_check_backend, create_proof_check_backend_storage, merge_storage_proofs, - ProofRecorder, ProvingBackend, ProvingBackendRecorder, StorageProof, + create_proof_check_backend, ProofRecorder, ProvingBackend, ProvingBackendRecorder, }; pub use trie_backend_essence::{TrieBackendStorage, Storage}; pub use trie_backend::TrieBackend; @@ -191,6 +190,7 @@ pub struct StateMachine<'a, B, H, N, Exec> changes_trie_state: Option>, _marker: PhantomData<(H, N)>, storage_transaction_cache: Option<&'a mut StorageTransactionCache>, + runtime_code: &'a RuntimeCode, } impl<'a, B, H, N, Exec> StateMachine<'a, B, H, N, Exec> where @@ -209,6 +209,7 @@ impl<'a, B, H, N, Exec> StateMachine<'a, B, H, N, Exec> where method: &'a str, call_data: &'a [u8], mut extensions: Extensions, + runtime_code: &'a RuntimeCode, ) -> Self { extensions.register(CallInWasmExt::new(exec.clone())); @@ -222,6 +223,7 @@ impl<'a, B, H, N, Exec> StateMachine<'a, B, H, N, Exec> where changes_trie_state, _marker: PhantomData, storage_transaction_cache: None, + runtime_code, } } @@ -292,6 +294,7 @@ impl<'a, B, H, N, Exec> StateMachine<'a, B, H, N, Exec> where let (result, was_native) = self.exec.call( &mut ext, + self.runtime_code, self.method, self.call_data, use_native, @@ -436,6 +439,7 @@ pub fn prove_execution( exec: &Exec, method: &str, call_data: &[u8], + runtime_code: &RuntimeCode, ) -> Result<(Vec, StorageProof), Box> where B: Backend, @@ -446,7 +450,14 @@ where { let trie_backend = backend.as_trie_backend() .ok_or_else(|| Box::new(ExecutionError::UnableToGenerateProof) as Box)?; - prove_execution_on_trie_backend::<_, _, N, _>(trie_backend, overlay, exec, method, call_data) + prove_execution_on_trie_backend::<_, _, N, _>( + trie_backend, + overlay, + exec, + method, + call_data, + runtime_code, + ) } /// Prove execution using the given trie backend, overlayed changes, and call executor. @@ -464,6 +475,7 @@ pub fn prove_execution_on_trie_backend( exec: &Exec, method: &str, call_data: &[u8], + runtime_code: &RuntimeCode, ) -> Result<(Vec, StorageProof), Box> where S: trie_backend_essence::TrieBackendStorage, @@ -474,7 +486,14 @@ where { let proving_backend = proving_backend::ProvingBackend::new(trie_backend); let mut sm = StateMachine::<_, H, N, Exec>::new( - &proving_backend, None, overlay, exec, method, call_data, Extensions::default(), + &proving_backend, + None, + overlay, + exec, + method, + call_data, + Extensions::default(), + runtime_code, ); let result = sm.execute_using_consensus_failure_handler::<_, NeverNativeValue, fn() -> _>( @@ -493,6 +512,7 @@ pub fn execution_proof_check( exec: &Exec, method: &str, call_data: &[u8], + runtime_code: &RuntimeCode, ) -> Result, Box> where H: Hasher, @@ -501,7 +521,14 @@ where N: crate::changes_trie::BlockNumber, { let trie_backend = create_proof_check_backend::(root.into(), proof)?; - execution_proof_check_on_trie_backend::<_, N, _>(&trie_backend, overlay, exec, method, call_data) + execution_proof_check_on_trie_backend::<_, N, _>( + &trie_backend, + overlay, + exec, + method, + call_data, + runtime_code, + ) } /// Check execution proof on proving backend, generated by `prove_execution` call. @@ -511,6 +538,7 @@ pub fn execution_proof_check_on_trie_backend( exec: &Exec, method: &str, call_data: &[u8], + runtime_code: &RuntimeCode, ) -> Result, Box> where H: Hasher, @@ -519,7 +547,14 @@ where N: crate::changes_trie::BlockNumber, { let mut sm = StateMachine::<_, H, N, Exec>::new( - trie_backend, None, overlay, exec, method, call_data, Extensions::default(), + trie_backend, + None, + overlay, + exec, + method, + call_data, + Extensions::default(), + runtime_code, ); sm.execute_using_consensus_failure_handler::<_, NeverNativeValue, fn() -> _>( @@ -692,7 +727,9 @@ mod tests { use super::*; use super::ext::Ext; use super::changes_trie::Configuration as ChangesTrieConfig; - use sp_core::{Blake2Hasher, map, traits::Externalities, storage::ChildStorageKey}; + use sp_core::{ + Blake2Hasher, map, traits::{Externalities, RuntimeCode}, storage::ChildStorageKey, + }; #[derive(Clone)] struct DummyCodeExecutor { @@ -714,6 +751,7 @@ mod tests { >( &self, ext: &mut E, + _: &RuntimeCode, _method: &str, _data: &[u8], use_native: bool, @@ -767,6 +805,7 @@ mod tests { fn execute_works() { let backend = trie_backend::tests::test_trie(); let mut overlayed_changes = Default::default(); + let wasm_code = RuntimeCode::empty(); let mut state_machine = StateMachine::new( &backend, @@ -781,6 +820,7 @@ mod tests { "test", &[], Default::default(), + &wasm_code, ); assert_eq!( @@ -794,6 +834,7 @@ mod tests { fn execute_works_with_native_else_wasm() { let backend = trie_backend::tests::test_trie(); let mut overlayed_changes = Default::default(); + let wasm_code = RuntimeCode::empty(); let mut state_machine = StateMachine::new( &backend, @@ -808,6 +849,7 @@ mod tests { "test", &[], Default::default(), + &wasm_code, ); assert_eq!(state_machine.execute(ExecutionStrategy::NativeElseWasm).unwrap(), vec![66]); @@ -818,6 +860,7 @@ mod tests { let mut consensus_failed = false; let backend = trie_backend::tests::test_trie(); let mut overlayed_changes = Default::default(); + let wasm_code = RuntimeCode::empty(); let mut state_machine = StateMachine::new( &backend, @@ -832,6 +875,7 @@ mod tests { "test", &[], Default::default(), + &wasm_code, ); assert!( @@ -864,6 +908,7 @@ mod tests { &executor, "test", &[], + &RuntimeCode::empty(), ).unwrap(); // check proof locally @@ -874,6 +919,7 @@ mod tests { &executor, "test", &[], + &RuntimeCode::empty(), ).unwrap(); // check that both results are correct diff --git a/primitives/state-machine/src/proving_backend.rs b/primitives/state-machine/src/proving_backend.rs index 672ec6ea794e4..60d891a3f0094 100644 --- a/primitives/state-machine/src/proving_backend.rs +++ b/primitives/state-machine/src/proving_backend.rs @@ -18,19 +18,19 @@ use std::sync::Arc; use parking_lot::RwLock; -use codec::{Decode, Encode, Codec}; +use codec::{Decode, Codec}; use log::debug; use hash_db::{Hasher, HashDB, EMPTY_PREFIX, Prefix}; use sp_trie::{ MemoryDB, default_child_trie_root, read_trie_value_with, read_child_trie_value_with, - record_all_keys + record_all_keys, StorageProof, }; pub use sp_trie::Recorder; pub use sp_trie::trie_types::{Layout, TrieError}; use crate::trie_backend::TrieBackend; use crate::trie_backend_essence::{Ephemeral, TrieBackendEssence, TrieBackendStorage}; use crate::{Error, ExecutionError, Backend}; -use std::collections::{HashMap, HashSet}; +use std::collections::HashMap; use crate::DBValue; use sp_core::storage::ChildInfo; @@ -40,82 +40,6 @@ pub struct ProvingBackendRecorder<'a, S: 'a + TrieBackendStorage, H: 'a + Has pub(crate) proof_recorder: &'a mut Recorder, } -/// A proof that some set of key-value pairs are included in the storage trie. The proof contains -/// the storage values so that the partial storage backend can be reconstructed by a verifier that -/// does not already have access to the key-value pairs. -/// -/// The proof consists of the set of serialized nodes in the storage trie accessed when looking up -/// the keys covered by the proof. Verifying the proof requires constructing the partial trie from -/// the serialized nodes and performing the key lookups. -#[derive(Debug, PartialEq, Eq, Clone, Encode, Decode)] -pub struct StorageProof { - trie_nodes: Vec>, -} - -impl StorageProof { - /// Constructs a storage proof from a subset of encoded trie nodes in a storage backend. - pub fn new(trie_nodes: Vec>) -> Self { - StorageProof { trie_nodes } - } - - /// Returns a new empty proof. - /// - /// An empty proof is capable of only proving trivial statements (ie. that an empty set of - /// key-value pairs exist in storage). - pub fn empty() -> Self { - StorageProof { - trie_nodes: Vec::new(), - } - } - - /// Returns whether this is an empty proof. - pub fn is_empty(&self) -> bool { - self.trie_nodes.is_empty() - } - - /// Create an iterator over trie nodes constructed from the proof. The nodes are not guaranteed - /// to be traversed in any particular order. - pub fn iter_nodes(self) -> StorageProofNodeIterator { - StorageProofNodeIterator::new(self) - } -} - -/// An iterator over trie nodes constructed from a storage proof. The nodes are not guaranteed to -/// be traversed in any particular order. -pub struct StorageProofNodeIterator { - inner: > as IntoIterator>::IntoIter, -} - -impl StorageProofNodeIterator { - fn new(proof: StorageProof) -> Self { - StorageProofNodeIterator { - inner: proof.trie_nodes.into_iter(), - } - } -} - -impl Iterator for StorageProofNodeIterator { - type Item = Vec; - - fn next(&mut self) -> Option { - self.inner.next() - } -} - -/// Merges multiple storage proofs covering potentially different sets of keys into one proof -/// covering all keys. The merged proof output may be smaller than the aggregate size of the input -/// proofs due to deduplication of trie nodes. -pub fn merge_storage_proofs(proofs: I) -> StorageProof - where I: IntoIterator -{ - let trie_nodes = proofs.into_iter() - .flat_map(|proof| proof.iter_nodes()) - .collect::>() - .into_iter() - .collect(); - StorageProof { trie_nodes } -} - impl<'a, S, H> ProvingBackendRecorder<'a, S, H> where S: TrieBackendStorage, @@ -222,7 +146,7 @@ impl<'a, S: 'a + TrieBackendStorage, H: 'a + Hasher> ProvingBackend<'a, S, H> let root = essence.root().clone(); let recorder = ProofRecorderBackend { backend: essence.backend_storage(), - proof_recorder: proof_recorder, + proof_recorder, }; ProvingBackend(TrieBackend::new(recorder, root)) } @@ -370,7 +294,7 @@ where H: Hasher, H::Out: Codec, { - let db = create_proof_check_backend_storage(proof); + let db = proof.into_memory_db(); if db.contains(&root, EMPTY_PREFIX) { Ok(TrieBackend::new(db, root)) @@ -379,20 +303,6 @@ where } } -/// Create in-memory storage of proof check backend. -pub fn create_proof_check_backend_storage( - proof: StorageProof, -) -> MemoryDB -where - H: Hasher, -{ - let mut db = MemoryDB::default(); - for item in proof.iter_nodes() { - db.insert(EMPTY_PREFIX, &item); - } - db -} - #[cfg(test)] mod tests { use crate::InMemoryBackend; diff --git a/primitives/trie/src/lib.rs b/primitives/trie/src/lib.rs index f6131c8ed5eed..80570a9792b10 100644 --- a/primitives/trie/src/lib.rs +++ b/primitives/trie/src/lib.rs @@ -21,6 +21,7 @@ mod error; mod node_header; mod node_codec; +mod storage_proof; mod trie_stream; use sp_std::boxed::Box; @@ -35,6 +36,7 @@ pub use error::Error; pub use trie_stream::TrieStream; /// The Substrate format implementation of `NodeCodec`. pub use node_codec::NodeCodec; +pub use storage_proof::StorageProof; /// Various re-exports from the `trie-db` crate. pub use trie_db::{ Trie, TrieMut, DBValue, Recorder, CError, Query, TrieLayout, TrieConfiguration, nibble_ops, TrieDBIterator, diff --git a/primitives/trie/src/storage_proof.rs b/primitives/trie/src/storage_proof.rs new file mode 100644 index 0000000000000..254adc2fcb48a --- /dev/null +++ b/primitives/trie/src/storage_proof.rs @@ -0,0 +1,109 @@ +// Copyright 2020 Parity Technologies (UK) Ltd. +// This file is part of Substrate. + +// Parity is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// Parity is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with Parity. If not, see . + +use sp_std::vec::Vec; +use codec::{Encode, Decode}; +use hash_db::{Hasher, HashDB}; + +/// A proof that some set of key-value pairs are included in the storage trie. The proof contains +/// the storage values so that the partial storage backend can be reconstructed by a verifier that +/// does not already have access to the key-value pairs. +/// +/// The proof consists of the set of serialized nodes in the storage trie accessed when looking up +/// the keys covered by the proof. Verifying the proof requires constructing the partial trie from +/// the serialized nodes and performing the key lookups. +#[derive(Debug, PartialEq, Eq, Clone, Encode, Decode)] +pub struct StorageProof { + trie_nodes: Vec>, +} + +impl StorageProof { + /// Constructs a storage proof from a subset of encoded trie nodes in a storage backend. + pub fn new(trie_nodes: Vec>) -> Self { + StorageProof { trie_nodes } + } + + /// Returns a new empty proof. + /// + /// An empty proof is capable of only proving trivial statements (ie. that an empty set of + /// key-value pairs exist in storage). + pub fn empty() -> Self { + StorageProof { + trie_nodes: Vec::new(), + } + } + + /// Returns whether this is an empty proof. + pub fn is_empty(&self) -> bool { + self.trie_nodes.is_empty() + } + + /// Create an iterator over trie nodes constructed from the proof. The nodes are not guaranteed + /// to be traversed in any particular order. + pub fn iter_nodes(self) -> StorageProofNodeIterator { + StorageProofNodeIterator::new(self) + } + + /// Creates a `MemoryDB` from `Self`. + pub fn into_memory_db(self) -> crate::MemoryDB { + self.into() + } + + /// Merges multiple storage proofs covering potentially different sets of keys into one proof + /// covering all keys. The merged proof output may be smaller than the aggregate size of the input + /// proofs due to deduplication of trie nodes. + pub fn merge(proofs: I) -> Self where I: IntoIterator { + let trie_nodes = proofs.into_iter() + .flat_map(|proof| proof.iter_nodes()) + .collect::>() + .into_iter() + .collect(); + + Self { trie_nodes } + } +} + +/// An iterator over trie nodes constructed from a storage proof. The nodes are not guaranteed to +/// be traversed in any particular order. +pub struct StorageProofNodeIterator { + inner: > as IntoIterator>::IntoIter, +} + +impl StorageProofNodeIterator { + fn new(proof: StorageProof) -> Self { + StorageProofNodeIterator { + inner: proof.trie_nodes.into_iter(), + } + } +} + +impl Iterator for StorageProofNodeIterator { + type Item = Vec; + + fn next(&mut self) -> Option { + self.inner.next() + } +} + +impl From for crate::MemoryDB { + fn from(proof: StorageProof) -> Self { + let mut db = crate::MemoryDB::default(); + for item in proof.iter_nodes() { + db.insert(crate::EMPTY_PREFIX, &item); + } + db + } +} diff --git a/test-utils/runtime/src/system.rs b/test-utils/runtime/src/system.rs index b410d317a1bab..073e06de81dfb 100644 --- a/test-utils/runtime/src/system.rs +++ b/test-utils/runtime/src/system.rs @@ -338,7 +338,7 @@ mod tests { use sp_io::TestExternalities; use substrate_test_runtime_client::{AccountKeyring, Sr25519Keyring}; use crate::{Header, Transfer, WASM_BINARY}; - use sp_core::{NeverNativeValue, map, traits::CodeExecutor}; + use sp_core::{NeverNativeValue, map, traits::{CodeExecutor, RuntimeCode}}; use sc_executor::{NativeExecutor, WasmExecutionMethod, native_executor_instance}; use sp_io::hashing::twox_128; @@ -401,8 +401,11 @@ mod tests { fn block_import_works_wasm() { block_import_works(|b, ext| { let mut ext = ext.ext(); + let runtime_code = RuntimeCode::from_externalities(&ext) + .expect("Code is part of the externalities"); executor().call::<_, NeverNativeValue, fn() -> _>( &mut ext, + &runtime_code, "Core_execute_block", &b.encode(), false, @@ -494,8 +497,11 @@ mod tests { fn block_import_with_transaction_works_wasm() { block_import_with_transaction_works(|b, ext| { let mut ext = ext.ext(); + let runtime_code = RuntimeCode::from_externalities(&ext) + .expect("Code is part of the externalities"); executor().call::<_, NeverNativeValue, fn() -> _>( &mut ext, + &runtime_code, "Core_execute_block", &b.encode(), false, diff --git a/utils/frame/benchmarking-cli/Cargo.toml b/utils/frame/benchmarking-cli/Cargo.toml index 93c62c3f9653e..89143ee9fe0ab 100644 --- a/utils/frame/benchmarking-cli/Cargo.toml +++ b/utils/frame/benchmarking-cli/Cargo.toml @@ -16,5 +16,6 @@ sc-client = { version = "0.8.0-alpha.2", path = "../../../client" } sc-client-db = { version = "0.8.0-alpha.2", path = "../../../client/db" } sc-executor = { version = "0.8.0-alpha.2", path = "../../../client/executor" } sp-runtime = { version = "2.0.0-alpha.2", path = "../../../primitives/runtime" } +sp-state-machine = { version = "0.8.0-alpha.2", path = "../../../primitives/state-machine" } structopt = "0.3.8" codec = { version = "1.2.0", package = "parity-scale-codec" } diff --git a/utils/frame/benchmarking-cli/src/lib.rs b/utils/frame/benchmarking-cli/src/lib.rs index 899419e5de5ad..1074944554ac2 100644 --- a/utils/frame/benchmarking-cli/src/lib.rs +++ b/utils/frame/benchmarking-cli/src/lib.rs @@ -121,6 +121,7 @@ impl BenchmarkCmd { self.repeat, ).encode(), Default::default(), + &sp_state_machine::backend::get_runtime_code(&state)?, ) .execute(strategy.into()) .map_err(|e| format!("Error executing runtime benchmark: {:?}", e))?;