diff --git a/bin/reth/src/dump_stage/hashing_account.rs b/bin/reth/src/dump_stage/hashing_account.rs index 4ae028b47..1692a8021 100644 --- a/bin/reth/src/dump_stage/hashing_account.rs +++ b/bin/reth/src/dump_stage/hashing_account.rs @@ -4,7 +4,8 @@ use crate::{ utils::DbTool, }; use eyre::Result; -use reth_db::{database::Database, table::TableImporter, tables, transaction::DbTx}; +use reth_db::{database::Database, table::TableImporter, tables}; +use reth_primitives::BlockNumber; use reth_provider::Transaction; use reth_stages::{stages::AccountHashingStage, Stage, StageId, UnwindInput}; use std::ops::DerefMut; @@ -12,29 +13,16 @@ use tracing::info; pub(crate) async fn dump_hashing_account_stage( db_tool: &mut DbTool<'_, DB>, - from: u64, - to: u64, + from: BlockNumber, + to: BlockNumber, output_db: &PlatformPath, should_run: bool, ) -> Result<()> { let (output_db, tip_block_number) = setup::(from, to, output_db, db_tool)?; // Import relevant AccountChangeSets - let tx = db_tool.db.tx()?; - let from_transition_rev = tx - .get::(from)? - .expect("there should be at least one.") - .transition_at_block(); - let to_transition_rev = tx - .get::(to)? - .expect("there should be at least one.") - .transition_after_block(); output_db.update(|tx| { - tx.import_table_with_range::( - &db_tool.db.tx()?, - Some(from_transition_rev), - to_transition_rev, - ) + tx.import_table_with_range::(&db_tool.db.tx()?, Some(from), to) })??; unwind_and_copy::(db_tool, from, tip_block_number, &output_db).await?; diff --git a/bin/reth/src/dump_stage/merkle.rs b/bin/reth/src/dump_stage/merkle.rs index 3d0357300..5208be41c 100644 --- a/bin/reth/src/dump_stage/merkle.rs +++ b/bin/reth/src/dump_stage/merkle.rs @@ -4,8 +4,8 @@ use crate::{ utils::DbTool, }; use eyre::Result; -use reth_db::{database::Database, table::TableImporter, tables, transaction::DbTx}; -use reth_primitives::MAINNET; +use reth_db::{database::Database, table::TableImporter, tables}; +use reth_primitives::{BlockNumber, MAINNET}; use reth_provider::Transaction; use reth_stages::{ stages::{AccountHashingStage, ExecutionStage, MerkleStage, StorageHashingStage}, @@ -16,8 +16,8 @@ use tracing::info; pub(crate) async fn dump_merkle_stage( db_tool: &mut DbTool<'_, DB>, - from: u64, - to: u64, + from: BlockNumber, + to: BlockNumber, output_db: &PlatformPath, should_run: bool, ) -> Result<()> { @@ -27,22 +27,8 @@ pub(crate) async fn dump_merkle_stage( tx.import_table_with_range::(&db_tool.db.tx()?, Some(from), to) })??; - let tx = db_tool.db.tx()?; - let from_transition_rev = tx - .get::(from)? - .expect("there should be at least one.") - .transition_at_block(); - let to_transition_rev = tx - .get::(to)? - .expect("there should be at least one.") - .transition_after_block(); - output_db.update(|tx| { - tx.import_table_with_range::( - &db_tool.db.tx()?, - Some(from_transition_rev), - to_transition_rev, - ) + tx.import_table_with_range::(&db_tool.db.tx()?, Some(from), to) })??; unwind_and_copy::(db_tool, (from, to), tip_block_number, &output_db).await?; diff --git a/bin/reth/src/test_eth_chain/runner.rs b/bin/reth/src/test_eth_chain/runner.rs index d2fee33fa..9d6118ab1 100644 --- a/bin/reth/src/test_eth_chain/runner.rs +++ b/bin/reth/src/test_eth_chain/runner.rs @@ -10,8 +10,8 @@ use reth_db::{ Error as DbError, }; use reth_primitives::{ - keccak256, Account as RethAccount, Address, Bytecode, ChainSpec, ForkCondition, Hardfork, - JsonU256, SealedBlock, SealedHeader, StorageEntry, H256, U256, + keccak256, Account as RethAccount, Address, Bytecode, ChainSpec, JsonU256, SealedBlock, + SealedHeader, StorageEntry, H256, U256, }; use reth_provider::Transaction; use reth_rlp::Decodable; @@ -130,8 +130,6 @@ pub async fn run_test(path: PathBuf) -> eyre::Result { debug!(target: "reth::cli", name, network = ?suite.network, "Running test"); let chain_spec: ChainSpec = suite.network.into(); - // if paris aka merge is not activated we dont have block rewards; - let has_block_reward = !matches!(chain_spec.fork(Hardfork::Paris), ForkCondition::Never); // Create db and acquire transaction let db = create_test_rw_db(); @@ -140,13 +138,13 @@ pub async fn run_test(path: PathBuf) -> eyre::Result { // insert genesis let header: SealedHeader = suite.genesis_block_header.into(); let genesis_block = SealedBlock { header, body: vec![], ommers: vec![], withdrawals: None }; - reth_provider::insert_canonical_block(&tx, genesis_block, None, has_block_reward)?; + reth_provider::insert_canonical_block(&tx, genesis_block, None)?; let mut last_block = None; suite.blocks.iter().try_for_each(|block| -> eyre::Result<()> { let decoded = SealedBlock::decode(&mut block.rlp.as_ref())?; last_block = Some(decoded.number); - reth_provider::insert_canonical_block(&tx, decoded, None, has_block_reward)?; + reth_provider::insert_canonical_block(&tx, decoded, None)?; Ok(()) })?; diff --git a/bin/reth/src/utils.rs b/bin/reth/src/utils.rs index 9bbb3d5fb..bea92485c 100644 --- a/bin/reth/src/utils.rs +++ b/bin/reth/src/utils.rs @@ -70,11 +70,11 @@ impl<'a, DB: Database> DbTool<'a, DB> { /// Seeds the database with some random data, only used for testing pub fn seed(&mut self, len: u64) -> Result<()> { info!(target: "reth::cli", "Generating random block range from 0 to {len}"); - let chain = random_block_range(0..len, Default::default(), 0..64); + let chain = random_block_range(0..=len - 1, Default::default(), 0..64); self.db.update(|tx| { chain.into_iter().try_for_each(|block| { - insert_canonical_block(tx, block, None, true)?; + insert_canonical_block(tx, block, None)?; Ok::<_, eyre::Error>(()) }) })??; diff --git a/crates/blockchain-tree/src/blockchain_tree/chain.rs b/crates/blockchain-tree/src/blockchain_tree/chain.rs index e205defd8..619a1031b 100644 --- a/crates/blockchain-tree/src/blockchain_tree/chain.rs +++ b/crates/blockchain-tree/src/blockchain_tree/chain.rs @@ -108,14 +108,10 @@ impl AppendableChain { .get(&parent_number) .ok_or(ExecError::BlockNumberNotFoundInChain { block_number: parent_number })?; - let revert_to_transition_id = self - .block_transitions() - .get(&parent.number) - .expect("Should have the transition ID for the parent block"); - let mut state = self.chain.state().clone(); + let mut state = self.state.clone(); // Revert state to the state after execution of the parent block - state.revert_to(*revert_to_transition_id); + state.revert_to(parent.number); // Revert changesets to get the state of the parent that we need to apply the change. let post_state_data = PostStateDataRef { @@ -133,13 +129,8 @@ impl AppendableChain { )?; state.extend(block_state); - let chain = Self { - chain: Chain { - block_transitions: BTreeMap::from([(block.number, state.transitions_count())]), - state, - blocks: BTreeMap::from([(block.number, block)]), - }, - }; + let chain = + Self { chain: Chain { state, blocks: BTreeMap::from([(block.number, block)]) } }; // If all is okay, return new chain back. Present chain is not modified. Ok(chain) @@ -209,8 +200,6 @@ impl AppendableChain { externals, )?; self.state.extend(block_state); - let transition_count = self.state.transitions_count(); - self.block_transitions.insert(block.number, transition_count); self.blocks.insert(block.number, block); Ok(()) } diff --git a/crates/blockchain-tree/src/blockchain_tree/mod.rs b/crates/blockchain-tree/src/blockchain_tree/mod.rs index 637581b6a..844dc2396 100644 --- a/crates/blockchain-tree/src/blockchain_tree/mod.rs +++ b/crates/blockchain-tree/src/blockchain_tree/mod.rs @@ -661,11 +661,12 @@ impl BlockchainTree let mut tx = Transaction::new(&self.externals.db)?; + let tip = tx.tip_number()?; // read block and execution result from database. and remove traces of block from tables. let blocks_and_execution = tx .take_block_and_execution_range( self.externals.chain_spec.as_ref(), - (revert_until + 1).., + (revert_until + 1)..=tip, ) .map_err(|e| ExecError::CanonicalRevert { inner: e.to_string() })?; @@ -716,7 +717,7 @@ mod tests { genesis.header.header.state_root = EMPTY_ROOT; let tx_mut = db.tx_mut().unwrap(); - insert_block(&tx_mut, genesis, None, false, Some((0, 0))).unwrap(); + insert_block(&tx_mut, genesis, None).unwrap(); // insert first 10 blocks for i in 0..10 { @@ -786,11 +787,9 @@ mod tests { #[tokio::test] async fn sanity_path() { - let data = BlockChainTestData::default(); - let (mut block1, exec1) = data.blocks[0].clone(); - block1.number = 11; - let (mut block2, exec2) = data.blocks[1].clone(); - block2.number = 12; + let data = BlockChainTestData::default_with_numbers(11, 12); + let (block1, exec1) = data.blocks[0].clone(); + let (block2, exec2) = data.blocks[1].clone(); // test pops execution results from vector, so order is from last to first. let externals = setup_externals(vec![exec2.clone(), exec1.clone(), exec2, exec1]); diff --git a/crates/consensus/auto-seal/src/lib.rs b/crates/consensus/auto-seal/src/lib.rs index d0418df51..b640946b3 100644 --- a/crates/consensus/auto-seal/src/lib.rs +++ b/crates/consensus/auto-seal/src/lib.rs @@ -69,10 +69,6 @@ impl Consensus for AutoSealConsensus { fn pre_validate_block(&self, _block: &SealedBlock) -> Result<(), ConsensusError> { Ok(()) } - - fn has_block_reward(&self, _total_difficulty: U256, _difficulty: U256) -> bool { - false - } } /// Builder type for configuring the setup diff --git a/crates/consensus/beacon/src/beacon_consensus.rs b/crates/consensus/beacon/src/beacon_consensus.rs index cfc747729..c5232676c 100644 --- a/crates/consensus/beacon/src/beacon_consensus.rs +++ b/crates/consensus/beacon/src/beacon_consensus.rs @@ -79,10 +79,6 @@ impl Consensus for BeaconConsensus { fn pre_validate_block(&self, block: &SealedBlock) -> Result<(), ConsensusError> { validation::validate_block_standalone(block, &self.chain_spec) } - - fn has_block_reward(&self, total_difficulty: U256, difficulty: U256) -> bool { - !self.chain_spec.fork(Hardfork::Paris).active_at_ttd(total_difficulty, difficulty) - } } /// Validates the header's extradata according to the beacon consensus rules. @@ -96,18 +92,3 @@ fn validate_header_extradata(header: &SealedHeader) -> Result<(), ConsensusError Ok(()) } } - -#[cfg(test)] -mod test { - use super::BeaconConsensus; - use reth_interfaces::consensus::Consensus; - use reth_primitives::{ChainSpecBuilder, U256}; - use std::sync::Arc; - - #[test] - fn test_has_block_reward_before_paris() { - let chain_spec = Arc::new(ChainSpecBuilder::mainnet().build()); - let consensus = BeaconConsensus::new(chain_spec); - assert!(consensus.has_block_reward(U256::ZERO, U256::ZERO)); - } -} diff --git a/crates/interfaces/src/consensus.rs b/crates/interfaces/src/consensus.rs index 5f232b67f..d5b435790 100644 --- a/crates/interfaces/src/consensus.rs +++ b/crates/interfaces/src/consensus.rs @@ -41,13 +41,6 @@ pub trait Consensus: Debug + Send + Sync { /// /// **This should not be called for the genesis block**. fn pre_validate_block(&self, block: &SealedBlock) -> Result<(), ConsensusError>; - - /// After the Merge (aka Paris) block rewards became obsolete. - /// - /// This flag is needed as reth's changeset is indexed on transaction level granularity. - /// - /// More info [here](https://github.com/paradigmxyz/reth/issues/237) - fn has_block_reward(&self, total_difficulty: U256, difficulty: U256) -> bool; } /// Consensus Errors diff --git a/crates/interfaces/src/p2p/bodies/downloader.rs b/crates/interfaces/src/p2p/bodies/downloader.rs index 69c6c461a..86a7698ae 100644 --- a/crates/interfaces/src/p2p/bodies/downloader.rs +++ b/crates/interfaces/src/p2p/bodies/downloader.rs @@ -2,7 +2,7 @@ use super::response::BlockResponse; use crate::p2p::error::DownloadResult; use futures::Stream; use reth_primitives::BlockNumber; -use std::ops::Range; +use std::ops::RangeInclusive; /// Body downloader return type. pub type BodyDownloaderResult = DownloadResult>; @@ -14,5 +14,5 @@ pub type BodyDownloaderResult = DownloadResult>; /// fulfilling these requests. pub trait BodyDownloader: Send + Sync + Stream + Unpin { /// Method for setting the download range. - fn set_download_range(&mut self, range: Range) -> DownloadResult<()>; + fn set_download_range(&mut self, range: RangeInclusive) -> DownloadResult<()>; } diff --git a/crates/interfaces/src/p2p/bodies/response.rs b/crates/interfaces/src/p2p/bodies/response.rs index dac7bf276..995d28f8b 100644 --- a/crates/interfaces/src/p2p/bodies/response.rs +++ b/crates/interfaces/src/p2p/bodies/response.rs @@ -1,7 +1,7 @@ use reth_primitives::{BlockNumber, SealedBlock, SealedHeader, U256}; /// The block response -#[derive(PartialEq, Eq, Debug)] +#[derive(PartialEq, Eq, Debug, Clone)] pub enum BlockResponse { /// Full block response (with transactions or ommers) Full(SealedBlock), diff --git a/crates/interfaces/src/p2p/error.rs b/crates/interfaces/src/p2p/error.rs index 70e22f6cd..658bbc99f 100644 --- a/crates/interfaces/src/p2p/error.rs +++ b/crates/interfaces/src/p2p/error.rs @@ -2,7 +2,7 @@ use super::headers::client::HeadersRequest; use crate::{consensus, db}; use reth_network_api::ReputationChangeKind; use reth_primitives::{BlockHashOrNumber, BlockNumber, Header, WithPeerId, H256}; -use std::ops::Range; +use std::ops::RangeInclusive; use thiserror::Error; use tokio::sync::{mpsc, oneshot}; @@ -200,7 +200,7 @@ pub enum DownloadError { #[error("Requested body range is invalid: {range:?}.")] InvalidBodyRange { /// Invalid block number range. - range: Range, + range: RangeInclusive, }, /* ==================== COMMON ERRORS ==================== */ /// Timed out while waiting for request id response. diff --git a/crates/interfaces/src/provider.rs b/crates/interfaces/src/provider.rs index c62f401f6..fc6f8cbd6 100644 --- a/crates/interfaces/src/provider.rs +++ b/crates/interfaces/src/provider.rs @@ -1,4 +1,4 @@ -use reth_primitives::{Address, BlockHash, BlockNumber, TransitionId, TxNumber, H256}; +use reth_primitives::{Address, BlockHash, BlockNumber, TxNumber, H256}; /// Bundled errors variants thrown by various providers. #[allow(missing_docs)] @@ -24,20 +24,20 @@ pub enum ProviderError { BlockTransition { block_number: BlockNumber }, /// The transition id was found for the given address and storage key, but the changeset was /// not found. - #[error("Storage ChangeSet address: ({address:?} key: {storage_key:?}) for transition:#{transition_id} does not exist")] + #[error("Storage ChangeSet address: ({address:?} key: {storage_key:?}) for block:#{block_number} does not exist")] StorageChangeset { - /// The transition id found for the address and storage key - transition_id: TransitionId, + /// The block number found for the address and storage key + block_number: BlockNumber, /// The account address address: Address, /// The storage key storage_key: H256, }, - /// The transition id was found for the given address, but the changeset was not found. - #[error("Account {address:?} ChangeSet for transition #{transition_id} does not exist")] + /// The block number was found for the given address, but the changeset was not found. + #[error("Account {address:?} ChangeSet for block #{block_number} does not exist")] AccountChangeset { - /// Transition id found for the address - transition_id: TransitionId, + /// Block number found for the address + block_number: BlockNumber, /// The account address address: Address, }, diff --git a/crates/interfaces/src/test_utils/generators.rs b/crates/interfaces/src/test_utils/generators.rs index f11bb1c5b..b00bdfb29 100644 --- a/crates/interfaces/src/test_utils/generators.rs +++ b/crates/interfaces/src/test_utils/generators.rs @@ -1,10 +1,14 @@ use rand::{distributions::uniform::SampleRange, seq::SliceRandom, thread_rng, Rng}; use reth_primitives::{ - proofs, sign_message, Account, Address, Bytes, Header, SealedBlock, SealedHeader, Signature, - StorageEntry, Transaction, TransactionKind, TransactionSigned, TxLegacy, H160, H256, U256, + proofs, sign_message, Account, Address, BlockNumber, Bytes, Header, SealedBlock, SealedHeader, + Signature, StorageEntry, Transaction, TransactionKind, TransactionSigned, TxLegacy, H160, H256, + U256, }; use secp256k1::{KeyPair, Message as SecpMessage, Secp256k1, SecretKey, SECP256K1}; -use std::{collections::BTreeMap, ops::Sub}; +use std::{ + collections::BTreeMap, + ops::{Range, RangeInclusive, Sub}, +}; // TODO(onbjerg): Maybe we should split this off to its own crate, or move the helpers to the // relevant crates? @@ -133,13 +137,13 @@ pub fn random_block( /// /// See [random_block] for considerations when validating the generated blocks. pub fn random_block_range( - block_numbers: std::ops::Range, + block_numbers: RangeInclusive, head: H256, - tx_count: std::ops::Range, + tx_count: Range, ) -> Vec { let mut rng = rand::thread_rng(); let mut blocks = - Vec::with_capacity(block_numbers.end.saturating_sub(block_numbers.start) as usize); + Vec::with_capacity(block_numbers.end().saturating_sub(*block_numbers.start()) as usize); for idx in block_numbers { blocks.push(random_block( idx, @@ -177,10 +181,9 @@ where let valid_addresses = state.keys().copied().collect(); - let num_transitions: usize = blocks.into_iter().map(|block| block.body.len()).sum(); - let mut transitions = Vec::with_capacity(num_transitions); + let mut transitions = Vec::new(); - (0..num_transitions).for_each(|i| { + blocks.into_iter().for_each(|block| { let mut transition = Vec::new(); let (from, to, mut transfer, new_entries) = random_account_change(&valid_addresses, n_changes.clone(), key_range.clone()); diff --git a/crates/interfaces/src/test_utils/headers.rs b/crates/interfaces/src/test_utils/headers.rs index 826b5621e..580f2b1ac 100644 --- a/crates/interfaces/src/test_utils/headers.rs +++ b/crates/interfaces/src/test_utils/headers.rs @@ -337,8 +337,4 @@ impl Consensus for TestConsensus { Ok(()) } } - - fn has_block_reward(&self, _: U256, _: U256) -> bool { - true - } } diff --git a/crates/net/downloaders/src/bodies/bodies.rs b/crates/net/downloaders/src/bodies/bodies.rs index e5143b873..ff7ec9950 100644 --- a/crates/net/downloaders/src/bodies/bodies.rs +++ b/crates/net/downloaders/src/bodies/bodies.rs @@ -20,7 +20,7 @@ use reth_tasks::{TaskSpawner, TokioTaskExecutor}; use std::{ cmp::Ordering, collections::BinaryHeap, - ops::{Range, RangeInclusive}, + ops::RangeInclusive, pin::Pin, sync::Arc, task::{Context, Poll}, @@ -60,7 +60,7 @@ pub struct BodiesDownloader { /// Maximum amount of received bodies to buffer internally. max_buffered_responses: usize, /// The range of block numbers for body download. - download_range: Range, + download_range: RangeInclusive, /// The latest block number returned. latest_queued_block_number: Option, /// Requests in progress @@ -82,11 +82,12 @@ where fn next_headers_request(&mut self) -> DownloadResult>> { let start_at = match self.in_progress_queue.last_requested_block_number { Some(num) => num + 1, - None => self.download_range.start, + None => *self.download_range.start(), }; - - let limit = self.download_range.end.saturating_sub(start_at).min(self.request_limit); - self.query_headers(start_at..self.download_range.end, limit) + // as the range is inclusive, we need to add 1 to the end. + let items_left = (self.download_range.end() + 1).saturating_sub(start_at); + let limit = items_left.min(self.request_limit); + self.query_headers(start_at..=*self.download_range.end(), limit) } /// Retrieve a batch of headers from the database starting from provided block number. @@ -101,7 +102,7 @@ where /// NOTE: The batches returned have a variable length. fn query_headers( &self, - range: Range, + range: RangeInclusive, max_non_empty: u64, ) -> DownloadResult>> { if range.is_empty() || max_non_empty == 0 { @@ -119,7 +120,7 @@ where // Collection of results let mut headers = Vec::::default(); - let mut current_block_num = range.start; + let mut current_block_num = *range.start(); // Collect headers while // 1. Current block number is in range @@ -149,7 +150,6 @@ where // Increment current block number current_block_num += 1; } - Ok(Some(headers).filter(|h| !h.is_empty())) } @@ -157,7 +157,7 @@ where fn next_expected_block_number(&self) -> BlockNumber { match self.latest_queued_block_number { Some(num) => num + 1, - None => self.download_range.start, + None => *self.download_range.start(), } } @@ -189,7 +189,7 @@ where // or all blocks have already been requested. self.in_progress_queue .last_requested_block_number - .map(|last| last + 1 == self.download_range.end) + .map(|last| last == *self.download_range.end()) .unwrap_or_default(); nothing_to_request && @@ -202,7 +202,7 @@ where /// /// Should be invoked upon encountering fatal error. fn clear(&mut self) { - self.download_range = Range::default(); + self.download_range = RangeInclusive::new(1, 0); self.latest_queued_block_number.take(); self.in_progress_queue.clear(); self.buffered_responses.clear(); @@ -278,7 +278,7 @@ where /// back into the buffer. /// If there are any bodies between the range start and last queued body that have not been /// downloaded or are not in progress, they will be re-requested. - fn set_download_range(&mut self, range: Range) -> DownloadResult<()> { + fn set_download_range(&mut self, range: RangeInclusive) -> DownloadResult<()> { // Check if the range is valid. if range.is_empty() { tracing::error!(target: "downloaders::bodies", ?range, "Range is invalid"); @@ -286,8 +286,8 @@ where } // Check if the provided range is the subset of the existing range. - let is_current_range_subset = - self.download_range.contains(&range.start) && range.end == self.download_range.end; + let is_current_range_subset = self.download_range.contains(range.start()) && + *range.end() == *self.download_range.end(); if is_current_range_subset { tracing::trace!(target: "downloaders::bodies", ?range, "Download range already in progress"); // The current range already includes requested. @@ -295,7 +295,7 @@ where } // Check if the provided range is the next expected range. - let is_next_consecutive_range = range.start == self.download_range.end; + let is_next_consecutive_range = *range.start() == *self.download_range.end() + 1; if is_next_consecutive_range { // New range received. tracing::trace!(target: "downloaders::bodies", ?range, "New download range set"); @@ -321,11 +321,9 @@ where fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.get_mut(); - if this.is_terminated() { return Poll::Ready(None) } - // Submit new requests and poll any in progress loop { // Poll requests @@ -387,13 +385,11 @@ where break } } - // All requests are handled, stream is finished if this.in_progress_queue.is_empty() { if this.queued_bodies.is_empty() { return Poll::Ready(None) } - let batch_size = this.stream_batch_size.min(this.queued_bodies.len()); let next_batch = this.queued_bodies.drain(..batch_size); this.metrics.total_flushed.increment(next_batch.len() as u64); @@ -525,7 +521,7 @@ impl BodiesDownloaderBuilder { concurrent_requests_range, in_progress_queue, metrics, - download_range: Default::default(), + download_range: RangeInclusive::new(1, 0), latest_queued_block_number: None, buffered_responses: Default::default(), queued_bodies: Default::default(), @@ -553,7 +549,7 @@ mod tests { async fn streams_bodies_in_order() { // Generate some random blocks let db = create_test_db::(EnvKind::RW); - let (headers, mut bodies) = generate_bodies(0..20); + let (headers, mut bodies) = generate_bodies(0..=19); insert_headers(&db, &headers); @@ -565,7 +561,7 @@ mod tests { Arc::new(TestConsensus::default()), db, ); - downloader.set_download_range(0..20).expect("failed to set download range"); + downloader.set_download_range(0..=19).expect("failed to set download range"); assert_matches!( downloader.next().await, @@ -580,7 +576,7 @@ mod tests { async fn requests_correct_number_of_times() { // Generate some random blocks let db = create_test_db::(EnvKind::RW); - let blocks = random_block_range(0..200, H256::zero(), 1..2); + let blocks = random_block_range(0..=199, H256::zero(), 1..2); let headers = blocks.iter().map(|block| block.header.clone()).collect::>(); let bodies = blocks @@ -604,7 +600,7 @@ mod tests { let mut downloader = BodiesDownloaderBuilder::default() .with_request_limit(request_limit) .build(client.clone(), Arc::new(TestConsensus::default()), db); - downloader.set_download_range(0..200).expect("failed to set download range"); + downloader.set_download_range(0..=199).expect("failed to set download range"); let _ = downloader.collect::>().await; assert_eq!(client.times_requested(), 20); @@ -616,7 +612,7 @@ mod tests { async fn streams_bodies_in_order_after_range_reset() { // Generate some random blocks let db = create_test_db::(EnvKind::RW); - let (headers, mut bodies) = generate_bodies(0..100); + let (headers, mut bodies) = generate_bodies(0..=99); insert_headers(&db, &headers); @@ -632,7 +628,7 @@ mod tests { let mut range_start = 0; while range_start < 100 { - downloader.set_download_range(range_start..100).expect("failed to set download range"); + downloader.set_download_range(range_start..=99).expect("failed to set download range"); assert_matches!( downloader.next().await, @@ -649,7 +645,7 @@ mod tests { async fn can_download_new_range_after_termination() { // Generate some random blocks let db = create_test_db::(EnvKind::RW); - let (headers, mut bodies) = generate_bodies(0..200); + let (headers, mut bodies) = generate_bodies(0..=199); insert_headers(&db, &headers); @@ -661,7 +657,7 @@ mod tests { ); // Set and download the first range - downloader.set_download_range(0..100).expect("failed to set download range"); + downloader.set_download_range(0..=99).expect("failed to set download range"); assert_matches!( downloader.next().await, Some(Ok(res)) => assert_eq!(res, zip_blocks(headers.iter().take(100), &mut bodies)) @@ -671,7 +667,7 @@ mod tests { assert!(downloader.next().await.is_none()); // Set and download the second range - downloader.set_download_range(100..200).expect("failed to set download range"); + downloader.set_download_range(100..=199).expect("failed to set download range"); assert_matches!( downloader.next().await, Some(Ok(res)) => assert_eq!(res, zip_blocks(headers.iter().skip(100), &mut bodies)) diff --git a/crates/net/downloaders/src/bodies/queue.rs b/crates/net/downloaders/src/bodies/queue.rs index e71e52096..216be123e 100644 --- a/crates/net/downloaders/src/bodies/queue.rs +++ b/crates/net/downloaders/src/bodies/queue.rs @@ -71,7 +71,6 @@ where None => last.number, }) .or(self.last_requested_block_number); - // Create request and push into the queue. self.inner.push( BodiesRequestFuture::new(client, consensus, priority, self.metrics.clone()) diff --git a/crates/net/downloaders/src/bodies/request.rs b/crates/net/downloaders/src/bodies/request.rs index f6bec7dfc..78e5db59a 100644 --- a/crates/net/downloaders/src/bodies/request.rs +++ b/crates/net/downloaders/src/bodies/request.rs @@ -266,7 +266,7 @@ mod tests { #[tokio::test] async fn request_submits_until_fulfilled() { // Generate some random blocks - let (headers, mut bodies) = generate_bodies(0..20); + let (headers, mut bodies) = generate_bodies(0..=19); let batch_size = 2; let client = Arc::new( diff --git a/crates/net/downloaders/src/bodies/task.rs b/crates/net/downloaders/src/bodies/task.rs index bf7ced932..c54893f61 100644 --- a/crates/net/downloaders/src/bodies/task.rs +++ b/crates/net/downloaders/src/bodies/task.rs @@ -9,7 +9,7 @@ use reth_primitives::BlockNumber; use reth_tasks::{TaskSpawner, TokioTaskExecutor}; use std::{ future::Future, - ops::Range, + ops::RangeInclusive, pin::Pin, task::{ready, Context, Poll}, }; @@ -22,7 +22,7 @@ use tokio_stream::wrappers::UnboundedReceiverStream; pub struct TaskDownloader { #[pin] from_downloader: UnboundedReceiverStream, - to_downloader: UnboundedSender>, + to_downloader: UnboundedSender>, } // === impl TaskDownloader === @@ -83,7 +83,7 @@ impl TaskDownloader { } impl BodyDownloader for TaskDownloader { - fn set_download_range(&mut self, range: Range) -> DownloadResult<()> { + fn set_download_range(&mut self, range: RangeInclusive) -> DownloadResult<()> { let _ = self.to_downloader.send(range); Ok(()) } @@ -99,7 +99,7 @@ impl Stream for TaskDownloader { /// A [BodyDownloader] that runs on its own task struct SpawnedDownloader { - updates: UnboundedReceiverStream>, + updates: UnboundedReceiverStream>, bodies_tx: UnboundedSender, downloader: T, } @@ -162,7 +162,7 @@ mod tests { reth_tracing::init_test_tracing(); let db = create_test_db::(EnvKind::RW); - let (headers, mut bodies) = generate_bodies(0..20); + let (headers, mut bodies) = generate_bodies(0..=19); insert_headers(&db, &headers); @@ -176,7 +176,7 @@ mod tests { ); let mut downloader = TaskDownloader::spawn(downloader); - downloader.set_download_range(0..20).expect("failed to set download range"); + downloader.set_download_range(0..=19).expect("failed to set download range"); assert_matches!( downloader.next().await, @@ -197,7 +197,7 @@ mod tests { ); let mut downloader = TaskDownloader::spawn(downloader); - downloader.set_download_range(0..0).expect("failed to set download range"); + downloader.set_download_range(1..=0).expect("failed to set download range"); assert_matches!(downloader.next().await, Some(Err(DownloadError::InvalidBodyRange { .. }))); } } diff --git a/crates/net/downloaders/src/test_utils/file_client.rs b/crates/net/downloaders/src/test_utils/file_client.rs index 50c570d4a..d222b7431 100644 --- a/crates/net/downloaders/src/test_utils/file_client.rs +++ b/crates/net/downloaders/src/test_utils/file_client.rs @@ -272,7 +272,7 @@ mod tests { async fn streams_bodies_from_buffer() { // Generate some random blocks let db = create_test_db::(EnvKind::RW); - let (headers, mut bodies) = generate_bodies(0..20); + let (headers, mut bodies) = generate_bodies(0..=19); insert_headers(&db, &headers); @@ -286,7 +286,7 @@ mod tests { Arc::new(TestConsensus::default()), db, ); - downloader.set_download_range(0..20).expect("failed to set download range"); + downloader.set_download_range(0..=19).expect("failed to set download range"); assert_matches!( downloader.next().await, @@ -330,7 +330,7 @@ mod tests { async fn test_download_headers_from_file() { // Generate some random blocks let db = create_test_db::(EnvKind::RW); - let (file, headers, mut bodies) = generate_bodies_file(0..20).await; + let (file, headers, mut bodies) = generate_bodies_file(0..=19).await; // now try to read them back let client = Arc::new(FileClient::from_file(file).await.unwrap()); @@ -355,7 +355,7 @@ mod tests { async fn test_download_bodies_from_file() { // Generate some random blocks let db = create_test_db::(EnvKind::RW); - let (file, headers, mut bodies) = generate_bodies_file(0..20).await; + let (file, headers, mut bodies) = generate_bodies_file(0..=19).await; // now try to read them back let client = Arc::new(FileClient::from_file(file).await.unwrap()); @@ -368,7 +368,7 @@ mod tests { Arc::new(TestConsensus::default()), db, ); - downloader.set_download_range(0..20).expect("failed to set download range"); + downloader.set_download_range(0..=19).expect("failed to set download range"); assert_matches!( downloader.next().await, diff --git a/crates/net/downloaders/src/test_utils/mod.rs b/crates/net/downloaders/src/test_utils/mod.rs index 0bc953e71..db2c8195c 100644 --- a/crates/net/downloaders/src/test_utils/mod.rs +++ b/crates/net/downloaders/src/test_utils/mod.rs @@ -4,7 +4,7 @@ use crate::bodies::test_utils::create_raw_bodies; use futures::SinkExt; use reth_interfaces::test_utils::generators::random_block_range; use reth_primitives::{BlockBody, SealedHeader, H256}; -use std::{collections::HashMap, io::SeekFrom}; +use std::{collections::HashMap, io::SeekFrom, ops::RangeInclusive}; use tokio::{ fs::File, io::{AsyncSeekExt, AsyncWriteExt, BufWriter}, @@ -16,7 +16,7 @@ pub(crate) const TEST_SCOPE: &str = "downloaders.test"; /// Generate a set of bodies and their corresponding block hashes pub(crate) fn generate_bodies( - rng: std::ops::Range, + rng: RangeInclusive, ) -> (Vec, HashMap) { let blocks = random_block_range(rng, H256::zero(), 0..2); @@ -41,9 +41,9 @@ pub(crate) fn generate_bodies( /// Generate a set of bodies, write them to a temporary file, and return the file along with the /// bodies and corresponding block hashes pub(crate) async fn generate_bodies_file( - rng: std::ops::Range, + rng: RangeInclusive, ) -> (tokio::fs::File, Vec, HashMap) { - let (headers, mut bodies) = generate_bodies(0..20); + let (headers, mut bodies) = generate_bodies(0..=19); let raw_block_bodies = create_raw_bodies(headers.clone().iter(), &mut bodies.clone()); let mut file: File = tempfile::tempfile().unwrap().into(); diff --git a/crates/payload/basic/src/lib.rs b/crates/payload/basic/src/lib.rs index b5c72a4b4..59a5ca7ba 100644 --- a/crates/payload/basic/src/lib.rs +++ b/crates/payload/basic/src/lib.rs @@ -449,6 +449,8 @@ fn build_payload( let mut total_fees = U256::ZERO; let base_fee = initialized_block_env.basefee.to::(); + let block_number = initialized_block_env.number.to::(); + for tx in best_txs { // ensure we still have capacity for this transaction if cumulative_gas_used + tx.gas_limit() > block_gas_limit { @@ -480,7 +482,7 @@ fn build_payload( let gas_used = result.gas_used(); // commit changes - commit_state_changes(&mut db, &mut post_state, state, true); + commit_state_changes(&mut db, &mut post_state, block_number, state, true); // add gas used by the transaction to cumulative gas used, before creating the receipt cumulative_gas_used += gas_used; @@ -520,7 +522,13 @@ fn build_payload( &attributes.withdrawals, ); for (address, increment) in balance_increments { - increment_account_balance(&mut db, &mut post_state, address, increment)?; + increment_account_balance( + &mut db, + &mut post_state, + block_number, + address, + increment, + )?; } // calculate withdrawals root diff --git a/crates/primitives/src/lib.rs b/crates/primitives/src/lib.rs index 5b4539d53..25f2d5dbb 100644 --- a/crates/primitives/src/lib.rs +++ b/crates/primitives/src/lib.rs @@ -95,8 +95,6 @@ pub type ChainId = u64; pub type StorageKey = H256; /// An account storage value. pub type StorageValue = U256; -/// The ID of block/transaction transition (represents state transition) -pub type TransitionId = u64; /// Solidity contract functions are addressed using the first four byte of the Keccak-256 hash of /// their signature pub type Selector = [u8; 4]; diff --git a/crates/revm/src/executor.rs b/crates/revm/src/executor.rs index e53986139..19122be34 100644 --- a/crates/revm/src/executor.rs +++ b/crates/revm/src/executor.rs @@ -9,7 +9,7 @@ use reth_blockchain_tree::post_state::PostState; use reth_consensus_common::calc; use reth_interfaces::executor::Error; use reth_primitives::{ - Account, Address, Block, Bloom, Bytecode, ChainSpec, Hardfork, Header, Receipt, + Account, Address, Block, BlockNumber, Bloom, Bytecode, ChainSpec, Hardfork, Header, Receipt, ReceiptWithBloom, TransactionSigned, Withdrawal, H256, U256, }; use reth_provider::{BlockExecutor, StateProvider}; @@ -103,12 +103,13 @@ where /// made in the transaction, which can be persisted to the database. fn commit_changes( &mut self, + block_number: BlockNumber, changes: hash_map::HashMap, has_state_clear_eip: bool, post_state: &mut PostState, ) { let db = self.db(); - commit_state_changes(db, post_state, changes, has_state_clear_eip); + commit_state_changes(db, post_state, block_number, changes, has_state_clear_eip); } /// Collect all balance changes at the end of the block. @@ -129,7 +130,11 @@ where } /// Irregular state change at Ethereum DAO hardfork - fn apply_dao_fork_changes(&mut self, post_state: &mut PostState) -> Result<(), Error> { + fn apply_dao_fork_changes( + &mut self, + block_number: BlockNumber, + post_state: &mut PostState, + ) -> Result<(), Error> { let db = self.db(); let mut drained_balance = U256::ZERO; @@ -142,12 +147,12 @@ where drained_balance += core::mem::take(&mut db_account.info.balance); let new = to_reth_acc(&db_account.info); // assume it is changeset as it is irregular state change - post_state.change_account(address, old, new); + post_state.change_account(block_number, address, old, new); } // add drained ether to beneficiary. let beneficiary = reth_blockchain_tree::eth_dao_fork::DAO_HARDFORK_BENEFICIARY; - self.increment_account_balance(beneficiary, drained_balance, post_state)?; + self.increment_account_balance(block_number, beneficiary, drained_balance, post_state)?; Ok(()) } @@ -155,11 +160,12 @@ where /// Increment the balance for the given account in the [PostState]. fn increment_account_balance( &mut self, + block_number: BlockNumber, address: Address, increment: U256, post_state: &mut PostState, ) -> Result<(), Error> { - increment_account_balance(self.db(), post_state, address, increment) + increment_account_balance(self.db(), post_state, block_number, address, increment) .map_err(|_| Error::ProviderError) } @@ -233,6 +239,7 @@ where // commit changes self.commit_changes( + block.number, state, self.chain_spec.fork(Hardfork::SpuriousDragon).active_at_block(block.number), &mut post_state, @@ -251,7 +258,6 @@ where // convert to reth log logs: result.into_logs().into_iter().map(into_reth_log).collect(), }); - post_state.finish_transition(); } Ok((post_state, cumulative_gas_used)) @@ -278,19 +284,13 @@ where // Add block rewards let balance_increments = self.post_block_balance_increments(block, total_difficulty); - let mut includes_block_transition = !balance_increments.is_empty(); for (address, increment) in balance_increments.into_iter() { - self.increment_account_balance(address, increment, &mut post_state)?; + self.increment_account_balance(block.number, address, increment, &mut post_state)?; } // Perform DAO irregular state change if self.chain_spec.fork(Hardfork::Dao).transitions_at_block(block.number) { - includes_block_transition = true; - self.apply_dao_fork_changes(&mut post_state)?; - } - - if includes_block_transition { - post_state.finish_transition(); + self.apply_dao_fork_changes(block.number, &mut post_state)?; } Ok(post_state) } @@ -325,6 +325,7 @@ where pub fn increment_account_balance( db: &mut CacheDB, post_state: &mut PostState, + block_number: BlockNumber, address: Address, increment: U256, ) -> Result<(), ::Error> @@ -344,6 +345,7 @@ where // if account was not present append `Created` changeset post_state.create_account( + block_number, address, Account { nonce: 0, balance: new.balance, bytecode_hash: None }, ) @@ -357,7 +359,7 @@ where beneficiary.account_state = AccountState::Touched; } // if account was present, append changed changeset. - post_state.change_account(address, old, new); + post_state.change_account(block_number, address, old, new); } } @@ -372,6 +374,7 @@ where pub fn commit_state_changes( db: &mut CacheDB, post_state: &mut PostState, + block_number: BlockNumber, changes: hash_map::HashMap, has_state_clear_eip: bool, ) where @@ -389,7 +392,7 @@ pub fn commit_state_changes( }; // Insert into `change` a old account and None for new account // and mark storage to be mapped - post_state.destroy_account(address, to_reth_acc(&db_account.info)); + post_state.destroy_account(block_number, address, to_reth_acc(&db_account.info)); // clear cached DB and mark account as not existing db_account.storage.clear(); @@ -418,7 +421,7 @@ pub fn commit_state_changes( let account = to_reth_acc(&entry.info); if !(has_state_clear_eip && account.is_empty()) { - post_state.create_account(address, account); + post_state.create_account(block_number, address, account); } entry } @@ -428,17 +431,22 @@ pub fn commit_state_changes( if matches!(entry.account_state, AccountState::NotExisting) { let account = to_reth_acc(&account.info); if !(has_state_clear_eip && account.is_empty()) { - post_state.create_account(address, account); + post_state.create_account(block_number, address, account); } } else if entry.info != account.info { post_state.change_account( + block_number, address, to_reth_acc(&entry.info), to_reth_acc(&account.info), ); } else if has_state_clear_eip && account.is_empty() { // The account was touched, but it is empty, so it should be deleted. - post_state.destroy_account(address, to_reth_acc(&account.info)); + post_state.destroy_account( + block_number, + address, + to_reth_acc(&account.info), + ); } entry.info = account.info.clone(); @@ -468,7 +476,7 @@ pub fn commit_state_changes( // Insert into change. if !storage_changeset.is_empty() { - post_state.change_storage(address, storage_changeset); + post_state.change_storage(block_number, address, storage_changeset); } } } @@ -495,6 +503,7 @@ pub fn verify_receipt<'a>( got: Box::new(logs_bloom), }) } + Ok(()) } @@ -589,10 +598,7 @@ mod tests { constants::ETH_TO_WEI, hex_literal::hex, keccak256, Account, Address, BlockNumber, Bytecode, Bytes, ChainSpecBuilder, ForkCondition, StorageKey, H256, MAINNET, U256, }; - use reth_provider::{ - post_state::{Change, Storage}, - AccountProvider, BlockHashProvider, StateProvider, - }; + use reth_provider::{post_state::Storage, AccountProvider, BlockHashProvider, StateProvider}; use reth_rlp::Decodable; use std::{collections::HashMap, str::FromStr}; @@ -726,12 +732,6 @@ mod tests { let mut executor = Executor::new(chain_spec, db); let post_state = executor.execute_and_verify_receipt(&block, U256::ZERO, None).unwrap(); - assert_eq!( - post_state.transitions_count(), - 2, - "Should executed two transitions (1 tx and 1 block reward)" - ); - let base_block_reward = ETH_TO_WEI * 2; let block_reward = calc::block_reward(base_block_reward, 1); @@ -787,50 +787,36 @@ mod tests { "Account should not be present in post-state since it was not changed" ); - // Check changes - const TX_TRANSITION_ID: u64 = 0; - const BLOCK_TRANSITION_ID: u64 = 1; - // Clone and sort to make the test deterministic - let mut changes = post_state.changes().to_vec(); - changes.sort_by_key(|change| (change.transition_id(), change.address())); assert_eq!( - changes, - &[ - // Storage changes on account 1 - Change::StorageChanged { - id: TX_TRANSITION_ID, - address: account1, - changeset: [(U256::from(1), (U256::ZERO, U256::from(2)))].into() - }, - // New account - Change::AccountCreated { - id: TX_TRANSITION_ID, - address: account2, - account: account2_info - }, - // Changed account - Change::AccountChanged { - id: TX_TRANSITION_ID, - address: account3, - old: account3_old_info, - new: account3_info - }, - // Block reward - Change::AccountChanged { - id: BLOCK_TRANSITION_ID, - address: account2, - old: account2_info, - new: account2_info_with_block_reward - }, - // Ommer reward - Change::AccountCreated { - id: BLOCK_TRANSITION_ID, - address: ommer_beneficiary, - account: ommer_beneficiary_info - }, - ], - "Changeset did not match" + post_state.account_changes(), + &BTreeMap::from([( + block.number, + BTreeMap::from([ + // New account + (account2, None), + // Changed account + (account3, Some(account3_old_info)), + // Ommer reward + (ommer_beneficiary, None) + ]) + ),]), + "Account changeset did not match" + ); + assert_eq!( + post_state.storage_changes(), + &BTreeMap::from([( + block.number, + BTreeMap::from([( + account1, + Storage { + wiped: false, + // Slot 1 changed from 0 to 2 + storage: BTreeMap::from([(U256::from(1), U256::ZERO)]) + } + )]) + )]), + "Storage changeset did not match" ); // Check final post-state @@ -903,11 +889,6 @@ mod tests { None, ) .unwrap(); - assert_eq!( - out.transitions_count(), - 1, - "Should only have 1 transition (the block transition)" - ); // Check if cache is set // beneficiary @@ -988,11 +969,6 @@ mod tests { let mut executor = Executor::new(chain_spec, db); let out = executor.execute_and_verify_receipt(&block, U256::ZERO, None).unwrap(); - assert_eq!( - out.transitions_count(), - 2, - "Should only have two transitions (the transaction and the block)" - ); assert_eq!(out.bytecodes().len(), 0, "Should have zero new bytecodes"); let post_account_caller = Account { @@ -1037,7 +1013,6 @@ mod tests { // execute chain and verify receipts let mut executor = Executor::new(chain_spec, db); let out = executor.execute_and_verify_receipt(&block, U256::ZERO, None).unwrap(); - assert_eq!(out.transitions_count(), 1, "Only one transition (the block transition)"); let withdrawal_sum = withdrawals.iter().fold(U256::ZERO, |sum, w| sum + w.amount_wei()); let beneficiary_account = executor.db().accounts.get(&withdrawal_beneficiary).unwrap(); @@ -1053,11 +1028,6 @@ mod tests { // Execute same block again let out = executor.execute_and_verify_receipt(&block, U256::ZERO, None).unwrap(); - assert_eq!( - out.transitions_count(), - 1, - "Should only have one transition (the block transition)" - ); assert_eq!( out.accounts().get(&withdrawal_beneficiary).unwrap(), @@ -1091,12 +1061,14 @@ mod tests { let mut executor = Executor::new(chain_spec, db); // touch account executor.commit_changes( + 1, hash_map::HashMap::from([(account, default_acc.clone())]), true, &mut PostState::default(), ); // destroy account executor.commit_changes( + 1, hash_map::HashMap::from([( account, RevmAccount { is_destroyed: true, is_touched: true, ..default_acc.clone() }, @@ -1106,6 +1078,7 @@ mod tests { ); // re-create account executor.commit_changes( + 1, hash_map::HashMap::from([( account, RevmAccount { is_touched: true, storage_cleared: true, ..default_acc.clone() }, @@ -1115,6 +1088,7 @@ mod tests { ); // touch account executor.commit_changes( + 1, hash_map::HashMap::from([(account, default_acc)]), true, &mut PostState::default(), diff --git a/crates/rpc/rpc-engine-api/src/engine_api.rs b/crates/rpc/rpc-engine-api/src/engine_api.rs index 283a3f995..5200e497c 100644 --- a/crates/rpc/rpc-engine-api/src/engine_api.rs +++ b/crates/rpc/rpc-engine-api/src/engine_api.rs @@ -484,7 +484,7 @@ mod tests { let (handle, api) = setup_engine_api(); let (start, count) = (1, 10); - let blocks = random_block_range(start..start + count, H256::default(), 0..2); + let blocks = random_block_range(start..=start + count - 1, H256::default(), 0..2); handle.client.extend_blocks(blocks.iter().cloned().map(|b| (b.hash(), b.unseal()))); let expected = @@ -499,7 +499,7 @@ mod tests { let (handle, api) = setup_engine_api(); let (start, count) = (1, 100); - let blocks = random_block_range(start..start + count, H256::default(), 0..2); + let blocks = random_block_range(start..=start + count - 1, H256::default(), 0..2); // Insert only blocks in ranges 1-25 and 50-75 let first_missing_range = 26..=50; diff --git a/crates/rpc/rpc-types/src/eth/engine/payload.rs b/crates/rpc/rpc-types/src/eth/engine/payload.rs index 17b68a6d7..7d30a1811 100644 --- a/crates/rpc/rpc-types/src/eth/engine/payload.rs +++ b/crates/rpc/rpc-types/src/eth/engine/payload.rs @@ -411,7 +411,7 @@ mod tests { #[test] fn payload_body_roundtrip() { - for block in random_block_range(0..100, H256::default(), 0..2) { + for block in random_block_range(0..=99, H256::default(), 0..2) { let unsealed = block.clone().unseal(); let payload_body: ExecutionPayloadBody = unsealed.into(); diff --git a/crates/stages/benches/setup/account_hashing.rs b/crates/stages/benches/setup/account_hashing.rs index 9d2a434d6..f4b2d86f1 100644 --- a/crates/stages/benches/setup/account_hashing.rs +++ b/crates/stages/benches/setup/account_hashing.rs @@ -53,12 +53,7 @@ fn find_stage_range(db: &Path) -> StageRange { } fn generate_testdata_db(num_blocks: u64) -> (PathBuf, StageRange) { - let opts = SeedOpts { - blocks: 0..num_blocks + 1, - accounts: 0..100_000, - txs: 100..150, - transitions: 10_000 + 1, - }; + let opts = SeedOpts { blocks: 0..=num_blocks, accounts: 0..100_000, txs: 100..150 }; let path = Path::new(env!("CARGO_MANIFEST_DIR")).join("testdata").join("account-hashing-bench"); diff --git a/crates/stages/benches/setup/mod.rs b/crates/stages/benches/setup/mod.rs index 2f5146229..9c04e9d42 100644 --- a/crates/stages/benches/setup/mod.rs +++ b/crates/stages/benches/setup/mod.rs @@ -109,7 +109,7 @@ pub(crate) fn txs_testdata(num_blocks: u64) -> PathBuf { .into_iter() .collect(); - let mut blocks = random_block_range(0..num_blocks + 1, H256::zero(), txs_range); + let mut blocks = random_block_range(0..=num_blocks, H256::zero(), txs_range); let (transitions, start_state) = random_transition_range( blocks.iter().take(2), diff --git a/crates/stages/src/pipeline/mod.rs b/crates/stages/src/pipeline/mod.rs index d328b89d5..7229093d4 100644 --- a/crates/stages/src/pipeline/mod.rs +++ b/crates/stages/src/pipeline/mod.rs @@ -371,7 +371,7 @@ where Err(err) => { self.listeners.notify(PipelineEvent::Error { stage_id }); - return if let StageError::Validation { block, error } = err { + let out = if let StageError::Validation { block, error } = err { warn!( target: "sync::pipeline", stage = %stage_id, @@ -402,7 +402,8 @@ where "Stage encountered a non-fatal error: {err}. Retrying" ); continue - } + }; + return out } } } diff --git a/crates/stages/src/stage.rs b/crates/stages/src/stage.rs index dc3cd117e..2fbdccef7 100644 --- a/crates/stages/src/stage.rs +++ b/crates/stages/src/stage.rs @@ -1,10 +1,9 @@ -use std::ops::RangeInclusive; - use crate::{error::StageError, id::StageId}; use async_trait::async_trait; use reth_db::database::Database; use reth_primitives::BlockNumber; use reth_provider::Transaction; +use std::{cmp::min, ops::RangeInclusive}; /// Stage execution input, see [Stage::execute]. #[derive(Debug, Default, PartialEq, Eq, Clone, Copy)] @@ -16,32 +15,42 @@ pub struct ExecInput { } impl ExecInput { + /// Return the progress of the stage or default. + pub fn stage_progress(&self) -> BlockNumber { + self.stage_progress.unwrap_or_default() + } + /// Return the progress of the previous stage or default. pub fn previous_stage_progress(&self) -> BlockNumber { self.previous_stage.as_ref().map(|(_, num)| *num).unwrap_or_default() } - /// Return next execution action. - /// - /// [ExecAction::Done] is returned if there are no blocks to execute in this stage. - /// [ExecAction::Run] is returned if the stage should proceed with execution. - pub fn next_action(&self, max_threshold: Option) -> ExecAction { - // Extract information about the stage progress - let stage_progress = self.stage_progress.unwrap_or_default(); - let previous_stage_progress = self.previous_stage_progress(); + /// Return next block range that needs to be executed. + pub fn next_block_range(&self) -> RangeInclusive { + let (range, _) = self.next_block_range_with_threshold(u64::MAX); + range + } - let start_block = stage_progress + 1; - let end_block = match max_threshold { - Some(threshold) => previous_stage_progress.min(stage_progress + threshold), - None => previous_stage_progress, - }; - let capped = end_block < previous_stage_progress; + /// Return true if this is the first block range to execute. + pub fn is_first_range(&self) -> bool { + self.stage_progress.is_none() + } - if start_block <= end_block { - ExecAction::Run { range: start_block..=end_block, capped } - } else { - ExecAction::Done { stage_progress, target: end_block } - } + /// Return the next block range to execute. + /// Return pair of the block range and if this is final block range. + pub fn next_block_range_with_threshold( + &self, + threshold: u64, + ) -> (RangeInclusive, bool) { + // plus +1 is to skip present block and allways start from block number 1, not 0. + let current_block = self.stage_progress.unwrap_or_default(); + let start = current_block + 1; + let target = self.previous_stage_progress(); + + let end = min(target, current_block.saturating_add(threshold)); + + let is_final_range = end == target; + (start..=end, is_final_range) } } @@ -56,6 +65,24 @@ pub struct UnwindInput { pub bad_block: Option, } +impl UnwindInput { + /// Return next block range that needs to be executed. + pub fn unwind_block_range(&self) -> RangeInclusive { + self.unwind_block_range_with_threshold(u64::MAX) + } + + /// Return the next block range to execute. + pub fn unwind_block_range_with_threshold(&self, threshold: u64) -> RangeInclusive { + // plus +1 is to skip present block. + let start = self.unwind_to + 1; + let mut end = self.stage_progress; + + end = min(end, start.saturating_add(threshold)); + + start..=end + } +} + /// The output of a stage execution. #[derive(Debug, PartialEq, Eq, Clone)] pub struct ExecOutput { @@ -65,6 +92,13 @@ pub struct ExecOutput { pub done: bool, } +impl ExecOutput { + /// Mark the stage as done, checkpointing at the given block number. + pub fn done(stage_progress: BlockNumber) -> Self { + Self { stage_progress, done: true } + } +} + /// The output of a stage unwinding. #[derive(Debug, PartialEq, Eq, Clone)] pub struct UnwindOutput { @@ -72,26 +106,6 @@ pub struct UnwindOutput { pub stage_progress: BlockNumber, } -/// Controls whether a stage should continue execution or not. -#[derive(Debug)] -pub enum ExecAction { - /// The stage should continue with execution. - Run { - /// The execution block range - range: RangeInclusive, - /// The flag indicating whether the range was capped - /// by some max blocks parameter - capped: bool, - }, - /// The stage should terminate since there are no blocks to execute. - Done { - /// The current stage progress - stage_progress: BlockNumber, - /// The execution target provided in [ExecInput]. - target: BlockNumber, - }, -} - /// A stage is a segmented part of the syncing process of the node. /// /// Each stage takes care of a well-defined task, such as downloading headers or executing @@ -125,29 +139,3 @@ pub trait Stage: Send + Sync { input: UnwindInput, ) -> Result; } - -/// Get the next execute action for the stage. Return if the stage has no -/// blocks to process. -macro_rules! exec_or_return { - ($input: expr, $log_target: literal) => { - match $input.next_action(None) { - // Next action cannot be capped without a threshold. - ExecAction::Run { range, capped: _capped } => range.into_inner(), - ExecAction::Done { stage_progress, target } => { - info!(target: $log_target, stage_progress, target, "Target block already reached"); - return Ok(ExecOutput { stage_progress, done: true }) - } - } - }; - ($input: expr, $threshold: expr, $log_target: literal) => { - match $input.next_action(Some($threshold)) { - ExecAction::Run { range, capped } => (range.into_inner(), capped), - ExecAction::Done { stage_progress, target } => { - info!(target: $log_target, stage_progress, target, "Target block already reached"); - return Ok(ExecOutput { stage_progress, done: true }) - } - } - }; -} - -pub(crate) use exec_or_return; diff --git a/crates/stages/src/stages/bodies.rs b/crates/stages/src/stages/bodies.rs index 518abfbea..4d1a26fb2 100644 --- a/crates/stages/src/stages/bodies.rs +++ b/crates/stages/src/stages/bodies.rs @@ -1,19 +1,15 @@ -use crate::{ - exec_or_return, ExecAction, ExecInput, ExecOutput, Stage, StageError, StageId, UnwindInput, - UnwindOutput, -}; +use crate::{ExecInput, ExecOutput, Stage, StageError, StageId, UnwindInput, UnwindOutput}; use futures_util::TryStreamExt; use reth_db::{ cursor::{DbCursorRO, DbCursorRW}, database::Database, models::{StoredBlockBodyIndices, StoredBlockOmmers, StoredBlockWithdrawals}, tables, - transaction::{DbTx, DbTxMut}, + transaction::DbTxMut, }; use reth_interfaces::{ consensus::Consensus, p2p::bodies::{downloader::BodyDownloader, response::BlockResponse}, - provider::ProviderError, }; use reth_provider::Transaction; use std::sync::Arc; @@ -28,7 +24,6 @@ pub const BODIES: StageId = StageId("Bodies"); /// The body stage downloads block bodies for all block headers stored locally in the database. /// /// # Empty blocks - /// /// Blocks with an ommers hash corresponding to no ommers *and* a transaction root corresponding to /// no transactions will not have a block body downloaded for them, since it would be meaningless to @@ -76,13 +71,16 @@ impl Stage for BodyStage { tx: &mut Transaction<'_, DB>, input: ExecInput, ) -> Result { - let (start_block, end_block) = exec_or_return!(input, "sync::stages::bodies"); + let range = input.next_block_range(); + if range.is_empty() { + let (from, to) = range.into_inner(); + info!(target: "sync::stages::bodies", from, "Target block already reached"); + return Ok(ExecOutput::done(to)) + } // Update the header range on the downloader - self.downloader.set_download_range(start_block..end_block + 1)?; - - // Cursor used to get total difficulty - let mut td_cursor = tx.cursor_read::()?; + self.downloader.set_download_range(range.clone())?; + let (from_block, to_block) = range.into_inner(); // Cursors used to write bodies, ommers and transactions let mut block_meta_cursor = tx.cursor_write::()?; @@ -91,11 +89,10 @@ impl Stage for BodyStage { let mut ommers_cursor = tx.cursor_write::()?; let mut withdrawals_cursor = tx.cursor_write::()?; - // Get id for the first transaction and first transition in the block - let (mut next_tx_num, mut next_transition_id) = tx.get_next_block_ids(start_block)?; + // Get id for the next tx_num of zero if there are no transactions. + let mut next_tx_num = tx_cursor.last()?.map(|(id, _)| id + 1).unwrap_or_default(); - let mut highest_block = input.stage_progress.unwrap_or_default(); - debug!(target: "sync::stages::bodies", stage_progress = highest_block, target = end_block, start_tx_id = next_tx_num, next_transition_id, "Commencing sync"); + debug!(target: "sync::stages::bodies", stage_progress = from_block, target = to_block, start_tx_id = next_tx_num, "Commencing sync"); // Task downloader can return `None` only if the response relaying channel was closed. This // is a fatal error to prevent the pipeline from running forever. @@ -103,15 +100,14 @@ impl Stage for BodyStage { self.downloader.try_next().await?.ok_or(StageError::ChannelClosed)?; trace!(target: "sync::stages::bodies", bodies_len = downloaded_bodies.len(), "Writing blocks"); + + let mut highest_block = from_block; for response in downloaded_bodies { // Write block let block_number = response.block_number(); - let difficulty = response.difficulty(); let first_tx_num = next_tx_num; - let first_transition_id = next_transition_id; let mut tx_count = 0; - let mut has_withdrawals = false; match response { BlockResponse::Full(block) => { tx_count = block.body.len() as u64; @@ -145,7 +141,6 @@ impl Stage for BodyStage { // Write withdrawals if any if let Some(withdrawals) = block.withdrawals { if !withdrawals.is_empty() { - has_withdrawals = true; withdrawals_cursor .append(block_number, StoredBlockWithdrawals { withdrawals })?; } @@ -154,31 +149,9 @@ impl Stage for BodyStage { BlockResponse::Empty(_) => {} }; - // The block transition marks the final state at the end of the block. - // Increment the transition if the block contains an addition block reward. - // If the block does not have a reward, the transition will be the same as the - // transition at the last transaction of this block. - let td = td_cursor - .seek(block_number)? - .ok_or(ProviderError::TotalDifficulty { number: block_number })? - .1; - let has_reward = self.consensus.has_block_reward(td.into(), difficulty); - let has_block_change = has_reward || has_withdrawals; - - // Increment transition id for each transaction, - // and by +1 if the block has its own state change (an block reward or withdrawals). - next_transition_id += tx_count + has_block_change as u64; - // insert block meta - block_meta_cursor.append( - block_number, - StoredBlockBodyIndices { - first_tx_num, - first_transition_id, - has_block_change, - tx_count, - }, - )?; + block_meta_cursor + .append(block_number, StoredBlockBodyIndices { first_tx_num, tx_count })?; highest_block = block_number; } @@ -186,8 +159,8 @@ impl Stage for BodyStage { // The stage is "done" if: // - We got fewer blocks than our target // - We reached our target and the target was not limited by the batch size of the stage - let done = highest_block == end_block; - info!(target: "sync::stages::bodies", stage_progress = highest_block, target = end_block, done, "Sync iteration finished"); + let done = highest_block == to_block; + info!(target: "sync::stages::bodies", stage_progress = highest_block, target = to_block, done, "Sync iteration finished"); Ok(ExecOutput { stage_progress: highest_block, done }) } @@ -432,7 +405,6 @@ mod tests { transaction::{DbTx, DbTxMut}, }; use reth_interfaces::{ - consensus::Consensus, p2p::{ bodies::{ client::{BodiesClient, BodiesFut}, @@ -451,7 +423,7 @@ mod tests { use reth_primitives::{BlockBody, BlockNumber, SealedBlock, SealedHeader, TxNumber, H256}; use std::{ collections::{HashMap, VecDeque}, - ops::Range, + ops::RangeInclusive, pin::Pin, sync::Arc, task::{Context, Poll}, @@ -526,17 +498,15 @@ mod tests { fn seed_execution(&mut self, input: ExecInput) -> Result { let start = input.stage_progress.unwrap_or_default(); - let end = input.previous_stage_progress() + 1; - let blocks = random_block_range(start..end, GENESIS_HASH, 0..2); + let end = input.previous_stage_progress(); + let blocks = random_block_range(start..=end, GENESIS_HASH, 0..2); self.tx.insert_headers_with_td(blocks.iter().map(|block| &block.header))?; if let Some(progress) = blocks.first() { // Insert last progress data self.tx.commit(|tx| { let body = StoredBlockBodyIndices { first_tx_num: 0, - first_transition_id: 0, tx_count: progress.body.len() as u64, - has_block_change: true, }; body.tx_num_range().try_for_each(|tx_num| { let transaction = random_signed_tx(); @@ -628,7 +598,6 @@ mod tests { self.tx.query(|tx| { // Acquire cursors on body related tables let mut headers_cursor = tx.cursor_read::()?; - let mut td_cursor = tx.cursor_read::()?; let mut bodies_cursor = tx.cursor_read::()?; let mut ommers_cursor = tx.cursor_read::()?; let mut transaction_cursor = tx.cursor_read::()?; @@ -640,7 +609,6 @@ mod tests { }; let mut prev_number: Option = None; - let mut expected_transition_id = 0; for entry in bodies_cursor.walk(Some(first_body_key))? { let (number, body) = entry?; @@ -675,23 +643,11 @@ mod tests { assert_eq!(tx_block_id, Some(number)); } - assert_eq!(body.first_transition_id, expected_transition_id); - for tx_id in body.tx_num_range() { let tx_entry = transaction_cursor.seek_exact(tx_id)?; assert!(tx_entry.is_some(), "Transaction is missing."); - // Increment expected id for each transaction transition. - expected_transition_id += 1; } - // Increment expected id for block reward. - let td = td_cursor - .seek(number)? - .expect("Missing TD for header") - .1; - if self.consensus.has_block_reward(td.into(), header.difficulty) { - expected_transition_id += 1; - } prev_number = Some(number); } @@ -747,13 +703,16 @@ mod tests { } impl BodyDownloader for TestBodyDownloader { - fn set_download_range(&mut self, range: Range) -> DownloadResult<()> { + fn set_download_range( + &mut self, + range: RangeInclusive, + ) -> DownloadResult<()> { self.headers = VecDeque::from(self.db.view(|tx| -> DownloadResult> { let mut header_cursor = tx.cursor_read::()?; let mut canonical_cursor = tx.cursor_read::()?; - let walker = canonical_cursor.walk_range(range.start..range.end)?; + let walker = canonical_cursor.walk_range(range)?; let mut headers = Vec::default(); for entry in walker { diff --git a/crates/stages/src/stages/execution.rs b/crates/stages/src/stages/execution.rs index 8fc7c1511..8dfb26d03 100644 --- a/crates/stages/src/stages/execution.rs +++ b/crates/stages/src/stages/execution.rs @@ -1,17 +1,14 @@ -use crate::{ - exec_or_return, ExecAction, ExecInput, ExecOutput, Stage, StageError, StageId, UnwindInput, - UnwindOutput, -}; +use crate::{ExecInput, ExecOutput, Stage, StageError, StageId, UnwindInput, UnwindOutput}; use metrics_core::Counter; use reth_db::{ cursor::{DbCursorRO, DbCursorRW, DbDupCursorRO}, database::Database, - models::TransitionIdAddress, + models::BlockNumberAddress, tables, transaction::{DbTx, DbTxMut}, }; use reth_metrics_derive::Metrics; -use reth_primitives::{Address, Block, BlockNumber, BlockWithSenders, U256}; +use reth_primitives::{Block, BlockNumber, BlockWithSenders, U256}; use reth_provider::{ post_state::PostState, BlockExecutor, ExecutorFactory, LatestStateProviderRef, Transaction, }; @@ -111,16 +108,14 @@ impl ExecutionStage { tx: &mut Transaction<'_, DB>, input: ExecInput, ) -> Result { - let ((start_block, end_block), capped) = - exec_or_return!(input, self.commit_threshold, "sync::stages::execution"); - let last_block = input.stage_progress.unwrap_or_default(); + let (range, is_final_range) = input.next_block_range_with_threshold(self.commit_threshold); // Create state provider with cached state let mut executor = self.executor_factory.with_sp(LatestStateProviderRef::new(&**tx)); // Fetch transactions, execute them and generate results let mut state = PostState::default(); - for block_number in start_block..=end_block { + for block_number in range.clone() { let (block, td) = Self::read_block_with_senders(tx, block_number)?; // Configure the executor to use the current state. @@ -137,17 +132,13 @@ impl ExecutionStage { state.extend(block_state); } - // put execution results to database - let first_transition_id = tx.get_block_transition(last_block)?; - let start = Instant::now(); - trace!(target: "sync::stages::execution", changes = state.changes().len(), accounts = state.accounts().len(), "Writing updated state to database"); - state.write_to_db(&**tx, first_transition_id)?; + trace!(target: "sync::stages::execution", accounts = state.accounts().len(), "Writing updated state to database"); + state.write_to_db(&**tx)?; trace!(target: "sync::stages::execution", took = ?Instant::now().duration_since(start), "Wrote state"); - let done = !capped; - info!(target: "sync::stages::execution", stage_progress = end_block, done, "Sync iteration finished"); - Ok(ExecOutput { stage_progress: end_block, done }) + info!(target: "sync::stages::execution", stage_progress = *range.end(), is_final_range, "Sync iteration finished"); + Ok(ExecOutput { stage_progress: *range.end(), done: is_final_range }) } } @@ -204,25 +195,16 @@ impl Stage for ExecutionStage { let mut account_changeset = tx.cursor_dup_write::()?; let mut storage_changeset = tx.cursor_dup_write::()?; - let from_transition_rev = tx.get_block_transition(input.unwind_to)?; - let to_transition_rev = tx.get_block_transition(input.stage_progress)?; + let block_range = input.unwind_to + 1..=input.stage_progress; - if from_transition_rev > to_transition_rev { - panic!("Unwind transition {} (stage progress block #{}) is higher than the transition {} of (unwind block #{})", from_transition_rev, input.stage_progress, to_transition_rev, input.unwind_to); - } - let num_of_tx = (to_transition_rev - from_transition_rev) as usize; - - // if there is no transaction ids, this means blocks were empty and block reward change set - // is not present. - if num_of_tx == 0 { + if block_range.is_empty() { return Ok(UnwindOutput { stage_progress: input.unwind_to }) } // get all batches for account change // Check if walk and walk_dup would do the same thing - let account_changeset_batch = account_changeset - .walk_range(from_transition_rev..to_transition_rev)? - .collect::, _>>()?; + let account_changeset_batch = + account_changeset.walk_range(block_range.clone())?.collect::, _>>()?; // revert all changes to PlainState for (_, changeset) in account_changeset_batch.into_iter().rev() { @@ -235,10 +217,7 @@ impl Stage for ExecutionStage { // get all batches for storage change let storage_changeset_batch = storage_changeset - .walk_range( - TransitionIdAddress((from_transition_rev, Address::zero())).. - TransitionIdAddress((to_transition_rev, Address::zero())), - )? + .walk_range(BlockNumberAddress::range(block_range.clone()))? .collect::, _>>()?; // revert all changes to PlainStorage @@ -258,17 +237,17 @@ impl Stage for ExecutionStage { // Discard unwinded changesets let mut rev_acc_changeset_walker = account_changeset.walk_back(None)?; - while let Some((transition_id, _)) = rev_acc_changeset_walker.next().transpose()? { - if transition_id < from_transition_rev { + while let Some((block_num, _)) = rev_acc_changeset_walker.next().transpose()? { + if block_num < *block_range.start() { break } // delete all changesets - tx.delete::(transition_id, None)?; + tx.delete::(block_num, None)?; } let mut rev_storage_changeset_walker = storage_changeset.walk_back(None)?; while let Some((key, _)) = rev_storage_changeset_walker.next().transpose()? { - if key.transition_id() < from_transition_rev { + if key.block_number() < *block_range.start() { break } // delete all changesets @@ -320,8 +299,8 @@ mod tests { let genesis = SealedBlock::decode(&mut genesis_rlp).unwrap(); let mut block_rlp = hex!("f90262f901f9a075c371ba45999d87f4542326910a11af515897aebce5265d3f6acd1f1161f82fa01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa098f2dcd87c8ae4083e7017a05456c14eea4b1db2032126e27b3b1563d57d7cc0a08151d548273f6683169524b66ca9fe338b9ce42bc3540046c828fd939ae23bcba03f4e5c2ec5b2170b711d97ee755c160457bb58d8daa338e835ec02ae6860bbabb901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000083020000018502540be40082a8798203e800a00000000000000000000000000000000000000000000000000000000000000000880000000000000000f863f861800a8405f5e10094100000000000000000000000000000000000000080801ba07e09e26678ed4fac08a249ebe8ed680bf9051a5e14ad223e4b2b9d26e0208f37a05f6e3f188e3e6eab7d7d3b6568f5eac7d687b08d307d3154ccd8c87b4630509bc0").as_slice(); let block = SealedBlock::decode(&mut block_rlp).unwrap(); - insert_canonical_block(tx.deref_mut(), genesis, None, true).unwrap(); - insert_canonical_block(tx.deref_mut(), block.clone(), None, true).unwrap(); + insert_canonical_block(tx.deref_mut(), genesis, None).unwrap(); + insert_canonical_block(tx.deref_mut(), block.clone(), None).unwrap(); tx.commit().unwrap(); // insert pre state @@ -409,8 +388,8 @@ mod tests { let genesis = SealedBlock::decode(&mut genesis_rlp).unwrap(); let mut block_rlp = hex!("f90262f901f9a075c371ba45999d87f4542326910a11af515897aebce5265d3f6acd1f1161f82fa01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa098f2dcd87c8ae4083e7017a05456c14eea4b1db2032126e27b3b1563d57d7cc0a08151d548273f6683169524b66ca9fe338b9ce42bc3540046c828fd939ae23bcba03f4e5c2ec5b2170b711d97ee755c160457bb58d8daa338e835ec02ae6860bbabb901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000083020000018502540be40082a8798203e800a00000000000000000000000000000000000000000000000000000000000000000880000000000000000f863f861800a8405f5e10094100000000000000000000000000000000000000080801ba07e09e26678ed4fac08a249ebe8ed680bf9051a5e14ad223e4b2b9d26e0208f37a05f6e3f188e3e6eab7d7d3b6568f5eac7d687b08d307d3154ccd8c87b4630509bc0").as_slice(); let block = SealedBlock::decode(&mut block_rlp).unwrap(); - insert_canonical_block(tx.deref_mut(), genesis, None, true).unwrap(); - insert_canonical_block(tx.deref_mut(), block.clone(), None, true).unwrap(); + insert_canonical_block(tx.deref_mut(), genesis, None).unwrap(); + insert_canonical_block(tx.deref_mut(), block.clone(), None).unwrap(); tx.commit().unwrap(); // variables @@ -476,8 +455,8 @@ mod tests { let genesis = SealedBlock::decode(&mut genesis_rlp).unwrap(); let mut block_rlp = hex!("f9025ff901f7a0c86e8cc0310ae7c531c758678ddbfd16fc51c8cef8cec650b032de9869e8b94fa01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa050554882fbbda2c2fd93fdc466db9946ea262a67f7a76cc169e714f105ab583da00967f09ef1dfed20c0eacfaa94d5cd4002eda3242ac47eae68972d07b106d192a0e3c8b47fbfc94667ef4cceb17e5cc21e3b1eebd442cebb27f07562b33836290db90100000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000008302000001830f42408238108203e800a00000000000000000000000000000000000000000000000000000000000000000880000000000000000f862f860800a83061a8094095e7baea6a6c7c4c2dfeb977efac326af552d8780801ba072ed817487b84ba367d15d2f039b5fc5f087d0a8882fbdf73e8cb49357e1ce30a0403d800545b8fc544f92ce8124e2255f8c3c6af93f28243a120585d4c4c6a2a3c0").as_slice(); let block = SealedBlock::decode(&mut block_rlp).unwrap(); - insert_canonical_block(tx.deref_mut(), genesis, None, true).unwrap(); - insert_canonical_block(tx.deref_mut(), block.clone(), None, true).unwrap(); + insert_canonical_block(tx.deref_mut(), genesis, None).unwrap(); + insert_canonical_block(tx.deref_mut(), block.clone(), None).unwrap(); tx.commit().unwrap(); // variables @@ -566,20 +545,15 @@ mod tests { assert_eq!( account_changesets, vec![ - (1, AccountBeforeTx { address: destroyed_address, info: Some(destroyed_info) }), - (1, AccountBeforeTx { address: beneficiary_address, info: None }), - (1, AccountBeforeTx { address: caller_address, info: Some(caller_info) }), ( - 2, - AccountBeforeTx { - address: beneficiary_address, - info: Some(Account { - nonce: 0, - balance: U256::from(0x230a0), - bytecode_hash: None - }) - } - ) + block.number, + AccountBeforeTx { address: destroyed_address, info: Some(destroyed_info) }, + ), + (block.number, AccountBeforeTx { address: beneficiary_address, info: None }), + ( + block.number, + AccountBeforeTx { address: caller_address, info: Some(caller_info) } + ), ] ); @@ -587,11 +561,11 @@ mod tests { storage_changesets, vec![ ( - (1, destroyed_address).into(), + (block.number, destroyed_address).into(), StorageEntry { key: H256::zero(), value: U256::ZERO } ), ( - (1, destroyed_address).into(), + (block.number, destroyed_address).into(), StorageEntry { key: H256::from_low_u64_be(1), value: U256::from(1u64) } ) ] diff --git a/crates/stages/src/stages/hashing_account.rs b/crates/stages/src/stages/hashing_account.rs index 8415e0fc4..8bbfc0b21 100644 --- a/crates/stages/src/stages/hashing_account.rs +++ b/crates/stages/src/stages/hashing_account.rs @@ -11,7 +11,11 @@ use reth_db::{ }; use reth_primitives::{keccak256, AccountHashingCheckpoint}; use reth_provider::Transaction; -use std::{fmt::Debug, ops::Range}; +use std::{ + cmp::max, + fmt::Debug, + ops::{Range, RangeInclusive}, +}; use tokio::sync::mpsc; use tracing::*; @@ -72,27 +76,26 @@ impl AccountHashingStage { } } -#[derive(Clone, Debug)] +// TODO: Rewrite this /// `SeedOpts` provides configuration parameters for calling `AccountHashingStage::seed` /// in unit tests or benchmarks to generate an initial database state for running the /// stage. /// /// In order to check the "full hashing" mode of the stage you want to generate more /// transitions than `AccountHashingStage.clean_threshold`. This requires: -/// 1. Creating enough blocks + transactions so there's enough transactions to generate +/// 1. Creating enough blocks so there's enough transactions to generate /// the required transition keys in the `BlockTransitionIndex` (which depends on the /// `TxTransitionIndex` internally) -/// 2. Setting `transitions > clean_threshold` so that there's enough diffs to actually +/// 2. Setting `blocks.len() > clean_threshold` so that there's enough diffs to actually /// take the 2nd codepath +#[derive(Clone, Debug)] pub struct SeedOpts { /// The range of blocks to be generated - pub blocks: Range, + pub blocks: RangeInclusive, /// The range of accounts to be generated pub accounts: Range, /// The range of transactions to be generated per block. pub txs: Range, - /// The number of transitions to go back, capped at the number of total txs - pub transitions: u64, } #[cfg(any(test, feature = "test-utils"))] @@ -113,12 +116,10 @@ impl AccountHashingStage { use reth_primitives::{Account, H256, U256}; use reth_provider::insert_canonical_block; - let blocks = random_block_range(opts.blocks, H256::zero(), opts.txs); - let num_transitions = blocks.iter().map(|b| b.body.len() as u64).sum(); - let transitions = std::cmp::min(opts.transitions, num_transitions); + let blocks = random_block_range(opts.blocks.clone(), H256::zero(), opts.txs); for block in blocks { - insert_canonical_block(&**tx, block, None, true).unwrap(); + insert_canonical_block(&**tx, block, None).unwrap(); } let mut accounts = random_eoa_account_range(opts.accounts); { @@ -129,18 +130,8 @@ impl AccountHashingStage { account_cursor.append(*addr, *acc)?; } - // seed account changeset - let last_transition = tx - .cursor_read::()? - .last()? - .unwrap() - .1 - .transition_after_block(); - - let first_transition = last_transition.checked_sub(transitions).unwrap_or_default(); - let mut acc_changeset_cursor = tx.cursor_write::()?; - for (t, (addr, acc)) in (first_transition..last_transition).zip(&accounts) { + for (t, (addr, acc)) in (opts.blocks).zip(&accounts) { let Account { nonce, balance, .. } = acc; let prev_acc = Account { nonce: nonce - 1, @@ -171,25 +162,24 @@ impl Stage for AccountHashingStage { tx: &mut Transaction<'_, DB>, input: ExecInput, ) -> Result { - let stage_progress = input.stage_progress.unwrap_or_default(); - let previous_stage_progress = input.previous_stage_progress(); - - // read account changeset, merge it into one changeset and calculate account hashes. - let from_transition = tx.get_block_transition(stage_progress)?; - let to_transition = tx.get_block_transition(previous_stage_progress)?; + let range = input.next_block_range(); + if range.is_empty() { + return Ok(ExecOutput::done(*range.end())) + } + let (from_block, to_block) = range.into_inner(); // if there are more blocks then threshold it is faster to go over Plain state and hash all // account otherwise take changesets aggregate the sets and apply hashing to // AccountHashing table. Also, if we start from genesis, we need to hash from scratch, as // genesis accounts are not in changeset. - if to_transition - from_transition > self.clean_threshold || stage_progress == 0 { + if to_block - from_block > self.clean_threshold || from_block == 1 { let mut checkpoint = self.get_checkpoint(tx)?; if checkpoint.address.is_none() || // Checkpoint is no longer valid if the range of transitions changed. // An already hashed account may have been changed with the new range, and therefore should be hashed again. - checkpoint.to != to_transition || - checkpoint.from != from_transition + checkpoint.from != from_block || + checkpoint.to != to_block { // clear table, load all accounts and hash it tx.clear::()?; @@ -208,7 +198,10 @@ impl Stage for AccountHashingStage { for chunk in &accounts_cursor .walk(start_address.clone())? .take(self.commit_threshold as usize) - .chunks(self.commit_threshold as usize / rayon::current_num_threads()) + .chunks( + max(self.commit_threshold as usize, rayon::current_num_threads()) / + rayon::current_num_threads(), + ) { // An _unordered_ channel to receive results from a rayon job let (tx, rx) = mpsc::unbounded_channel(); @@ -253,19 +246,21 @@ impl Stage for AccountHashingStage { if let Some((next_address, _)) = &next_address { checkpoint.address = Some(next_address.key().unwrap()); - checkpoint.from = from_transition; - checkpoint.to = to_transition; + checkpoint.from = from_block; + checkpoint.to = to_block; } self.save_checkpoint(tx, checkpoint)?; if next_address.is_some() { - return Ok(ExecOutput { stage_progress, done: false }) + // from block is correct here as were are iteration over state for this + // particular block + return Ok(ExecOutput { stage_progress: input.stage_progress(), done: false }) } } else { // Aggregate all transition changesets and and make list of account that have been // changed. - let lists = tx.get_addresses_of_changed_accounts(from_transition, to_transition)?; + let lists = tx.get_addresses_of_changed_accounts(from_block..=to_block)?; // iterate over plain state and get newest value. // Assumption we are okay to make is that plainstate represent // `previous_stage_progress` state. @@ -287,11 +282,10 @@ impl Stage for AccountHashingStage { // There is no threshold on account unwind, we will always take changesets and // apply past values to HashedAccount table. - let from_transition_rev = tx.get_block_transition(input.unwind_to)?; - let to_transition_rev = tx.get_block_transition(input.stage_progress)?; + let range = input.unwind_block_range(); // Aggregate all transition changesets and and make list of account that have been changed. - tx.unwind_account_hashing(from_transition_rev..to_transition_rev)?; + tx.unwind_account_hashing(range)?; Ok(UnwindOutput { stage_progress: input.unwind_to }) } @@ -301,8 +295,8 @@ impl Stage for AccountHashingStage { mod tests { use super::*; use crate::test_utils::{ - stage_test_suite_ext, ExecuteStageTestRunner, TestRunnerError, UnwindStageTestRunner, - PREV_STAGE_ID, + stage_test_suite_ext, ExecuteStageTestRunner, StageTestRunner, TestRunnerError, + UnwindStageTestRunner, PREV_STAGE_ID, }; use assert_matches::assert_matches; use reth_primitives::{Account, U256}; @@ -333,6 +327,59 @@ mod tests { assert!(runner.validate_execution(input, result.ok()).is_ok(), "execution validation"); } + #[tokio::test] + async fn execute_clean_account_hashing_with_commit_threshold() { + let (previous_stage, stage_progress) = (20, 10); + // Set up the runner + let mut runner = AccountHashingTestRunner::default(); + runner.set_clean_threshold(1); + runner.set_commit_threshold(5); + + let mut input = ExecInput { + previous_stage: Some((PREV_STAGE_ID, previous_stage)), + stage_progress: Some(stage_progress), + }; + + runner.seed_execution(input).expect("failed to seed execution"); + + // first run, hash first five account. + let rx = runner.execute(input); + let result = rx.await.unwrap(); + + assert_matches!(result, Ok(ExecOutput {done, stage_progress}) if done == false && stage_progress == 10); + assert_eq!(runner.tx.table::().unwrap().len(), 5); + let fifth_address = runner + .tx + .query(|tx| { + let (address, _) = tx + .cursor_read::()? + .walk(None)? + .skip(5) + .next() + .unwrap() + .unwrap(); + Ok(address) + }) + .unwrap(); + + let stage_progress = runner.stage().get_checkpoint(&runner.tx.inner()).unwrap(); + assert_eq!( + stage_progress, + AccountHashingCheckpoint { address: Some(fifth_address), from: 11, to: 20 } + ); + + // second run, hash next five account. + input.stage_progress = Some(result.unwrap().stage_progress); + let rx = runner.execute(input); + let result = rx.await.unwrap(); + + assert_matches!(result, Ok(ExecOutput {done, stage_progress}) if done == true && stage_progress == 20); + assert_eq!(runner.tx.table::().unwrap().len(), 10); + + // Validate the stage execution + assert!(runner.validate_execution(input, result.ok()).is_ok(), "execution validation"); + } + mod test_utils { use super::*; use crate::{ @@ -437,10 +484,9 @@ mod tests { Ok(AccountHashingStage::seed( &mut self.tx.inner(), SeedOpts { - blocks: 0..input.previous_stage_progress() + 1, - accounts: 0..2, + blocks: 1..=input.previous_stage_progress(), + accounts: 0..10, txs: 0..3, - transitions: 2, }, ) .unwrap()) diff --git a/crates/stages/src/stages/hashing_storage.rs b/crates/stages/src/stages/hashing_storage.rs index abcccf535..875c6cf09 100644 --- a/crates/stages/src/stages/hashing_storage.rs +++ b/crates/stages/src/stages/hashing_storage.rs @@ -4,11 +4,11 @@ use reth_codecs::Compact; use reth_db::{ cursor::DbDupCursorRO, database::Database, - models::TransitionIdAddress, + models::BlockNumberAddress, tables, transaction::{DbTx, DbTxMut}, }; -use reth_primitives::{keccak256, Address, StorageEntry, StorageHashingCheckpoint}; +use reth_primitives::{keccak256, StorageEntry, StorageHashingCheckpoint}; use reth_provider::Transaction; use std::{collections::BTreeMap, fmt::Debug}; use tracing::*; @@ -20,7 +20,7 @@ pub const STORAGE_HASHING: StageId = StageId("StorageHashing"); /// This is preparation before generating intermediate hashes and calculating Merkle tree root. #[derive(Debug)] pub struct StorageHashingStage { - /// The threshold (in number of state transitions) for switching between incremental + /// The threshold (in number of blocks) for switching between incremental /// hashing and full storage hashing. pub clean_threshold: u64, /// The maximum number of slots to process before committing. @@ -83,25 +83,24 @@ impl Stage for StorageHashingStage { tx: &mut Transaction<'_, DB>, input: ExecInput, ) -> Result { - let stage_progress = input.stage_progress.unwrap_or_default(); - let previous_stage_progress = input.previous_stage_progress(); - - // read storage changeset, merge it into one changeset and calculate storage hashes. - let from_transition = tx.get_block_transition(stage_progress)?; - let to_transition = tx.get_block_transition(previous_stage_progress)?; + let range = input.next_block_range(); + if range.is_empty() { + return Ok(ExecOutput::done(*range.end())) + } + let (from_block, to_block) = range.into_inner(); // if there are more blocks then threshold it is faster to go over Plain state and hash all // account otherwise take changesets aggregate the sets and apply hashing to // AccountHashing table. Also, if we start from genesis, we need to hash from scratch, as // genesis accounts are not in changeset, along with their storages. - if to_transition - from_transition > self.clean_threshold || stage_progress == 0 { + if to_block - from_block > self.clean_threshold || from_block == 1 { let mut checkpoint = self.get_checkpoint(tx)?; if checkpoint.address.is_none() || - // Checkpoint is no longer valid if the range of transitions changed. + // Checkpoint is no longer valid if the range of blocks changed. // An already hashed storage may have been changed with the new range, and therefore should be hashed again. - checkpoint.to != to_transition || - checkpoint.from != from_transition + checkpoint.to != to_block || + checkpoint.from != from_block { tx.clear::()?; @@ -173,20 +172,21 @@ impl Stage for StorageHashingStage { if let Some(address) = ¤t_key { checkpoint.address = Some(*address); checkpoint.storage = current_subkey; - checkpoint.from = from_transition; - checkpoint.to = to_transition; + checkpoint.from = from_block; + checkpoint.to = to_block; } self.save_checkpoint(tx, checkpoint)?; if current_key.is_some() { - return Ok(ExecOutput { stage_progress, done: false }) + // `from_block` is correct here as were are iteration over state for this + // particular block. + return Ok(ExecOutput { stage_progress: input.stage_progress(), done: false }) } } else { - // Aggregate all transition changesets and and make list of storages that have been + // Aggregate all changesets and and make list of storages that have been // changed. - let lists = - tx.get_addresses_and_keys_of_changed_storages(from_transition, to_transition)?; + let lists = tx.get_addresses_and_keys_of_changed_storages(from_block..=to_block)?; // iterate over plain state and get newest storage value. // Assumption we are okay with is that plain state represent // `previous_stage_progress` state. @@ -204,13 +204,9 @@ impl Stage for StorageHashingStage { tx: &mut Transaction<'_, DB>, input: UnwindInput, ) -> Result { - let from_transition_rev = tx.get_block_transition(input.unwind_to)?; - let to_transition_rev = tx.get_block_transition(input.stage_progress)?; + let range = input.unwind_block_range(); - tx.unwind_storage_hashing( - TransitionIdAddress((from_transition_rev, Address::zero())).. - TransitionIdAddress((to_transition_rev, Address::zero())), - )?; + tx.unwind_storage_hashing(BlockNumberAddress::range(range))?; Ok(UnwindOutput { stage_progress: input.unwind_to }) } @@ -223,15 +219,16 @@ mod tests { stage_test_suite_ext, ExecuteStageTestRunner, StageTestRunner, TestRunnerError, TestTransaction, UnwindStageTestRunner, PREV_STAGE_ID, }; + use assert_matches::assert_matches; use reth_db::{ cursor::{DbCursorRO, DbCursorRW}, mdbx::{tx::Tx, WriteMap, RW}, - models::{StoredBlockBodyIndices, TransitionIdAddress}, + models::{BlockNumberAddress, StoredBlockBodyIndices}, }; use reth_interfaces::test_utils::generators::{ random_block_range, random_contract_account_range, }; - use reth_primitives::{SealedBlock, StorageEntry, H256, U256}; + use reth_primitives::{Address, SealedBlock, StorageEntry, H256, U256}; stage_test_suite_ext!(StorageHashingTestRunner, storage_hashing); @@ -278,6 +275,100 @@ mod tests { } } + #[tokio::test] + async fn execute_clean_account_hashing_with_commit_threshold() { + let (previous_stage, stage_progress) = (500, 100); + // Set up the runner + let mut runner = StorageHashingTestRunner::default(); + runner.set_clean_threshold(1); + runner.set_commit_threshold(500); + + let mut input = ExecInput { + previous_stage: Some((PREV_STAGE_ID, previous_stage)), + stage_progress: Some(stage_progress), + }; + + runner.seed_execution(input).expect("failed to seed execution"); + + // first run, hash first half of storages. + let rx = runner.execute(input); + let result = rx.await.unwrap(); + assert_matches!(result, Ok(ExecOutput {done, stage_progress}) if done == false && stage_progress == 100); + assert_eq!(runner.tx.table::().unwrap().len(), 500); + let (progress_address, progress_key) = runner + .tx + .query(|tx| { + let (address, entry) = tx + .cursor_read::()? + .walk(None)? + .skip(500) + .next() + .unwrap() + .unwrap(); + Ok((address, entry.key)) + }) + .unwrap(); + + let stage_progress = runner.stage().get_checkpoint(&runner.tx.inner()).unwrap(); + let progress_key = stage_progress.storage.map(|_| progress_key); + assert_eq!( + stage_progress, + StorageHashingCheckpoint { + address: Some(progress_address), + storage: progress_key, + from: 101, + to: 500 + } + ); + + // second run with commit threshold of 2 to check if subkey is set. + runner.set_commit_threshold(2); + let rx = runner.execute(input); + let result = rx.await.unwrap(); + assert_matches!(result, Ok(ExecOutput {done, stage_progress}) if done == false && stage_progress == 100); + assert_eq!(runner.tx.table::().unwrap().len(), 502); + let (progress_address, progress_key) = runner + .tx + .query(|tx| { + let (address, entry) = tx + .cursor_read::()? + .walk(None)? + .skip(502) + .next() + .unwrap() + .unwrap(); + Ok((address, entry.key)) + }) + .unwrap(); + + let stage_progress = runner.stage().get_checkpoint(&runner.tx.inner()).unwrap(); + let progress_key = stage_progress.storage.map(|_| progress_key); + assert_eq!( + stage_progress, + StorageHashingCheckpoint { + address: Some(progress_address), + storage: progress_key, + from: 101, + to: 500 + } + ); + + // third last run, hash rest of storages. + runner.set_commit_threshold(1000); + input.stage_progress = Some(result.unwrap().stage_progress); + let rx = runner.execute(input); + let result = rx.await.unwrap(); + + assert_matches!(result, Ok(ExecOutput {done, stage_progress}) if done == true && stage_progress == 500); + assert_eq!( + runner.tx.table::().unwrap().len(), + runner.tx.table::().unwrap().len() + ); + + // Validate the stage execution + assert!(runner.validate_execution(input, result.ok()).is_ok(), "execution validation"); + } + struct StorageHashingTestRunner { tx: TestTransaction, commit_threshold: u64, @@ -310,22 +401,22 @@ mod tests { type Seed = Vec; fn seed_execution(&mut self, input: ExecInput) -> Result { - let stage_progress = input.stage_progress.unwrap_or_default(); - let end = input.previous_stage_progress() + 1; + let stage_progress = input.stage_progress.unwrap_or_default() + 1; + let end = input.previous_stage_progress(); let n_accounts = 31; let mut accounts = random_contract_account_range(&mut (0..n_accounts)); - let blocks = random_block_range(stage_progress..end, H256::zero(), 0..3); + let blocks = random_block_range(stage_progress..=end, H256::zero(), 0..3); self.tx.insert_headers(blocks.iter().map(|block| &block.header))?; let iter = blocks.iter(); - let (mut next_transition_id, mut next_tx_num) = (0, 0); - let mut first_transition_id = next_transition_id; + let mut next_tx_num = 0; let mut first_tx_num = next_tx_num; for progress in iter { // Insert last progress data + let block_number = progress.number; self.tx.commit(|tx| { progress.body.iter().try_for_each(|transaction| { tx.put::(transaction.hash(), next_tx_num)?; @@ -342,14 +433,13 @@ mod tests { }; self.insert_storage_entry( tx, - (next_transition_id, *addr).into(), + (block_number, *addr).into(), new_entry, progress.header.number == stage_progress, )?; } next_tx_num += 1; - next_transition_id += 1; Ok(()) })?; @@ -358,24 +448,20 @@ mod tests { if has_reward { self.insert_storage_entry( tx, - (next_transition_id, Address::random()).into(), + (block_number, Address::random()).into(), StorageEntry { key: keccak256("mining"), value: U256::from(rand::random::()), }, progress.header.number == stage_progress, )?; - next_transition_id += 1; } let body = StoredBlockBodyIndices { first_tx_num, - first_transition_id, tx_count: progress.body.len() as u64, - has_block_change: has_reward, }; - first_transition_id = next_transition_id; first_tx_num = next_tx_num; tx.put::(progress.number, body) @@ -448,7 +534,7 @@ mod tests { fn insert_storage_entry( &self, tx: &Tx<'_, RW, WriteMap>, - tid_address: TransitionIdAddress, + tid_address: BlockNumberAddress, entry: StorageEntry, hash: bool, ) -> Result<(), reth_db::Error> { @@ -486,25 +572,20 @@ mod tests { fn unwind_storage(&self, input: UnwindInput) -> Result<(), TestRunnerError> { tracing::debug!("unwinding storage..."); - let target_transition = self - .tx - .inner() - .get_block_transition(input.unwind_to) - .map_err(|e| TestRunnerError::Internal(Box::new(e)))?; - + let target_block = input.unwind_to; self.tx.commit(|tx| { let mut storage_cursor = tx.cursor_dup_write::()?; let mut changeset_cursor = tx.cursor_dup_read::()?; let mut rev_changeset_walker = changeset_cursor.walk_back(None)?; - while let Some((tid_address, entry)) = rev_changeset_walker.next().transpose()? { - if tid_address.transition_id() < target_transition { + while let Some((bn_address, entry)) = rev_changeset_walker.next().transpose()? { + if bn_address.block_number() < target_block { break } if storage_cursor - .seek_by_key_subkey(tid_address.address(), entry.key)? + .seek_by_key_subkey(bn_address.address(), entry.key)? .filter(|e| e.key == entry.key) .is_some() { @@ -512,7 +593,7 @@ mod tests { } if entry.value != U256::ZERO { - storage_cursor.upsert(tid_address.address(), entry)?; + storage_cursor.upsert(bn_address.address(), entry)?; } } Ok(()) diff --git a/crates/stages/src/stages/index_account_history.rs b/crates/stages/src/stages/index_account_history.rs index 0c28aa79e..f5e2abd36 100644 --- a/crates/stages/src/stages/index_account_history.rs +++ b/crates/stages/src/stages/index_account_history.rs @@ -36,24 +36,18 @@ impl Stage for IndexAccountHistoryStage { tx: &mut Transaction<'_, DB>, input: ExecInput, ) -> Result { - let stage_progress = input.stage_progress.unwrap_or_default(); - let previous_stage_progress = input.previous_stage_progress(); + let (range, is_final_range) = input.next_block_range_with_threshold(self.commit_threshold); - // read account changeset, merge it into one changeset and calculate account hashes. - let from_transition = tx.get_block_transition(stage_progress)?; - // NOTE: can probably done more probabilistic take of bundles with transition but it is - // guess game for later. Transitions better reflect amount of work. - let to_block = - std::cmp::min(stage_progress + self.commit_threshold, previous_stage_progress); - let to_transition = tx.get_block_transition(to_block)?; + if range.is_empty() { + return Ok(ExecOutput::done(*range.end())) + } - let indices = - tx.get_account_transition_ids_from_changeset(from_transition, to_transition)?; + let indices = tx.get_account_transition_ids_from_changeset(range.clone())?; // Insert changeset to history index tx.insert_account_history_index(indices)?; info!(target: "sync::stages::index_account_history", "Stage finished"); - Ok(ExecOutput { stage_progress: to_block, done: to_block == previous_stage_progress }) + Ok(ExecOutput { stage_progress: *range.end(), done: is_final_range }) } /// Unwind the stage. @@ -63,10 +57,9 @@ impl Stage for IndexAccountHistoryStage { input: UnwindInput, ) -> Result { info!(target: "sync::stages::index_account_history", to_block = input.unwind_to, "Unwinding"); - let from_transition_rev = tx.get_block_transition(input.unwind_to)?; - let to_transition_rev = tx.get_block_transition(input.stage_progress)?; + let range = input.unwind_block_range(); - tx.unwind_account_history_indices(from_transition_rev..to_transition_rev)?; + tx.unwind_account_history_indices(range)?; // from HistoryIndex higher than that number. Ok(UnwindOutput { stage_progress: input.unwind_to }) @@ -86,7 +79,7 @@ mod tests { }, tables, transaction::DbTxMut, - TransitionList, + BlockNumberList, }; use reth_primitives::{hex_literal::hex, H160}; @@ -98,15 +91,15 @@ mod tests { /// Shard for account fn shard(shard_index: u64) -> ShardedKey { - ShardedKey { key: ADDRESS, highest_transition_id: shard_index } + ShardedKey { key: ADDRESS, highest_block_number: shard_index } } - fn list(list: &[usize]) -> TransitionList { - TransitionList::new(list).unwrap() + fn list(list: &[usize]) -> BlockNumberList { + BlockNumberList::new(list).unwrap() } fn cast( - table: Vec<(ShardedKey, TransitionList)>, + table: Vec<(ShardedKey, BlockNumberList)>, ) -> BTreeMap, Vec> { table .into_iter() @@ -123,27 +116,19 @@ mod tests { // we just need first and last tx.put::( 0, - StoredBlockBodyIndices { - first_transition_id: 0, - tx_count: 3, - ..Default::default() - }, + StoredBlockBodyIndices { tx_count: 3, ..Default::default() }, ) .unwrap(); tx.put::( 5, - StoredBlockBodyIndices { - first_transition_id: 3, - tx_count: 5, - ..Default::default() - }, + StoredBlockBodyIndices { tx_count: 5, ..Default::default() }, ) .unwrap(); // setup changeset that are going to be applied to history index tx.put::(4, acc()).unwrap(); - tx.put::(6, acc()).unwrap(); + tx.put::(5, acc()).unwrap(); Ok(()) }) .unwrap() @@ -181,7 +166,7 @@ mod tests { // verify let table = cast(tx.table::().unwrap()); - assert_eq!(table, BTreeMap::from([(shard(u64::MAX), vec![4, 6]),])); + assert_eq!(table, BTreeMap::from([(shard(u64::MAX), vec![4, 5])])); // unwind unwind(&tx, 5, 0).await; @@ -209,7 +194,7 @@ mod tests { // verify let table = cast(tx.table::().unwrap()); - assert_eq!(table, BTreeMap::from([(shard(u64::MAX), vec![1, 2, 3, 4, 6]),])); + assert_eq!(table, BTreeMap::from([(shard(u64::MAX), vec![1, 2, 3, 4, 5]),])); // unwind unwind(&tx, 5, 0).await; @@ -240,7 +225,7 @@ mod tests { let table = cast(tx.table::().unwrap()); assert_eq!( table, - BTreeMap::from([(shard(3), full_list.clone()), (shard(u64::MAX), vec![4, 6])]) + BTreeMap::from([(shard(3), full_list.clone()), (shard(u64::MAX), vec![4, 5])]) ); // unwind @@ -270,7 +255,7 @@ mod tests { // verify close_full_list.push(4); - close_full_list.push(6); + close_full_list.push(5); let table = cast(tx.table::().unwrap()); assert_eq!(table, BTreeMap::from([(shard(u64::MAX), close_full_list.clone()),])); @@ -308,7 +293,7 @@ mod tests { let table = cast(tx.table::().unwrap()); assert_eq!( table, - BTreeMap::from([(shard(4), close_full_list.clone()), (shard(u64::MAX), vec![6])]) + BTreeMap::from([(shard(4), close_full_list.clone()), (shard(u64::MAX), vec![5])]) ); // unwind @@ -345,7 +330,7 @@ mod tests { BTreeMap::from([ (shard(1), full_list.clone()), (shard(2), full_list.clone()), - (shard(u64::MAX), vec![2, 3, 4, 6]) + (shard(u64::MAX), vec![2, 3, 4, 5]) ]) ); diff --git a/crates/stages/src/stages/index_storage_history.rs b/crates/stages/src/stages/index_storage_history.rs index f4f4643f5..c9d5fd4d1 100644 --- a/crates/stages/src/stages/index_storage_history.rs +++ b/crates/stages/src/stages/index_storage_history.rs @@ -1,6 +1,5 @@ use crate::{ExecInput, ExecOutput, Stage, StageError, StageId, UnwindInput, UnwindOutput}; -use reth_db::{database::Database, models::TransitionIdAddress}; -use reth_primitives::Address; +use reth_db::{database::Database, models::BlockNumberAddress}; use reth_provider::Transaction; use std::fmt::Debug; use tracing::*; @@ -37,24 +36,18 @@ impl Stage for IndexStorageHistoryStage { tx: &mut Transaction<'_, DB>, input: ExecInput, ) -> Result { - let stage_progress = input.stage_progress.unwrap_or_default(); - let previous_stage_progress = input.previous_stage_progress(); + let target = input.previous_stage_progress(); + let (range, is_final_range) = input.next_block_range_with_threshold(self.commit_threshold); - // read storge changeset, merge it into one changeset and calculate account hashes. - let from_transition = tx.get_block_transition(stage_progress)?; + if range.is_empty() { + return Ok(ExecOutput::done(target)) + } - // NOTE: can probably done more probabilistic take of bundles with transition but it is - // guess game for later. Transitions better reflect amount of work. - let to_block = - std::cmp::min(stage_progress + self.commit_threshold, previous_stage_progress); - let to_transition = tx.get_block_transition(to_block)?; - - let indices = - tx.get_storage_transition_ids_from_changeset(from_transition, to_transition)?; + let indices = tx.get_storage_transition_ids_from_changeset(range.clone())?; tx.insert_storage_history_index(indices)?; info!(target: "sync::stages::index_storage_history", "Stage finished"); - Ok(ExecOutput { stage_progress: to_block, done: to_block == previous_stage_progress }) + Ok(ExecOutput { stage_progress: *range.end(), done: is_final_range }) } /// Unwind the stage. @@ -64,13 +57,9 @@ impl Stage for IndexStorageHistoryStage { input: UnwindInput, ) -> Result { info!(target: "sync::stages::index_account_history", to_block = input.unwind_to, "Unwinding"); - let from_transition_rev = tx.get_block_transition(input.unwind_to)?; - let to_transition_rev = tx.get_block_transition(input.stage_progress)?; + let range = input.unwind_block_range(); - tx.unwind_storage_history_indices( - TransitionIdAddress((from_transition_rev, Address::zero())).. - TransitionIdAddress((to_transition_rev, Address::zero())), - )?; + tx.unwind_storage_history_indices(BlockNumberAddress::range(range))?; Ok(UnwindOutput { stage_progress: input.unwind_to }) } @@ -86,11 +75,11 @@ mod tests { use reth_db::{ models::{ storage_sharded_key::{StorageShardedKey, NUM_OF_INDICES_IN_SHARD}, - ShardedKey, StoredBlockBodyIndices, TransitionIdAddress, + BlockNumberAddress, ShardedKey, StoredBlockBodyIndices, }, tables, transaction::DbTxMut, - TransitionList, + BlockNumberList, }; use reth_primitives::{hex_literal::hex, StorageEntry, H160, H256, U256}; @@ -103,24 +92,24 @@ mod tests { StorageEntry { key, value: U256::ZERO } } - fn trns(transition_id: u64) -> TransitionIdAddress { - TransitionIdAddress((transition_id, ADDRESS)) + fn trns(transition_id: u64) -> BlockNumberAddress { + BlockNumberAddress((transition_id, ADDRESS)) } /// Shard for account fn shard(shard_index: u64) -> StorageShardedKey { StorageShardedKey { address: ADDRESS, - sharded_key: ShardedKey { key: STORAGE_KEY, highest_transition_id: shard_index }, + sharded_key: ShardedKey { key: STORAGE_KEY, highest_block_number: shard_index }, } } - fn list(list: &[usize]) -> TransitionList { - TransitionList::new(list).unwrap() + fn list(list: &[usize]) -> BlockNumberList { + BlockNumberList::new(list).unwrap() } fn cast( - table: Vec<(StorageShardedKey, TransitionList)>, + table: Vec<(StorageShardedKey, BlockNumberList)>, ) -> BTreeMap> { table .into_iter() @@ -137,27 +126,19 @@ mod tests { // we just need first and last tx.put::( 0, - StoredBlockBodyIndices { - first_transition_id: 0, - tx_count: 3, - ..Default::default() - }, + StoredBlockBodyIndices { tx_count: 3, ..Default::default() }, ) .unwrap(); tx.put::( 5, - StoredBlockBodyIndices { - first_transition_id: 3, - tx_count: 5, - ..Default::default() - }, + StoredBlockBodyIndices { tx_count: 5, ..Default::default() }, ) .unwrap(); // setup changeset that are going to be applied to history index tx.put::(trns(4), storage(STORAGE_KEY)).unwrap(); - tx.put::(trns(6), storage(STORAGE_KEY)).unwrap(); + tx.put::(trns(5), storage(STORAGE_KEY)).unwrap(); Ok(()) }) .unwrap() @@ -195,7 +176,7 @@ mod tests { // verify let table = cast(tx.table::().unwrap()); - assert_eq!(table, BTreeMap::from([(shard(u64::MAX), vec![4, 6]),])); + assert_eq!(table, BTreeMap::from([(shard(u64::MAX), vec![4, 5]),])); // unwind unwind(&tx, 5, 0).await; @@ -223,7 +204,7 @@ mod tests { // verify let table = cast(tx.table::().unwrap()); - assert_eq!(table, BTreeMap::from([(shard(u64::MAX), vec![1, 2, 3, 4, 6]),])); + assert_eq!(table, BTreeMap::from([(shard(u64::MAX), vec![1, 2, 3, 4, 5]),])); // unwind unwind(&tx, 5, 0).await; @@ -257,7 +238,7 @@ mod tests { let table = cast(tx.table::().unwrap()); assert_eq!( table, - BTreeMap::from([(shard(3), full_list.clone()), (shard(u64::MAX), vec![4, 6])]) + BTreeMap::from([(shard(3), full_list.clone()), (shard(u64::MAX), vec![4, 5])]) ); // unwind @@ -287,7 +268,7 @@ mod tests { // verify close_full_list.push(4); - close_full_list.push(6); + close_full_list.push(5); let table = cast(tx.table::().unwrap()); assert_eq!(table, BTreeMap::from([(shard(u64::MAX), close_full_list.clone()),])); @@ -325,7 +306,7 @@ mod tests { let table = cast(tx.table::().unwrap()); assert_eq!( table, - BTreeMap::from([(shard(4), close_full_list.clone()), (shard(u64::MAX), vec![6])]) + BTreeMap::from([(shard(4), close_full_list.clone()), (shard(u64::MAX), vec![5])]) ); // unwind @@ -362,7 +343,7 @@ mod tests { BTreeMap::from([ (shard(1), full_list.clone()), (shard(2), full_list.clone()), - (shard(u64::MAX), vec![2, 3, 4, 6]) + (shard(u64::MAX), vec![2, 3, 4, 5]) ]) ); diff --git a/crates/stages/src/stages/merkle.rs b/crates/stages/src/stages/merkle.rs index dff56db1d..575a05530 100644 --- a/crates/stages/src/stages/merkle.rs +++ b/crates/stages/src/stages/merkle.rs @@ -115,33 +115,31 @@ impl Stage for MerkleStage { MerkleStage::Both { clean_threshold } => *clean_threshold, }; - let stage_progress = input.stage_progress.unwrap_or_default(); - let previous_stage_progress = input.previous_stage_progress(); + let range = input.next_block_range(); + let (from_block, to_block) = range.clone().into_inner(); + let current_blook = input.previous_stage_progress(); - let from_transition = tx.get_block_transition(stage_progress)?; - let to_transition = tx.get_block_transition(previous_stage_progress)?; + let block_root = tx.get_header(current_blook)?.state_root; - let block_root = tx.get_header(previous_stage_progress)?.state_root; - - let trie_root = if from_transition == to_transition { + let trie_root = if range.is_empty() { block_root - } else if to_transition - from_transition > threshold || stage_progress == 0 { + } else if to_block - from_block > threshold || from_block == 1 { // if there are more blocks than threshold it is faster to rebuild the trie - debug!(target: "sync::stages::merkle::exec", current = ?stage_progress, target = ?previous_stage_progress, "Rebuilding trie"); + debug!(target: "sync::stages::merkle::exec", current = ?current_blook, target = ?to_block, "Rebuilding trie"); tx.clear::()?; tx.clear::()?; StateRoot::new(tx.deref_mut()).root(None).map_err(|e| StageError::Fatal(Box::new(e)))? } else { - debug!(target: "sync::stages::merkle::exec", current = ?stage_progress, target = - ?previous_stage_progress, "Updating trie"); // Iterate over - StateRoot::incremental_root(tx.deref_mut(), from_transition..to_transition, None) + debug!(target: "sync::stages::merkle::exec", current = ?current_blook, target = + ?to_block, "Updating trie"); // Iterate over + StateRoot::incremental_root(tx.deref_mut(), range, None) .map_err(|e| StageError::Fatal(Box::new(e)))? }; - self.validate_state_root(trie_root, block_root, previous_stage_progress)?; + self.validate_state_root(trie_root, block_root, to_block)?; info!(target: "sync::stages::merkle::exec", "Stage finished"); - Ok(ExecOutput { stage_progress: previous_stage_progress, done: true }) + Ok(ExecOutput { stage_progress: to_block, done: true }) } /// Unwind the stage. @@ -150,6 +148,7 @@ impl Stage for MerkleStage { tx: &mut Transaction<'_, DB>, input: UnwindInput, ) -> Result { + let range = input.unwind_block_range(); if matches!(self, MerkleStage::Execution { .. }) { info!(target: "sync::stages::merkle::exec", "Stage is always skipped"); return Ok(UnwindOutput { stage_progress: input.unwind_to }) @@ -161,14 +160,10 @@ impl Stage for MerkleStage { return Ok(UnwindOutput { stage_progress: input.unwind_to }) } - let from_transition = tx.get_block_transition(input.unwind_to)?; - let to_transition = tx.get_block_transition(input.stage_progress)?; - // Unwind trie only if there are transitions - if from_transition < to_transition { - let block_root = - StateRoot::incremental_root(tx.deref_mut(), from_transition..to_transition, None) - .map_err(|e| StageError::Fatal(Box::new(e)))?; + if !range.is_empty() { + let block_root = StateRoot::incremental_root(tx.deref_mut(), range, None) + .map_err(|e| StageError::Fatal(Box::new(e)))?; let target_root = tx.get_header(input.unwind_to)?.state_root; self.validate_state_root(block_root, target_root, input.unwind_to)?; } else { @@ -288,7 +283,8 @@ mod tests { fn seed_execution(&mut self, input: ExecInput) -> Result { let stage_progress = input.stage_progress.unwrap_or_default(); - let end = input.previous_stage_progress() + 1; + let start = stage_progress + 1; + let end = input.previous_stage_progress(); let num_of_accounts = 31; let accounts = random_contract_account_range(&mut (0..num_of_accounts)) @@ -313,7 +309,7 @@ mod tests { let head_hash = sealed_head.hash(); let mut blocks = vec![sealed_head]; - blocks.extend(random_block_range((stage_progress + 1)..end, head_hash, 0..3)); + blocks.extend(random_block_range(start..=end, head_hash, 0..3)); self.tx.insert_blocks(blocks.iter(), None)?; let (transitions, final_state) = random_transition_range( @@ -322,7 +318,8 @@ mod tests { 0..3, 0..256, ); - self.tx.insert_transitions(transitions, None)?; + // add block changeset from block 1. + self.tx.insert_transitions(transitions, Some(start))?; self.tx.insert_accounts_and_storages(final_state)?; // Calculate state root @@ -349,7 +346,7 @@ mod tests { Ok(state_root_prehashed(accounts.into_iter())) })?; - let last_block_number = end - 1; + let last_block_number = end; self.tx.commit(|tx| { let mut last_header = tx.get::(last_block_number)?.unwrap(); last_header.state_root = root; @@ -371,12 +368,7 @@ mod tests { impl UnwindStageTestRunner for MerkleTestRunner { fn before_unwind(&self, input: UnwindInput) -> Result<(), TestRunnerError> { - let target_transition = self - .tx - .inner() - .get_block_transition(input.unwind_to) - .map_err(|e| TestRunnerError::Internal(Box::new(e))) - .unwrap(); + let target_block = input.unwind_to + 1; self.tx .commit(|tx| { @@ -392,7 +384,7 @@ mod tests { while let Some((tid_address, entry)) = rev_changeset_walker.next().transpose().unwrap() { - if tid_address.transition_id() < target_transition { + if tid_address.block_number() < target_block { break } @@ -420,10 +412,10 @@ mod tests { tx.cursor_dup_write::().unwrap(); let mut rev_changeset_walker = changeset_cursor.walk_back(None).unwrap(); - while let Some((transition_id, account_before_tx)) = + while let Some((block_number, account_before_tx)) = rev_changeset_walker.next().transpose().unwrap() { - if transition_id < target_transition { + if block_number < target_block { break } diff --git a/crates/stages/src/stages/sender_recovery.rs b/crates/stages/src/stages/sender_recovery.rs index dcf4b4a57..3fa65f844 100644 --- a/crates/stages/src/stages/sender_recovery.rs +++ b/crates/stages/src/stages/sender_recovery.rs @@ -1,7 +1,4 @@ -use crate::{ - exec_or_return, ExecAction, ExecInput, ExecOutput, Stage, StageError, StageId, UnwindInput, - UnwindOutput, -}; +use crate::{ExecInput, ExecOutput, Stage, StageError, StageId, UnwindInput, UnwindOutput}; use itertools::Itertools; use reth_db::{ cursor::{DbCursorRO, DbCursorRW}, @@ -53,20 +50,22 @@ impl Stage for SenderRecoveryStage { tx: &mut Transaction<'_, DB>, input: ExecInput, ) -> Result { - let ((start_block, end_block), capped) = - exec_or_return!(input, self.commit_threshold, "sync::stages::sender_recovery"); - let done = !capped; + let (range, is_final_range) = input.next_block_range_with_threshold(self.commit_threshold); + if range.is_empty() { + return Ok(ExecOutput::done(*range.end())) + } + let (start_block, end_block) = range.clone().into_inner(); // Look up the start index for the transaction range - let first_tx_num = tx.get_block_meta(start_block)?.first_tx_num(); + let first_tx_num = tx.block_body_indices(start_block)?.first_tx_num(); // Look up the end index for transaction range (inclusive) - let last_tx_num = tx.get_block_meta(end_block)?.last_tx_num(); + let last_tx_num = tx.block_body_indices(end_block)?.last_tx_num(); // No transactions to walk over if first_tx_num > last_tx_num { info!(target: "sync::stages::sender_recovery", first_tx_num, last_tx_num, "Target transaction already reached"); - return Ok(ExecOutput { stage_progress: end_block, done }) + return Ok(ExecOutput { stage_progress: end_block, done: is_final_range }) } // Acquire the cursor for inserting elements @@ -131,8 +130,8 @@ impl Stage for SenderRecoveryStage { } } - info!(target: "sync::stages::sender_recovery", stage_progress = end_block, done, "Sync iteration finished"); - Ok(ExecOutput { stage_progress: end_block, done }) + info!(target: "sync::stages::sender_recovery", stage_progress = end_block, is_final_range, "Sync iteration finished"); + Ok(ExecOutput { stage_progress: end_block, done: is_final_range }) } /// Unwind the stage. @@ -143,7 +142,7 @@ impl Stage for SenderRecoveryStage { ) -> Result { info!(target: "sync::stages::sender_recovery", to_block = input.unwind_to, "Unwinding"); // Lookup latest tx id that we should unwind to - let latest_tx_id = tx.get_block_meta(input.unwind_to)?.last_tx_num(); + let latest_tx_id = tx.block_body_indices(input.unwind_to)?.last_tx_num(); tx.unwind_table_by_num::(latest_tx_id)?; Ok(UnwindOutput { stage_progress: input.unwind_to }) } @@ -190,7 +189,7 @@ mod tests { // Insert blocks with a single transaction at block `stage_progress + 10` let non_empty_block_number = stage_progress + 10; - let blocks = (stage_progress..input.previous_stage_progress() + 1) + let blocks = (stage_progress..=input.previous_stage_progress()) .map(|number| { random_block(number, None, Some((number == non_empty_block_number) as u8), None) }) @@ -274,7 +273,7 @@ mod tests { /// 2. If the is no requested block entry in the bodies table, /// but [tables::TxSenders] is not empty. fn ensure_no_senders_by_block(&self, block: BlockNumber) -> Result<(), TestRunnerError> { - let body_result = self.tx.inner().get_block_meta(block); + let body_result = self.tx.inner().block_body_indices(block); match body_result { Ok(body) => self .tx @@ -305,9 +304,9 @@ mod tests { fn seed_execution(&mut self, input: ExecInput) -> Result { let stage_progress = input.stage_progress.unwrap_or_default(); - let end = input.previous_stage_progress() + 1; + let end = input.previous_stage_progress(); - let blocks = random_block_range(stage_progress..end, H256::zero(), 0..2); + let blocks = random_block_range(stage_progress..=end, H256::zero(), 0..2); self.tx.insert_blocks(blocks.iter(), None)?; Ok(blocks) } diff --git a/crates/stages/src/stages/total_difficulty.rs b/crates/stages/src/stages/total_difficulty.rs index 69f1ccad9..b8f4c76cd 100644 --- a/crates/stages/src/stages/total_difficulty.rs +++ b/crates/stages/src/stages/total_difficulty.rs @@ -1,7 +1,4 @@ -use crate::{ - exec_or_return, ExecAction, ExecInput, ExecOutput, Stage, StageError, StageId, UnwindInput, - UnwindOutput, -}; +use crate::{ExecInput, ExecOutput, Stage, StageError, StageId, UnwindInput, UnwindOutput}; use reth_db::{ cursor::{DbCursorRO, DbCursorRW}, database::Database, @@ -56,8 +53,8 @@ impl Stage for TotalDifficultyStage { tx: &mut Transaction<'_, DB>, input: ExecInput, ) -> Result { - let ((start_block, end_block), capped) = - exec_or_return!(input, self.commit_threshold, "sync::stages::total_difficulty"); + let (range, is_final_range) = input.next_block_range_with_threshold(self.commit_threshold); + let (start_block, end_block) = range.clone().into_inner(); debug!(target: "sync::stages::total_difficulty", start_block, end_block, "Commencing sync"); @@ -76,7 +73,7 @@ impl Stage for TotalDifficultyStage { debug!(target: "sync::stages::total_difficulty", ?td, block_number = last_header_number, "Last total difficulty entry"); // Acquire canonical walker - let walker = cursor_canonical.walk_range(start_block..=end_block)?; + let walker = cursor_canonical.walk_range(range)?; // Walk over newly inserted headers, update & insert td for entry in walker { @@ -93,9 +90,8 @@ impl Stage for TotalDifficultyStage { cursor_td.append(number, td.into())?; } - let done = !capped; - info!(target: "sync::stages::total_difficulty", stage_progress = end_block, done, "Sync iteration finished"); - Ok(ExecOutput { done, stage_progress: end_block }) + info!(target: "sync::stages::total_difficulty", stage_progress = end_block, is_final_range, "Sync iteration finished"); + Ok(ExecOutput { stage_progress: end_block, done: is_final_range }) } /// Unwind the stage. diff --git a/crates/stages/src/stages/tx_lookup.rs b/crates/stages/src/stages/tx_lookup.rs index 305477c08..db7930d60 100644 --- a/crates/stages/src/stages/tx_lookup.rs +++ b/crates/stages/src/stages/tx_lookup.rs @@ -1,7 +1,4 @@ -use crate::{ - exec_or_return, ExecAction, ExecInput, ExecOutput, Stage, StageError, StageId, UnwindInput, - UnwindOutput, -}; +use crate::{ExecInput, ExecOutput, Stage, StageError, StageId, UnwindInput, UnwindOutput}; use rayon::prelude::*; use reth_db::{ cursor::{DbCursorRO, DbCursorRW}, @@ -18,7 +15,7 @@ pub const TRANSACTION_LOOKUP: StageId = StageId("TransactionLookup"); /// The transaction lookup stage. /// /// This stage walks over the bodies table, and sets the transaction hash of each transaction in a -/// block to the corresponding `TransitionId` at each block. This is written to the +/// block to the corresponding `BlockNumber` at each block. This is written to the /// [`tables::TxHashNumber`] This is used for looking up changesets via the transaction hash. #[derive(Debug, Clone)] pub struct TransactionLookupStage { @@ -52,8 +49,11 @@ impl Stage for TransactionLookupStage { tx: &mut Transaction<'_, DB>, input: ExecInput, ) -> Result { - let ((start_block, end_block), capped) = - exec_or_return!(input, self.commit_threshold, "sync::stages::transaction_lookup"); + let (range, is_final_range) = input.next_block_range_with_threshold(self.commit_threshold); + if range.is_empty() { + return Ok(ExecOutput::done(*range.end())) + } + let (start_block, end_block) = range.into_inner(); debug!(target: "sync::stages::transaction_lookup", start_block, end_block, "Commencing sync"); @@ -106,9 +106,8 @@ impl Stage for TransactionLookupStage { } } - let done = !capped; - info!(target: "sync::stages::transaction_lookup", stage_progress = end_block, done, "Sync iteration finished"); - Ok(ExecOutput { done, stage_progress: end_block }) + info!(target: "sync::stages::transaction_lookup", stage_progress = end_block, is_final_range, "Sync iteration finished"); + Ok(ExecOutput { done: is_final_range, stage_progress: end_block }) } /// Unwind the stage. @@ -254,7 +253,7 @@ mod tests { /// 2. If the is no requested block entry in the bodies table, /// but [tables::TxHashNumber] is not empty. fn ensure_no_hash_by_block(&self, number: BlockNumber) -> Result<(), TestRunnerError> { - let body_result = self.tx.inner().get_block_meta(number); + let body_result = self.tx.inner().block_body_indices(number); match body_result { Ok(body) => self.tx.ensure_no_entry_above_by_value::( body.last_tx_num(), @@ -286,9 +285,9 @@ mod tests { fn seed_execution(&mut self, input: ExecInput) -> Result { let stage_progress = input.stage_progress.unwrap_or_default(); - let end = input.previous_stage_progress() + 1; + let end = input.previous_stage_progress(); - let blocks = random_block_range(stage_progress..end, H256::zero(), 0..2); + let blocks = random_block_range(stage_progress..=end, H256::zero(), 0..2); self.tx.insert_blocks(blocks.iter(), None)?; Ok(blocks) } diff --git a/crates/stages/src/test_utils/test_db.rs b/crates/stages/src/test_utils/test_db.rs index 310190c28..26d9d3210 100644 --- a/crates/stages/src/test_utils/test_db.rs +++ b/crates/stages/src/test_utils/test_db.rs @@ -239,9 +239,7 @@ impl TestTransaction { block.number, StoredBlockBodyIndices { first_tx_num: next_tx_num, - first_transition_id: next_tx_num, tx_count: block.body.len() as u64, - has_block_change: false, }, )?; block.body.iter().try_for_each(|body_tx| { diff --git a/crates/storage/codecs/derive/src/compact/mod.rs b/crates/storage/codecs/derive/src/compact/mod.rs index 7bbd27900..fddcbd6de 100644 --- a/crates/storage/codecs/derive/src/compact/mod.rs +++ b/crates/storage/codecs/derive/src/compact/mod.rs @@ -162,7 +162,7 @@ pub fn get_bit_size(ftype: &str) -> u8 { match ftype { "bool" | "Option" => 1, "TxType" => 2, - "u64" | "BlockNumber" | "TxNumber" | "ChainId" | "TransitionId" | "NumTransactions" => 4, + "u64" | "BlockNumber" | "TxNumber" | "ChainId" | "NumTransactions" => 4, "u128" => 5, "U256" | "TxHash" => 6, _ => 0, diff --git a/crates/storage/db/src/tables/codecs/fuzz/mod.rs b/crates/storage/db/src/tables/codecs/fuzz/mod.rs index 8c89155a7..3fabd992b 100644 --- a/crates/storage/db/src/tables/codecs/fuzz/mod.rs +++ b/crates/storage/db/src/tables/codecs/fuzz/mod.rs @@ -86,5 +86,5 @@ macro_rules! impl_fuzzer_value_with_input { }; } -impl_fuzzer_key!(TransitionIdAddress); +impl_fuzzer_key!(BlockNumberAddress); impl_fuzzer_value_with_input!((IntegerList, IntegerListInput)); diff --git a/crates/storage/db/src/tables/mod.rs b/crates/storage/db/src/tables/mod.rs index 76de1e671..d40da2478 100644 --- a/crates/storage/db/src/tables/mod.rs +++ b/crates/storage/db/src/tables/mod.rs @@ -25,7 +25,7 @@ use crate::{ tables::{ codecs::CompactU256, models::{ - accounts::{AccountBeforeTx, TransitionIdAddress}, + accounts::{AccountBeforeTx, BlockNumberAddress}, blocks::{HeaderHash, StoredBlockOmmers}, storage_sharded_key::StorageShardedKey, ShardedKey, StoredBlockBodyIndices, StoredBlockWithdrawals, @@ -35,7 +35,7 @@ use crate::{ use reth_primitives::{ trie::{BranchNodeCompact, StorageTrieEntry, StoredNibbles, StoredNibblesSubKey}, Account, Address, BlockHash, BlockNumber, Bytecode, Header, IntegerList, Receipt, StorageEntry, - TransactionSigned, TransitionId, TxHash, TxNumber, H256, + TransactionSigned, TxHash, TxNumber, H256, }; /// Enum for the types of tables present in libmdbx. @@ -151,8 +151,7 @@ table!( ); table!( - /// Stores block indices that contains indexes of transaction and transitions, - /// number of transactions and if block has a block change (block reward or withdrawals). + /// Stores block indices that contains indexes of transaction and the count of them. /// /// More information about stored indices can be found in the [`StoredBlockBodyIndices`] struct. ( BlockBodyIndices ) BlockNumber | StoredBlockBodyIndices @@ -209,61 +208,61 @@ dupsort!( ); table!( - /// Stores pointers to transition changeset with changes for each account key. + /// Stores pointers to block changeset with changes for each account key. /// - /// Last shard key of the storage will contains `u64::MAX` `TransitionId`, + /// Last shard key of the storage will contains `u64::MAX` `BlockNumber`, /// this would allows us small optimization on db access when change is in plain state. /// /// Imagine having shards as: /// * `Address | 100` /// * `Address | u64::MAX` /// - /// What we need to find is id that is one greater than N. Db `seek` function allows us to fetch + /// What we need to find is number that is one greater than N. Db `seek` function allows us to fetch /// the shard that equal or more than asked. For example: /// * For N=50 we would get first shard. /// * for N=150 we would get second shard. - /// * If max transition id is 200 and we ask for N=250 we would fetch last shard and + /// * If max block number is 200 and we ask for N=250 we would fetch last shard and /// know that needed entry is in `AccountPlainState`. /// * If there were no shard we would get `None` entry or entry of different storage key. /// /// Code example can be found in `reth_provider::HistoricalStateProviderRef` - ( AccountHistory ) ShardedKey
| TransitionList + ( AccountHistory ) ShardedKey
| BlockNumberList ); table!( - /// Stores pointers to transition changeset with changes for each storage key. + /// Stores pointers to block number changeset with changes for each storage key. /// - /// Last shard key of the storage will contains `u64::MAX` `TransitionId`, + /// Last shard key of the storage will contains `u64::MAX` `BlockNumber`, /// this would allows us small optimization on db access when change is in plain state. /// /// Imagine having shards as: /// * `Address | StorageKey | 100` /// * `Address | StorageKey | u64::MAX` /// - /// What we need to find is id that is one greater than N. Db `seek` function allows us to fetch + /// What we need to find is number that is one greater than N. Db `seek` function allows us to fetch /// the shard that equal or more than asked. For example: /// * For N=50 we would get first shard. /// * for N=150 we would get second shard. - /// * If max transition id is 200 and we ask for N=250 we would fetch last shard and + /// * If max block number is 200 and we ask for N=250 we would fetch last shard and /// know that needed entry is in `StoragePlainState`. /// * If there were no shard we would get `None` entry or entry of different storage key. /// /// Code example can be found in `reth_provider::HistoricalStateProviderRef` - ( StorageHistory ) StorageShardedKey | TransitionList + ( StorageHistory ) StorageShardedKey | BlockNumberList ); dupsort!( /// Stores the state of an account before a certain transaction changed it. /// Change on state can be: account is created, selfdestructed, touched while empty /// or changed (balance,nonce). - ( AccountChangeSet ) TransitionId | [Address] AccountBeforeTx + ( AccountChangeSet ) BlockNumber | [Address] AccountBeforeTx ); dupsort!( /// Stores the state of a storage key before a certain transaction changed it. /// If [`StorageEntry::value`] is zero, this means storage was not existing /// and needs to be removed. - ( StorageChangeSet ) TransitionIdAddress | [H256] StorageEntry + ( StorageChangeSet ) BlockNumberAddress | [H256] StorageEntry ); table!( @@ -309,14 +308,9 @@ table!( ( SyncStageProgress ) StageId | Vec ); -/// /// Alias Types /// List with transaction numbers. -pub type TransitionList = IntegerList; +pub type BlockNumberList = IntegerList; /// Encoded stage id. pub type StageId = String; - -// -// TODO: Temporary types, until they're properly defined alongside with the Encode and Decode Trait -// diff --git a/crates/storage/db/src/tables/models/accounts.rs b/crates/storage/db/src/tables/models/accounts.rs index c48760933..2141ea60e 100644 --- a/crates/storage/db/src/tables/models/accounts.rs +++ b/crates/storage/db/src/tables/models/accounts.rs @@ -1,12 +1,14 @@ //! Account related models and types. +use std::ops::{Range, RangeInclusive}; + use crate::{ impl_fixed_arbitrary, table::{Decode, Encode}, Error, }; use reth_codecs::Compact; -use reth_primitives::{Account, Address, TransitionId}; +use reth_primitives::{Account, Address, BlockNumber}; use serde::{Deserialize, Serialize}; /// Account as it is saved inside [`AccountChangeSet`][crate::tables::AccountChangeSet]. @@ -43,16 +45,23 @@ impl Compact for AccountBeforeTx { } } -/// [`TransitionId`] concatenated with [`Address`]. Used as the key for +/// [`BlockNumber`] concatenated with [`Address`]. Used as the key for /// [`StorageChangeSet`](crate::tables::StorageChangeSet) /// /// Since it's used as a key, it isn't compressed when encoding it. #[derive(Debug, Default, Copy, Clone, PartialEq, Eq, Serialize, Deserialize, Ord, PartialOrd)] -pub struct TransitionIdAddress(pub (TransitionId, Address)); +pub struct BlockNumberAddress(pub (BlockNumber, Address)); + +impl BlockNumberAddress { + /// Create a new Range from `start` to `end` + /// + /// Note: End is inclusive + pub fn range(range: RangeInclusive) -> Range { + (*range.start(), Address::zero()).into()..(*range.end() + 1, Address::zero()).into() + } -impl TransitionIdAddress { /// Return the transition id - pub fn transition_id(&self) -> TransitionId { + pub fn block_number(&self) -> BlockNumber { self.0 .0 } @@ -61,19 +70,19 @@ impl TransitionIdAddress { self.0 .1 } - /// Consumes `Self` and returns [`TransitionId`], [`Address`] - pub fn take(self) -> (TransitionId, Address) { + /// Consumes `Self` and returns [`BlockNumber`], [`Address`] + pub fn take(self) -> (BlockNumber, Address) { (self.0 .0, self.0 .1) } } -impl From<(u64, Address)> for TransitionIdAddress { +impl From<(BlockNumber, Address)> for BlockNumberAddress { fn from(tpl: (u64, Address)) -> Self { - TransitionIdAddress(tpl) + BlockNumberAddress(tpl) } } -impl Encode for TransitionIdAddress { +impl Encode for BlockNumberAddress { type Encoded = [u8; 28]; fn encode(self) -> Self::Encoded { @@ -88,17 +97,17 @@ impl Encode for TransitionIdAddress { } } -impl Decode for TransitionIdAddress { +impl Decode for BlockNumberAddress { fn decode>(value: B) -> Result { let value = value.as_ref(); let num = u64::from_be_bytes(value[..8].try_into().map_err(|_| Error::DecodeError)?); let hash = Address::from_slice(&value[8..]); - Ok(TransitionIdAddress((num, hash))) + Ok(BlockNumberAddress((num, hash))) } } -impl_fixed_arbitrary!(TransitionIdAddress, 28); +impl_fixed_arbitrary!(BlockNumberAddress, 28); #[cfg(test)] mod test { @@ -110,7 +119,7 @@ mod test { fn test_tx_number_address() { let num = 1u64; let hash = Address::from_str("ba5e000000000000000000000000000000000000").unwrap(); - let key = TransitionIdAddress((num, hash)); + let key = BlockNumberAddress((num, hash)); let mut bytes = [0u8; 28]; bytes[..8].copy_from_slice(&num.to_be_bytes()); @@ -119,7 +128,7 @@ mod test { let encoded = Encode::encode(key); assert_eq!(encoded, bytes); - let decoded: TransitionIdAddress = Decode::decode(encoded).unwrap(); + let decoded: BlockNumberAddress = Decode::decode(encoded).unwrap(); assert_eq!(decoded, key); } @@ -127,7 +136,7 @@ mod test { fn test_tx_number_address_rand() { let mut bytes = [0u8; 28]; thread_rng().fill(bytes.as_mut_slice()); - let key = TransitionIdAddress::arbitrary(&mut Unstructured::new(&bytes)).unwrap(); + let key = BlockNumberAddress::arbitrary(&mut Unstructured::new(&bytes)).unwrap(); assert_eq!(bytes, Encode::encode(key)); } } diff --git a/crates/storage/db/src/tables/models/blocks.rs b/crates/storage/db/src/tables/models/blocks.rs index 0ed27bc9a..c62734307 100644 --- a/crates/storage/db/src/tables/models/blocks.rs +++ b/crates/storage/db/src/tables/models/blocks.rs @@ -1,7 +1,7 @@ //! Block related models and types. use reth_codecs::{main_codec, Compact}; -use reth_primitives::{Header, TransitionId, TxNumber, Withdrawal, H256}; +use reth_primitives::{Header, TxNumber, Withdrawal, H256}; use std::ops::Range; /// Total number of transactions. @@ -19,18 +19,11 @@ pub struct StoredBlockBodyIndices { /// Note: If the block is empty, this is the number of the first transaction /// in the next non-empty block. pub first_tx_num: TxNumber, - /// The id of the first transition in this block. - /// - /// NOTE: If the block is empty, this is the id of the first transition - /// in the next non-empty block. - pub first_transition_id: TransitionId, /// The total number of transactions in the block /// /// NOTE: Number of transitions is equal to number of transactions with /// additional transition for block change if block has block reward or withdrawal. pub tx_count: NumTransactions, - /// Flags if there is additional transition changeset of the withdrawal or block reward. - pub has_block_change: bool, } impl StoredBlockBodyIndices { @@ -39,36 +32,6 @@ impl StoredBlockBodyIndices { self.first_tx_num..self.first_tx_num + self.tx_count } - /// Return the range of transition ids for this block. - pub fn transition_range(&self) -> Range { - self.transition_at_block()..self.transition_after_block() - } - - /// Return transition id of the state after block executed. - /// This transitions is used with the history index to represent the state after this - /// block execution. - /// - /// Because we are storing old values of the changeset in the history index, we need - /// transition of one after, to fetch correct values of the past state - /// - /// NOTE: This is the same as the first transition id of the next block. - pub fn transition_after_block(&self) -> TransitionId { - self.first_transition_id + self.tx_count + (self.has_block_change as u64) - } - - /// Return transition id of the state at the block execution. - /// This transitions is used with the history index to represent the state - /// before the block execution. - /// - /// Because we are storing old values of the changeset in the history index, we need - /// transition of one after, to fetch correct values of the past state - /// - /// NOTE: If block does not have transitions (empty block) then this is the same - /// as the first transition id of the next block. - pub fn transition_at_block(&self) -> TransitionId { - self.first_transition_id - } - /// Return the index of last transaction in this block unless the block /// is empty in which case it refers to the last transaction in a previous /// non-empty block @@ -97,12 +60,6 @@ impl StoredBlockBodyIndices { pub fn tx_count(&self) -> NumTransactions { self.tx_count } - - /// Return flag signifying whether the block has additional - /// transition changeset (withdrawal or uncle/block rewards). - pub fn has_block_change(&self) -> bool { - self.has_block_change - } } /// The storage representation of a block ommers. @@ -143,31 +100,15 @@ mod test { } #[test] - fn block_meta_indices() { + fn block_indices() { let first_tx_num = 10; - let first_transition_id = 14; let tx_count = 6; - let has_block_change = true; - let mut block_meta = StoredBlockBodyIndices { - first_tx_num, - first_transition_id, - tx_count, - has_block_change, - }; + let block_indices = StoredBlockBodyIndices { first_tx_num, tx_count }; - assert_eq!(block_meta.first_tx_num(), first_tx_num); - assert_eq!(block_meta.last_tx_num(), first_tx_num + tx_count - 1); - assert_eq!(block_meta.next_tx_num(), first_tx_num + tx_count); - assert_eq!(block_meta.tx_count(), tx_count); - assert!(block_meta.has_block_change()); - assert_eq!(block_meta.transition_at_block(), first_transition_id); - assert_eq!(block_meta.transition_after_block(), first_transition_id + tx_count + 1); - assert_eq!(block_meta.tx_num_range(), first_tx_num..first_tx_num + tx_count); - assert_eq!( - block_meta.transition_range(), - first_transition_id..first_transition_id + tx_count + 1 - ); - block_meta.has_block_change = false; - assert_eq!(block_meta.transition_after_block(), first_transition_id + tx_count); + assert_eq!(block_indices.first_tx_num(), first_tx_num); + assert_eq!(block_indices.last_tx_num(), first_tx_num + tx_count - 1); + assert_eq!(block_indices.next_tx_num(), first_tx_num + tx_count); + assert_eq!(block_indices.tx_count(), tx_count); + assert_eq!(block_indices.tx_num_range(), first_tx_num..first_tx_num + tx_count); } } diff --git a/crates/storage/db/src/tables/models/sharded_key.rs b/crates/storage/db/src/tables/models/sharded_key.rs index 867b8e7f4..5ea1c3aa8 100644 --- a/crates/storage/db/src/tables/models/sharded_key.rs +++ b/crates/storage/db/src/tables/models/sharded_key.rs @@ -4,7 +4,7 @@ use crate::{ table::{Decode, Encode}, Error, }; -use reth_primitives::TransitionId; +use reth_primitives::BlockNumber; /// Number of indices in one shard. pub const NUM_OF_INDICES_IN_SHARD: usize = 100; @@ -12,21 +12,21 @@ pub const NUM_OF_INDICES_IN_SHARD: usize = 100; /// Sometimes data can be too big to be saved for a single key. This helps out by dividing the data /// into different shards. Example: /// -/// `Address | 200` -> data is from transition 0 to 200. +/// `Address | 200` -> data is from block 0 to 200. /// -/// `Address | 300` -> data is from transaction 201 to 300. +/// `Address | 300` -> data is from block 201 to 300. #[derive(Debug, Default, Clone, Eq, PartialEq, Ord, PartialOrd)] pub struct ShardedKey { /// The key for this type. pub key: T, - /// Highest transition id to which `value` is related to. - pub highest_transition_id: TransitionId, + /// Highest block number to which `value` is related to. + pub highest_block_number: BlockNumber, } impl ShardedKey { /// Creates a new `ShardedKey`. - pub fn new(key: T, highest_transition_id: TransitionId) -> Self { - ShardedKey { key, highest_transition_id } + pub fn new(key: T, highest_block_number: BlockNumber) -> Self { + ShardedKey { key, highest_block_number } } } @@ -39,7 +39,7 @@ where fn encode(self) -> Self::Encoded { let mut buf: Vec = Encode::encode(self.key).into(); - buf.extend_from_slice(&self.highest_transition_id.to_be_bytes()); + buf.extend_from_slice(&self.highest_block_number.to_be_bytes()); buf } } diff --git a/crates/storage/db/src/tables/models/storage_sharded_key.rs b/crates/storage/db/src/tables/models/storage_sharded_key.rs index d43ebe564..a27c90f96 100644 --- a/crates/storage/db/src/tables/models/storage_sharded_key.rs +++ b/crates/storage/db/src/tables/models/storage_sharded_key.rs @@ -4,7 +4,7 @@ use crate::{ table::{Decode, Encode}, Error, }; -use reth_primitives::{TransitionId, H160, H256}; +use reth_primitives::{BlockNumber, H160, H256}; use super::ShardedKey; @@ -27,8 +27,8 @@ pub struct StorageShardedKey { impl StorageShardedKey { /// Creates a new `StorageShardedKey`. - pub fn new(address: H160, storage_key: H256, highest_transition_id: TransitionId) -> Self { - Self { address, sharded_key: ShardedKey { key: storage_key, highest_transition_id } } + pub fn new(address: H160, storage_key: H256, highest_block_number: BlockNumber) -> Self { + Self { address, sharded_key: ShardedKey { key: storage_key, highest_block_number } } } } @@ -38,7 +38,7 @@ impl Encode for StorageShardedKey { fn encode(self) -> Self::Encoded { let mut buf: Vec = Encode::encode(self.address).into(); buf.extend_from_slice(&Encode::encode(self.sharded_key.key)); - buf.extend_from_slice(&self.sharded_key.highest_transition_id.to_be_bytes()); + buf.extend_from_slice(&self.sharded_key.highest_block_number.to_be_bytes()); buf } } diff --git a/crates/storage/provider/Cargo.toml b/crates/storage/provider/Cargo.toml index 65bfc4442..784373063 100644 --- a/crates/storage/provider/Cargo.toml +++ b/crates/storage/provider/Cargo.toml @@ -33,6 +33,7 @@ parking_lot = { version = "0.12", optional = true } [dev-dependencies] reth-db = { path = "../db", features = ["test-utils"] } reth-primitives = { path = "../../primitives", features = ["arbitrary", "test-utils"] } +reth-rlp = { path = "../../rlp" } parking_lot = "0.12" [features] diff --git a/crates/storage/provider/src/chain.rs b/crates/storage/provider/src/chain.rs index 900854a89..765032672 100644 --- a/crates/storage/provider/src/chain.rs +++ b/crates/storage/provider/src/chain.rs @@ -4,7 +4,7 @@ use crate::PostState; use reth_interfaces::{executor::Error as ExecError, Error}; use reth_primitives::{ BlockHash, BlockNumHash, BlockNumber, ForkBlock, Receipt, SealedBlock, SealedBlockWithSenders, - TransitionId, TxHash, + TxHash, }; use std::collections::BTreeMap; @@ -23,12 +23,6 @@ pub struct Chain { pub state: PostState, /// All blocks in this chain. pub blocks: BTreeMap, - /// A mapping of each block number in the chain to the highest transition ID in the chain's - /// state after execution of the block. - /// - /// This is used to revert changes in the state until a certain block number when the chain is - /// split. - pub block_transitions: BTreeMap, } impl Chain { @@ -47,11 +41,6 @@ impl Chain { self.blocks.iter().find_map(|(num, block)| (block.hash() == block_hash).then_some(*num)) } - /// Return block inner transition ids. - pub fn block_transitions(&self) -> &BTreeMap { - &self.block_transitions - } - /// Returns the block with matching hash. pub fn block(&self, block_hash: BlockHash) -> Option<&SealedBlock> { self.blocks @@ -65,12 +54,11 @@ impl Chain { return Some(self.state.clone()) } - if let Some(&transition_id) = self.block_transitions.get(&block_number) { + if self.blocks.get(&block_number).is_some() { let mut state = self.state.clone(); - state.revert_to(transition_id); + state.revert_to(block_number); return Some(state) } - None } @@ -113,41 +101,13 @@ impl Chain { /// Create new chain with given blocks and post state. pub fn new(blocks: Vec<(SealedBlockWithSenders, PostState)>) -> Self { let mut state = PostState::default(); - let mut block_transitions = BTreeMap::new(); let mut block_num_hash = BTreeMap::new(); for (block, block_state) in blocks.into_iter() { state.extend(block_state); - block_transitions.insert(block.number, state.transitions_count()); block_num_hash.insert(block.number, block); } - Self { state, block_transitions, blocks: block_num_hash } - } - - /// Merge two chains by appending the given chain into the current one. - /// - /// The state of accounts for this chain is set to the state of the newest chain. - pub fn append_chain(&mut self, chain: Chain) -> Result<(), Error> { - let chain_tip = self.tip(); - if chain_tip.hash != chain.fork_block_hash() { - return Err(ExecError::AppendChainDoesntConnect { - chain_tip: chain_tip.num_hash(), - other_chain_fork: chain.fork_block().into_components(), - } - .into()) - } - - // Insert blocks from other chain - self.blocks.extend(chain.blocks.into_iter()); - let current_transition_count = self.state.transitions_count(); - self.state.extend(chain.state); - - // Update the block transition mapping, shifting the transition ID by the current number of - // transitions in *this* chain - for (block_number, transition_id) in chain.block_transitions.iter() { - self.block_transitions.insert(*block_number, transition_id + current_transition_count); - } - Ok(()) + Self { state, blocks: block_num_hash } } /// Get all receipts with attachment. @@ -169,6 +129,26 @@ impl Chain { receipt_attch } + /// Merge two chains by appending the given chain into the current one. + /// + /// The state of accounts for this chain is set to the state of the newest chain. + pub fn append_chain(&mut self, chain: Chain) -> Result<(), Error> { + let chain_tip = self.tip(); + if chain_tip.hash != chain.fork_block_hash() { + return Err(ExecError::AppendChainDoesntConnect { + chain_tip: chain_tip.num_hash(), + other_chain_fork: chain.fork_block().into_components(), + } + .into()) + } + + // Insert blocks from other chain + self.blocks.extend(chain.blocks.into_iter()); + self.state.extend(chain.state); + + Ok(()) + } + /// Split this chain at the given block. /// /// The given block will be the first block in the first returned chain. @@ -210,22 +190,12 @@ impl Chain { let higher_number_blocks = self.blocks.split_off(&(block_number + 1)); let mut canonical_state = std::mem::take(&mut self.state); - let new_state = canonical_state.split_at( - *self.block_transitions.get(&block_number).expect("Unknown block transition ID"), - ); + let new_state = canonical_state.split_at(block_number); self.state = new_state; ChainSplit::Split { - canonical: Chain { - state: canonical_state, - block_transitions: BTreeMap::new(), - blocks: self.blocks, - }, - pending: Chain { - state: self.state, - block_transitions: self.block_transitions, - blocks: higher_number_blocks, - }, + canonical: Chain { state: canonical_state, blocks: self.blocks }, + pending: Chain { state: self.state, blocks: higher_number_blocks }, } } } @@ -349,16 +319,13 @@ mod tests { fn test_number_split() { let mut base_state = PostState::default(); let account = Account { nonce: 10, ..Default::default() }; - base_state.create_account(H160([1; 20]), account); - base_state.finish_transition(); + base_state.create_account(1, H160([1; 20]), account); let mut block_state1 = PostState::default(); - block_state1.create_account(H160([2; 20]), Account::default()); - block_state1.finish_transition(); + block_state1.create_account(2, H160([2; 20]), Account::default()); let mut block_state2 = PostState::default(); - block_state2.create_account(H160([3; 20]), Account::default()); - block_state2.finish_transition(); + block_state2.create_account(3, H160([3; 20]), Account::default()); let mut block1 = SealedBlockWithSenders::default(); let block1_hash = H256([15; 32]); @@ -378,19 +345,13 @@ mod tests { ]); let mut split1_state = chain.state.clone(); - let split2_state = split1_state.split_at(*chain.block_transitions.get(&1).unwrap()); + let split2_state = split1_state.split_at(1); - let chain_split1 = Chain { - state: split1_state, - block_transitions: BTreeMap::new(), - blocks: BTreeMap::from([(1, block1.clone())]), - }; + let chain_split1 = + Chain { state: split1_state, blocks: BTreeMap::from([(1, block1.clone())]) }; - let chain_split2 = Chain { - state: split2_state, - block_transitions: chain.block_transitions.clone(), - blocks: BTreeMap::from([(2, block2.clone())]), - }; + let chain_split2 = + Chain { state: split2_state, blocks: BTreeMap::from([(2, block2.clone())]) }; // return tip state assert_eq!(chain.state_at_block(block2.number), Some(chain.state.clone())); diff --git a/crates/storage/provider/src/post_state.rs b/crates/storage/provider/src/post_state.rs index c008134ed..a71aaf344 100644 --- a/crates/storage/provider/src/post_state.rs +++ b/crates/storage/provider/src/post_state.rs @@ -1,17 +1,30 @@ //! Output of execution. use reth_db::{ cursor::{DbCursorRO, DbCursorRW, DbDupCursorRO, DbDupCursorRW}, - models::{AccountBeforeTx, TransitionIdAddress}, + models::{AccountBeforeTx, BlockNumberAddress}, tables, transaction::{DbTx, DbTxMut}, Error as DbError, }; use reth_primitives::{ - bloom::logs_bloom, proofs::calculate_receipt_root_ref, Account, Address, Bloom, Bytecode, Log, - Receipt, StorageEntry, TransitionId, H256, U256, + bloom::logs_bloom, proofs::calculate_receipt_root_ref, Account, Address, BlockNumber, Bloom, + Bytecode, Log, Receipt, StorageEntry, H256, U256, }; use std::collections::BTreeMap; +/// Storage for an account with the old and new values for each slot: (slot -> (old, new)). +pub type StorageChangeset = BTreeMap; + +/// A mapping of `block -> address -> account` that represents what accounts were changed, and what +/// their state were prior to that change. +/// +/// If the prior state was `None`, then the account is new. +pub type AccountChanges = BTreeMap>>; + +/// A mapping of `block -> account -> slot -> old value` that represents what slots were changed, +/// and what their values were prior to that change. +pub type StorageChanges = BTreeMap>; + /// Storage for an account. /// /// # Wiped Storage @@ -29,104 +42,7 @@ pub struct Storage { pub storage: BTreeMap, } -/// Storage for an account with the old and new values for each slot. -/// TODO: Do we actually need (old, new) anymore, or is (old) sufficient? (Check the writes) -/// If we don't, we can unify this and [Storage]. -pub type StorageChangeset = BTreeMap; - -/// A change to the state of accounts or storage. -#[derive(Debug, Clone, Eq, PartialEq)] -pub enum Change { - /// A new account was created. - AccountCreated { - /// The ID of the transition this change is a part of. - id: TransitionId, - /// The address of the account that was created. - address: Address, - /// The account. - account: Account, - }, - /// An existing account was changed. - AccountChanged { - /// The ID of the transition this change is a part of. - id: TransitionId, - /// The address of the account that was changed. - address: Address, - /// The account before the change. - old: Account, - /// The account after the change. - new: Account, - }, - /// Storage slots for an account were changed. - StorageChanged { - /// The ID of the transition this change is a part of. - id: TransitionId, - /// The address of the account associated with the storage slots. - address: Address, - /// The storage changeset. - changeset: StorageChangeset, - }, - /// Storage was wiped - StorageWiped { - /// The ID of the transition this change is a part of. - id: TransitionId, - /// The address of the account whose storage was wiped. - address: Address, - }, - /// An account was destroyed. - /// - /// This removes all of the information associated with the account. An accompanying - /// [Change::StorageWiped] will also be present to mark the deletion of storage. - /// - /// If a change to an account satisfies the conditions for EIP-158, this change variant is also - /// applied instead of the change that would otherwise have happened. - AccountDestroyed { - /// The ID of the transition this change is a part of. - id: TransitionId, - /// The address of the destroyed account. - address: Address, - /// The account before it was destroyed. - old: Account, - }, -} - -impl Change { - /// Get the transition ID for the change - pub fn transition_id(&self) -> TransitionId { - match self { - Change::AccountChanged { id, .. } | - Change::AccountCreated { id, .. } | - Change::StorageChanged { id, .. } | - Change::StorageWiped { id, .. } | - Change::AccountDestroyed { id, .. } => *id, - } - } - - /// Get the address of the account this change operates on. - pub fn address(&self) -> Address { - match self { - Change::AccountChanged { address, .. } | - Change::AccountCreated { address, .. } | - Change::StorageChanged { address, .. } | - Change::StorageWiped { address, .. } | - Change::AccountDestroyed { address, .. } => *address, - } - } - - /// Set the transition ID of this change. - pub fn set_transition_id(&mut self, new_id: TransitionId) { - match self { - Change::AccountChanged { ref mut id, .. } | - Change::AccountCreated { ref mut id, .. } | - Change::StorageChanged { ref mut id, .. } | - Change::StorageWiped { ref mut id, .. } | - Change::AccountDestroyed { ref mut id, .. } => { - *id = new_id; - } - } - } -} - +// todo: rewrite all the docs for this /// The state of accounts after execution of one or more transactions, including receipts and new /// bytecode. /// @@ -144,9 +60,6 @@ impl Change { /// /// # Transitions /// -/// Each [Change] has an `id` field that marks what transition it is part of. Each transaction is -/// its own transition, but there may be 0 or 1 transitions associated with the block. -/// /// The block level transition includes: /// /// - Block rewards @@ -154,13 +67,6 @@ impl Change { /// - Withdrawals /// - The irregular state change for the DAO hardfork /// -/// [PostState::finish_transition] *must* be called after every transaction, and after every block. -/// -/// The first transaction executed and added to the [PostState] has a transition ID of 0, the next -/// one a transition ID of 1, and so on. If the [PostState] is for a single block, and the number of -/// transitions ([PostState::transitions_count]) is greater than the number of transactions in the -/// block, then the last transition is the block transition. -/// /// For multi-block [PostState]s it is not possible to figure out what transition ID maps on to a /// transaction or a block. /// @@ -169,10 +75,8 @@ impl Change { /// Since most [PostState]s in reth are for multiple blocks it is better to pre-allocate capacity /// for receipts and changes, which [PostState::new] does, and thus it (or /// [PostState::with_tx_capacity]) should be preferred to using the [Default] implementation. -#[derive(Debug, Clone, Eq, PartialEq)] +#[derive(Debug, Clone, Default, Eq, PartialEq)] pub struct PostState { - /// The ID of the current transition. - current_transition_id: TransitionId, /// The state of all modified accounts after execution. /// /// If the value contained is `None`, then the account should be deleted. @@ -182,26 +86,20 @@ pub struct PostState { /// If the contained [Storage] is marked as wiped, then all storage values should be cleared /// from the database. storage: BTreeMap, - /// The changes to state that happened during execution - changes: Vec, + /// The state of accounts before they were changed in the given block. + /// + /// If the value is `None`, then the account is new, otherwise it is a change. + account_changes: AccountChanges, + /// The state of account storage before it was changed in the given block. + /// + /// This map only contains old values for storage slots. + storage_changes: StorageChanges, /// New code created during the execution bytecode: BTreeMap, /// The receipt(s) of the executed transaction(s). receipts: Vec, } -/// Used to determine preallocation sizes of [PostState]'s internal [Vec]s. It denotes the number of -/// best-guess changes each transaction causes to state. -const BEST_GUESS_CHANGES_PER_TX: usize = 8; - -/// How many [Change]s to preallocate for in [PostState]. -/// -/// This is just a guesstimate based on: -/// -/// - Each block having ~200-300 transactions -/// - Each transaction having some amount of changes -const PREALLOC_CHANGES_SIZE: usize = 256 * BEST_GUESS_CHANGES_PER_TX; - impl PostState { /// Create an empty [PostState]. pub fn new() -> Self { @@ -210,11 +108,7 @@ impl PostState { /// Create an empty [PostState] with pre-allocated space for a certain amount of transactions. pub fn with_tx_capacity(txs: usize) -> Self { - Self { - changes: Vec::with_capacity(txs * BEST_GUESS_CHANGES_PER_TX), - receipts: Vec::with_capacity(txs), - ..Default::default() - } + Self { receipts: Vec::with_capacity(txs), ..Default::default() } } /// Get the latest state of all changed accounts. @@ -222,6 +116,16 @@ impl PostState { &self.accounts } + /// Get a reference to all the account changes + pub fn account_changes(&self) -> &AccountChanges { + &self.account_changes + } + + /// Get a reference to all the storage changes + pub fn storage_changes(&self) -> &StorageChanges { + &self.storage_changes + } + /// Get the latest state for a specific account. /// /// # Returns @@ -243,11 +147,6 @@ impl PostState { self.storage.get(address) } - /// Get the changes causing this [PostState]. - pub fn changes(&self) -> &[Change] { - &self.changes - } - /// Get the newly created bytecodes pub fn bytecodes(&self) -> &BTreeMap { &self.bytecode @@ -278,52 +177,87 @@ impl PostState { calculate_receipt_root_ref(self.receipts().iter().map(Into::into)) } - /// Get the number of transitions causing this [PostState] - pub fn transitions_count(&self) -> TransitionId { - self.current_transition_id - } - + // todo: note overwrite behavior, i.e. changes in `other` take precedent /// Extend this [PostState] with the changes in another [PostState]. - pub fn extend(&mut self, other: PostState) { - if other.changes.is_empty() { - return + pub fn extend(&mut self, mut other: PostState) { + // Update plain state + self.accounts.extend(other.accounts); + for (address, their_storage) in other.storage { + let our_storage = self.storage.entry(address).or_default(); + if their_storage.wiped { + our_storage.wiped = true; + our_storage.storage.clear(); + } + our_storage.storage.extend(their_storage.storage); } - self.changes.reserve(other.changes.len()); + // Insert account change sets + for (block_number, account_changes) in std::mem::take(&mut other.account_changes) { + let block = self.account_changes.entry(block_number).or_default(); + for (address, account) in account_changes { + if block.contains_key(&address) { + continue + } + block.insert(address, account); + } + } - let mut next_transition_id = self.current_transition_id; - for mut change in other.changes.into_iter() { - next_transition_id = self.current_transition_id + change.transition_id(); - change.set_transition_id(next_transition_id); - self.add_and_apply(change); + // Insert storage change sets + for (block_number, storage_changes) in std::mem::take(&mut other.storage_changes) { + for (address, their_storage) in storage_changes { + let our_storage = self + .storage_changes + .entry(block_number) + .or_default() + .entry(address) + .or_default(); + + if their_storage.wiped { + our_storage.wiped = true; + } + for (slot, value) in their_storage.storage { + our_storage.storage.entry(slot).or_insert(value); + } + } } self.receipts.extend(other.receipts); self.bytecode.extend(other.bytecode); - self.current_transition_id = next_transition_id + 1; } - /// Reverts each change up to and including any change that is part of `transition_id`. + /// Reverts each change up to and including any change that is part of `block_number`. /// /// The reverted changes are removed from this post-state, and their effects are reverted. - /// - /// The reverted changes are returned. - pub fn revert_to(&mut self, transition_id: TransitionId) -> Vec { - let mut changes_to_revert = Vec::new(); - self.changes.retain(|change| { - if change.transition_id() >= transition_id { - changes_to_revert.push(change.clone()); + pub fn revert_to(&mut self, target_block_number: BlockNumber) { + let mut account_changes_to_revert = BTreeMap::new(); + self.account_changes.retain(|block_number, accounts| { + if *block_number > target_block_number { + account_changes_to_revert.insert(*block_number, accounts.clone()); false } else { true } }); - - for change in changes_to_revert.iter_mut().rev() { - change.set_transition_id(change.transition_id() - transition_id as TransitionId); - self.revert(change.clone()); + for (_, accounts) in account_changes_to_revert.into_iter().rev() { + self.accounts.extend(accounts); + } + + let mut storage_changes_to_revert = BTreeMap::new(); + self.storage_changes.retain(|block_number, storages| { + if *block_number > target_block_number { + storage_changes_to_revert.insert(*block_number, storages.clone()); + false + } else { + true + } + }); + for (_, storages) in storage_changes_to_revert.into_iter().rev() { + for (address, storage) in storages { + self.storage.entry(address).and_modify(|head_storage| { + head_storage.wiped = storage.wiped; + head_storage.storage.extend(storage.clone().storage); + }); + } } - self.current_transition_id = transition_id as TransitionId; - changes_to_revert } /// Reverts each change up to and including any change that is part of `transition_id`. @@ -338,66 +272,92 @@ impl PostState { /// 1. This post-state has the changes reverted /// 2. The returned post-state does *not* have the changes reverted, but only contains the /// descriptions of the changes that were reverted in the first post-state. - pub fn split_at(&mut self, transition_id: TransitionId) -> Self { + pub fn split_at(&mut self, revert_to_block: BlockNumber) -> Self { // Clone ourselves let mut non_reverted_state = self.clone(); // Revert the desired changes - let reverted_changes = self.revert_to(transition_id); + self.revert_to(revert_to_block); - // Compute the new `current_transition_id` for `non_reverted_state`. - let new_transition_id = - reverted_changes.last().map(|c| c.transition_id()).unwrap_or_default(); - non_reverted_state.changes = reverted_changes; - non_reverted_state.current_transition_id = new_transition_id + 1; + // Remove all changes in the returned post-state that were not reverted + non_reverted_state + .storage_changes + .retain(|block_number, _| *block_number > revert_to_block); + non_reverted_state + .account_changes + .retain(|block_number, _| *block_number > revert_to_block); non_reverted_state } /// Add a newly created account to the post-state. - pub fn create_account(&mut self, address: Address, account: Account) { - self.add_and_apply(Change::AccountCreated { - id: self.current_transition_id, - address, - account, - }); + pub fn create_account( + &mut self, + block_number: BlockNumber, + address: Address, + account: Account, + ) { + self.accounts.insert(address, Some(account)); + self.account_changes.entry(block_number).or_default().entry(address).or_insert(None); } /// Add a changed account to the post-state. /// /// If the account also has changed storage values, [PostState::change_storage] should also be /// called. - pub fn change_account(&mut self, address: Address, old: Account, new: Account) { - self.add_and_apply(Change::AccountChanged { - id: self.current_transition_id, - address, - old, - new, - }); + pub fn change_account( + &mut self, + block_number: BlockNumber, + address: Address, + old: Account, + new: Account, + ) { + self.accounts.insert(address, Some(new)); + self.account_changes.entry(block_number).or_default().entry(address).or_insert(Some(old)); } /// Mark an account as destroyed. - pub fn destroy_account(&mut self, address: Address, account: Account) { - self.add_and_apply(Change::AccountDestroyed { - id: self.current_transition_id, - address, - old: account, - }); - self.add_and_apply(Change::StorageWiped { id: self.current_transition_id, address }); + pub fn destroy_account( + &mut self, + block_number: BlockNumber, + address: Address, + account: Account, + ) { + self.accounts.insert(address, None); + self.account_changes + .entry(block_number) + .or_default() + .entry(address) + .or_insert(Some(account)); + let storage = self.storage.entry(address).or_default(); + storage.wiped = true; + storage.storage.clear(); + let storage_changes = + self.storage_changes.entry(block_number).or_default().entry(address).or_default(); + storage_changes.wiped = true; } /// Add changed storage values to the post-state. - pub fn change_storage(&mut self, address: Address, changeset: StorageChangeset) { - self.add_and_apply(Change::StorageChanged { - id: self.current_transition_id, - address, - changeset, - }); + pub fn change_storage( + &mut self, + block_number: BlockNumber, + address: Address, + changeset: StorageChangeset, + ) { + self.storage + .entry(address) + .or_default() + .storage + .extend(changeset.iter().map(|(slot, (_, new))| (*slot, *new))); + let storage_changes = + self.storage_changes.entry(block_number).or_default().entry(address).or_default(); + for (slot, (old, _)) in changeset.into_iter() { + storage_changes.storage.entry(slot).or_insert(old); + } } /// Add new bytecode to the post-state. pub fn add_bytecode(&mut self, code_hash: H256, bytecode: Bytecode) { - // TODO: Is this faster than just doing `.insert`? // Assumption: `insert` will override the value if present, but since the code hash for a // given bytecode will always be the same, we are overriding with the same value. // @@ -413,137 +373,45 @@ impl PostState { self.receipts.push(receipt); } - /// Mark all prior changes as being part of one transition, and start a new one. - pub fn finish_transition(&mut self) { - self.current_transition_id += 1; - } - - /// Add a new change, and apply its transformations to the current state - pub fn add_and_apply(&mut self, change: Change) { - match &change { - Change::AccountCreated { address, account, .. } | - Change::AccountChanged { address, new: account, .. } => { - self.accounts.insert(*address, Some(*account)); - } - Change::AccountDestroyed { address, .. } => { - self.accounts.insert(*address, None); - } - Change::StorageChanged { address, changeset, .. } => { - let storage = self.storage.entry(*address).or_default(); - for (slot, (_, current_value)) in changeset { - storage.storage.insert(*slot, *current_value); - } - } - Change::StorageWiped { address, .. } => { - let storage = self.storage.entry(*address).or_default(); - storage.wiped = true; - storage.storage.clear(); - } - } - - self.changes.push(change); - } - - /// Revert a change, applying the inverse of its transformations to the current state. - fn revert(&mut self, change: Change) { - match &change { - Change::AccountCreated { address, .. } => { - self.accounts.remove(address); - } - Change::AccountChanged { address, old, .. } => { - self.accounts.insert(*address, Some(*old)); - } - Change::AccountDestroyed { address, old, .. } => { - self.accounts.insert(*address, Some(*old)); - } - Change::StorageChanged { address, changeset, .. } => { - let storage = self.storage.entry(*address).or_default(); - for (slot, (old_value, _)) in changeset { - storage.storage.insert(*slot, *old_value); - } - } - Change::StorageWiped { address, .. } => { - let storage = self.storage.entry(*address).or_default(); - storage.wiped = false; - } - } - } - /// Write the post state to the database. - pub fn write_to_db<'a, TX: DbTxMut<'a> + DbTx<'a>>( - mut self, - tx: &TX, - first_transition_id: TransitionId, - ) -> Result<(), DbError> { - // Collect and sort changesets by their key to improve write performance - let mut changesets = std::mem::take(&mut self.changes); - changesets - .sort_unstable_by_key(|changeset| (changeset.transition_id(), changeset.address())); - - // Partition changesets into account and storage changes - let (account_changes, storage_changes): (Vec, Vec) = - changesets.into_iter().partition(|changeset| { - matches!( - changeset, - Change::AccountChanged { .. } | - Change::AccountCreated { .. } | - Change::AccountDestroyed { .. } - ) - }); - + pub fn write_to_db<'a, TX: DbTxMut<'a> + DbTx<'a>>(self, tx: &TX) -> Result<(), DbError> { // Write account changes - tracing::trace!(target: "provider::post_state", len = account_changes.len(), "Writing account changes"); + tracing::trace!(target: "provider::post_state", "Writing account changes"); let mut account_changeset_cursor = tx.cursor_dup_write::()?; - for changeset in account_changes.into_iter() { - match changeset { - Change::AccountDestroyed { id, address, old } | - Change::AccountChanged { id, address, old, .. } => { - let destroyed = matches!(changeset, Change::AccountDestroyed { .. }); - let transition_id = first_transition_id + id; - tracing::trace!(target: "provider::post_state", transition_id, ?address, ?old, destroyed, "Account changed"); - account_changeset_cursor - .append_dup(transition_id, AccountBeforeTx { address, info: Some(old) })?; - } - Change::AccountCreated { id, address, .. } => { - let transition_id = first_transition_id + id; - tracing::trace!(target: "provider::post_state", transition_id, ?address, "Account created"); - account_changeset_cursor - .append_dup(transition_id, AccountBeforeTx { address, info: None })?; - } - _ => unreachable!(), + for (block_number, account_changes) in self.account_changes.into_iter() { + for (address, info) in account_changes.into_iter() { + tracing::trace!(target: "provider::post_state", block_number, ?address, old = ?info, "Account changed"); + account_changeset_cursor + .append_dup(block_number, AccountBeforeTx { address, info })?; } } // Write storage changes - tracing::trace!(target: "provider::post_state", len = storage_changes.len(), "Writing storage changes"); + tracing::trace!(target: "provider::post_state", "Writing storage changes"); let mut storages_cursor = tx.cursor_dup_write::()?; let mut storage_changeset_cursor = tx.cursor_dup_write::()?; - for changeset in storage_changes.into_iter() { - match changeset { - Change::StorageChanged { id, address, changeset } => { - let storage_id = TransitionIdAddress((first_transition_id + id, address)); - - for (key, (old_value, _)) in changeset { - tracing::trace!(target: "provider::post_state", ?storage_id, ?key, ?old_value, "Storage changed"); - storage_changeset_cursor.append_dup( - storage_id, - StorageEntry { key: H256(key.to_be_bytes()), value: old_value }, - )?; - } - } - Change::StorageWiped { id, address } => { - let storage_id = TransitionIdAddress((first_transition_id + id, address)); + for (block_number, storage_changes) in self.storage_changes.into_iter() { + for (address, mut storage) in storage_changes.into_iter() { + let storage_id = BlockNumberAddress((block_number, address)); + if storage.wiped { if let Some((_, entry)) = storages_cursor.seek_exact(address)? { tracing::trace!(target: "provider::post_state", ?storage_id, key = ?entry.key, "Storage wiped"); - storage_changeset_cursor.append_dup(storage_id, entry)?; + storage.storage.insert(entry.key.into(), entry.value); while let Some(entry) = storages_cursor.next_dup_val()? { - storage_changeset_cursor.append_dup(storage_id, entry)?; + storage.storage.insert(entry.key.into(), entry.value); } } } - _ => unreachable!(), + + for (slot, old_value) in storage.storage { + tracing::trace!(target: "provider::post_state", ?storage_id, ?slot, ?old_value, "Storage changed"); + storage_changeset_cursor.append_dup( + storage_id, + StorageEntry { key: H256(slot.to_be_bytes()), value: old_value }, + )?; + } } } @@ -592,14 +460,13 @@ impl PostState { bytecodes_cursor.upsert(hash, bytecode)?; } - // write the receipts of the transactions + // Write the receipts of the transactions let mut receipts_cursor = tx.cursor_write::()?; - let mut next_tx_num = if let Some(last_tx) = receipts_cursor.last()?.map(|(tx_num, _)| tx_num) { last_tx + 1 } else { - // the very first tx + // The very first tx 0 }; for receipt in self.receipts.into_iter() { @@ -611,19 +478,6 @@ impl PostState { } } -impl Default for PostState { - fn default() -> Self { - Self { - current_transition_id: 0, - accounts: Default::default(), - storage: Default::default(), - changes: Vec::with_capacity(PREALLOC_CHANGES_SIZE), - bytecode: Default::default(), - receipts: vec![], - } - } -} - #[cfg(test)] mod tests { use super::*; @@ -639,42 +493,33 @@ mod tests { #[test] fn extend_empty() { let mut a = PostState::new(); - assert_eq!(a.current_transition_id, 0); // Extend empty poststate with another empty poststate a.extend(PostState::new()); - assert_eq!(a.current_transition_id, 0); // Add single transition and extend with empty poststate - a.create_account(Address::zero(), Account::default()); - a.finish_transition(); - let transition_id = a.current_transition_id; + a.create_account(1, Address::zero(), Account::default()); a.extend(PostState::new()); - assert_eq!(a.current_transition_id, transition_id); + assert_eq!(a.account_changes.iter().fold(0, |len, (_, changes)| len + changes.len()), 1); } #[test] fn extend() { let mut a = PostState::new(); - a.create_account(Address::zero(), Account::default()); - a.destroy_account(Address::zero(), Account::default()); - a.finish_transition(); + a.create_account(1, Address::zero(), Account::default()); + a.destroy_account(1, Address::zero(), Account::default()); - assert_eq!(a.transitions_count(), 1); - assert_eq!(a.changes().len(), 3); + assert_eq!(a.account_changes.iter().fold(0, |len, (_, changes)| len + changes.len()), 1); let mut b = PostState::new(); - b.create_account(Address::repeat_byte(0xff), Account::default()); - b.finish_transition(); + b.create_account(2, Address::repeat_byte(0xff), Account::default()); - assert_eq!(b.transitions_count(), 1); - assert_eq!(b.changes.len(), 1); + assert_eq!(b.account_changes.iter().fold(0, |len, (_, changes)| len + changes.len()), 1); let mut c = a.clone(); c.extend(b.clone()); - assert_eq!(c.transitions_count(), 2); - assert_eq!(c.changes.len(), a.changes.len() + b.changes.len()); + assert_eq!(c.account_changes.iter().fold(0, |len, (_, changes)| len + changes.len()), 2); } #[test] @@ -692,10 +537,10 @@ mod tests { let account_b_changed = Account { balance: U256::from(3), nonce: 3, bytecode_hash: None }; // 0x00.. is created - post_state.create_account(address_a, account_a); + post_state.create_account(1, address_a, account_a); // 0x11.. is changed (balance + 1, nonce + 1) - post_state.change_account(address_b, account_b, account_b_changed); - post_state.write_to_db(&tx, 0).expect("Could not write post state to DB"); + post_state.change_account(1, address_b, account_b, account_b_changed); + post_state.write_to_db(&tx).expect("Could not write post state to DB"); // Check plain state assert_eq!( @@ -714,20 +559,20 @@ mod tests { .cursor_dup_read::() .expect("Could not open changeset cursor"); assert_eq!( - changeset_cursor.seek_exact(0).expect("Could not read account change set"), - Some((0, AccountBeforeTx { address: address_a, info: None })), + changeset_cursor.seek_exact(1).expect("Could not read account change set"), + Some((1, AccountBeforeTx { address: address_a, info: None })), "Account A changeset is wrong" ); assert_eq!( changeset_cursor.next_dup().expect("Changeset table is malformed"), - Some((0, AccountBeforeTx { address: address_b, info: Some(account_b) })), + Some((1, AccountBeforeTx { address: address_b, info: Some(account_b) })), "Account B changeset is wrong" ); let mut post_state = PostState::new(); // 0x11.. is destroyed - post_state.destroy_account(address_b, account_b_changed); - post_state.write_to_db(&tx, 1).expect("Could not write second post state to DB"); + post_state.destroy_account(2, address_b, account_b_changed); + post_state.write_to_db(&tx).expect("Could not write second post state to DB"); // Check new plain state for account B assert_eq!( @@ -738,8 +583,8 @@ mod tests { // Check change set assert_eq!( - changeset_cursor.seek_exact(1).expect("Could not read account change set"), - Some((1, AccountBeforeTx { address: address_b, info: Some(account_b_changed) })), + changeset_cursor.seek_exact(2).expect("Could not read account change set"), + Some((2, AccountBeforeTx { address: address_b, info: Some(account_b_changed) })), "Account B changeset is wrong after deletion" ); } @@ -764,9 +609,9 @@ mod tests { // 0x01 => 1 => 2 let storage_b_changeset = BTreeMap::from([(U256::from(1), (U256::from(1), U256::from(2)))]); - post_state.change_storage(address_a, storage_a_changeset); - post_state.change_storage(address_b, storage_b_changeset); - post_state.write_to_db(&tx, 0).expect("Could not write post state to DB"); + post_state.change_storage(1, address_a, storage_a_changeset); + post_state.change_storage(1, address_b, storage_b_changeset); + post_state.write_to_db(&tx).expect("Could not write post state to DB"); // Check plain storage state let mut storage_cursor = tx @@ -811,9 +656,9 @@ mod tests { .cursor_dup_read::() .expect("Could not open storage changeset cursor"); assert_eq!( - changeset_cursor.seek_exact(TransitionIdAddress((0, address_a))).unwrap(), + changeset_cursor.seek_exact(BlockNumberAddress((1, address_a))).unwrap(), Some(( - TransitionIdAddress((0, address_a)), + BlockNumberAddress((1, address_a)), StorageEntry { key: H256::zero(), value: U256::from(0) } )), "Slot 0 for account A should have changed from 0" @@ -821,7 +666,7 @@ mod tests { assert_eq!( changeset_cursor.next_dup().unwrap(), Some(( - TransitionIdAddress((0, address_a)), + BlockNumberAddress((1, address_a)), StorageEntry { key: H256::from(U256::from(1).to_be_bytes()), value: U256::from(0) } )), "Slot 1 for account A should have changed from 0" @@ -833,9 +678,9 @@ mod tests { ); assert_eq!( - changeset_cursor.seek_exact(TransitionIdAddress((0, address_b))).unwrap(), + changeset_cursor.seek_exact(BlockNumberAddress((1, address_b))).unwrap(), Some(( - TransitionIdAddress((0, address_b)), + BlockNumberAddress((1, address_b)), StorageEntry { key: H256::from(U256::from(1).to_be_bytes()), value: U256::from(1) } )), "Slot 1 for account B should have changed from 1" @@ -848,8 +693,8 @@ mod tests { // Delete account A let mut post_state = PostState::new(); - post_state.destroy_account(address_a, Account::default()); - post_state.write_to_db(&tx, 1).expect("Could not write post state to DB"); + post_state.destroy_account(2, address_a, Account::default()); + post_state.write_to_db(&tx).expect("Could not write post state to DB"); assert_eq!( storage_cursor.seek_exact(address_a).unwrap(), @@ -858,9 +703,9 @@ mod tests { ); assert_eq!( - changeset_cursor.seek_exact(TransitionIdAddress((1, address_a))).unwrap(), + changeset_cursor.seek_exact(BlockNumberAddress((2, address_a))).unwrap(), Some(( - TransitionIdAddress((1, address_a)), + BlockNumberAddress((2, address_a)), StorageEntry { key: H256::zero(), value: U256::from(1) } )), "Slot 0 for account A should have changed from 1 on deletion" @@ -868,7 +713,7 @@ mod tests { assert_eq!( changeset_cursor.next_dup().unwrap(), Some(( - TransitionIdAddress((1, address_a)), + BlockNumberAddress((2, address_a)), StorageEntry { key: H256::from(U256::from(1).to_be_bytes()), value: U256::from(2) } )), "Slot 1 for account A should have changed from 2 on deletion" @@ -902,14 +747,11 @@ mod tests { let mut state = PostState::new(); // Create some storage for account A (simulates a contract deployment) - state.change_storage(address_a, storage_changeset_one); - state.finish_transition(); + state.change_storage(1, address_a, storage_changeset_one); // Next transition destroys the account (selfdestruct) - state.destroy_account(address_a, Account::default()); - state.finish_transition(); + state.destroy_account(2, address_a, Account::default()); // Next transition recreates account A with some storage (simulates a contract deployment) - state.change_storage(address_a, storage_changeset_two); - state.finish_transition(); + state.change_storage(3, address_a, storage_changeset_two); // All the storage of account A has to be deleted in the database (wiped) assert!( @@ -928,23 +770,317 @@ mod tests { fn revert_to() { let mut state = PostState::new(); state.create_account( + 1, Address::repeat_byte(0), Account { nonce: 1, balance: U256::from(1), bytecode_hash: None }, ); - state.finish_transition(); - let revert_to = state.current_transition_id; + let revert_to = 1; state.create_account( + 2, Address::repeat_byte(0xff), Account { nonce: 2, balance: U256::from(2), bytecode_hash: None }, ); - state.finish_transition(); - assert_eq!(state.transitions_count(), 2); - assert_eq!(state.accounts().len(), 2); + assert_eq!( + state.account_changes.iter().fold(0, |len, (_, changes)| len + changes.len()), + 2 + ); - let reverted_changes = state.revert_to(revert_to); - assert_eq!(state.accounts().len(), 1); - assert_eq!(state.transitions_count(), 1); - assert_eq!(reverted_changes.len(), 1); + state.revert_to(revert_to); + assert_eq!( + state.account_changes.iter().fold(0, |len, (_, changes)| len + changes.len()), + 1 + ); + } + + /// Checks that if an account is touched multiple times in the same block, + /// then the old value from the first change is kept and not overwritten. + /// + /// This is important because post states from different transactions in the same block may see + /// different states of the same account as the old value, but the changeset should reflect the + /// state of the account before the block. + #[test] + fn account_changesets_keep_old_values() { + let mut state = PostState::new(); + let block = 1; + let address = Address::repeat_byte(0); + + // A transaction in block 1 creates the account + state.create_account( + block, + address, + Account { nonce: 1, balance: U256::from(1), bytecode_hash: None }, + ); + + // A transaction in block 1 then changes the same account + state.change_account( + block, + address, + Account { nonce: 1, balance: U256::from(1), bytecode_hash: None }, + Account { nonce: 1, balance: U256::from(2), bytecode_hash: None }, + ); + + // The value in the changeset for the account should be `None` since this was an account + // creation + assert_eq!( + state.account_changes(), + &BTreeMap::from([(block, BTreeMap::from([(address, None)]))]), + "The changeset for the account is incorrect" + ); + + // The latest state of the account should be: nonce = 1, balance = 2, bytecode hash = None + assert_eq!( + state.accounts.get(&address).unwrap(), + &Some(Account { nonce: 1, balance: U256::from(2), bytecode_hash: None }), + "The latest state of the account is incorrect" + ); + + // Another transaction in block 1 then changes the account yet again + state.change_account( + block, + address, + Account { nonce: 1, balance: U256::from(2), bytecode_hash: None }, + Account { nonce: 2, balance: U256::from(1), bytecode_hash: None }, + ); + + // The value in the changeset for the account should still be `None` + assert_eq!( + state.account_changes(), + &BTreeMap::from([(block, BTreeMap::from([(address, None)]))]), + "The changeset for the account is incorrect" + ); + + // The latest state of the account should be: nonce = 2, balance = 1, bytecode hash = None + assert_eq!( + state.accounts.get(&address).unwrap(), + &Some(Account { nonce: 2, balance: U256::from(1), bytecode_hash: None }), + "The latest state of the account is incorrect" + ); + } + + /// Checks that if a storage slot is touched multiple times in the same block, + /// then the old value from the first change is kept and not overwritten. + /// + /// This is important because post states from different transactions in the same block may see + /// different states of the same account as the old value, but the changeset should reflect the + /// state of the account before the block. + #[test] + fn storage_changesets_keep_old_values() { + let mut state = PostState::new(); + let block = 1; + let address = Address::repeat_byte(0); + + // A transaction in block 1 changes: + // + // Slot 0: 0 -> 1 + // Slot 1: 3 -> 4 + state.change_storage( + block, + address, + BTreeMap::from([ + (U256::from(0), (U256::from(0), U256::from(1))), + (U256::from(1), (U256::from(3), U256::from(4))), + ]), + ); + + // A transaction in block 1 changes: + // + // Slot 0: 1 -> 2 + // Slot 1: 4 -> 5 + state.change_storage( + block, + address, + BTreeMap::from([ + (U256::from(0), (U256::from(1), U256::from(2))), + (U256::from(1), (U256::from(4), U256::from(5))), + ]), + ); + + // The storage changeset for the account in block 1 should now be: + // + // Slot 0: 0 (the value before the first tx in the block) + // Slot 1: 3 + assert_eq!( + state.storage_changes(), + &BTreeMap::from([( + block, + BTreeMap::from([( + address, + Storage { + storage: BTreeMap::from([ + (U256::from(0), U256::from(0)), + (U256::from(1), U256::from(3)) + ]), + wiped: false, + } + )]) + )]), + "The changeset for the storage is incorrect" + ); + + // The latest state of the storage should be: + // + // Slot 0: 2 + // Slot 1: 5 + assert_eq!( + state.storage(), + &BTreeMap::from([( + address, + Storage { + storage: BTreeMap::from([ + (U256::from(0), U256::from(2)), + (U256::from(1), U256::from(5)) + ]), + wiped: false + } + )]), + "The latest state of the storage is incorrect" + ); + } + + /// Tests that the oldest value for changesets is kept when extending a post state from another + /// post state. + /// + /// In other words, this tests the same cases as `account_changesets_keep_old_values` and + /// `storage_changesets_keep_old_values`, but in the case where accounts/slots are changed in + /// different post states that are then merged. + #[test] + fn extending_preserves_changesets() { + let mut a = PostState::new(); + let mut b = PostState::new(); + let block = 1; + let address = Address::repeat_byte(0); + + // The first state (a) represents a transaction that creates an account with some storage + // slots + // + // Expected changeset state: + // - Account: None + // - Storage: Slot 0: 0 + a.create_account( + block, + address, + Account { nonce: 1, balance: U256::from(1), bytecode_hash: None }, + ); + a.change_storage( + block, + address, + BTreeMap::from([(U256::from(0), (U256::from(0), U256::from(1)))]), + ); + assert_eq!( + a.account_changes(), + &BTreeMap::from([(block, BTreeMap::from([(address, None)]))]), + "The changeset for the account is incorrect in state A" + ); + assert_eq!( + a.storage_changes(), + &BTreeMap::from([( + block, + BTreeMap::from([( + address, + Storage { + storage: BTreeMap::from([(U256::from(0), U256::from(0)),]), + wiped: false, + } + )]) + )]), + "The changeset for the storage is incorrect in state A" + ); + + // The second state (b) represents a transaction that changes some slots and account info + // for the same account + // + // Expected changeset state is the same, i.e.: + // - Account: None + // - Storage: Slot 0: 0 + b.change_account( + block, + address, + Account { nonce: 1, balance: U256::from(1), bytecode_hash: None }, + Account { nonce: 1, balance: U256::from(10), bytecode_hash: None }, + ); + b.change_storage( + block, + address, + BTreeMap::from([(U256::from(0), (U256::from(1), U256::from(2)))]), + ); + assert_eq!( + b.account_changes(), + &BTreeMap::from([( + block, + BTreeMap::from([( + address, + Some(Account { nonce: 1, balance: U256::from(1), bytecode_hash: None }) + )]) + )]), + "The changeset for the account is incorrect in state B" + ); + assert_eq!( + b.storage_changes(), + &BTreeMap::from([( + block, + BTreeMap::from([( + address, + Storage { + storage: BTreeMap::from([(U256::from(0), U256::from(1)),]), + wiped: false, + } + )]) + )]), + "The changeset for the storage is incorrect in state B" + ); + + // Now we merge the states + a.extend(b); + + // The expected state is: + // + // Changesets: + // - Account: None + // - Storage: Slot 0: 0 + // + // Accounts: + // - Nonce 1, balance 10, bytecode hash None + // + // Storage: + // - Slot 0: 2 + assert_eq!( + a.account_changes(), + &BTreeMap::from([(block, BTreeMap::from([(address, None)]))]), + "The changeset for the account is incorrect in the merged state" + ); + assert_eq!( + a.storage_changes(), + &BTreeMap::from([( + block, + BTreeMap::from([( + address, + Storage { + storage: BTreeMap::from([(U256::from(0), U256::from(0)),]), + wiped: false, + } + )]) + )]), + "The changeset for the storage is incorrect in the merged state" + ); + assert_eq!( + a.accounts(), + &BTreeMap::from([( + address, + Some(Account { nonce: 1, balance: U256::from(10), bytecode_hash: None }) + )]), + "The state of accounts in the merged state is incorrect" + ); + assert_eq!( + a.storage(), + &BTreeMap::from([( + address, + Storage { + storage: BTreeMap::from([(U256::from(0), U256::from(2)),]), + wiped: false + } + )]), + "The latest state of the storage is incorrect in the merged state" + ); } } diff --git a/crates/storage/provider/src/providers/mod.rs b/crates/storage/provider/src/providers/mod.rs index 473a17933..4c2a338b7 100644 --- a/crates/storage/provider/src/providers/mod.rs +++ b/crates/storage/provider/src/providers/mod.rs @@ -415,32 +415,30 @@ impl StateProviderFactory for ShareableDatabase { Ok(Box::new(LatestStateProvider::new(self.db.tx()?))) } - fn history_by_block_number(&self, block_number: BlockNumber) -> Result> { + fn history_by_block_number( + &self, + mut block_number: BlockNumber, + ) -> Result> { let tx = self.db.tx()?; - // get transition id - let transition = tx - .get::(block_number)? - .ok_or(ProviderError::BlockTransition { block_number })? - .transition_after_block(); + // +1 as the changeset that we want is the one that was applied after this block. + block_number += 1; - Ok(Box::new(HistoricalStateProvider::new(tx, transition))) + Ok(Box::new(HistoricalStateProvider::new(tx, block_number))) } fn history_by_block_hash(&self, block_hash: BlockHash) -> Result> { let tx = self.db.tx()?; // get block number - let block_number = tx + let mut block_number = tx .get::(block_hash)? .ok_or(ProviderError::BlockHash { block_hash })?; - // get transition id - let transition = tx - .get::(block_number)? - .ok_or(ProviderError::BlockTransition { block_number })? - .transition_after_block(); + // +1 as the changeset that we want is the one that was applied after this block. + // as the changeset contains old values. + block_number += 1; - Ok(Box::new(HistoricalStateProvider::new(tx, transition))) + Ok(Box::new(HistoricalStateProvider::new(tx, block_number))) } fn pending( diff --git a/crates/storage/provider/src/providers/state/historical.rs b/crates/storage/provider/src/providers/state/historical.rs index eff0d0e9e..cdf282141 100644 --- a/crates/storage/provider/src/providers/state/historical.rs +++ b/crates/storage/provider/src/providers/state/historical.rs @@ -10,7 +10,7 @@ use reth_db::{ }; use reth_interfaces::Result; use reth_primitives::{ - Account, Address, BlockNumber, Bytecode, Bytes, StorageKey, StorageValue, TransitionId, H256, + Account, Address, BlockNumber, Bytecode, Bytes, StorageKey, StorageValue, H256, }; use std::marker::PhantomData; @@ -25,48 +25,48 @@ use std::marker::PhantomData; pub struct HistoricalStateProviderRef<'a, 'b, TX: DbTx<'a>> { /// Transaction tx: &'b TX, - /// Transition is main indexer of account and storage changes - transition: TransitionId, + /// Block number is main index for the history state of accounts and storages. + block_number: BlockNumber, /// Phantom lifetime `'a` _phantom: PhantomData<&'a TX>, } impl<'a, 'b, TX: DbTx<'a>> HistoricalStateProviderRef<'a, 'b, TX> { /// Create new StateProvider from history transaction number - pub fn new(tx: &'b TX, transition: TransitionId) -> Self { - Self { tx, transition, _phantom: PhantomData {} } + pub fn new(tx: &'b TX, block_number: BlockNumber) -> Self { + Self { tx, block_number, _phantom: PhantomData {} } } } impl<'a, 'b, TX: DbTx<'a>> AccountProvider for HistoricalStateProviderRef<'a, 'b, TX> { /// Get basic account information. fn basic_account(&self, address: Address) -> Result> { - // history key to search IntegerList of transition id changesets. - let history_key = ShardedKey::new(address, self.transition); + // history key to search IntegerList of block number changesets. + let history_key = ShardedKey::new(address, self.block_number); - let changeset_transition_id = self + let changeset_block_number = self .tx .cursor_read::()? .seek(history_key)? .filter(|(key, _)| key.key == address) .map(|(_, list)| { - list.0.enable_rank().successor(self.transition as usize).map(|i| i as u64) + list.0.enable_rank().successor(self.block_number as usize).map(|i| i as u64) }); - // if changeset transition id is present we are getting value from changeset - if let Some(Some(changeset_transition_id)) = changeset_transition_id { + // if changeset of the block is present we are getting value from that changeset + if let Some(Some(changeset_block_number)) = changeset_block_number { let account = self .tx .cursor_dup_read::()? - .seek_by_key_subkey(changeset_transition_id, address)? + .seek_by_key_subkey(changeset_block_number, address)? .filter(|acc| acc.address == address) .ok_or(ProviderError::AccountChangeset { - transition_id: changeset_transition_id, + block_number: changeset_block_number, address, })?; Ok(account.info) } else { // if changeset is not present that means that there was history shard but we need to - // use newest value from plain state + // use newest value from plain state. Or zero if none. Ok(self.tx.get::(address)?) } } @@ -95,27 +95,27 @@ impl<'a, 'b, TX: DbTx<'a>> BlockHashProvider for HistoricalStateProviderRef<'a, impl<'a, 'b, TX: DbTx<'a>> StateProvider for HistoricalStateProviderRef<'a, 'b, TX> { /// Get storage. fn storage(&self, address: Address, storage_key: StorageKey) -> Result> { - // history key to search IntegerList of transition id changesets. - let history_key = StorageShardedKey::new(address, storage_key, self.transition); + // history key to search IntegerList of block changesets. + let history_key = StorageShardedKey::new(address, storage_key, self.block_number); - let changeset_transition_id = self + let changeset_block_number = self .tx .cursor_read::()? .seek(history_key)? .filter(|(key, _)| key.address == address && key.sharded_key.key == storage_key) .map(|(_, list)| { - list.0.enable_rank().successor(self.transition as usize).map(|i| i as u64) + list.0.enable_rank().successor(self.block_number as usize).map(|i| i as u64) }); // if changeset transition id is present we are getting value from changeset - if let Some(Some(changeset_transition_id)) = changeset_transition_id { + if let Some(Some(changeset_block_number)) = changeset_block_number { let storage_entry = self .tx .cursor_dup_read::()? - .seek_by_key_subkey((changeset_transition_id, address).into(), storage_key)? + .seek_by_key_subkey((changeset_block_number, address).into(), storage_key)? .filter(|entry| entry.key == storage_key) .ok_or(ProviderError::StorageChangeset { - transition_id: changeset_transition_id, + block_number: changeset_block_number, address, storage_key, })?; @@ -151,22 +151,22 @@ impl<'a, 'b, TX: DbTx<'a>> StateProvider for HistoricalStateProviderRef<'a, 'b, pub struct HistoricalStateProvider<'a, TX: DbTx<'a>> { /// Database transaction tx: TX, - /// Transition is main indexer of account and storage changes - transition: TransitionId, + /// State at the block number is the main indexer of the state. + block_number: BlockNumber, /// Phantom lifetime `'a` _phantom: PhantomData<&'a TX>, } impl<'a, TX: DbTx<'a>> HistoricalStateProvider<'a, TX> { /// Create new StateProvider from history transaction number - pub fn new(tx: TX, transition: TransitionId) -> Self { - Self { tx, transition, _phantom: PhantomData {} } + pub fn new(tx: TX, block_number: BlockNumber) -> Self { + Self { tx, block_number, _phantom: PhantomData {} } } /// Returns a new provider that takes the `TX` as reference #[inline(always)] fn as_ref<'b>(&'b self) -> HistoricalStateProviderRef<'a, 'b, TX> { - HistoricalStateProviderRef::new(&self.tx, self.transition) + HistoricalStateProviderRef::new(&self.tx, self.block_number) } } @@ -184,7 +184,7 @@ mod tests { models::{storage_sharded_key::StorageShardedKey, AccountBeforeTx, ShardedKey}, tables, transaction::{DbTx, DbTxMut}, - TransitionList, + BlockNumberList, }; use reth_primitives::{hex_literal::hex, Account, StorageEntry, H160, H256, U256}; @@ -204,13 +204,13 @@ mod tests { let tx = db.tx_mut().unwrap(); tx.put::( - ShardedKey { key: ADDRESS, highest_transition_id: 7 }, - TransitionList::new([3, 7]).unwrap(), + ShardedKey { key: ADDRESS, highest_block_number: 7 }, + BlockNumberList::new([3, 7]).unwrap(), ) .unwrap(); tx.put::( - ShardedKey { key: ADDRESS, highest_transition_id: u64::MAX }, - TransitionList::new([10, 15]).unwrap(), + ShardedKey { key: ADDRESS, highest_block_number: u64::MAX }, + BlockNumberList::new([10, 15]).unwrap(), ) .unwrap(); @@ -291,17 +291,17 @@ mod tests { tx.put::( StorageShardedKey { address: ADDRESS, - sharded_key: ShardedKey { key: STORAGE, highest_transition_id: 7 }, + sharded_key: ShardedKey { key: STORAGE, highest_block_number: 7 }, }, - TransitionList::new([3, 7]).unwrap(), + BlockNumberList::new([3, 7]).unwrap(), ) .unwrap(); tx.put::( StorageShardedKey { address: ADDRESS, - sharded_key: ShardedKey { key: STORAGE, highest_transition_id: u64::MAX }, + sharded_key: ShardedKey { key: STORAGE, highest_block_number: u64::MAX }, }, - TransitionList::new([10, 15]).unwrap(), + BlockNumberList::new([10, 15]).unwrap(), ) .unwrap(); diff --git a/crates/storage/provider/src/test_utils/blocks.rs b/crates/storage/provider/src/test_utils/blocks.rs index 7e4ec1c47..2ba17875a 100644 --- a/crates/storage/provider/src/test_utils/blocks.rs +++ b/crates/storage/provider/src/test_utils/blocks.rs @@ -3,8 +3,8 @@ use crate::{post_state::PostState, Transaction}; use reth_db::{database::Database, models::StoredBlockBodyIndices, tables}; use reth_primitives::{ - hex_literal::hex, Account, Header, SealedBlock, SealedBlockWithSenders, Withdrawal, H160, H256, - U256, + hex_literal::hex, Account, BlockNumber, Header, SealedBlock, SealedBlockWithSenders, + Withdrawal, H160, H256, U256, }; use reth_rlp::Decodable; use std::collections::BTreeMap; @@ -54,9 +54,20 @@ pub struct BlockChainTestData { pub blocks: Vec<(SealedBlockWithSenders, PostState)>, } +impl BlockChainTestData { + /// Create test data with two blocks that are connected, specifying their block numbers. + pub fn default_with_numbers(one: BlockNumber, two: BlockNumber) -> Self { + let one = block1(one); + let hash = one.0.hash; + Self { genesis: genesis(), blocks: vec![one, block2(two, hash)] } + } +} + impl Default for BlockChainTestData { fn default() -> Self { - Self { genesis: genesis(), blocks: vec![block1(), block2()] } + let one = block1(1); + let hash = one.0.hash; + Self { genesis: genesis(), blocks: vec![one, block2(2, hash)] } } } @@ -72,12 +83,12 @@ pub fn genesis() -> SealedBlock { } /// Block one that points to genesis -fn block1() -> (SealedBlockWithSenders, PostState) { +fn block1(number: BlockNumber) -> (SealedBlockWithSenders, PostState) { let mut block_rlp = hex!("f9025ff901f7a0c86e8cc0310ae7c531c758678ddbfd16fc51c8cef8cec650b032de9869e8b94fa01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa050554882fbbda2c2fd93fdc466db9946ea262a67f7a76cc169e714f105ab583da00967f09ef1dfed20c0eacfaa94d5cd4002eda3242ac47eae68972d07b106d192a0e3c8b47fbfc94667ef4cceb17e5cc21e3b1eebd442cebb27f07562b33836290db90100000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000008302000001830f42408238108203e800a00000000000000000000000000000000000000000000000000000000000000000880000000000000000f862f860800a83061a8094095e7baea6a6c7c4c2dfeb977efac326af552d8780801ba072ed817487b84ba367d15d2f039b5fc5f087d0a8882fbdf73e8cb49357e1ce30a0403d800545b8fc544f92ce8124e2255f8c3c6af93f28243a120585d4c4c6a2a3c0").as_slice(); let mut block = SealedBlock::decode(&mut block_rlp).unwrap(); block.withdrawals = Some(vec![Withdrawal::default()]); let mut header = block.header.clone().unseal(); - header.number = 1; + header.number = number; header.state_root = H256(hex!("5d035ccb3e75a9057452ff060b773b213ec1fc353426174068edfc3971a0b6bd")); header.parent_hash = H256::zero(); @@ -86,57 +97,50 @@ fn block1() -> (SealedBlockWithSenders, PostState) { let mut post_state = PostState::default(); // Transaction changes post_state.create_account( + number, + H160([0x61; 20]), + Account { nonce: 1, balance: U256::from(10), bytecode_hash: None }, + ); + post_state.create_account( + number, H160([0x60; 20]), Account { nonce: 1, balance: U256::from(10), bytecode_hash: None }, ); post_state.change_storage( + number, H160([0x60; 20]), BTreeMap::from([(U256::from(5), (U256::ZERO, U256::from(10)))]), ); - post_state.finish_transition(); - // Block changes - post_state.create_account( - H160([0x61; 20]), - Account { nonce: 1, balance: U256::from(10), bytecode_hash: None }, - ); - post_state.finish_transition(); (SealedBlockWithSenders { block, senders: vec![H160([0x30; 20])] }, post_state) } /// Block two that points to block 1 -fn block2() -> (SealedBlockWithSenders, PostState) { +fn block2(number: BlockNumber, parent_hash: H256) -> (SealedBlockWithSenders, PostState) { let mut block_rlp = hex!("f9025ff901f7a0c86e8cc0310ae7c531c758678ddbfd16fc51c8cef8cec650b032de9869e8b94fa01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347942adc25665018aa1fe0e6bc666dac8fc2697ff9baa050554882fbbda2c2fd93fdc466db9946ea262a67f7a76cc169e714f105ab583da00967f09ef1dfed20c0eacfaa94d5cd4002eda3242ac47eae68972d07b106d192a0e3c8b47fbfc94667ef4cceb17e5cc21e3b1eebd442cebb27f07562b33836290db90100000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000008302000001830f42408238108203e800a00000000000000000000000000000000000000000000000000000000000000000880000000000000000f862f860800a83061a8094095e7baea6a6c7c4c2dfeb977efac326af552d8780801ba072ed817487b84ba367d15d2f039b5fc5f087d0a8882fbdf73e8cb49357e1ce30a0403d800545b8fc544f92ce8124e2255f8c3c6af93f28243a120585d4c4c6a2a3c0").as_slice(); let mut block = SealedBlock::decode(&mut block_rlp).unwrap(); block.withdrawals = Some(vec![Withdrawal::default()]); let mut header = block.header.clone().unseal(); - header.number = 2; + header.number = number; header.state_root = H256(hex!("90101a13dd059fa5cca99ed93d1dc23657f63626c5b8f993a2ccbdf7446b64f8")); // parent_hash points to block1 hash - header.parent_hash = - H256(hex!("d846db2ab174c492cfe985c18fa75b154e20572bc33bb1c67cf5d2995791bdb7")); + header.parent_hash = parent_hash; block.header = header.seal_slow(); let mut post_state = PostState::default(); - // Transaction changes + // block changes post_state.change_account( + number, H160([0x60; 20]), Account { nonce: 1, balance: U256::from(10), bytecode_hash: None }, - Account { nonce: 2, balance: U256::from(15), bytecode_hash: None }, + Account { nonce: 3, balance: U256::from(20), bytecode_hash: None }, ); post_state.change_storage( + number, H160([0x60; 20]), BTreeMap::from([(U256::from(5), (U256::from(10), U256::from(15)))]), ); - post_state.finish_transition(); - // Block changes - post_state.change_account( - H160([0x60; 20]), - Account { nonce: 2, balance: U256::from(15), bytecode_hash: None }, - Account { nonce: 3, balance: U256::from(20), bytecode_hash: None }, - ); - post_state.finish_transition(); (SealedBlockWithSenders { block, senders: vec![H160([0x31; 20])] }, post_state) } diff --git a/crates/storage/provider/src/transaction.rs b/crates/storage/provider/src/transaction.rs index 4c4b1b982..b043e97a2 100644 --- a/crates/storage/provider/src/transaction.rs +++ b/crates/storage/provider/src/transaction.rs @@ -1,6 +1,6 @@ use crate::{ insert_canonical_block, - post_state::{Change, PostState, StorageChangeset}, + post_state::{PostState, StorageChangeset}, }; use itertools::{izip, Itertools}; use reth_db::{ @@ -10,24 +10,23 @@ use reth_db::{ models::{ sharded_key, storage_sharded_key::{self, StorageShardedKey}, - AccountBeforeTx, ShardedKey, StoredBlockBodyIndices, TransitionIdAddress, + AccountBeforeTx, BlockNumberAddress, ShardedKey, StoredBlockBodyIndices, }, table::Table, tables, transaction::{DbTx, DbTxMut, DbTxMutGAT}, - TransitionList, + BlockNumberList, }; use reth_interfaces::{db::Error as DbError, provider::ProviderError}; use reth_primitives::{ - keccak256, proofs::EMPTY_ROOT, Account, Address, BlockHash, BlockNumber, ChainSpec, Hardfork, - Header, SealedBlock, SealedBlockWithSenders, StorageEntry, TransactionSignedEcRecovered, - TransitionId, TxNumber, H256, U256, + keccak256, Account, Address, BlockHash, BlockNumber, ChainSpec, Hardfork, Header, SealedBlock, + SealedBlockWithSenders, StorageEntry, TransactionSignedEcRecovered, TxNumber, H256, U256, }; use reth_trie::{StateRoot, StateRootError}; use std::{ collections::{btree_map::Entry, BTreeMap, BTreeSet}, fmt::Debug, - ops::{Bound, Deref, DerefMut, Range, RangeBounds}, + ops::{Deref, DerefMut, Range, RangeBounds, RangeInclusive}, }; /// A container for any DB transaction that will open a new inner transaction when the current @@ -97,6 +96,11 @@ where self.db } + /// Get lastest block number. + pub fn tip_number(&self) -> Result { + Ok(self.cursor_read::()?.last()?.unwrap_or_default().0) + } + /// Commit the current inner transaction and open a new one. /// /// # Panics @@ -140,7 +144,7 @@ where } /// Query the block body by number. - pub fn get_block_meta( + pub fn block_body_indices( &self, number: BlockNumber, ) -> Result { @@ -150,28 +154,16 @@ where Ok(body) } - /// Query the last transition of the block by [BlockNumber] key - pub fn get_block_transition(&self, key: BlockNumber) -> Result { - let last_transition_id = self - .get::(key)? - .ok_or(ProviderError::BlockTransition { block_number: key })? - .transition_after_block(); - Ok(last_transition_id) - } - /// Get the next start transaction id and transition for the `block` by looking at the previous /// block. Returns Zero/Zero for Genesis. - pub fn get_next_block_ids( - &self, - block: BlockNumber, - ) -> Result<(TxNumber, TransitionId), TransactionError> { + pub fn first_block_number(&self, block: BlockNumber) -> Result { if block == 0 { - return Ok((0, 0)) + return Ok(0) } let prev_number = block - 1; - let prev_body = self.get_block_meta(prev_number)?; - Ok((prev_body.first_tx_num + prev_body.tx_count, prev_body.transition_after_block())) + let prev_body = self.block_body_indices(prev_number)?; + Ok(prev_body.first_tx_num + prev_body.tx_count) } /// Query the block header by number @@ -200,24 +192,22 @@ where } /// Unwind the table to a provided block - pub(crate) fn unwind_table( - &self, - block: BlockNumber, - mut selector: F, - ) -> Result<(), DbError> + /// + /// Note: Key is not inclusive and specified key would stay in db. + pub(crate) fn unwind_table(&self, key: u64, mut selector: F) -> Result<(), DbError> where DB: Database, T: Table, - F: FnMut(T::Key) -> BlockNumber, + F: FnMut(T::Key) -> u64, { let mut cursor = self.cursor_write::()?; let mut reverse_walker = cursor.walk_back(None)?; - while let Some(Ok((key, _))) = reverse_walker.next() { - if selector(key.clone()) <= block { + while let Some(Ok((entry_key, _))) = reverse_walker.next() { + if selector(entry_key.clone()) <= key { break } - self.delete::(key, None)?; + self.delete::(entry_key, None)?; } Ok(()) } @@ -309,31 +299,11 @@ where self.get_take_block_range::(chain_spec, range) } - /// Transverse over changesets and plain state and recreated the execution results. - /// - /// Return results from database. - pub fn get_block_execution_result_range( - &self, - range: impl RangeBounds + Clone, - ) -> Result, TransactionError> { - self.get_take_block_execution_result_range::(range) - } - - /// Transverse over changesets and plain state and recreated the execution results. - /// - /// Get results and remove them from database - pub fn take_block_execution_result_range( - &self, - range: impl RangeBounds + Clone, - ) -> Result, TransactionError> { - self.get_take_block_execution_result_range::(range) - } - /// Get range of blocks and its execution result pub fn get_block_and_execution_range( &self, chain_spec: &ChainSpec, - range: impl RangeBounds + Clone, + range: RangeInclusive, ) -> Result, TransactionError> { self.get_take_block_and_execution_range::(chain_spec, range) } @@ -342,7 +312,7 @@ where pub fn take_block_and_execution_range( &self, chain_spec: &ChainSpec, - range: impl RangeBounds + Clone, + range: RangeInclusive, ) -> Result, TransactionError> { self.get_take_block_and_execution_range::(chain_spec, range) } @@ -350,7 +320,7 @@ where /// Unwind and clear account hashing pub fn unwind_account_hashing( &self, - range: Range, + range: RangeInclusive, ) -> Result<(), TransactionError> { let mut hashed_accounts = self.cursor_write::()?; @@ -389,7 +359,7 @@ where /// Unwind and clear storage hashing pub fn unwind_storage_hashing( &self, - range: Range, + range: Range, ) -> Result<(), TransactionError> { let mut hashed_storage = self.cursor_dup_write::()?; @@ -403,7 +373,7 @@ where .fold( BTreeMap::new(), |mut accounts: BTreeMap<(Address, H256), U256>, - (TransitionIdAddress((_, address)), storage_entry)| { + (BlockNumberAddress((_, address)), storage_entry)| { accounts.insert((address, storage_entry.key), storage_entry.value); accounts }, @@ -435,7 +405,7 @@ where /// Unwind and clear account history indices pub fn unwind_account_history_indices( &self, - range: Range, + range: RangeInclusive, ) -> Result<(), TransactionError> { let mut cursor = self.cursor_write::()?; @@ -463,7 +433,7 @@ where // there are items in list self.put::( ShardedKey::new(address, u64::MAX), - TransitionList::new(shard_part) + BlockNumberList::new(shard_part) .expect("There is at least one element in list and it is sorted."), )?; } @@ -474,7 +444,7 @@ where /// Unwind and clear storage history indices pub fn unwind_storage_history_indices( &self, - range: Range, + range: Range, ) -> Result<(), TransactionError> { let mut cursor = self.cursor_write::()?; @@ -491,7 +461,7 @@ where BTreeMap::new(), |mut accounts: BTreeMap<(Address, H256), u64>, (index, storage)| { // we just need address and lowest transition id. - accounts.insert((index.address(), storage.key), index.transition_id()); + accounts.insert((index.address(), storage.key), index.block_number()); accounts }, ); @@ -504,7 +474,7 @@ where // there are items in list self.put::( StorageShardedKey::new(address, storage_key, u64::MAX), - TransitionList::new(shard_part) + BlockNumberList::new(shard_part) .expect("There is at least one element in list and it is sorted."), )?; } @@ -522,31 +492,25 @@ where if blocks.is_empty() { return Ok(()) } - let tip = blocks.last().unwrap(); - let new_tip_number = tip.number; - let new_tip_hash = tip.hash; - let expected_state_root = tip.state_root; - - let fork_block_number = blocks.first().unwrap().number.saturating_sub(1); - - let first_transition_id = self.get_block_transition(fork_block_number)?; - - let num_transitions = state.transitions_count(); + let new_tip = blocks.last().unwrap(); + let new_tip_number = new_tip.number; // Write state and changesets to the database - state.write_to_db(self.deref_mut(), first_transition_id)?; + state.write_to_db(self.deref_mut())?; + + let first_number = blocks.first().unwrap().number; + + let last = blocks.last().unwrap(); + let last_block_number = last.number; + let last_block_hash = last.hash(); + let expected_state_root = last.state_root; // Insert the blocks for block in blocks { self.insert_block(block)?; } - self.insert_hashes( - first_transition_id, - first_transition_id + num_transitions, - new_tip_number, - new_tip_hash, - expected_state_root, - )?; + + self.insert_hashes(first_number..=last_block_number, last_block_hash, expected_state_root)?; // Update pipeline progress self.update_pipeline_stages(new_tip_number)?; @@ -564,24 +528,23 @@ where /// /// This assumes that we are using beacon consensus and that the block is post-merge, which /// means that the block will have no block reward. + /// TODO do multi block insertion. pub fn insert_block(&mut self, block: SealedBlockWithSenders) -> Result<(), TransactionError> { // Header, Body, SenderRecovery, TD, TxLookup stages let (block, senders) = block.into_components(); + let range = block.number..=block.number; - let block_meta = - insert_canonical_block(self.deref_mut(), block, Some(senders), false).unwrap(); + insert_canonical_block(self.deref_mut(), block, Some(senders)).unwrap(); - let from = block_meta.transition_at_block(); - let to = block_meta.transition_after_block(); // account history stage { - let indices = self.get_account_transition_ids_from_changeset(from, to)?; + let indices = self.get_account_transition_ids_from_changeset(range.clone())?; self.insert_account_history_index(indices)?; } // storage history stage { - let indices = self.get_storage_transition_ids_from_changeset(from, to)?; + let indices = self.get_storage_transition_ids_from_changeset(range)?; self.insert_storage_history_index(indices)?; } @@ -597,45 +560,36 @@ where /// The resulting state root is compared with `expected_state_root`. pub fn insert_hashes( &mut self, - from_transition_id: TransitionId, - to_transition_id: TransitionId, - current_block_number: BlockNumber, - current_block_hash: H256, + range: RangeInclusive, + end_block_hash: H256, expected_state_root: H256, ) -> Result<(), TransactionError> { // storage hashing stage { - let lists = self - .get_addresses_and_keys_of_changed_storages(from_transition_id, to_transition_id)?; + let lists = self.get_addresses_and_keys_of_changed_storages(range.clone())?; let storages = self.get_plainstate_storages(lists.into_iter())?; self.insert_storage_for_hashing(storages.into_iter())?; } // account hashing stage { - let lists = - self.get_addresses_of_changed_accounts(from_transition_id, to_transition_id)?; + let lists = self.get_addresses_of_changed_accounts(range.clone())?; let accounts = self.get_plainstate_accounts(lists.into_iter())?; self.insert_account_for_hashing(accounts.into_iter())?; } // merkle tree { - let state_root = StateRoot::incremental_root( - self.deref_mut(), - from_transition_id..to_transition_id, - None, - )?; + let state_root = StateRoot::incremental_root(self.deref_mut(), range.clone(), None)?; if state_root != expected_state_root { return Err(TransactionError::StateTrieRootMismatch { got: state_root, expected: expected_state_root, - block_number: current_block_number, - block_hash: current_block_hash, + block_number: *range.end(), + block_hash: end_block_hash, }) } } - Ok(()) } @@ -855,31 +809,14 @@ where /// 5. Get all receipts from table fn get_take_block_execution_result_range( &self, - range: impl RangeBounds + Clone, + range: RangeInclusive, ) -> Result, TransactionError> { - // We are not removing block meta as it is used to get block transitions. - let block_transition = - self.get_or_take::(range.clone())?; - - if block_transition.is_empty() { + if range.is_empty() { return Ok(Vec::new()) } - // get block transition of parent block. - let from = block_transition - .first() - .expect("Check for empty is already done") - .1 - .transition_at_block(); - let to = block_transition - .last() - .expect("Check for empty is already done") - .1 - .transition_after_block(); - - // NOTE: Just get block bodies dont remove them - // it is connection point for bodies getter and execution result getter. - let block_bodies = self.get_or_take::(range)?; + // We are not removing block meta as it is used to get block transitions. + let block_bodies = self.get_or_take::(range.clone())?; // get transaction receipts let from_transaction_num = @@ -889,32 +826,29 @@ where let receipts = self.get_or_take::(from_transaction_num..=to_transaction_num)?; - // get saved previous values - let from_storage: TransitionIdAddress = (from, Address::zero()).into(); - let to_storage: TransitionIdAddress = (to, Address::zero()).into(); + let storage_range = BlockNumberAddress::range(range.clone()); let storage_changeset = - self.get_or_take::(from_storage..to_storage)?; - let account_changeset = self.get_or_take::(from..to)?; + self.get_or_take::(storage_range)?; + let account_changeset = self.get_or_take::(range)?; // iterate previous value and get plain state value to create changeset // Double option around Account represent if Account state is know (first option) and // account is removed (Second Option) type LocalPlainState = BTreeMap>, BTreeMap)>; - type Changesets = BTreeMap>; let mut local_plain_state: LocalPlainState = BTreeMap::new(); // iterate in reverse and get plain state. // Bundle execution changeset to its particular transaction and block - let mut all_changesets: Changesets = BTreeMap::new(); + let mut block_states: BTreeMap = BTreeMap::new(); let mut plain_accounts_cursor = self.cursor_write::()?; let mut plain_storage_cursor = self.cursor_dup_write::()?; // add account changeset changes - for (transition_id, account_before) in account_changeset.into_iter().rev() { + for (block_number, account_before) in account_changeset.into_iter().rev() { let AccountBeforeTx { info: old_info, address } = account_before; let new_info = match local_plain_state.entry(address) { Entry::Vacant(entry) => { @@ -928,38 +862,26 @@ where } }; - let change = match (old_info, new_info) { + let post_state = block_states.entry(block_number).or_default(); + match (old_info, new_info) { (Some(old), Some(new)) => { if new != old { - Change::AccountChanged { - id: transition_id, - address, - old, - new, - } + post_state.change_account(block_number, address, old, new); } else { unreachable!("Junk data in database: an account changeset did not represent any change"); } } - (None, Some(account)) => Change::AccountCreated { - id: transition_id, - address, - account - }, - (Some(old), None) => Change::AccountDestroyed { - id: transition_id, - address, - old - }, + (None, Some(account)) => post_state.create_account(block_number, address, account), + (Some(old), None) => + post_state.destroy_account(block_number, address, old), (None, None) => unreachable!("Junk data in database: an account changeset transitioned from no account to no account"), }; - all_changesets.entry(transition_id).or_default().push(change); } // add storage changeset changes - let mut storage_changes: BTreeMap = BTreeMap::new(); - for (transition_and_address, storage_entry) in storage_changeset.into_iter().rev() { - let TransitionIdAddress((_, address)) = transition_and_address; + let mut storage_changes: BTreeMap = BTreeMap::new(); + for (block_and_address, storage_entry) in storage_changeset.into_iter().rev() { + let BlockNumberAddress((_, address)) = block_and_address; let new_storage = match local_plain_state.entry(address).or_default().1.entry(storage_entry.key) { Entry::Vacant(entry) => { @@ -974,20 +896,20 @@ where std::mem::replace(entry.get_mut(), storage_entry.value) } }; - storage_changes.entry(transition_and_address).or_default().insert( + storage_changes.entry(block_and_address).or_default().insert( U256::from_be_bytes(storage_entry.key.0), (storage_entry.value, new_storage), ); } - for (TransitionIdAddress((transition_id, address)), storage_changeset) in + for (BlockNumberAddress((block_number, address)), storage_changeset) in storage_changes.into_iter() { - all_changesets.entry(transition_id).or_default().push(Change::StorageChanged { - id: transition_id, + block_states.entry(block_number).or_default().change_storage( + block_number, address, - changeset: storage_changeset, - }); + storage_changeset, + ); } if TAKE { @@ -1026,90 +948,41 @@ where } // iterate over block body and create ExecutionResult - let mut block_exec_results = Vec::new(); - let mut block_transition_iter = block_transition.into_iter(); - let mut next_transition_id = from; - let mut receipt_iter = receipts.into_iter(); // loop break if we are at the end of the blocks. - for (_, block_body) in block_bodies.into_iter() { - let mut block_post_state = PostState::new(); + for (block_number, block_body) in block_bodies.into_iter() { for tx_num in block_body.tx_num_range() { - if let Some(changes) = all_changesets.remove(&next_transition_id) { - for mut change in changes.into_iter() { - change - .set_transition_id(block_post_state.transitions_count() as TransitionId); - block_post_state.add_and_apply(change); - } - } if let Some((receipt_tx_num, receipt)) = receipt_iter.next() { if tx_num != receipt_tx_num { - block_post_state.add_receipt(receipt) + block_states.entry(block_number).or_default().add_receipt(receipt); } } - block_post_state.finish_transition(); - next_transition_id += 1; } - - let Some((_,block_meta)) = block_transition_iter.next() else { break}; - // if block transition points to 1+next transition id it means that there is block - // changeset. - if block_meta.has_block_change() { - if let Some(changes) = all_changesets.remove(&next_transition_id) { - for mut change in changes.into_iter() { - change - .set_transition_id(block_post_state.transitions_count() as TransitionId); - block_post_state.add_and_apply(change); - } - block_post_state.finish_transition(); - next_transition_id += 1; - } - } - block_exec_results.push(block_post_state) } - Ok(block_exec_results) + Ok(block_states.into_values().collect()) } /// Return range of blocks and its execution result pub fn get_take_block_and_execution_range( &self, chain_spec: &ChainSpec, - range: impl RangeBounds + Clone, + range: RangeInclusive, ) -> Result, TransactionError> { if TAKE { - let (from_transition, parent_number, parent_state_root) = match range.start_bound() { - Bound::Included(n) => { - let parent_number = n.saturating_sub(1); - let transition = self.get_block_transition(parent_number)?; - let parent = self.get_header(parent_number)?; - (transition, parent_number, parent.state_root) - } - Bound::Excluded(n) => { - let transition = self.get_block_transition(*n)?; - let parent = self.get_header(*n)?; - (transition, *n, parent.state_root) - } - Bound::Unbounded => (0, 0, EMPTY_ROOT), - }; - let to_transition = match range.end_bound() { - Bound::Included(n) => self.get_block_transition(*n)?, - Bound::Excluded(n) => self.get_block_transition(n.saturating_sub(1))?, - Bound::Unbounded => TransitionId::MAX, - }; + let storage_range = BlockNumberAddress::range(range.clone()); - let transition_range = from_transition..to_transition; - let zero = Address::zero(); - let transition_storage_range = - (from_transition, zero).into()..(to_transition, zero).into(); - - self.unwind_account_hashing(transition_range.clone())?; - self.unwind_account_history_indices(transition_range.clone())?; - self.unwind_storage_hashing(transition_storage_range.clone())?; - self.unwind_storage_history_indices(transition_storage_range)?; + self.unwind_account_hashing(range.clone())?; + self.unwind_account_history_indices(range.clone())?; + self.unwind_storage_hashing(storage_range.clone())?; + self.unwind_storage_history_indices(storage_range)?; // merkle tree - let new_state_root = StateRoot::incremental_root(self.deref(), transition_range, None)?; + let new_state_root = StateRoot::incremental_root(self.deref(), range.clone(), None)?; + + let parent_number = range.start().saturating_sub(1); + let parent_state_root = self.get_header(parent_number)?.state_root; + // state root should be always correct as we are reverting state. // but for sake of double verification we will check it again. if new_state_root != parent_state_root { @@ -1164,15 +1037,11 @@ where /// Iterate over account changesets and return all account address that were changed. pub fn get_addresses_and_keys_of_changed_storages( &self, - from: TransitionId, - to: TransitionId, + range: RangeInclusive, ) -> Result>, TransactionError> { Ok(self .cursor_read::()? - .walk_range( - TransitionIdAddress((from, Address::zero())).. - TransitionIdAddress((to, Address::zero())), - )? + .walk_range(BlockNumberAddress::range(range))? .collect::, _>>()? .into_iter() // fold all storages and save its old state so we can remove it from HashedStorage @@ -1180,14 +1049,14 @@ where .fold( BTreeMap::new(), |mut accounts: BTreeMap>, - (TransitionIdAddress((_, address)), storage_entry)| { + (BlockNumberAddress((_, address)), storage_entry)| { accounts.entry(address).or_default().insert(storage_entry.key); accounts }, )) } - /// Get plainstate storages + /// Get plainstate storages #[allow(clippy::type_complexity)] pub fn get_plainstate_storages( &self, @@ -1252,12 +1121,11 @@ where /// Iterate over account changesets and return all account address that were changed. pub fn get_addresses_of_changed_accounts( &self, - from: TransitionId, - to: TransitionId, + range: RangeInclusive, ) -> Result, TransactionError> { Ok(self .cursor_read::()? - .walk_range(from..to)? + .walk_range(range)? .collect::, _>>()? .into_iter() // fold all account to one set of changed accounts @@ -1308,15 +1176,15 @@ where } /// Get all transaction ids where account got changed. + /// + /// NOTE: Get inclusive range of blocks. pub fn get_storage_transition_ids_from_changeset( &self, - from: TransitionId, - to: TransitionId, + range: RangeInclusive, ) -> Result>, TransactionError> { let storage_changeset = self .cursor_read::()? - .walk(Some((from, Address::zero()).into()))? - .take_while(|res| res.as_ref().map(|(k, _)| k.transition_id() < to).unwrap_or_default()) + .walk_range(BlockNumberAddress::range(range))? .collect::, _>>()?; // fold all storages to one set of changes @@ -1326,7 +1194,7 @@ where storages .entry((index.address(), storage.key)) .or_default() - .push(index.transition_id()); + .push(index.block_number()); storages }, ); @@ -1335,15 +1203,15 @@ where } /// Get all transaction ids where account got changed. + /// + /// NOTE: Get inclusive range of blocks. pub fn get_account_transition_ids_from_changeset( &self, - from: TransitionId, - to: TransitionId, + range: RangeInclusive, ) -> Result>, TransactionError> { let account_changesets = self .cursor_read::()? - .walk(Some(from))? - .take_while(|res| res.as_ref().map(|(k, _)| *k < to).unwrap_or_default()) + .walk_range(range)? .collect::, _>>()?; let account_transtions = account_changesets @@ -1384,16 +1252,16 @@ where StorageShardedKey::new( address, storage_key, - *list.last().expect("Chuck does not return empty list") as TransitionId, + *list.last().expect("Chuck does not return empty list") as BlockNumber, ), - TransitionList::new(list).expect("Indices are presorted and not empty"), + BlockNumberList::new(list).expect("Indices are presorted and not empty"), ) })?; // Insert last list with u64::MAX if let Some(last_list) = last_chunk { self.put::( StorageShardedKey::new(address, storage_key, u64::MAX), - TransitionList::new(last_list).expect("Indices are presorted and not empty"), + BlockNumberList::new(last_list).expect("Indices are presorted and not empty"), )?; } } @@ -1422,16 +1290,16 @@ where self.put::( ShardedKey::new( address, - *list.last().expect("Chuck does not return empty list") as TransitionId, + *list.last().expect("Chuck does not return empty list") as BlockNumber, ), - TransitionList::new(list).expect("Indices are presorted and not empty"), + BlockNumberList::new(list).expect("Indices are presorted and not empty"), ) })?; // Insert last list with u64::MAX if let Some(last_list) = last_chunk { self.put::( ShardedKey::new(address, u64::MAX), - TransitionList::new(last_list).expect("Indices are presorted and not empty"), + BlockNumberList::new(last_list).expect("Indices are presorted and not empty"), )? } } @@ -1455,7 +1323,7 @@ fn unwind_account_history_shards( tables::AccountHistory, >, address: Address, - transition_id: TransitionId, + block_number: BlockNumber, ) -> Result, TransactionError> { let mut item = cursor.seek_exact(ShardedKey::new(address, u64::MAX))?; @@ -1468,13 +1336,13 @@ fn unwind_account_history_shards( // check first item and if it is more and eq than `transition_id` delete current // item. let first = list.iter(0).next().expect("List can't empty"); - if first >= transition_id as usize { + if first >= block_number as usize { item = cursor.prev()?; continue - } else if transition_id <= sharded_key.highest_transition_id { + } else if block_number <= sharded_key.highest_block_number { // if first element is in scope whole list would be removed. // so at least this first element is present. - return Ok(list.iter(0).take_while(|i| *i < transition_id as usize).collect::>()) + return Ok(list.iter(0).take_while(|i| *i < block_number as usize).collect::>()) } else { let new_list = list.iter(0).collect::>(); return Ok(new_list) @@ -1493,7 +1361,7 @@ fn unwind_storage_history_shards( >, address: Address, storage_key: H256, - transition_id: TransitionId, + block_number: BlockNumber, ) -> Result, TransactionError> { let mut item = cursor.seek_exact(StorageShardedKey::new(address, storage_key, u64::MAX))?; @@ -1509,13 +1377,13 @@ fn unwind_storage_history_shards( // check first item and if it is more and eq than `transition_id` delete current // item. let first = list.iter(0).next().expect("List can't empty"); - if first >= transition_id as usize { + if first >= block_number as usize { item = cursor.prev()?; continue - } else if transition_id <= storage_sharded_key.sharded_key.highest_transition_id { + } else if block_number <= storage_sharded_key.sharded_key.highest_block_number { // if first element is in scope whole list would be removed. // so at least this first element is present. - return Ok(list.iter(0).take_while(|i| *i < transition_id as usize).collect::>()) + return Ok(list.iter(0).take_while(|i| *i < block_number as usize).collect::>()) } else { return Ok(list.iter(0).collect::>()) } @@ -1556,7 +1424,7 @@ mod test { TransactionsProvider, }; use reth_db::mdbx::test_utils::create_test_rw_db; - use reth_primitives::{ChainSpecBuilder, TransitionId, MAINNET}; + use reth_primitives::{ChainSpecBuilder, MAINNET}; use std::{ops::DerefMut, sync::Arc}; #[test] @@ -1576,54 +1444,26 @@ mod test { let (block1, exec_res1) = data.blocks[0].clone(); let (block2, exec_res2) = data.blocks[1].clone(); - insert_canonical_block(tx.deref_mut(), data.genesis.clone(), None, false).unwrap(); + insert_canonical_block(tx.deref_mut(), data.genesis.clone(), None).unwrap(); + assert_genesis_block(&tx, data.genesis); - exec_res1.clone().write_to_db(tx.deref_mut(), 0).unwrap(); - tx.insert_block(block1.clone()).unwrap(); - tx.insert_hashes( - 0, - exec_res1.transitions_count() as TransitionId, - block1.number, - block1.hash, - block1.state_root, - ) - .unwrap(); + tx.append_blocks_with_post_state(vec![block1.clone()], exec_res1.clone()).unwrap(); // get one block let get = tx.get_block_and_execution_range(&chain_spec, 1..=1).unwrap(); - assert_eq!(get, vec![(block1.clone(), exec_res1.clone())]); + let get_block = get[0].0.clone(); + let get_state = get[0].1.clone(); + assert_eq!(get_block, block1.clone()); + assert_eq!(get_state, exec_res1.clone()); // take one block let take = tx.take_block_and_execution_range(&chain_spec, 1..=1).unwrap(); assert_eq!(take, vec![(block1.clone(), exec_res1.clone())]); assert_genesis_block(&tx, genesis.clone()); - exec_res1.clone().write_to_db(tx.deref_mut(), 0).unwrap(); - tx.insert_block(block1.clone()).unwrap(); - tx.insert_hashes( - 0, - exec_res1.transitions_count() as TransitionId, - block1.number, - block1.hash, - block1.state_root, - ) - .unwrap(); - - exec_res2 - .clone() - .write_to_db(tx.deref_mut(), exec_res1.transitions_count() as TransitionId) - .unwrap(); - tx.insert_block(block2.clone()).unwrap(); - tx.insert_hashes( - exec_res1.transitions_count() as TransitionId, - (exec_res1.transitions_count() + exec_res2.transitions_count()) as TransitionId, - 2, - block2.hash, - block2.state_root, - ) - .unwrap(); - + tx.append_blocks_with_post_state(vec![block1.clone()], exec_res1.clone()).unwrap(); + tx.append_blocks_with_post_state(vec![block2.clone()], exec_res2.clone()).unwrap(); tx.commit().unwrap(); // Check that transactions map onto blocks correctly. @@ -1652,10 +1492,10 @@ mod test { // get two blocks let get = tx.get_block_and_execution_range(&chain_spec, 1..=2).unwrap(); - assert_eq!( - get, - vec![(block1.clone(), exec_res1.clone()), (block2.clone(), exec_res2.clone())] - ); + assert_eq!(get[0].0, block1.clone()); + assert_eq!(get[1].0, block2.clone()); + assert_eq!(get[0].1, exec_res1.clone()); + assert_eq!(get[1].1, exec_res2.clone()); // take two blocks let get = tx.take_block_and_execution_range(&chain_spec, 1..=2).unwrap(); @@ -1682,7 +1522,8 @@ mod test { let (block1, exec_res1) = data.blocks[0].clone(); let (block2, exec_res2) = data.blocks[1].clone(); - insert_canonical_block(tx.deref_mut(), data.genesis.clone(), None, false).unwrap(); + insert_canonical_block(tx.deref_mut(), data.genesis.clone(), None).unwrap(); + assert_genesis_block(&tx, data.genesis); tx.append_blocks_with_post_state(vec![block1.clone()], exec_res1.clone()).unwrap(); diff --git a/crates/storage/provider/src/utils.rs b/crates/storage/provider/src/utils.rs index 3e154c60e..cafbb6568 100644 --- a/crates/storage/provider/src/utils.rs +++ b/crates/storage/provider/src/utils.rs @@ -1,9 +1,10 @@ use reth_db::{ + cursor::DbCursorRO, models::{StoredBlockBodyIndices, StoredBlockOmmers, StoredBlockWithdrawals}, tables, transaction::{DbTx, DbTxMut}, }; -use reth_interfaces::{provider::ProviderError, Result}; +use reth_interfaces::Result; use reth_primitives::{Address, SealedBlock}; /// Insert block data into corresponding tables. Used mainly for testing & internal tooling. @@ -21,8 +22,6 @@ pub fn insert_block<'a, TX: DbTxMut<'a> + DbTx<'a>>( tx: &TX, block: SealedBlock, senders: Option>, - has_block_reward: bool, - parent_tx_num_transition_id: Option<(u64, u64)>, ) -> Result { let block_number = block.number; tx.put::(block.number, block.hash())?; @@ -49,22 +48,11 @@ pub fn insert_block<'a, TX: DbTxMut<'a> + DbTx<'a>>( )?; } - let parent_block_meta = if let Some(parent_tx_num_transition_id) = parent_tx_num_transition_id { - StoredBlockBodyIndices { - first_transition_id: parent_tx_num_transition_id.1, - first_tx_num: parent_tx_num_transition_id.0, - tx_count: 0, - has_block_change: false, - } - } else if block.number == 0 { - StoredBlockBodyIndices::default() - } else { - let prev_block_num = block.number - 1; - tx.get::(prev_block_num)? - .ok_or(ProviderError::BlockBodyIndices { number: prev_block_num })? - }; + let mut next_tx_num = + tx.cursor_read::()?.last()?.map(|(n, _)| n + 1).unwrap_or_default(); + let first_tx_num = next_tx_num; + let tx_count = block.body.len() as u64; - let mut next_tx_num = parent_block_meta.next_tx_num(); let senders_len = senders.as_ref().map(|s| s.len()); let tx_iter = if Some(block.body.len()) == senders_len { @@ -88,10 +76,8 @@ pub fn insert_block<'a, TX: DbTxMut<'a> + DbTx<'a>>( next_tx_num += 1; } - let mut has_withdrawals = false; if let Some(withdrawals) = block.withdrawals { if !withdrawals.is_empty() { - has_withdrawals = true; tx.put::( block_number, StoredBlockWithdrawals { withdrawals }, @@ -99,21 +85,14 @@ pub fn insert_block<'a, TX: DbTxMut<'a> + DbTx<'a>>( } } - let has_block_change = has_block_reward || has_withdrawals; + let block_indices = StoredBlockBodyIndices { first_tx_num, tx_count }; + tx.put::(block_number, block_indices.clone())?; - let block_meta = StoredBlockBodyIndices { - first_transition_id: parent_block_meta.transition_after_block(), - first_tx_num: parent_block_meta.next_tx_num(), - tx_count, - has_block_change, - }; - tx.put::(block_number, block_meta.clone())?; - - if !block_meta.is_empty() { - tx.put::(block_meta.last_tx_num(), block_number)?; + if !block_indices.is_empty() { + tx.put::(block_indices.last_tx_num(), block_number)?; } - Ok(block_meta) + Ok(block_indices) } /// Inserts canonical block in blockchain. Parent tx num and transition id is taken from @@ -122,7 +101,6 @@ pub fn insert_canonical_block<'a, TX: DbTxMut<'a> + DbTx<'a>>( tx: &TX, block: SealedBlock, senders: Option>, - has_block_reward: bool, ) -> Result { - insert_block(tx, block, senders, has_block_reward, None) + insert_block(tx, block, senders) } diff --git a/crates/trie/src/prefix_set/loader.rs b/crates/trie/src/prefix_set/loader.rs index 3b628163e..af368a617 100644 --- a/crates/trie/src/prefix_set/loader.rs +++ b/crates/trie/src/prefix_set/loader.rs @@ -3,13 +3,13 @@ use crate::Nibbles; use derive_more::Deref; use reth_db::{ cursor::DbCursorRO, - models::{AccountBeforeTx, TransitionIdAddress}, + models::{AccountBeforeTx, BlockNumberAddress}, tables, transaction::DbTx, Error, }; -use reth_primitives::{keccak256, Address, StorageEntry, TransitionId, H256}; -use std::{collections::HashMap, ops::Range}; +use reth_primitives::{keccak256, BlockNumber, StorageEntry, H256}; +use std::{collections::HashMap, ops::RangeInclusive}; /// A wrapper around a database transaction that loads prefix sets within a given transition range. #[derive(Deref)] @@ -29,7 +29,7 @@ where /// Load all account and storage changes for the given transition id range. pub fn load( self, - tid_range: Range, + range: RangeInclusive, ) -> Result<(PrefixSet, HashMap), Error> { // Initialize prefix sets. let mut account_prefix_set = PrefixSet::default(); @@ -37,7 +37,7 @@ where // Walk account changeset and insert account prefixes. let mut account_cursor = self.cursor_read::()?; - for account_entry in account_cursor.walk_range(tid_range.clone())? { + for account_entry in account_cursor.walk_range(range.clone())? { let (_, AccountBeforeTx { address, .. }) = account_entry?; account_prefix_set.insert(Nibbles::unpack(keccak256(address))); } @@ -45,10 +45,9 @@ where // Walk storage changeset and insert storage prefixes as well as account prefixes if missing // from the account prefix set. let mut storage_cursor = self.cursor_dup_read::()?; - let start = TransitionIdAddress((tid_range.start, Address::zero())); - let end = TransitionIdAddress((tid_range.end, Address::zero())); - for storage_entry in storage_cursor.walk_range(start..end)? { - let (TransitionIdAddress((_, address)), StorageEntry { key, .. }) = storage_entry?; + let storage_range = BlockNumberAddress::range(range); + for storage_entry in storage_cursor.walk_range(storage_range)? { + let (BlockNumberAddress((_, address)), StorageEntry { key, .. }) = storage_entry?; let hashed_address = keccak256(address); account_prefix_set.insert(Nibbles::unpack(hashed_address)); storage_prefix_set diff --git a/crates/trie/src/trie.rs b/crates/trie/src/trie.rs index e242ca38f..ea868938f 100644 --- a/crates/trie/src/trie.rs +++ b/crates/trie/src/trie.rs @@ -16,10 +16,10 @@ use reth_primitives::{ keccak256, proofs::EMPTY_ROOT, trie::{BranchNodeCompact, StorageTrieEntry, StoredNibblesSubKey}, - Address, StorageEntry, TransitionId, H256, + Address, BlockNumber, StorageEntry, H256, }; use reth_rlp::Encodable; -use std::{collections::HashMap, ops::Range, sync::mpsc}; +use std::{collections::HashMap, ops::RangeInclusive, sync::mpsc}; /// The branch node update sender pub type BranchNodeUpdateSender = mpsc::Sender; @@ -77,11 +77,11 @@ impl<'a, 'tx, TX: DbTx<'tx> + DbTxMut<'tx>> StateRoot<'a, TX> { /// The updated state root hash. pub fn incremental_root( tx: &'a TX, - tid_range: Range, + range: RangeInclusive, branch_node_sender: Option, ) -> Result { tracing::debug!(target: "loader", "incremental state root"); - let (account_prefixes, storage_prefixes) = PrefixSetLoader::new(tx).load(tid_range)?; + let (account_prefixes, storage_prefixes) = PrefixSetLoader::new(tx).load(range)?; let this = Self::new(tx) .with_changed_account_prefixes(account_prefixes) .with_changed_storage_prefixes(storage_prefixes); @@ -162,7 +162,7 @@ impl<'a, 'tx, TX: DbTx<'tx> + DbTxMut<'tx>> StateRoot<'a, TX> { let account = EthAccount::from(account).with_storage_root(storage_root); let mut account_rlp = Vec::with_capacity(account.length()); - account.encode(&mut account_rlp); + account.encode(&mut &mut account_rlp); hash_builder.add_leaf(account_nibbles, &account_rlp); diff --git a/docs/design/database.md b/docs/design/database.md index 16f8e840e..27bdeb5d8 100644 --- a/docs/design/database.md +++ b/docs/design/database.md @@ -54,12 +54,12 @@ BlockTransitionIdIndex { } AccountHistory { H256 Account "PK" - TransitionList TransitionIdList "List of transitions where account was changed" + BlockNumberList TransitionIdList "List of transitions where account was changed" } StorageHistory { H256 Account "PK" H256 StorageKey "PK" - TransitionList TransitionIdList "List of transitions where account storage entry was changed" + BlockNumberList TransitionIdList "List of transitions where account storage entry was changed" } AccountChangeSet { u64 TransitionId "PK"