Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
38 commits
Select commit Hold shift + click to select a range
eb7d8c6
perf(engine): return sorted data from compute_trie_input
yongkangc Oct 28, 2025
43ceb89
refactor(engine): optimize trie input handling in compute_trie_input
yongkangc Oct 28, 2025
3fd4996
update call site
yongkangc Oct 28, 2025
9b83edd
fix fmt
yongkangc Oct 28, 2025
7c85ed3
refactor(chain-state): update to sorted trie structures
yongkangc Oct 29, 2025
4249c66
added sorted for test
yongkangc Oct 29, 2025
08c9a1f
refactor(engine): optimize payload validation with sorted trie input
yongkangc Oct 29, 2025
23a9338
hashed_state convert to sorted
yongkangc Oct 29, 2025
861579e
refactor(trie): streamline TrieInput creation from sorted blocks
yongkangc Oct 29, 2025
87642e4
refactor(engine): ExecutedBlock to Arc
yongkangc Oct 30, 2025
2fbcbb4
refactor(multiproof): remove MultiProofConfig struct and related methods
yongkangc Oct 30, 2025
9e1fa4a
refactor(engine): simplify trie input handling in payload validation …
yongkangc Oct 30, 2025
0e2c3e7
refactor(trie): enhance TrieInput with efficient prepend and Arc usage
yongkangc Oct 30, 2025
5f51ed3
refactor: reference arc directly
yongkangc Oct 30, 2025
b346694
feat(trie): add extend_from_sorted for extending and converting hashe…
yongkangc Oct 30, 2025
56ee954
extend_from_sorted
yongkangc Oct 30, 2025
6fd8fd2
added comment
yongkangc Oct 30, 2025
acfb780
added comment
yongkangc Oct 30, 2025
d3e56a2
added extend_from_sorted
yongkangc Oct 30, 2025
6c5cd7c
clear data
yongkangc Oct 30, 2025
76babb4
remove into
yongkangc Oct 30, 2025
66c73e5
Merge branch 'main' into yk/compute_trie2
yongkangc Oct 30, 2025
665ae71
remove drain into sorted
yongkangc Oct 30, 2025
4ca196a
rm clear
yongkangc Oct 30, 2025
2667271
rm clear
yongkangc Oct 30, 2025
dae97c6
change function signature to rm alloc
yongkangc Oct 30, 2025
d251dee
comments
yongkangc Oct 30, 2025
28b84bf
added back clear
yongkangc Oct 30, 2025
990cb15
destructure instead of arc
yongkangc Oct 30, 2025
5f90e0f
comment
yongkangc Oct 30, 2025
cdff945
rm alloc, use trie destructuring
yongkangc Oct 30, 2025
b15a062
Refactor payload validator to improve block handling and reduce alloc…
yongkangc Oct 30, 2025
ba70ee7
fast path
yongkangc Oct 30, 2025
37bdb2d
fmt
yongkangc Oct 30, 2025
0e0e2d7
Enhance TrieInputSorted and HashedPostStateSorted with new methods fo…
yongkangc Oct 30, 2025
062aa27
Revert "Enhance TrieInputSorted and HashedPostStateSorted with new me…
yongkangc Oct 30, 2025
ceb77c0
added tests
yongkangc Oct 30, 2025
f1454d7
fmt
yongkangc Oct 30, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
18 changes: 9 additions & 9 deletions crates/chain-state/src/in_memory.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ use reth_primitives_traits::{
SignedTransaction,
};
use reth_storage_api::StateProviderBox;
use reth_trie::{updates::TrieUpdates, HashedPostState};
use reth_trie::{updates::TrieUpdatesSorted, HashedPostStateSorted};
use std::{collections::BTreeMap, sync::Arc, time::Instant};
use tokio::sync::{broadcast, watch};

Expand Down Expand Up @@ -725,10 +725,10 @@ pub struct ExecutedBlock<N: NodePrimitives = EthPrimitives> {
pub recovered_block: Arc<RecoveredBlock<N::Block>>,
/// Block's execution outcome.
pub execution_output: Arc<ExecutionOutcome<N::Receipt>>,
/// Block's hashed state.
pub hashed_state: Arc<HashedPostState>,
/// Trie updates that result from calculating the state root for the block.
pub trie_updates: Arc<TrieUpdates>,
/// Block's sorted hashed state.
pub hashed_state: Arc<HashedPostStateSorted>,
/// Sorted trie updates that result from calculating the state root for the block.
pub trie_updates: Arc<TrieUpdatesSorted>,
}

