From d358142d2fd14e06a9906e02bd8b9c75f273620a Mon Sep 17 00:00:00 2001 From: Arsenii Kulikov Date: Thu, 23 Jan 2025 04:18:01 +0400 Subject: [PATCH] fix: remove persisted_trie_updates --- crates/chain-state/src/in_memory.rs | 73 ++++++++----- crates/chain-state/src/memory_overlay.rs | 9 +- crates/chain-state/src/test_utils.rs | 12 +-- crates/engine/tree/src/engine.rs | 4 +- crates/engine/tree/src/persistence.rs | 8 +- crates/engine/tree/src/tree/mod.rs | 100 ++++++++---------- .../ethereum/engine-primitives/src/payload.rs | 10 +- crates/ethereum/payload/src/lib.rs | 18 ++-- crates/optimism/payload/src/builder.rs | 18 ++-- crates/optimism/payload/src/payload.rs | 10 +- crates/payload/builder/src/test_utils.rs | 4 +- crates/payload/primitives/src/traits.rs | 4 +- .../src/providers/blockchain_provider.rs | 82 +++++++------- .../provider/src/providers/consistent.rs | 24 +++-- crates/storage/provider/src/writer/mod.rs | 10 +- 15 files changed, 212 insertions(+), 174 deletions(-) diff --git a/crates/chain-state/src/in_memory.rs b/crates/chain-state/src/in_memory.rs index 38c74fb0860f..8c3caec85f45 100644 --- a/crates/chain-state/src/in_memory.rs +++ b/crates/chain-state/src/in_memory.rs @@ -240,7 +240,7 @@ impl CanonicalInMemoryState { /// Updates the pending block with the given block. /// /// Note: This assumes that the parent block of the pending block is canonical. - pub fn set_pending_block(&self, pending: ExecutedBlock) { + pub fn set_pending_block(&self, pending: ExecutedBlockWithTrieUpdates) { // fetch the state of the pending block's parent block let parent = self.state_by_hash(pending.recovered_block().parent_hash()); let pending = BlockState::with_parent(pending, parent); @@ -254,9 +254,10 @@ impl CanonicalInMemoryState { /// /// This removes all reorged blocks and appends the new blocks to the tracked chain and connects /// them to their parent blocks. - fn update_blocks(&self, new_blocks: I, reorged: I) + fn update_blocks(&self, new_blocks: I, reorged: R) where - I: IntoIterator>, + I: IntoIterator>, + R: IntoIterator>, { { // acquire locks, starting with the numbers lock @@ -601,7 +602,7 @@ impl CanonicalInMemoryState { #[derive(Debug, PartialEq, Eq, Clone)] pub struct BlockState { /// The executed block that determines the state after this block has been executed. - block: ExecutedBlock, + block: ExecutedBlockWithTrieUpdates, /// The block's parent block if it exists. parent: Option>>, } @@ -609,12 +610,15 @@ pub struct BlockState { #[allow(dead_code)] impl BlockState { /// [`BlockState`] constructor. - pub const fn new(block: ExecutedBlock) -> Self { + pub const fn new(block: ExecutedBlockWithTrieUpdates) -> Self { Self { block, parent: None } } /// [`BlockState`] constructor with parent. - pub const fn with_parent(block: ExecutedBlock, parent: Option>) -> Self { + pub const fn with_parent( + block: ExecutedBlockWithTrieUpdates, + parent: Option>, + ) -> Self { Self { block, parent } } @@ -628,12 +632,12 @@ impl BlockState { } /// Returns the executed block that determines the state. - pub fn block(&self) -> ExecutedBlock { + pub fn block(&self) -> ExecutedBlockWithTrieUpdates { self.block.clone() } /// Returns a reference to the executed block that determines the state. - pub const fn block_ref(&self) -> &ExecutedBlock { + pub const fn block_ref(&self) -> &ExecutedBlockWithTrieUpdates { &self.block } @@ -787,7 +791,7 @@ impl BlockState { } /// Represents an executed block stored in-memory. -#[derive(Clone, Debug, PartialEq, Eq, Default)] +#[derive(Clone, Debug, PartialEq, Eq)] pub struct ExecutedBlock { /// Recovered Block pub recovered_block: Arc>, @@ -795,21 +799,19 @@ pub struct ExecutedBlock { pub execution_output: Arc>, /// Block's hashed state. pub hashed_state: Arc, - /// Trie updates that result of applying the block. - pub trie: Arc, } -impl ExecutedBlock { - /// [`ExecutedBlock`] constructor. - pub const fn new( - recovered_block: Arc>, - execution_output: Arc>, - hashed_state: Arc, - trie: Arc, - ) -> Self { - Self { recovered_block, execution_output, hashed_state, trie } +impl Default for ExecutedBlock { + fn default() -> Self { + Self { + recovered_block: Default::default(), + execution_output: Default::default(), + hashed_state: Default::default(), + } } +} +impl ExecutedBlock { /// Returns a reference to [`RecoveredBlock`] pub fn recovered_block(&self) -> &RecoveredBlock { &self.recovered_block @@ -824,6 +826,29 @@ impl ExecutedBlock { pub fn hashed_state(&self) -> &HashedPostState { &self.hashed_state } +} + +/// An [`ExecutedBlock`] with its [`TrieUpdates`]. +#[derive(Clone, Debug, PartialEq, Eq, Default, derive_more::Deref, derive_more::DerefMut)] +pub struct ExecutedBlockWithTrieUpdates { + /// Inner [`ExecutedBlock`]. + #[deref] + #[deref_mut] + pub block: ExecutedBlock, + /// Trie updates that result of applying the block. + pub trie: Arc, +} + +impl ExecutedBlockWithTrieUpdates { + /// [`ExecutedBlock`] constructor. + pub const fn new( + recovered_block: Arc>, + execution_output: Arc>, + hashed_state: Arc, + trie: Arc, + ) -> Self { + Self { block: ExecutedBlock { recovered_block, execution_output, hashed_state }, trie } + } /// Returns a reference to the trie updates for the block pub fn trie_updates(&self) -> &TrieUpdates { @@ -837,13 +862,13 @@ pub enum NewCanonicalChain { /// A simple append to the current canonical head Commit { /// all blocks that lead back to the canonical head - new: Vec>, + new: Vec>, }, /// A reorged chain consists of two chains that trace back to a shared ancestor block at which /// point they diverge. Reorg { /// All blocks of the _new_ chain - new: Vec>, + new: Vec>, /// All blocks of the _old_ chain old: Vec>, }, @@ -1247,7 +1272,7 @@ mod tests { block1.recovered_block().hash() ); - let chain = NewCanonicalChain::Reorg { new: vec![block2.clone()], old: vec![block1] }; + let chain = NewCanonicalChain::Reorg { new: vec![block2.clone()], old: vec![block1.block] }; state.update_chain(chain); assert_eq!( state.head_state().unwrap().block_ref().recovered_block().hash(), @@ -1530,7 +1555,7 @@ mod tests { // Test reorg notification let chain_reorg = NewCanonicalChain::Reorg { new: vec![block1a.clone(), block2a.clone()], - old: vec![block1.clone(), block2.clone()], + old: vec![block1.block.clone(), block2.block.clone()], }; assert_eq!( diff --git a/crates/chain-state/src/memory_overlay.rs b/crates/chain-state/src/memory_overlay.rs index 51594890b8ac..38c384dcdb5c 100644 --- a/crates/chain-state/src/memory_overlay.rs +++ b/crates/chain-state/src/memory_overlay.rs @@ -1,4 +1,4 @@ -use super::ExecutedBlock; +use super::ExecutedBlockWithTrieUpdates; use alloy_consensus::BlockHeader; use alloy_primitives::{ keccak256, map::B256HashMap, Address, BlockNumber, Bytes, StorageKey, StorageValue, B256, @@ -23,7 +23,7 @@ pub struct MemoryOverlayStateProviderRef<'a, N: NodePrimitives = reth_primitives /// Historical state provider for state lookups that are not found in in-memory blocks. pub(crate) historical: Box, /// The collection of executed parent blocks. Expected order is newest to oldest. - pub(crate) in_memory: Vec>, + pub(crate) in_memory: Vec>, /// Lazy-loaded in-memory trie data. pub(crate) trie_state: OnceLock, } @@ -40,7 +40,10 @@ impl<'a, N: NodePrimitives> MemoryOverlayStateProviderRef<'a, N> { /// - `in_memory` - the collection of executed ancestor blocks in reverse. /// - `historical` - a historical state provider for the latest ancestor block stored in the /// database. - pub fn new(historical: Box, in_memory: Vec>) -> Self { + pub fn new( + historical: Box, + in_memory: Vec>, + ) -> Self { Self { historical, in_memory, trie_state: OnceLock::new() } } diff --git a/crates/chain-state/src/test_utils.rs b/crates/chain-state/src/test_utils.rs index cee6802e66ec..57964d71dba5 100644 --- a/crates/chain-state/src/test_utils.rs +++ b/crates/chain-state/src/test_utils.rs @@ -1,7 +1,7 @@ use core::marker::PhantomData; use crate::{ - in_memory::ExecutedBlock, CanonStateNotification, CanonStateNotifications, + in_memory::ExecutedBlockWithTrieUpdates, CanonStateNotification, CanonStateNotifications, CanonStateSubscriptions, }; use alloy_consensus::{ @@ -210,11 +210,11 @@ impl TestBlockBuilder { block_number: BlockNumber, receipts: Receipts, parent_hash: B256, - ) -> ExecutedBlock { + ) -> ExecutedBlockWithTrieUpdates { let block_with_senders = self.generate_random_block(block_number, parent_hash); let (block, senders) = block_with_senders.split_sealed(); - ExecutedBlock::new( + ExecutedBlockWithTrieUpdates::new( Arc::new(RecoveredBlock::new_sealed(block, senders)), Arc::new(ExecutionOutcome::new( BundleState::default(), @@ -232,7 +232,7 @@ impl TestBlockBuilder { &mut self, receipts: Receipts, parent_hash: B256, - ) -> ExecutedBlock { + ) -> ExecutedBlockWithTrieUpdates { let number = rand::thread_rng().gen::(); self.get_executed_block(number, receipts, parent_hash) } @@ -242,7 +242,7 @@ impl TestBlockBuilder { &mut self, block_number: BlockNumber, parent_hash: B256, - ) -> ExecutedBlock { + ) -> ExecutedBlockWithTrieUpdates { self.get_executed_block(block_number, Receipts { receipt_vec: vec![vec![]] }, parent_hash) } @@ -250,7 +250,7 @@ impl TestBlockBuilder { pub fn get_executed_blocks( &mut self, range: Range, - ) -> impl Iterator + '_ { + ) -> impl Iterator + '_ { let mut parent_hash = B256::default(); range.map(move |number| { let current_parent_hash = parent_hash; diff --git a/crates/engine/tree/src/engine.rs b/crates/engine/tree/src/engine.rs index 1e721627becf..2d6f66d12fb6 100644 --- a/crates/engine/tree/src/engine.rs +++ b/crates/engine/tree/src/engine.rs @@ -7,7 +7,7 @@ use crate::{ }; use alloy_primitives::B256; use futures::{Stream, StreamExt}; -use reth_chain_state::ExecutedBlock; +use reth_chain_state::ExecutedBlockWithTrieUpdates; use reth_engine_primitives::{BeaconConsensusEngineEvent, BeaconEngineMessage, EngineTypes}; use reth_primitives::{NodePrimitives, RecoveredBlock}; use reth_primitives_traits::Block; @@ -245,7 +245,7 @@ pub enum EngineApiRequest { /// A request received from the consensus engine. Beacon(BeaconEngineMessage), /// Request to insert an already executed block, e.g. via payload building. - InsertExecutedBlock(ExecutedBlock), + InsertExecutedBlock(ExecutedBlockWithTrieUpdates), } impl Display for EngineApiRequest { diff --git a/crates/engine/tree/src/persistence.rs b/crates/engine/tree/src/persistence.rs index 5c9d1357d2c7..ffe401cc04c4 100644 --- a/crates/engine/tree/src/persistence.rs +++ b/crates/engine/tree/src/persistence.rs @@ -1,7 +1,7 @@ use crate::metrics::PersistenceMetrics; use alloy_consensus::BlockHeader; use alloy_eips::BlockNumHash; -use reth_chain_state::ExecutedBlock; +use reth_chain_state::ExecutedBlockWithTrieUpdates; use reth_errors::ProviderError; use reth_primitives::{EthPrimitives, NodePrimitives}; use reth_provider::{ @@ -140,7 +140,7 @@ where fn on_save_blocks( &self, - blocks: Vec>, + blocks: Vec>, ) -> Result, PersistenceError> { debug!(target: "engine::persistence", first=?blocks.first().map(|b| b.recovered_block.num_hash()), last=?blocks.last().map(|b| b.recovered_block.num_hash()), "Saving range of blocks"); let start_time = Instant::now(); @@ -181,7 +181,7 @@ pub enum PersistenceAction { /// /// First, header, transaction, and receipt-related data should be written to static files. /// Then the execution history-related data will be written to the database. - SaveBlocks(Vec>, oneshot::Sender>), + SaveBlocks(Vec>, oneshot::Sender>), /// Removes block data above the given block number from the database. /// @@ -258,7 +258,7 @@ impl PersistenceHandle { /// If there are no blocks to persist, then `None` is sent in the sender. pub fn save_blocks( &self, - blocks: Vec>, + blocks: Vec>, tx: oneshot::Sender>, ) -> Result<(), SendError>> { self.send_action(PersistenceAction::SaveBlocks(blocks, tx)) diff --git a/crates/engine/tree/src/tree/mod.rs b/crates/engine/tree/src/tree/mod.rs index eeed8afa9e3f..870e6366d4ab 100644 --- a/crates/engine/tree/src/tree/mod.rs +++ b/crates/engine/tree/src/tree/mod.rs @@ -21,7 +21,8 @@ use alloy_rpc_types_engine::{ use block_buffer::BlockBuffer; use error::{InsertBlockError, InsertBlockErrorKind, InsertBlockFatalError}; use reth_chain_state::{ - CanonicalInMemoryState, ExecutedBlock, MemoryOverlayStateProvider, NewCanonicalChain, + CanonicalInMemoryState, ExecutedBlock, ExecutedBlockWithTrieUpdates, + MemoryOverlayStateProvider, NewCanonicalChain, }; use reth_consensus::{Consensus, FullConsensus, PostExecutionInput}; pub use reth_engine_primitives::InvalidBlockHook; @@ -103,19 +104,15 @@ pub struct TreeState { /// __All__ unique executed blocks by block hash that are connected to the canonical chain. /// /// This includes blocks of all forks. - blocks_by_hash: HashMap>, + blocks_by_hash: HashMap>, /// Executed blocks grouped by their respective block number. /// /// This maps unique block number to all known blocks for that height. /// /// Note: there can be multiple blocks at the same height due to forks. - blocks_by_number: BTreeMap>>, + blocks_by_number: BTreeMap>>, /// Map of any parent block hash to its children. parent_to_child: HashMap>, - /// Map of hash to trie updates for canonical blocks that are persisted but not finalized. - /// - /// Contains the block number for easy removal. - persisted_trie_updates: HashMap)>, /// Currently tracked canonical head of the chain. current_canonical_head: BlockNumHash, } @@ -128,7 +125,6 @@ impl TreeState { blocks_by_number: BTreeMap::new(), current_canonical_head, parent_to_child: HashMap::default(), - persisted_trie_updates: HashMap::default(), } } @@ -138,7 +134,7 @@ impl TreeState { } /// Returns the [`ExecutedBlock`] by hash. - fn executed_block_by_hash(&self, hash: B256) -> Option<&ExecutedBlock> { + fn executed_block_by_hash(&self, hash: B256) -> Option<&ExecutedBlockWithTrieUpdates> { self.blocks_by_hash.get(&hash) } @@ -151,7 +147,7 @@ impl TreeState { /// newest to oldest. And the parent hash of the oldest block that is missing from the buffer. /// /// Returns `None` if the block for the given hash is not found. - fn blocks_by_hash(&self, hash: B256) -> Option<(B256, Vec>)> { + fn blocks_by_hash(&self, hash: B256) -> Option<(B256, Vec>)> { let block = self.blocks_by_hash.get(&hash).cloned()?; let mut parent_hash = block.recovered_block().parent_hash(); let mut blocks = vec![block]; @@ -164,7 +160,7 @@ impl TreeState { } /// Insert executed block into the state. - fn insert_executed(&mut self, executed: ExecutedBlock) { + fn insert_executed(&mut self, executed: ExecutedBlockWithTrieUpdates) { let hash = executed.recovered_block().hash(); let parent_hash = executed.recovered_block().parent_hash(); let block_number = executed.recovered_block().number(); @@ -195,7 +191,10 @@ impl TreeState { /// ## Returns /// /// The removed block and the block hashes of its children. - fn remove_by_hash(&mut self, hash: B256) -> Option<(ExecutedBlock, HashSet)> { + fn remove_by_hash( + &mut self, + hash: B256, + ) -> Option<(ExecutedBlockWithTrieUpdates, HashSet)> { let executed = self.blocks_by_hash.remove(&hash)?; // Remove this block from collection of children of its parent block. @@ -270,11 +269,6 @@ impl TreeState { debug!(target: "engine::tree", num_hash=?executed.recovered_block().num_hash(), "Attempting to remove block walking back from the head"); if let Some((removed, _)) = self.remove_by_hash(executed.recovered_block().hash()) { debug!(target: "engine::tree", num_hash=?removed.recovered_block().num_hash(), "Removed block walking back from the head"); - // finally, move the trie updates - self.persisted_trie_updates.insert( - removed.recovered_block().hash(), - (removed.recovered_block().number(), removed.trie), - ); } } } @@ -305,9 +299,6 @@ impl TreeState { } } - // remove trie updates that are below the finalized block - self.persisted_trie_updates.retain(|_, (block_num, _)| *block_num > finalized_num); - // The only block that should remain at the `finalized` number now, is the finalized // block, if it exists. // @@ -907,7 +898,8 @@ where // This is only done for in-memory blocks, because we should not have persisted any blocks // that are _above_ the current canonical head. while current_number > current_canonical_number { - if let Some(block) = self.executed_block_by_hash(current_hash)? { + if let Some(block) = self.state.tree_state.executed_block_by_hash(current_hash).cloned() + { current_hash = block.recovered_block().parent_hash(); current_number -= 1; new_chain.push(block); @@ -935,7 +927,7 @@ where // If the canonical chain is ahead of the new chain, // gather all blocks until new head number. while current_canonical_number > current_number { - if let Some(block) = self.executed_block_by_hash(old_hash)? { + if let Some(block) = self.canonical_block_by_hash(old_hash)? { old_chain.push(block.clone()); old_hash = block.recovered_block().parent_hash(); current_canonical_number -= 1; @@ -952,7 +944,7 @@ where // Walk both chains from specified hashes at same height until // a common ancestor (fork block) is reached. while old_hash != current_hash { - if let Some(block) = self.executed_block_by_hash(old_hash)? { + if let Some(block) = self.canonical_block_by_hash(old_hash)? { old_hash = block.recovered_block().parent_hash(); old_chain.push(block); } else { @@ -961,7 +953,8 @@ where return Ok(None); } - if let Some(block) = self.executed_block_by_hash(current_hash)? { + if let Some(block) = self.state.tree_state.executed_block_by_hash(current_hash).cloned() + { current_hash = block.recovered_block().parent_hash(); new_chain.push(block); } else { @@ -1528,7 +1521,7 @@ where /// Returns a batch of consecutive canonical blocks to persist in the range /// `(last_persisted_number .. canonical_head - threshold]` . The expected /// order is oldest -> newest. - fn get_canonical_blocks_to_persist(&self) -> Vec> { + fn get_canonical_blocks_to_persist(&self) -> Vec> { let mut blocks_to_persist = Vec::new(); let mut current_hash = self.state.tree_state.canonical_block_hash(); let last_persisted_number = self.persistence_state.last_persisted_block.number; @@ -1581,19 +1574,13 @@ where /// has in memory. /// /// For finalized blocks, this will return `None`. - fn executed_block_by_hash(&self, hash: B256) -> ProviderResult>> { + fn canonical_block_by_hash(&self, hash: B256) -> ProviderResult>> { trace!(target: "engine::tree", ?hash, "Fetching executed block by hash"); // check memory first - let block = self.state.tree_state.executed_block_by_hash(hash).cloned(); - - if block.is_some() { - return Ok(block) + if let Some(block) = self.state.tree_state.executed_block_by_hash(hash).cloned() { + return Ok(Some(block.block)) } - let Some((_, updates)) = self.state.tree_state.persisted_trie_updates.get(&hash) else { - return Ok(None) - }; - let (block, senders) = self .provider .sealed_block_with_senders(hash.into(), TransactionVariant::WithHash)? @@ -1607,7 +1594,6 @@ where Ok(Some(ExecutedBlock { recovered_block: Arc::new(RecoveredBlock::new_sealed(block, senders)), - trie: updates.clone(), execution_output: Arc::new(execution_output), hashed_state: Arc::new(hashed_state), })) @@ -2046,7 +2032,6 @@ where self.update_reorg_metrics(old.len()); self.reinsert_reorged_blocks(new.clone()); - self.reinsert_reorged_blocks(old.clone()); } // update the tracked in-memory state with the new chain @@ -2073,7 +2058,7 @@ where } /// This reinserts any blocks in the new chain that do not already exist in the tree - fn reinsert_reorged_blocks(&mut self, new_chain: Vec>) { + fn reinsert_reorged_blocks(&mut self, new_chain: Vec>) { for block in new_chain { if self .state @@ -2374,13 +2359,15 @@ where self.metrics.block_validation.record_state_root(&trie_output, root_elapsed.as_secs_f64()); debug!(target: "engine::tree", ?root_elapsed, block=?sealed_block.num_hash(), "Calculated state root"); - let executed: ExecutedBlock = ExecutedBlock { - recovered_block: Arc::new(RecoveredBlock::new_sealed( - sealed_block.as_ref().clone(), - block.senders().to_vec(), - )), - execution_output: Arc::new(ExecutionOutcome::from((output, block_number))), - hashed_state: Arc::new(hashed_state), + let executed: ExecutedBlockWithTrieUpdates = ExecutedBlockWithTrieUpdates { + block: ExecutedBlock { + recovered_block: Arc::new(RecoveredBlock::new_sealed( + sealed_block.as_ref().clone(), + block.senders().to_vec(), + )), + execution_output: Arc::new(ExecutionOutcome::from((output, block_number))), + hashed_state: Arc::new(hashed_state), + }, trie: Arc::new(trie_output), }; @@ -2886,7 +2873,7 @@ mod tests { >, to_tree_tx: Sender, Block>>, from_tree_rx: UnboundedReceiver, - blocks: Vec, + blocks: Vec, action_rx: Receiver, executor_provider: MockExecutorProvider, block_builder: TestBlockBuilder, @@ -2957,7 +2944,7 @@ mod tests { } } - fn with_blocks(mut self, blocks: Vec) -> Self { + fn with_blocks(mut self, blocks: Vec) -> Self { let mut blocks_by_hash = HashMap::default(); let mut blocks_by_number = BTreeMap::new(); let mut state_by_hash = HashMap::default(); @@ -2982,7 +2969,6 @@ mod tests { blocks_by_number, current_canonical_head: blocks.last().unwrap().recovered_block().num_hash(), parent_to_child, - persisted_trie_updates: HashMap::default(), }; let last_executed_block = blocks.last().unwrap().clone(); @@ -3702,20 +3688,24 @@ mod tests { let chain_b = test_block_builder.create_fork(&last_block, 10); for block in &chain_a { - test_harness.tree.state.tree_state.insert_executed(ExecutedBlock { - recovered_block: Arc::new(block.clone()), - execution_output: Arc::new(ExecutionOutcome::default()), - hashed_state: Arc::new(HashedPostState::default()), + test_harness.tree.state.tree_state.insert_executed(ExecutedBlockWithTrieUpdates { + block: ExecutedBlock { + recovered_block: Arc::new(block.clone()), + execution_output: Arc::new(ExecutionOutcome::default()), + hashed_state: Arc::new(HashedPostState::default()), + }, trie: Arc::new(TrieUpdates::default()), }); } test_harness.tree.state.tree_state.set_canonical_head(chain_a.last().unwrap().num_hash()); for block in &chain_b { - test_harness.tree.state.tree_state.insert_executed(ExecutedBlock { - recovered_block: Arc::new(block.clone()), - execution_output: Arc::new(ExecutionOutcome::default()), - hashed_state: Arc::new(HashedPostState::default()), + test_harness.tree.state.tree_state.insert_executed(ExecutedBlockWithTrieUpdates { + block: ExecutedBlock { + recovered_block: Arc::new(block.clone()), + execution_output: Arc::new(ExecutionOutcome::default()), + hashed_state: Arc::new(HashedPostState::default()), + }, trie: Arc::new(TrieUpdates::default()), }); } diff --git a/crates/ethereum/engine-primitives/src/payload.rs b/crates/ethereum/engine-primitives/src/payload.rs index beb99d027087..eabad4fb131b 100644 --- a/crates/ethereum/engine-primitives/src/payload.rs +++ b/crates/ethereum/engine-primitives/src/payload.rs @@ -9,7 +9,7 @@ use alloy_rpc_types_engine::{ ExecutionPayloadV1, PayloadAttributes, PayloadId, }; use core::convert::Infallible; -use reth_chain_state::ExecutedBlock; +use reth_chain_state::ExecutedBlockWithTrieUpdates; use reth_payload_primitives::{BuiltPayload, PayloadBuilderAttributes}; use reth_primitives::{EthPrimitives, SealedBlock}; use reth_rpc_types_compat::engine::payload::{ @@ -28,7 +28,7 @@ pub struct EthBuiltPayload { /// The built block pub(crate) block: Arc, /// Block execution data for the payload, if any. - pub(crate) executed_block: Option, + pub(crate) executed_block: Option, /// The fees of the block pub(crate) fees: U256, /// The blobs, proofs, and commitments in the block. If the block is pre-cancun, this will be @@ -48,7 +48,7 @@ impl EthBuiltPayload { id: PayloadId, block: Arc, fees: U256, - executed_block: Option, + executed_block: Option, requests: Option, ) -> Self { Self { id, block, executed_block, fees, sidecars: Vec::new(), requests } @@ -100,7 +100,7 @@ impl BuiltPayload for EthBuiltPayload { self.fees } - fn executed_block(&self) -> Option { + fn executed_block(&self) -> Option { self.executed_block.clone() } @@ -120,7 +120,7 @@ impl BuiltPayload for &EthBuiltPayload { (**self).fees() } - fn executed_block(&self) -> Option { + fn executed_block(&self) -> Option { self.executed_block.clone() } diff --git a/crates/ethereum/payload/src/lib.rs b/crates/ethereum/payload/src/lib.rs index b4f313aa4e68..dd28ca0ffaa4 100644 --- a/crates/ethereum/payload/src/lib.rs +++ b/crates/ethereum/payload/src/lib.rs @@ -19,7 +19,7 @@ use reth_basic_payload_builder::{ commit_withdrawals, is_better_payload, BuildArguments, BuildOutcome, PayloadBuilder, PayloadConfig, }; -use reth_chain_state::ExecutedBlock; +use reth_chain_state::{ExecutedBlock, ExecutedBlockWithTrieUpdates}; use reth_chainspec::{ChainSpec, ChainSpecProvider}; use reth_errors::RethError; use reth_evm::{ @@ -481,13 +481,15 @@ where debug!(target: "payload_builder", id=%attributes.id, sealed_block_header = ?sealed_block.sealed_header(), "sealed built block"); // create the executed block data - let executed = ExecutedBlock { - recovered_block: Arc::new(RecoveredBlock::new_sealed( - sealed_block.as_ref().clone(), - executed_senders, - )), - execution_output: Arc::new(execution_outcome), - hashed_state: Arc::new(hashed_state), + let executed = ExecutedBlockWithTrieUpdates { + block: ExecutedBlock { + recovered_block: Arc::new(RecoveredBlock::new_sealed( + sealed_block.as_ref().clone(), + executed_senders, + )), + execution_output: Arc::new(execution_outcome), + hashed_state: Arc::new(hashed_state), + }, trie: Arc::new(trie_output), }; diff --git a/crates/optimism/payload/src/builder.rs b/crates/optimism/payload/src/builder.rs index 18d4b552ec0e..686280631d1c 100644 --- a/crates/optimism/payload/src/builder.rs +++ b/crates/optimism/payload/src/builder.rs @@ -14,7 +14,7 @@ use alloy_rpc_types_engine::PayloadId; use op_alloy_consensus::{OpDepositReceipt, OpTxType}; use op_alloy_rpc_types_engine::OpPayloadAttributes; use reth_basic_payload_builder::*; -use reth_chain_state::ExecutedBlock; +use reth_chain_state::{ExecutedBlock, ExecutedBlockWithTrieUpdates}; use reth_chainspec::{ChainSpecProvider, EthereumHardforks}; use reth_evm::{ env::EvmEnv, system_calls::SystemCaller, ConfigureEvm, Evm, NextBlockEnvAttributes, @@ -435,13 +435,15 @@ where debug!(target: "payload_builder", id=%ctx.attributes().payload_id(), sealed_block_header = ?sealed_block.header(), "sealed built block"); // create the executed block data - let executed: ExecutedBlock = ExecutedBlock { - recovered_block: Arc::new(RecoveredBlock::new_sealed( - sealed_block.as_ref().clone(), - info.executed_senders, - )), - execution_output: Arc::new(execution_outcome), - hashed_state: Arc::new(hashed_state), + let executed: ExecutedBlockWithTrieUpdates = ExecutedBlockWithTrieUpdates { + block: ExecutedBlock { + recovered_block: Arc::new(RecoveredBlock::new_sealed( + sealed_block.as_ref().clone(), + info.executed_senders, + )), + execution_output: Arc::new(execution_outcome), + hashed_state: Arc::new(hashed_state), + }, trie: Arc::new(trie_output), }; diff --git a/crates/optimism/payload/src/payload.rs b/crates/optimism/payload/src/payload.rs index 10c4f2780cd5..26269b7e754b 100644 --- a/crates/optimism/payload/src/payload.rs +++ b/crates/optimism/payload/src/payload.rs @@ -11,7 +11,7 @@ use op_alloy_consensus::{encode_holocene_extra_data, EIP1559ParamError}; /// Re-export for use in downstream arguments. pub use op_alloy_rpc_types_engine::OpPayloadAttributes; use op_alloy_rpc_types_engine::{OpExecutionPayloadEnvelopeV3, OpExecutionPayloadEnvelopeV4}; -use reth_chain_state::ExecutedBlock; +use reth_chain_state::ExecutedBlockWithTrieUpdates; use reth_chainspec::EthereumHardforks; use reth_optimism_chainspec::OpChainSpec; use reth_optimism_primitives::{OpBlock, OpPrimitives, OpTransactionSigned}; @@ -137,7 +137,7 @@ pub struct OpBuiltPayload { /// The built block pub(crate) block: Arc>, /// Block execution data for the payload, if any. - pub(crate) executed_block: Option>, + pub(crate) executed_block: Option>, /// The fees of the block pub(crate) fees: U256, /// The blobs, proofs, and commitments in the block. If the block is pre-cancun, this will be @@ -159,7 +159,7 @@ impl OpBuiltPayload { fees: U256, chain_spec: Arc, attributes: OpPayloadBuilderAttributes, - executed_block: Option>, + executed_block: Option>, ) -> Self { Self { id, block, executed_block, fees, sidecars: Vec::new(), chain_spec, attributes } } @@ -196,7 +196,7 @@ impl BuiltPayload for OpBuiltPayload { self.fees } - fn executed_block(&self) -> Option> { + fn executed_block(&self) -> Option> { self.executed_block.clone() } @@ -216,7 +216,7 @@ impl BuiltPayload for &OpBuiltPayload { (**self).fees() } - fn executed_block(&self) -> Option> { + fn executed_block(&self) -> Option> { self.executed_block.clone() } diff --git a/crates/payload/builder/src/test_utils.rs b/crates/payload/builder/src/test_utils.rs index 9cd680ce6521..f38c8e0038c9 100644 --- a/crates/payload/builder/src/test_utils.rs +++ b/crates/payload/builder/src/test_utils.rs @@ -6,7 +6,7 @@ use crate::{ }; use alloy_primitives::U256; -use reth_chain_state::{CanonStateNotification, ExecutedBlock}; +use reth_chain_state::{CanonStateNotification, ExecutedBlockWithTrieUpdates}; use reth_payload_builder_primitives::PayloadBuilderError; use reth_payload_primitives::{PayloadKind, PayloadTypes}; use reth_primitives::Block; @@ -90,7 +90,7 @@ impl PayloadJob for TestPayloadJob { self.attr.payload_id(), Arc::new(Block::default().seal_slow()), U256::ZERO, - Some(ExecutedBlock::default()), + Some(ExecutedBlockWithTrieUpdates::default()), Some(Default::default()), )) } diff --git a/crates/payload/primitives/src/traits.rs b/crates/payload/primitives/src/traits.rs index 22e068f22972..18100c796233 100644 --- a/crates/payload/primitives/src/traits.rs +++ b/crates/payload/primitives/src/traits.rs @@ -6,7 +6,7 @@ use alloy_eips::{ use alloy_primitives::{Address, B256, U256}; use alloy_rpc_types_engine::{PayloadAttributes as EthPayloadAttributes, PayloadId}; use core::fmt; -use reth_chain_state::ExecutedBlock; +use reth_chain_state::ExecutedBlockWithTrieUpdates; use reth_primitives::{NodePrimitives, SealedBlock}; /// Represents a built payload type that contains a built `SealedBlock` and can be converted into @@ -22,7 +22,7 @@ pub trait BuiltPayload: Send + Sync + fmt::Debug { fn fees(&self) -> U256; /// Returns the entire execution data for the built block, if available. - fn executed_block(&self) -> Option> { + fn executed_block(&self) -> Option> { None } diff --git a/crates/storage/provider/src/providers/blockchain_provider.rs b/crates/storage/provider/src/providers/blockchain_provider.rs index d94f9cdd8f93..6ef8ab5a9ae2 100644 --- a/crates/storage/provider/src/providers/blockchain_provider.rs +++ b/crates/storage/provider/src/providers/blockchain_provider.rs @@ -787,7 +787,7 @@ mod tests { use rand::Rng; use reth_chain_state::{ test_utils::TestBlockBuilder, CanonStateNotification, CanonStateSubscriptions, - CanonicalInMemoryState, ExecutedBlock, NewCanonicalChain, + CanonicalInMemoryState, ExecutedBlock, ExecutedBlockWithTrieUpdates, NewCanonicalChain, }; use reth_chainspec::{ ChainSpec, ChainSpecBuilder, ChainSpecProvider, EthereumHardfork, MAINNET, @@ -930,7 +930,7 @@ mod tests { let execution_outcome = ExecutionOutcome { receipts: block_receipts.into(), ..Default::default() }; - ExecutedBlock::new( + ExecutedBlockWithTrieUpdates::new( Arc::new(RecoveredBlock::new_sealed(block.clone(), senders)), execution_outcome.into(), Default::default(), @@ -1059,7 +1059,7 @@ mod tests { let in_memory_block_senders = first_in_mem_block.senders().expect("failed to recover senders"); let chain = NewCanonicalChain::Commit { - new: vec![ExecutedBlock::new( + new: vec![ExecutedBlockWithTrieUpdates::new( Arc::new(RecoveredBlock::new_sealed( first_in_mem_block.clone(), in_memory_block_senders, @@ -1095,13 +1095,15 @@ mod tests { assert_eq!(provider.find_block_by_hash(first_db_block.hash(), BlockSource::Pending)?, None); // Insert the last block into the pending state - provider.canonical_in_memory_state.set_pending_block(ExecutedBlock { - recovered_block: Arc::new(RecoveredBlock::new_sealed( - last_in_mem_block.clone(), - Default::default(), - )), - execution_output: Default::default(), - hashed_state: Default::default(), + provider.canonical_in_memory_state.set_pending_block(ExecutedBlockWithTrieUpdates { + block: ExecutedBlock { + recovered_block: Arc::new(RecoveredBlock::new_sealed( + last_in_mem_block.clone(), + Default::default(), + )), + execution_output: Default::default(), + hashed_state: Default::default(), + }, trie: Default::default(), }); @@ -1153,7 +1155,7 @@ mod tests { let in_memory_block_senders = first_in_mem_block.senders().expect("failed to recover senders"); let chain = NewCanonicalChain::Commit { - new: vec![ExecutedBlock::new( + new: vec![ExecutedBlockWithTrieUpdates::new( Arc::new(RecoveredBlock::new_sealed( first_in_mem_block.clone(), in_memory_block_senders, @@ -1207,13 +1209,15 @@ mod tests { ); // Set the block as pending - provider.canonical_in_memory_state.set_pending_block(ExecutedBlock { - recovered_block: Arc::new(RecoveredBlock::new_sealed( - block.clone(), - block.senders().unwrap(), - )), - execution_output: Default::default(), - hashed_state: Default::default(), + provider.canonical_in_memory_state.set_pending_block(ExecutedBlockWithTrieUpdates { + block: ExecutedBlock { + recovered_block: Arc::new(RecoveredBlock::new_sealed( + block.clone(), + block.senders().unwrap(), + )), + execution_output: Default::default(), + hashed_state: Default::default(), + }, trie: Default::default(), }); @@ -1290,7 +1294,7 @@ mod tests { let in_memory_block_senders = first_in_mem_block.senders().expect("failed to recover senders"); let chain = NewCanonicalChain::Commit { - new: vec![ExecutedBlock::new( + new: vec![ExecutedBlockWithTrieUpdates::new( Arc::new(RecoveredBlock::new_sealed( first_in_mem_block.clone(), in_memory_block_senders, @@ -1855,7 +1859,7 @@ mod tests { .first() .map(|block| { let senders = block.senders().expect("failed to recover senders"); - ExecutedBlock::new( + ExecutedBlockWithTrieUpdates::new( Arc::new(RecoveredBlock::new_sealed(block.clone(), senders)), Arc::new(ExecutionOutcome { bundle: BundleState::new( @@ -1990,15 +1994,19 @@ mod tests { // adding a pending block to state can test pending() and pending_state_by_hash() function let pending_block = database_blocks[database_blocks.len() - 1].clone(); - only_database_provider.canonical_in_memory_state.set_pending_block(ExecutedBlock { - recovered_block: Arc::new(RecoveredBlock::new_sealed( - pending_block.clone(), - Default::default(), - )), - execution_output: Default::default(), - hashed_state: Default::default(), - trie: Default::default(), - }); + only_database_provider.canonical_in_memory_state.set_pending_block( + ExecutedBlockWithTrieUpdates { + block: ExecutedBlock { + recovered_block: Arc::new(RecoveredBlock::new_sealed( + pending_block.clone(), + Default::default(), + )), + execution_output: Default::default(), + hashed_state: Default::default(), + }, + trie: Default::default(), + }, + ); assert_eq!( pending_block.hash(), @@ -2113,13 +2121,15 @@ mod tests { // Set the pending block in memory let pending_block = in_memory_blocks.last().unwrap(); - provider.canonical_in_memory_state.set_pending_block(ExecutedBlock { - recovered_block: Arc::new(RecoveredBlock::new_sealed( - pending_block.clone(), - Default::default(), - )), - execution_output: Default::default(), - hashed_state: Default::default(), + provider.canonical_in_memory_state.set_pending_block(ExecutedBlockWithTrieUpdates { + block: ExecutedBlock { + recovered_block: Arc::new(RecoveredBlock::new_sealed( + pending_block.clone(), + Default::default(), + )), + execution_output: Default::default(), + hashed_state: Default::default(), + }, trie: Default::default(), }); diff --git a/crates/storage/provider/src/providers/consistent.rs b/crates/storage/provider/src/providers/consistent.rs index c2b273885b98..c752dd52113c 100644 --- a/crates/storage/provider/src/providers/consistent.rs +++ b/crates/storage/provider/src/providers/consistent.rs @@ -1479,7 +1479,7 @@ mod tests { use alloy_primitives::B256; use itertools::Itertools; use rand::Rng; - use reth_chain_state::{ExecutedBlock, NewCanonicalChain}; + use reth_chain_state::{ExecutedBlock, ExecutedBlockWithTrieUpdates, NewCanonicalChain}; use reth_db::models::AccountBeforeTx; use reth_execution_types::ExecutionOutcome; use reth_primitives::{RecoveredBlock, SealedBlock}; @@ -1581,7 +1581,7 @@ mod tests { let in_memory_block_senders = first_in_mem_block.senders().expect("failed to recover senders"); let chain = NewCanonicalChain::Commit { - new: vec![ExecutedBlock::new( + new: vec![ExecutedBlockWithTrieUpdates::new( Arc::new(RecoveredBlock::new_sealed( first_in_mem_block.clone(), in_memory_block_senders, @@ -1623,13 +1623,15 @@ mod tests { ); // Insert the last block into the pending state - provider.canonical_in_memory_state.set_pending_block(ExecutedBlock { - recovered_block: Arc::new(RecoveredBlock::new_sealed( - last_in_mem_block.clone(), - Default::default(), - )), - execution_output: Default::default(), - hashed_state: Default::default(), + provider.canonical_in_memory_state.set_pending_block(ExecutedBlockWithTrieUpdates { + block: ExecutedBlock { + recovered_block: Arc::new(RecoveredBlock::new_sealed( + last_in_mem_block.clone(), + Default::default(), + )), + execution_output: Default::default(), + hashed_state: Default::default(), + }, trie: Default::default(), }); @@ -1689,7 +1691,7 @@ mod tests { let in_memory_block_senders = first_in_mem_block.senders().expect("failed to recover senders"); let chain = NewCanonicalChain::Commit { - new: vec![ExecutedBlock::new( + new: vec![ExecutedBlockWithTrieUpdates::new( Arc::new(RecoveredBlock::new_sealed( first_in_mem_block.clone(), in_memory_block_senders, @@ -1795,7 +1797,7 @@ mod tests { .first() .map(|block| { let senders = block.senders().expect("failed to recover senders"); - ExecutedBlock::new( + ExecutedBlockWithTrieUpdates::new( Arc::new(RecoveredBlock::new_sealed(block.clone(), senders)), Arc::new(ExecutionOutcome { bundle: BundleState::new( diff --git a/crates/storage/provider/src/writer/mod.rs b/crates/storage/provider/src/writer/mod.rs index d1cc61600db8..d23428236cc6 100644 --- a/crates/storage/provider/src/writer/mod.rs +++ b/crates/storage/provider/src/writer/mod.rs @@ -4,7 +4,7 @@ use crate::{ StorageLocation, TrieWriter, }; use alloy_consensus::BlockHeader; -use reth_chain_state::ExecutedBlock; +use reth_chain_state::{ExecutedBlock, ExecutedBlockWithTrieUpdates}; use reth_db::transaction::{DbTx, DbTxMut}; use reth_errors::ProviderResult; use reth_primitives::{NodePrimitives, StaticFileSegment}; @@ -132,7 +132,7 @@ where + StaticFileProviderFactory, { /// Writes executed blocks and receipts to storage. - pub fn save_blocks(&self, blocks: Vec>) -> ProviderResult<()> + pub fn save_blocks(&self, blocks: Vec>) -> ProviderResult<()> where N: NodePrimitives, ProviderDB: BlockWriter + StateWriter, @@ -160,7 +160,11 @@ where // * trie updates (cannot naively extend, need helper) // * indices (already done basically) // Insert the blocks - for ExecutedBlock { recovered_block, execution_output, hashed_state, trie } in blocks { + for ExecutedBlockWithTrieUpdates { + block: ExecutedBlock { recovered_block, execution_output, hashed_state }, + trie, + } in blocks + { self.database() .insert_block(Arc::unwrap_or_clone(recovered_block), StorageLocation::Both)?;