impl<N: NodePrimitives> Default for ExecutedBlock<N> {
Expand Down Expand Up @@ -763,13 +763,13 @@ impl<N: NodePrimitives> ExecutedBlock<N> {

/// Returns a reference to the hashed state result of the execution outcome
#[inline]
pub fn hashed_state(&self) -> &HashedPostState {
pub fn hashed_state(&self) -> &HashedPostStateSorted {
&self.hashed_state
}

/// Returns a reference to the trie updates resulting from the execution outcome
#[inline]
pub fn trie_updates(&self) -> &TrieUpdates {
pub fn trie_updates(&self) -> &TrieUpdatesSorted {
&self.trie_updates
}

Expand Down Expand Up @@ -875,8 +875,8 @@ mod tests {
StateProofProvider, StateProvider, StateRootProvider, StorageRootProvider,
};
use reth_trie::{
AccountProof, HashedStorage, MultiProof, MultiProofTargets, StorageMultiProof,
StorageProof, TrieInput,
updates::TrieUpdates, AccountProof, HashedPostState, HashedStorage, MultiProof,
MultiProofTargets, StorageMultiProof, StorageProof, TrieInput,
};

fn create_mock_state(
Expand Down
2 changes: 1 addition & 1 deletion crates/chain-state/src/memory_overlay.rs
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ impl<'a, N: NodePrimitives> MemoryOverlayStateProviderRef<'a, N> {
/// Return lazy-loaded trie state aggregated from in-memory blocks.
fn trie_input(&self) -> &TrieInput {
self.trie_input.get_or_init(|| {
TrieInput::from_blocks(
TrieInput::from_blocks_sorted(
self.in_memory
.iter()
.rev()
Expand Down
6 changes: 3 additions & 3 deletions crates/chain-state/src/test_utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ use reth_primitives_traits::{
SignedTransaction,
};
use reth_storage_api::NodePrimitivesProvider;
use reth_trie::{root::state_root_unhashed, updates::TrieUpdates, HashedPostState};
use reth_trie::{root::state_root_unhashed, updates::TrieUpdatesSorted, HashedPostStateSorted};
use revm_database::BundleState;
use revm_state::AccountInfo;
use std::{
Expand Down Expand Up @@ -216,8 +216,8 @@ impl<N: NodePrimitives> TestBlockBuilder<N> {
block_number,
vec![Requests::default()],
)),
hashed_state: Arc::new(HashedPostState::default()),
trie_updates: Arc::new(TrieUpdates::default()),
hashed_state: Arc::new(HashedPostStateSorted::default()),
trie_updates: Arc::new(TrieUpdatesSorted::default()),
}
}

Expand Down
4 changes: 2 additions & 2 deletions crates/engine/tree/src/tree/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1759,8 +1759,8 @@ where
Ok(Some(ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(block, senders)),
execution_output: Arc::new(execution_output),
hashed_state: Arc::new(hashed_state),
trie_updates: Arc::new(trie_updates.into()),
hashed_state: Arc::new(hashed_state.into_sorted()),
trie_updates: Arc::new(trie_updates),
}))
}

Expand Down
34 changes: 2 additions & 32 deletions crates/engine/tree/src/tree/payload_processor/multiproof.rs
Original file line number Diff line number Diff line change
Expand Up @@ -13,9 +13,8 @@ use metrics::{Gauge, Histogram};
use reth_metrics::Metrics;
use reth_revm::state::EvmState;
use reth_trie::{
added_removed_keys::MultiAddedRemovedKeys, prefix_set::TriePrefixSetsMut,
updates::TrieUpdatesSorted, DecodedMultiProof, HashedPostState, HashedPostStateSorted,
HashedStorage, MultiProofTargets, TrieInput,
added_removed_keys::MultiAddedRemovedKeys, DecodedMultiProof, HashedPostState, HashedStorage,
MultiProofTargets,
};
use reth_trie_parallel::{
proof::ParallelProof,
Expand Down Expand Up @@ -56,35 +55,6 @@ impl SparseTrieUpdate {
}
}

/// Common configuration for multi proof tasks
#[derive(Debug, Clone, Default)]
pub(crate) struct MultiProofConfig {
/// The sorted collection of cached in-memory intermediate trie nodes that
/// can be reused for computation.
pub nodes_sorted: Arc<TrieUpdatesSorted>,
/// The sorted in-memory overlay hashed state.
pub state_sorted: Arc<HashedPostStateSorted>,
/// The collection of prefix sets for the computation. Since the prefix sets _always_
/// invalidate the in-memory nodes, not all keys from `state_sorted` might be present here,
/// if we have cached nodes for them.
pub prefix_sets: Arc<TriePrefixSetsMut>,
}

impl MultiProofConfig {
/// Creates a new state root config from the trie input.
///
/// This returns a cleared [`TrieInput`] so that we can reuse any allocated space in the
/// [`TrieInput`].
pub(crate) fn from_input(mut input: TrieInput) -> (TrieInput, Self) {
let config = Self {
nodes_sorted: Arc::new(input.nodes.drain_into_sorted()),
state_sorted: Arc::new(input.state.drain_into_sorted()),
prefix_sets: Arc::new(input.prefix_sets.clone()),
};
(input.cleared(), config)
}
}

/// Messages used internally by the multi proof task.
#[derive(Debug)]
pub(super) enum MultiProofMessage {
Expand Down
80 changes: 39 additions & 41 deletions crates/engine/tree/src/tree/payload_validator.rs
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@ use crate::tree::{
error::{InsertBlockError, InsertBlockErrorKind, InsertPayloadError},
executor::WorkloadExecutor,
instrumented_state::InstrumentedStateProvider,
payload_processor::{multiproof::MultiProofConfig, PayloadProcessor},
payload_processor::PayloadProcessor,
precompile_cache::{CachedPrecompile, CachedPrecompileMetrics, PrecompileCacheMap},
sparse_trie::StateRootComputeOutcome,
EngineApiMetrics, EngineApiTreeState, ExecutionEnv, PayloadHandle, StateProviderBuilder,
Expand Down Expand Up @@ -38,7 +38,7 @@ use reth_provider::{
StateRootProvider, TrieReader,
};
use reth_revm::db::State;
use reth_trie::{updates::TrieUpdates, HashedPostState, TrieInput};
use reth_trie::{updates::TrieUpdates, HashedPostState, TrieInputSorted};
use reth_trie_parallel::root::{ParallelStateRoot, ParallelStateRootError};
use std::{collections::HashMap, sync::Arc, time::Instant};
use tracing::{debug, debug_span, error, info, instrument, trace, warn};
Expand Down Expand Up @@ -121,8 +121,6 @@ where
metrics: EngineApiMetrics,
/// Validator for the payload.
validator: V,
/// A cleared trie input, kept around to be reused so allocations can be minimized.
trie_input: Option<TrieInput>,
}

impl<N, P, Evm, V> BasicEngineValidator<P, Evm, V>
Expand Down Expand Up @@ -166,7 +164,6 @@ where
invalid_block_hook,
metrics: EngineApiMetrics::default(),
validator,
trie_input: Default::default(),
}
}

Expand Down Expand Up @@ -531,8 +528,8 @@ where
Ok(ExecutedBlock {
recovered_block: Arc::new(block),
execution_output: Arc::new(ExecutionOutcome::from((output, block_num_hash.number))),
hashed_state: Arc::new(hashed_state),
trie_updates: Arc::new(trie_output),
hashed_state: Arc::new(hashed_state.into_sorted()),
trie_updates: Arc::new(trie_output.into_sorted()),
})
}

Expand Down Expand Up @@ -642,26 +639,22 @@ where
hashed_state: &HashedPostState,
state: &EngineApiTreeState<N>,
) -> Result<(B256, TrieUpdates), ParallelStateRootError> {
let (mut input, block_hash) = self.compute_trie_input(parent_hash, state, None)?;
let (mut input, block_hash) = self.compute_trie_input(parent_hash, state)?;

// Extend with block we are validating root for.
input.append_ref(hashed_state);

// Convert the TrieInput into a MultProofConfig, since everything uses the sorted
// forms of the state/trie fields.
let (_, multiproof_config) = MultiProofConfig::from_input(input);
let TrieInputSorted { nodes, state, prefix_sets: prefix_sets_mut } = input;

let factory = OverlayStateProviderFactory::new(self.provider.clone())
.with_block_hash(Some(block_hash))
.with_trie_overlay(Some(multiproof_config.nodes_sorted))
.with_hashed_state_overlay(Some(multiproof_config.state_sorted));
.with_trie_overlay(Some(nodes))
.with_hashed_state_overlay(Some(state));

// The `hashed_state` argument is already taken into account as part of the overlay, but we
// need to use the prefix sets which were generated from it to indicate to the
// ParallelStateRoot which parts of the trie need to be recomputed.
let prefix_sets = Arc::into_inner(multiproof_config.prefix_sets)
.expect("MultiProofConfig was never cloned")
.freeze();
let prefix_sets = prefix_sets_mut.freeze();

ParallelStateRoot::new(factory, prefix_sets).incremental_root_with_updates()
}
Expand Down Expand Up @@ -760,31 +753,23 @@ where
> {
match strategy {
StateRootStrategy::StateRootTask => {
// get allocated trie input if it exists
let allocated_trie_input = self.trie_input.take();

// Compute trie input
let trie_input_start = Instant::now();
let (trie_input, block_hash) =
self.compute_trie_input(parent_hash, state, allocated_trie_input)?;
let (trie_input, block_hash) = self.compute_trie_input(parent_hash, state)?;

self.metrics
.block_validation
.trie_input_duration
.record(trie_input_start.elapsed().as_secs_f64());

// Convert the TrieInput into a MultProofConfig, since everything uses the sorted
// forms of the state/trie fields.
let (trie_input, multiproof_config) = MultiProofConfig::from_input(trie_input);
self.trie_input.replace(trie_input);
// Create OverlayStateProviderFactory with sorted trie data for multiproofs
let TrieInputSorted { nodes, state, .. } = trie_input;

// Create OverlayStateProviderFactory with the multiproof config, for use with
// multiproofs.
let multiproof_provider_factory =
OverlayStateProviderFactory::new(self.provider.clone())
.with_block_hash(Some(block_hash))
.with_trie_overlay(Some(multiproof_config.nodes_sorted))
.with_hashed_state_overlay(Some(multiproof_config.state_sorted));
.with_trie_overlay(Some(nodes))
.with_hashed_state_overlay(Some(state));

// Use state root task only if prefix sets are empty, otherwise proof generation is
// too expensive because it requires walking all paths in every proof.
Expand Down Expand Up @@ -912,16 +897,19 @@ where
/// Computes the trie input at the provided parent hash, as well as the block number of the
/// highest persisted ancestor.
///
/// The goal of this function is to take in-memory blocks and generate a [`TrieInput`] that
/// serves as an overlay to the database blocks.
/// The goal of this function is to take in-memory blocks and generate a [`TrieInputSorted`]
/// that serves as an overlay to the database blocks.
///
/// It works as follows:
/// 1. Collect in-memory blocks that are descendants of the provided parent hash using
/// [`crate::tree::TreeState::blocks_by_hash`]. This returns the highest persisted ancestor
/// hash (`block_hash`) and the list of in-memory descendant blocks.
/// 2. Extend the `TrieInput` with the contents of these in-memory blocks (from oldest to
/// 2. Extend the `TrieInputSorted` with the contents of these in-memory blocks (from oldest to
/// newest) to build the overlay state and trie updates that sit on top of the database view
/// anchored at `block_hash`.
///
/// The in-memory blocks are already sorted, so we can directly append them without expensive
/// sorting operations.
#[instrument(
level = "debug",
target = "engine::tree::payload_validator",
Expand All @@ -932,11 +920,7 @@ where
&self,
parent_hash: B256,
state: &EngineApiTreeState<N>,
allocated_trie_input: Option<TrieInput>,
) -> ProviderResult<(TrieInput, B256)> {
// get allocated trie input or use a default trie input
let mut input = allocated_trie_input.unwrap_or_default();

) -> ProviderResult<(TrieInputSorted, B256)> {
let (block_hash, blocks) =
state.tree_state.blocks_by_hash(parent_hash).unwrap_or_else(|| (parent_hash, vec![]));

Expand All @@ -946,10 +930,24 @@ where
debug!(target: "engine::tree::payload_validator", historical = ?block_hash, blocks = blocks.len(), "Parent found in memory");
}

// Extend with contents of parent in-memory blocks.
input.extend_with_blocks(
blocks.iter().rev().map(|block| (block.hashed_state(), block.trie_updates())),
);
// Extend with contents of parent in-memory blocks directly in sorted form.
let mut input = TrieInputSorted::default();
let mut blocks_iter = blocks.iter().rev();

if let Some(first) = blocks_iter.next() {
input.state = Arc::clone(&first.hashed_state);
input.nodes = Arc::clone(&first.trie_updates);

// Only clone and mutate if there are multiple in-memory blocks.
if blocks.len() > 1 {
let state_mut = Arc::make_mut(&mut input.state);
let nodes_mut = Arc::make_mut(&mut input.nodes);
for block in blocks_iter {
state_mut.extend_ref(block.hashed_state());
nodes_mut.extend_ref(block.trie_updates());
}
}
}

Ok((input, block_hash))
}
Expand Down
8 changes: 4 additions & 4 deletions crates/engine/tree/src/tree/tests.rs
Original file line number Diff line number Diff line change
Expand Up @@ -829,8 +829,8 @@ fn test_tree_state_on_new_head_deep_fork() {
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()),
trie_updates: Arc::new(TrieUpdates::default()),
hashed_state: Arc::new(HashedPostState::default().into_sorted()),
trie_updates: Arc::new(TrieUpdates::default().into_sorted()),
});
}
test_harness.tree.state.tree_state.set_canonical_head(chain_a.last().unwrap().num_hash());
Expand All @@ -839,8 +839,8 @@ fn test_tree_state_on_new_head_deep_fork() {
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()),
trie_updates: Arc::new(TrieUpdates::default()),
hashed_state: Arc::new(HashedPostState::default().into_sorted()),
trie_updates: Arc::new(TrieUpdates::default().into_sorted()),
});
}

Expand Down
2 changes: 1 addition & 1 deletion crates/optimism/flashblocks/src/worker.rs
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ where
ExecutedBlock {
recovered_block: block.into(),
execution_output: Arc::new(execution_outcome),
hashed_state: Arc::new(hashed_state),
hashed_state: Arc::new(hashed_state.into_sorted()),
trie_updates: Arc::default(),
},
);
Expand Down
4 changes: 2 additions & 2 deletions crates/optimism/payload/src/builder.rs
Original file line number Diff line number Diff line change
Expand Up @@ -389,8 +389,8 @@ impl<Txs> OpBuilder<'_, Txs> {
let executed: ExecutedBlock<N> = ExecutedBlock {
recovered_block: Arc::new(block),
execution_output: Arc::new(execution_outcome),
hashed_state: Arc::new(hashed_state),
trie_updates: Arc::new(trie_updates),
hashed_state: Arc::new(hashed_state.into_sorted()),
trie_updates: Arc::new(trie_updates.into_sorted()),
};

let no_tx_pool = ctx.attributes().no_tx_pool();
Expand Down
11 changes: 6 additions & 5 deletions crates/ress/provider/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -156,11 +156,12 @@ where
// NOTE: there might be a race condition where target ancestor hash gets evicted from the
// database.
let witness_state_provider = self.provider.state_by_block_hash(ancestor_hash)?;
let mut trie_input = TrieInput::default();
for block in executed_ancestors.into_iter().rev() {
let trie_updates = block.trie_updates.as_ref();
trie_input.append_cached_ref(trie_updates, &block.hashed_state);
}
let trie_input = TrieInput::from_blocks_sorted(
executed_ancestors
.iter()
.rev()
.map(|block| (block.hashed_state.as_ref(), block.trie_updates.as_ref())),
);
let mut hashed_state = db.into_state();
hashed_state.extend(record.hashed_state);

Expand Down
4 changes: 2 additions & 2 deletions crates/rpc/rpc-eth-api/src/helpers/pending_block.rs
Original file line number Diff line number Diff line change
Expand Up @@ -372,8 +372,8 @@ pub trait LoadPendingBlock:
Ok(ExecutedBlock {
recovered_block: block.into(),
execution_output: Arc::new(execution_outcome),
hashed_state: Arc::new(hashed_state),
trie_updates: Arc::new(trie_updates),
hashed_state: Arc::new(hashed_state.into_sorted()),
trie_updates: Arc::new(trie_updates.into_sorted()),
})
}
}
Expand Down
Loading