feat(trie): Merge trie changesets changes into main (#19068)

Co-authored-by: Alexey Shekhirin <5773434+shekhirin@users.noreply.github.com>
Co-authored-by: Roman Hodulák <roman.hodulak@polyglot-software.com>
Co-authored-by: Dan Cline <6798349+Rjected@users.noreply.github.com>
This commit is contained in:
Brian Picciano
2025-10-16 11:52:35 +02:00
committed by GitHub
parent 3de82cf2bd
commit be94d0d393
82 changed files with 3778 additions and 1119 deletions

View File

@@ -2,7 +2,7 @@
use criterion::{black_box, criterion_group, criterion_main, Criterion};
use reth_chain_state::{
test_utils::TestBlockBuilder, ExecutedBlockWithTrieUpdates, MemoryOverlayStateProviderRef,
test_utils::TestBlockBuilder, ExecutedBlock, MemoryOverlayStateProviderRef,
};
use reth_ethereum_primitives::EthPrimitives;
use reth_storage_api::{noop::NoopProvider, BlockHashReader};
@@ -84,10 +84,7 @@ fn bench_canonical_hashes_range(c: &mut Criterion) {
fn setup_provider_with_blocks(
num_blocks: usize,
) -> (
MemoryOverlayStateProviderRef<'static, EthPrimitives>,
Vec<ExecutedBlockWithTrieUpdates<EthPrimitives>>,
) {
) -> (MemoryOverlayStateProviderRef<'static, EthPrimitives>, Vec<ExecutedBlock<EthPrimitives>>) {
let mut builder = TestBlockBuilder::<EthPrimitives>::default();
let blocks: Vec<_> = builder.get_executed_blocks(1000..1000 + num_blocks as u64).collect();

View File

@@ -242,7 +242,7 @@ impl<N: NodePrimitives> CanonicalInMemoryState<N> {
/// 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: ExecutedBlockWithTrieUpdates<N>) {
pub fn set_pending_block(&self, pending: ExecutedBlock<N>) {
// 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);
@@ -258,7 +258,7 @@ impl<N: NodePrimitives> CanonicalInMemoryState<N> {
/// them to their parent blocks.
fn update_blocks<I, R>(&self, new_blocks: I, reorged: R)
where
I: IntoIterator<Item = ExecutedBlockWithTrieUpdates<N>>,
I: IntoIterator<Item = ExecutedBlock<N>>,
R: IntoIterator<Item = ExecutedBlock<N>>,
{
{
@@ -568,22 +568,19 @@ impl<N: NodePrimitives> CanonicalInMemoryState<N> {
#[derive(Debug, PartialEq, Eq, Clone)]
pub struct BlockState<N: NodePrimitives = EthPrimitives> {
/// The executed block that determines the state after this block has been executed.
block: ExecutedBlockWithTrieUpdates<N>,
block: ExecutedBlock<N>,
/// The block's parent block if it exists.
parent: Option<Arc<BlockState<N>>>,
}
impl<N: NodePrimitives> BlockState<N> {
/// [`BlockState`] constructor.
pub const fn new(block: ExecutedBlockWithTrieUpdates<N>) -> Self {
pub const fn new(block: ExecutedBlock<N>) -> Self {
Self { block, parent: None }
}
/// [`BlockState`] constructor with parent.
pub const fn with_parent(
block: ExecutedBlockWithTrieUpdates<N>,
parent: Option<Arc<Self>>,
) -> Self {
pub const fn with_parent(block: ExecutedBlock<N>, parent: Option<Arc<Self>>) -> Self {
Self { block, parent }
}
@@ -597,12 +594,12 @@ impl<N: NodePrimitives> BlockState<N> {
}
/// Returns the executed block that determines the state.
pub fn block(&self) -> ExecutedBlockWithTrieUpdates<N> {
pub fn block(&self) -> ExecutedBlock<N> {
self.block.clone()
}
/// Returns a reference to the executed block that determines the state.
pub const fn block_ref(&self) -> &ExecutedBlockWithTrieUpdates<N> {
pub const fn block_ref(&self) -> &ExecutedBlock<N> {
&self.block
}
@@ -730,6 +727,8 @@ pub struct ExecutedBlock<N: NodePrimitives = EthPrimitives> {
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>,
}
impl<N: NodePrimitives> Default for ExecutedBlock<N> {
@@ -738,6 +737,7 @@ impl<N: NodePrimitives> Default for ExecutedBlock<N> {
recovered_block: Default::default(),
execution_output: Default::default(),
hashed_state: Default::default(),
trie_updates: Default::default(),
}
}
}
@@ -767,6 +767,12 @@ impl<N: NodePrimitives> ExecutedBlock<N> {
&self.hashed_state
}
/// Returns a reference to the trie updates resulting from the execution outcome
#[inline]
pub fn trie_updates(&self) -> &TrieUpdates {
&self.trie_updates
}
/// Returns a [`BlockNumber`] of the block.
#[inline]
pub fn block_number(&self) -> BlockNumber {
@@ -774,127 +780,20 @@ impl<N: NodePrimitives> ExecutedBlock<N> {
}
}
/// Trie updates that result from calculating the state root for the block.
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum ExecutedTrieUpdates {
/// Trie updates present. State root was calculated, and the trie updates can be applied to the
/// database.
Present(Arc<TrieUpdates>),
/// Trie updates missing. State root was calculated, but the trie updates cannot be applied to
/// the current database state. To apply the updates, the state root must be recalculated, and
/// new trie updates must be generated.
///
/// This can happen when processing fork chain blocks that are building on top of the
/// historical database state. Since we don't store the historical trie state, we cannot
/// generate the trie updates for it.
Missing,
}
impl ExecutedTrieUpdates {
/// Creates a [`ExecutedTrieUpdates`] with present but empty trie updates.
pub fn empty() -> Self {
Self::Present(Arc::default())
}
/// Sets the trie updates to the provided value as present.
pub fn set_present(&mut self, updates: Arc<TrieUpdates>) {
*self = Self::Present(updates);
}
/// Takes the present trie updates, leaving the state as missing.
pub fn take_present(&mut self) -> Option<Arc<TrieUpdates>> {
match self {
Self::Present(updates) => {
let updates = core::mem::take(updates);
*self = Self::Missing;
Some(updates)
}
Self::Missing => None,
}
}
/// Returns a reference to the trie updates if present.
#[allow(clippy::missing_const_for_fn)] // false positive
pub fn as_ref(&self) -> Option<&TrieUpdates> {
match self {
Self::Present(updates) => Some(updates),
Self::Missing => None,
}
}
/// Returns `true` if the trie updates are present.
pub const fn is_present(&self) -> bool {
matches!(self, Self::Present(_))
}
/// Returns `true` if the trie updates are missing.
pub const fn is_missing(&self) -> bool {
matches!(self, Self::Missing)
}
}
/// An [`ExecutedBlock`] with its [`TrieUpdates`].
///
/// We store it as separate type because [`TrieUpdates`] are only available for blocks stored in
/// memory and can't be obtained for canonical persisted blocks.
#[derive(
Clone, Debug, PartialEq, Eq, derive_more::Deref, derive_more::DerefMut, derive_more::Into,
)]
pub struct ExecutedBlockWithTrieUpdates<N: NodePrimitives = EthPrimitives> {
/// Inner [`ExecutedBlock`].
#[deref]
#[deref_mut]
#[into]
pub block: ExecutedBlock<N>,
/// Trie updates that result from calculating the state root for the block.
///
/// If [`ExecutedTrieUpdates::Missing`], the trie updates should be computed when persisting
/// the block **on top of the canonical parent**.
pub trie: ExecutedTrieUpdates,
}
impl<N: NodePrimitives> ExecutedBlockWithTrieUpdates<N> {
/// [`ExecutedBlock`] constructor.
pub const fn new(
recovered_block: Arc<RecoveredBlock<N::Block>>,
execution_output: Arc<ExecutionOutcome<N::Receipt>>,
hashed_state: Arc<HashedPostState>,
trie: ExecutedTrieUpdates,
) -> Self {
Self { block: ExecutedBlock { recovered_block, execution_output, hashed_state }, trie }
}
/// Returns a reference to the trie updates for the block, if present.
#[inline]
pub fn trie_updates(&self) -> Option<&TrieUpdates> {
self.trie.as_ref()
}
/// Converts the value into [`SealedBlock`].
pub fn into_sealed_block(self) -> SealedBlock<N::Block> {
let block = Arc::unwrap_or_clone(self.block.recovered_block);
block.into_sealed_block()
}
}
/// Non-empty chain of blocks.
#[derive(Debug)]
pub enum NewCanonicalChain<N: NodePrimitives = EthPrimitives> {
/// A simple append to the current canonical head
Commit {
/// all blocks that lead back to the canonical head
new: Vec<ExecutedBlockWithTrieUpdates<N>>,
new: Vec<ExecutedBlock<N>>,
},
/// 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<ExecutedBlockWithTrieUpdates<N>>,
new: Vec<ExecutedBlock<N>>,
/// All blocks of the _old_ chain
///
/// These are not [`ExecutedBlockWithTrieUpdates`] because we don't always have the trie
/// updates for the old canonical chain. For example, in case of node being restarted right
/// before the reorg [`TrieUpdates`] can't be fetched from database.
old: Vec<ExecutedBlock<N>>,
},
}
@@ -1257,7 +1156,7 @@ mod tests {
block1.recovered_block().hash()
);
let chain = NewCanonicalChain::Reorg { new: vec![block2.clone()], old: vec![block1.block] };
let chain = NewCanonicalChain::Reorg { new: vec![block2.clone()], old: vec![block1] };
state.update_chain(chain);
assert_eq!(
state.head_state().unwrap().block_ref().recovered_block().hash(),
@@ -1539,7 +1438,7 @@ mod tests {
// Test reorg notification
let chain_reorg = NewCanonicalChain::Reorg {
new: vec![block1a.clone(), block2a.clone()],
old: vec![block1.block.clone(), block2.block.clone()],
old: vec![block1.clone(), block2.clone()],
};
assert_eq!(

View File

@@ -1,4 +1,4 @@
use super::ExecutedBlockWithTrieUpdates;
use super::ExecutedBlock;
use alloy_consensus::BlockHeader;
use alloy_primitives::{keccak256, Address, BlockNumber, Bytes, StorageKey, StorageValue, B256};
use reth_errors::ProviderResult;
@@ -24,7 +24,7 @@ pub struct MemoryOverlayStateProviderRef<
/// Historical state provider for state lookups that are not found in memory blocks.
pub(crate) historical: Box<dyn StateProvider + 'a>,
/// The collection of executed parent blocks. Expected order is newest to oldest.
pub(crate) in_memory: Vec<ExecutedBlockWithTrieUpdates<N>>,
pub(crate) in_memory: Vec<ExecutedBlock<N>>,
/// Lazy-loaded in-memory trie data.
pub(crate) trie_input: OnceLock<TrieInput>,
}
@@ -41,10 +41,7 @@ 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<dyn StateProvider + 'a>,
in_memory: Vec<ExecutedBlockWithTrieUpdates<N>>,
) -> Self {
pub fn new(historical: Box<dyn StateProvider + 'a>, in_memory: Vec<ExecutedBlock<N>>) -> Self {
Self { historical, in_memory, trie_input: OnceLock::new() }
}
@@ -60,7 +57,7 @@ impl<'a, N: NodePrimitives> MemoryOverlayStateProviderRef<'a, N> {
self.in_memory
.iter()
.rev()
.map(|block| (block.hashed_state.as_ref(), block.trie.as_ref())),
.map(|block| (block.hashed_state.as_ref(), block.trie_updates.as_ref())),
)
})
}

View File

@@ -1,6 +1,6 @@
use crate::{
in_memory::ExecutedBlockWithTrieUpdates, CanonStateNotification, CanonStateNotifications,
CanonStateSubscriptions, ExecutedTrieUpdates,
in_memory::ExecutedBlock, CanonStateNotification, CanonStateNotifications,
CanonStateSubscriptions,
};
use alloy_consensus::{Header, SignableTransaction, TxEip1559, TxReceipt, EMPTY_ROOT_HASH};
use alloy_eips::{
@@ -23,7 +23,7 @@ use reth_primitives_traits::{
SignedTransaction,
};
use reth_storage_api::NodePrimitivesProvider;
use reth_trie::{root::state_root_unhashed, HashedPostState};
use reth_trie::{root::state_root_unhashed, updates::TrieUpdates, HashedPostState};
use revm_database::BundleState;
use revm_state::AccountInfo;
use std::{
@@ -198,45 +198,45 @@ impl<N: NodePrimitives> TestBlockBuilder<N> {
fork
}
/// Gets an [`ExecutedBlockWithTrieUpdates`] with [`BlockNumber`], receipts and parent hash.
/// Gets an [`ExecutedBlock`] with [`BlockNumber`], receipts and parent hash.
fn get_executed_block(
&mut self,
block_number: BlockNumber,
receipts: Vec<Vec<Receipt>>,
parent_hash: B256,
) -> ExecutedBlockWithTrieUpdates {
) -> ExecutedBlock {
let block_with_senders = self.generate_random_block(block_number, parent_hash);
let (block, senders) = block_with_senders.split_sealed();
ExecutedBlockWithTrieUpdates::new(
Arc::new(RecoveredBlock::new_sealed(block, senders)),
Arc::new(ExecutionOutcome::new(
ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(block, senders)),
execution_output: Arc::new(ExecutionOutcome::new(
BundleState::default(),
receipts,
block_number,
vec![Requests::default()],
)),
Arc::new(HashedPostState::default()),
ExecutedTrieUpdates::empty(),
)
hashed_state: Arc::new(HashedPostState::default()),
trie_updates: Arc::new(TrieUpdates::default()),
}
}
/// Generates an [`ExecutedBlockWithTrieUpdates`] that includes the given receipts.
/// Generates an [`ExecutedBlock`] that includes the given receipts.
pub fn get_executed_block_with_receipts(
&mut self,
receipts: Vec<Vec<Receipt>>,
parent_hash: B256,
) -> ExecutedBlockWithTrieUpdates {
) -> ExecutedBlock {
let number = rand::rng().random::<u64>();
self.get_executed_block(number, receipts, parent_hash)
}
/// Generates an [`ExecutedBlockWithTrieUpdates`] with the given [`BlockNumber`].
/// Generates an [`ExecutedBlock`] with the given [`BlockNumber`].
pub fn get_executed_block_with_number(
&mut self,
block_number: BlockNumber,
parent_hash: B256,
) -> ExecutedBlockWithTrieUpdates {
) -> ExecutedBlock {
self.get_executed_block(block_number, vec![vec![]], parent_hash)
}
@@ -244,7 +244,7 @@ impl<N: NodePrimitives> TestBlockBuilder<N> {
pub fn get_executed_blocks(
&mut self,
range: Range<u64>,
) -> impl Iterator<Item = ExecutedBlockWithTrieUpdates> + '_ {
) -> impl Iterator<Item = ExecutedBlock> + '_ {
let mut parent_hash = B256::default();
range.map(move |number| {
let current_parent_hash = parent_hash;

View File

@@ -15,7 +15,7 @@ use reth_db_common::{
};
use reth_node_api::{HeaderTy, ReceiptTy, TxTy};
use reth_node_core::args::StageEnum;
use reth_provider::{DBProvider, DatabaseProviderFactory, StaticFileProviderFactory};
use reth_provider::{DBProvider, DatabaseProviderFactory, StaticFileProviderFactory, TrieWriter};
use reth_prune::PruneSegment;
use reth_stages::StageId;
use reth_static_file_types::StaticFileSegment;
@@ -138,6 +138,10 @@ impl<C: ChainSpecParser> Command<C> {
None,
)?;
}
StageEnum::MerkleChangeSets => {
provider_rw.clear_trie_changesets()?;
reset_stage_checkpoint(tx, StageId::MerkleChangeSets)?;
}
StageEnum::AccountHistory | StageEnum::StorageHistory => {
tx.clear::<tables::AccountsHistory>()?;
tx.clear::<tables::StoragesHistory>()?;

View File

@@ -15,10 +15,7 @@ use reth_db::DatabaseEnv;
use reth_downloaders::{bodies::noop::NoopBodiesDownloader, headers::noop::NoopHeaderDownloader};
use reth_evm::ConfigureEvm;
use reth_exex::ExExManagerHandle;
use reth_provider::{
providers::ProviderNodeTypes, BlockExecutionWriter, BlockNumReader, ChainStateBlockReader,
ChainStateBlockWriter, ProviderFactory, StaticFileProviderFactory,
};
use reth_provider::{providers::ProviderNodeTypes, BlockNumReader, ProviderFactory};
use reth_stages::{
sets::{DefaultStages, OfflineStages},
stages::ExecutionStage,
@@ -60,55 +57,22 @@ impl<C: ChainSpecParser<ChainSpec: EthChainSpec + EthereumHardforks>> Command<C>
let components = components(provider_factory.chain_spec());
let highest_static_file_block = provider_factory
.static_file_provider()
.get_highest_static_files()
.max_block_num()
.filter(|highest_static_file_block| *highest_static_file_block > target);
// Execute a pipeline unwind if the start of the range overlaps the existing static
// files. If that's the case, then copy all available data from MDBX to static files, and
// only then, proceed with the unwind.
//
// We also execute a pipeline unwind if `offline` is specified, because we need to only
// unwind the data associated with offline stages.
if highest_static_file_block.is_some() || self.offline {
if self.offline {
info!(target: "reth::cli", "Performing an unwind for offline-only data!");
}
if let Some(highest_static_file_block) = highest_static_file_block {
info!(target: "reth::cli", ?target, ?highest_static_file_block, "Executing a pipeline unwind.");
} else {
info!(target: "reth::cli", ?target, "Executing a pipeline unwind.");
}
info!(target: "reth::cli", prune_config=?config.prune, "Using prune settings");
// This will build an offline-only pipeline if the `offline` flag is enabled
let mut pipeline =
self.build_pipeline(config, provider_factory, components.evm_config().clone())?;
// Move all applicable data from database to static files.
pipeline.move_to_static_files()?;
pipeline.unwind(target, None)?;
} else {
info!(target: "reth::cli", ?target, "Executing a database unwind.");
let provider = provider_factory.provider_rw()?;
provider
.remove_block_and_execution_above(target)
.map_err(|err| eyre::eyre!("Transaction error on unwind: {err}"))?;
// update finalized block if needed
let last_saved_finalized_block_number = provider.last_finalized_block_number()?;
if last_saved_finalized_block_number.is_none_or(|f| f > target) {
provider.save_finalized_block_number(target)?;
}
provider.commit()?;
if self.offline {
info!(target: "reth::cli", "Performing an unwind for offline-only data!");
}
let highest_static_file_block = provider_factory.provider()?.last_block_number()?;
info!(target: "reth::cli", ?target, ?highest_static_file_block, prune_config=?config.prune, "Executing a pipeline unwind.");
// This will build an offline-only pipeline if the `offline` flag is enabled
let mut pipeline =
self.build_pipeline(config, provider_factory, components.evm_config().clone())?;
// Move all applicable data from database to static files.
pipeline.move_to_static_files()?;
pipeline.unwind(target, None)?;
info!(target: "reth::cli", ?target, "Unwound blocks");
Ok(())

View File

@@ -440,7 +440,7 @@ pub struct PruneConfig {
impl Default for PruneConfig {
fn default() -> Self {
Self { block_interval: DEFAULT_BLOCK_INTERVAL, segments: PruneModes::none() }
Self { block_interval: DEFAULT_BLOCK_INTERVAL, segments: PruneModes::default() }
}
}
@@ -464,6 +464,7 @@ impl PruneConfig {
account_history,
storage_history,
bodies_history,
merkle_changesets,
receipts_log_filter,
},
} = other;
@@ -480,6 +481,8 @@ impl PruneConfig {
self.segments.account_history = self.segments.account_history.or(account_history);
self.segments.storage_history = self.segments.storage_history.or(storage_history);
self.segments.bodies_history = self.segments.bodies_history.or(bodies_history);
// Merkle changesets is not optional, so we just replace it if provided
self.segments.merkle_changesets = merkle_changesets;
if self.segments.receipts_log_filter.0.is_empty() && !receipts_log_filter.0.is_empty() {
self.segments.receipts_log_filter = receipts_log_filter;
@@ -1001,6 +1004,7 @@ receipts = 'full'
account_history: None,
storage_history: Some(PruneMode::Before(5000)),
bodies_history: None,
merkle_changesets: PruneMode::Before(0),
receipts_log_filter: ReceiptsLogPruneConfig(BTreeMap::from([(
Address::random(),
PruneMode::Full,
@@ -1017,6 +1021,7 @@ receipts = 'full'
account_history: Some(PruneMode::Distance(2000)),
storage_history: Some(PruneMode::Distance(3000)),
bodies_history: None,
merkle_changesets: PruneMode::Distance(10000),
receipts_log_filter: ReceiptsLogPruneConfig(BTreeMap::from([
(Address::random(), PruneMode::Distance(1000)),
(Address::random(), PruneMode::Before(2000)),
@@ -1035,6 +1040,7 @@ receipts = 'full'
assert_eq!(config1.segments.receipts, Some(PruneMode::Distance(1000)));
assert_eq!(config1.segments.account_history, Some(PruneMode::Distance(2000)));
assert_eq!(config1.segments.storage_history, Some(PruneMode::Before(5000)));
assert_eq!(config1.segments.merkle_changesets, PruneMode::Distance(10000));
assert_eq!(config1.segments.receipts_log_filter, original_filter);
}

View File

@@ -10,7 +10,7 @@ use core::{
fmt::{Display, Formatter, Result},
time::Duration,
};
use reth_chain_state::ExecutedBlockWithTrieUpdates;
use reth_chain_state::ExecutedBlock;
use reth_ethereum_primitives::EthPrimitives;
use reth_primitives_traits::{NodePrimitives, SealedBlock, SealedHeader};
@@ -24,11 +24,11 @@ pub enum ConsensusEngineEvent<N: NodePrimitives = EthPrimitives> {
/// The fork choice state was updated, and the current fork choice status
ForkchoiceUpdated(ForkchoiceState, ForkchoiceStatus),
/// A block was added to the fork chain.
ForkBlockAdded(ExecutedBlockWithTrieUpdates<N>, Duration),
ForkBlockAdded(ExecutedBlock<N>, Duration),
/// A new block was received from the consensus engine
BlockReceived(BlockNumHash),
/// A block was added to the canonical chain, and the elapsed time validating the block
CanonicalBlockAdded(ExecutedBlockWithTrieUpdates<N>, Duration),
CanonicalBlockAdded(ExecutedBlock<N>, Duration),
/// A canonical chain was committed, and the elapsed time committing the data
CanonicalChainCommitted(Box<SealedHeader<N::BlockHeader>>, Duration),
/// The consensus engine processed an invalid block.

View File

@@ -7,7 +7,7 @@ use crate::{
};
use alloy_primitives::B256;
use futures::{Stream, StreamExt};
use reth_chain_state::ExecutedBlockWithTrieUpdates;
use reth_chain_state::ExecutedBlock;
use reth_engine_primitives::{BeaconEngineMessage, ConsensusEngineEvent};
use reth_ethereum_primitives::EthPrimitives;
use reth_payload_primitives::PayloadTypes;
@@ -246,7 +246,7 @@ pub enum EngineApiRequest<T: PayloadTypes, N: NodePrimitives> {
/// A request received from the consensus engine.
Beacon(BeaconEngineMessage<T>),
/// Request to insert an already executed block, e.g. via payload building.
InsertExecutedBlock(ExecutedBlockWithTrieUpdates<N>),
InsertExecutedBlock(ExecutedBlock<N>),
}
impl<T: PayloadTypes, N: NodePrimitives> Display for EngineApiRequest<T, N> {

View File

@@ -1,7 +1,7 @@
use crate::metrics::PersistenceMetrics;
use alloy_consensus::BlockHeader;
use alloy_eips::BlockNumHash;
use reth_chain_state::ExecutedBlockWithTrieUpdates;
use reth_chain_state::ExecutedBlock;
use reth_errors::ProviderError;
use reth_ethereum_primitives::EthPrimitives;
use reth_primitives_traits::NodePrimitives;
@@ -140,7 +140,7 @@ where
fn on_save_blocks(
&self,
blocks: Vec<ExecutedBlockWithTrieUpdates<N::Primitives>>,
blocks: Vec<ExecutedBlock<N::Primitives>>,
) -> Result<Option<BlockNumHash>, 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();
@@ -180,7 +180,7 @@ pub enum PersistenceAction<N: NodePrimitives = EthPrimitives> {
///
/// 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<ExecutedBlockWithTrieUpdates<N>>, oneshot::Sender<Option<BlockNumHash>>),
SaveBlocks(Vec<ExecutedBlock<N>>, oneshot::Sender<Option<BlockNumHash>>),
/// Removes block data above the given block number from the database.
///
@@ -257,7 +257,7 @@ impl<T: NodePrimitives> PersistenceHandle<T> {
/// If there are no blocks to persist, then `None` is sent in the sender.
pub fn save_blocks(
&self,
blocks: Vec<ExecutedBlockWithTrieUpdates<T>>,
blocks: Vec<ExecutedBlock<T>>,
tx: oneshot::Sender<Option<BlockNumHash>>,
) -> Result<(), SendError<PersistenceAction<T>>> {
self.send_action(PersistenceAction::SaveBlocks(blocks, tx))

View File

@@ -1,7 +1,6 @@
//! Internal errors for the tree module.
use alloy_consensus::BlockHeader;
use alloy_primitives::B256;
use reth_consensus::ConsensusError;
use reth_errors::{BlockExecutionError, BlockValidationError, ProviderError};
use reth_evm::execute::InternalBlockExecutionError;
@@ -19,20 +18,6 @@ pub enum AdvancePersistenceError {
/// A provider error
#[error(transparent)]
Provider(#[from] ProviderError),
/// Missing ancestor.
///
/// This error occurs when we need to compute the state root for a block with missing trie
/// updates, but the ancestor block is not available. State root computation requires the state
/// from the parent block as a starting point.
///
/// A block may be missing the trie updates when it's a fork chain block building on top of the
/// historical database state. Since we don't store the historical trie state, we cannot
/// generate the trie updates for it until the moment when database is unwound to the canonical
/// chain.
///
/// Also see [`reth_chain_state::ExecutedTrieUpdates::Missing`].
#[error("Missing ancestor with hash {0}")]
MissingAncestor(B256),
}
#[derive(thiserror::Error)]

View File

@@ -13,10 +13,8 @@ use alloy_rpc_types_engine::{
ForkchoiceState, PayloadStatus, PayloadStatusEnum, PayloadValidationError,
};
use error::{InsertBlockError, InsertBlockFatalError};
use persistence_state::CurrentPersistenceAction;
use reth_chain_state::{
CanonicalInMemoryState, ExecutedBlock, ExecutedBlockWithTrieUpdates, ExecutedTrieUpdates,
MemoryOverlayStateProvider, NewCanonicalChain,
CanonicalInMemoryState, ExecutedBlock, MemoryOverlayStateProvider, NewCanonicalChain,
};
use reth_consensus::{Consensus, FullConsensus};
use reth_engine_primitives::{
@@ -31,14 +29,12 @@ use reth_payload_primitives::{
};
use reth_primitives_traits::{NodePrimitives, RecoveredBlock, SealedBlock, SealedHeader};
use reth_provider::{
providers::ConsistentDbView, BlockNumReader, BlockReader, DBProvider, DatabaseProviderFactory,
HashedPostStateProvider, ProviderError, StateProviderBox, StateProviderFactory, StateReader,
StateRootProvider, TransactionVariant,
providers::ConsistentDbView, BlockReader, DatabaseProviderFactory, HashedPostStateProvider,
ProviderError, StateProviderBox, StateProviderFactory, StateReader, TransactionVariant,
TrieReader,
};
use reth_revm::database::StateProviderDatabase;
use reth_stages_api::ControlFlow;
use reth_trie::{HashedPostState, TrieInput};
use reth_trie_db::DatabaseHashedPostState;
use revm::state::EvmState;
use state::TreeState;
use std::{
@@ -78,7 +74,6 @@ pub use payload_processor::*;
pub use payload_validator::{BasicEngineValidator, EngineValidator};
pub use persistence_state::PersistenceState;
pub use reth_engine_primitives::TreeConfig;
use reth_trie::KeccakKeyHasher;
pub mod state;
@@ -101,7 +96,7 @@ pub struct StateProviderBuilder<N: NodePrimitives, P> {
/// The historical block hash to fetch state from.
historical: B256,
/// The blocks that form the chain from historical to target and are in memory.
overlay: Option<Vec<ExecutedBlockWithTrieUpdates<N>>>,
overlay: Option<Vec<ExecutedBlock<N>>>,
}
impl<N: NodePrimitives, P> StateProviderBuilder<N, P> {
@@ -110,7 +105,7 @@ impl<N: NodePrimitives, P> StateProviderBuilder<N, P> {
pub const fn new(
provider_factory: P,
historical: B256,
overlay: Option<Vec<ExecutedBlockWithTrieUpdates<N>>>,
overlay: Option<Vec<ExecutedBlock<N>>>,
) -> Self {
Self { provider_factory, historical, overlay }
}
@@ -318,6 +313,7 @@ where
+ StateProviderFactory
+ StateReader<Receipt = N::Receipt>
+ HashedPostStateProvider
+ TrieReader
+ Clone
+ 'static,
<P as DatabaseProviderFactory>::Provider:
@@ -823,7 +819,7 @@ where
for block_num in (new_head_number + 1)..=current_head_number {
if let Some(block_state) = self.canonical_in_memory_state.state_by_number(block_num) {
let executed_block = block_state.block_ref().block.clone();
let executed_block = block_state.block_ref().clone();
old_blocks.push(executed_block);
debug!(
target: "engine::tree",
@@ -855,14 +851,9 @@ where
// Try to load the canonical ancestor's block
match self.canonical_block_by_hash(new_head_hash)? {
Some(executed_block) => {
let block_with_trie = ExecutedBlockWithTrieUpdates {
block: executed_block,
trie: ExecutedTrieUpdates::Missing,
};
// Perform the reorg to properly handle the unwind
self.canonical_in_memory_state.update_chain(NewCanonicalChain::Reorg {
new: vec![block_with_trie],
new: vec![executed_block],
old: old_blocks,
});
@@ -915,13 +906,8 @@ where
// Try to load the block from storage
if let Some(executed_block) = self.canonical_block_by_hash(block_hash)? {
let block_with_trie = ExecutedBlockWithTrieUpdates {
block: executed_block,
trie: ExecutedTrieUpdates::Missing,
};
self.canonical_in_memory_state
.update_chain(NewCanonicalChain::Commit { new: vec![block_with_trie] });
.update_chain(NewCanonicalChain::Commit { new: vec![executed_block] });
debug!(
target: "engine::tree",
@@ -976,29 +962,6 @@ where
Ok(true)
}
/// Returns the persisting kind for the input block.
fn persisting_kind_for(&self, block: BlockWithParent) -> PersistingKind {
// Check that we're currently persisting.
let Some(action) = self.persistence_state.current_action() else {
return PersistingKind::NotPersisting
};
// Check that the persistince action is saving blocks, not removing them.
let CurrentPersistenceAction::SavingBlocks { highest } = action else {
return PersistingKind::PersistingNotDescendant
};
// The block being validated can only be a descendant if its number is higher than
// the highest block persisting. Otherwise, it's likely a fork of a lower block.
if block.block.number > highest.number &&
self.state.tree_state.is_descendant(*highest, block)
{
return PersistingKind::PersistingDescendant
}
// In all other cases, the block is not a descendant.
PersistingKind::PersistingNotDescendant
}
/// Invoked when we receive a new forkchoice update message. Calls into the blockchain tree
/// to resolve chain forks and ensure that the Execution Layer is working with the latest valid
/// chain.
@@ -1305,7 +1268,7 @@ where
/// Helper method to save blocks and set the persistence state. This ensures we keep track of
/// the current persistence action while we're saving blocks.
fn persist_blocks(&mut self, blocks_to_persist: Vec<ExecutedBlockWithTrieUpdates<N>>) {
fn persist_blocks(&mut self, blocks_to_persist: Vec<ExecutedBlock<N>>) {
if blocks_to_persist.is_empty() {
debug!(target: "engine::tree", "Returned empty set of blocks to persist");
return
@@ -1696,17 +1659,9 @@ 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.
///
/// If any blocks are missing trie updates, all blocks are persisted, not taking `threshold`
/// into account.
///
/// For those blocks that didn't have the trie updates calculated, runs the state root
/// calculation, and saves the trie updates.
///
/// Returns an error if the state root calculation fails.
fn get_canonical_blocks_to_persist(
&mut self,
) -> Result<Vec<ExecutedBlockWithTrieUpdates<N>>, AdvancePersistenceError> {
&self,
) -> Result<Vec<ExecutedBlock<N>>, AdvancePersistenceError> {
// We will calculate the state root using the database, so we need to be sure there are no
// changes
debug_assert!(!self.persistence_state.in_progress());
@@ -1715,27 +1670,16 @@ where
let mut current_hash = self.state.tree_state.canonical_block_hash();
let last_persisted_number = self.persistence_state.last_persisted_block.number;
let canonical_head_number = self.state.tree_state.canonical_block_number();
let all_blocks_have_trie_updates = self
.state
.tree_state
.blocks_by_hash
.values()
.all(|block| block.trie_updates().is_some());
let target_number = if all_blocks_have_trie_updates {
// Persist only up to block buffer target if all blocks have trie updates
canonical_head_number.saturating_sub(self.config.memory_block_buffer_target())
} else {
// Persist all blocks if any block is missing trie updates
canonical_head_number
};
// Persist only up to block buffer target
let target_number =
canonical_head_number.saturating_sub(self.config.memory_block_buffer_target());
debug!(
target: "engine::tree",
?current_hash,
?last_persisted_number,
?canonical_head_number,
?all_blocks_have_trie_updates,
?target_number,
"Returning canonical blocks to persist"
);
@@ -1754,48 +1698,6 @@ where
// Reverse the order so that the oldest block comes first
blocks_to_persist.reverse();
// Calculate missing trie updates
for block in &mut blocks_to_persist {
if block.trie.is_present() {
continue
}
debug!(
target: "engine::tree",
block = ?block.recovered_block().num_hash(),
"Calculating trie updates before persisting"
);
let provider = self
.state_provider_builder(block.recovered_block().parent_hash())?
.ok_or(AdvancePersistenceError::MissingAncestor(
block.recovered_block().parent_hash(),
))?
.build()?;
let mut trie_input = self.compute_trie_input(
self.persisting_kind_for(block.recovered_block.block_with_parent()),
self.provider.database_provider_ro()?,
block.recovered_block().parent_hash(),
None,
)?;
// Extend with block we are generating trie updates for.
trie_input.append_ref(block.hashed_state());
let (_root, updates) = provider.state_root_from_nodes_with_updates(trie_input)?;
debug_assert_eq!(_root, block.recovered_block().state_root());
// Update trie updates in both tree state and blocks to persist that we return
let trie_updates = Arc::new(updates);
let tree_state_block = self
.state
.tree_state
.blocks_by_hash
.get_mut(&block.recovered_block().hash())
.expect("blocks to persist are constructed from tree state blocks");
tree_state_block.trie.set_present(trie_updates.clone());
block.trie.set_present(trie_updates);
}
Ok(blocks_to_persist)
}
@@ -1834,7 +1736,7 @@ where
trace!(target: "engine::tree", ?hash, "Fetching executed block by hash");
// check memory first
if let Some(block) = self.state.tree_state.executed_block_by_hash(hash) {
return Ok(Some(block.block.clone()))
return Ok(Some(block.clone()))
}
let (block, senders) = self
@@ -1847,11 +1749,13 @@ where
.get_state(block.header().number())?
.ok_or_else(|| ProviderError::StateForNumberNotFound(block.header().number()))?;
let hashed_state = self.provider.hashed_post_state(execution_output.state());
let trie_updates = self.provider.get_block_trie_updates(block.number())?;
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()),
}))
}
@@ -2289,25 +2193,7 @@ where
self.update_reorg_metrics(old.len());
self.reinsert_reorged_blocks(new.clone());
// Try reinserting the reorged canonical chain. This is only possible if we have
// `persisted_trie_updates` for those blocks.
let old = old
.iter()
.filter_map(|block| {
let trie = self
.state
.tree_state
.persisted_trie_updates
.get(&block.recovered_block.hash())?
.1
.clone();
Some(ExecutedBlockWithTrieUpdates {
block: block.clone(),
trie: ExecutedTrieUpdates::Present(trie),
})
})
.collect::<Vec<_>>();
self.reinsert_reorged_blocks(old);
self.reinsert_reorged_blocks(old.clone());
}
// update the tracked in-memory state with the new chain
@@ -2334,7 +2220,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<ExecutedBlockWithTrieUpdates<N>>) {
fn reinsert_reorged_blocks(&mut self, new_chain: Vec<ExecutedBlock<N>>) {
for block in new_chain {
if self
.state
@@ -2505,11 +2391,7 @@ where
&mut self,
block_id: BlockWithParent,
input: Input,
execute: impl FnOnce(
&mut V,
Input,
TreeCtx<'_, N>,
) -> Result<ExecutedBlockWithTrieUpdates<N>, Err>,
execute: impl FnOnce(&mut V, Input, TreeCtx<'_, N>) -> Result<ExecutedBlock<N>, Err>,
convert_to_block: impl FnOnce(&mut Self, Input) -> Result<RecoveredBlock<N::Block>, Err>,
) -> Result<InsertPayloadOk, Err>
where
@@ -2604,109 +2486,6 @@ where
Ok(InsertPayloadOk::Inserted(BlockStatus::Valid))
}
/// Computes the trie input at the provided parent hash.
///
/// The goal of this function is to take in-memory blocks and generate a [`TrieInput`] 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
/// [`TreeState::blocks_by_hash`].
/// 2. If the persistence is in progress, and the block that we're computing the trie input for
/// is a descendant of the currently persisting blocks, we need to be sure that in-memory
/// blocks are not overlapping with the database blocks that may have been already persisted.
/// To do that, we're filtering out in-memory blocks that are lower than the highest database
/// block.
/// 3. Once in-memory blocks are collected and optionally filtered, we compute the
/// [`HashedPostState`] from them.
fn compute_trie_input<TP: DBProvider + BlockNumReader>(
&self,
persisting_kind: PersistingKind,
provider: TP,
parent_hash: B256,
allocated_trie_input: Option<TrieInput>,
) -> ProviderResult<TrieInput> {
// get allocated trie input or use a default trie input
let mut input = allocated_trie_input.unwrap_or_default();
let best_block_number = provider.best_block_number()?;
let (mut historical, mut blocks) = self
.state
.tree_state
.blocks_by_hash(parent_hash)
.map_or_else(|| (parent_hash.into(), vec![]), |(hash, blocks)| (hash.into(), blocks));
// If the current block is a descendant of the currently persisting blocks, then we need to
// filter in-memory blocks, so that none of them are already persisted in the database.
if persisting_kind.is_descendant() {
// Iterate over the blocks from oldest to newest.
while let Some(block) = blocks.last() {
let recovered_block = block.recovered_block();
if recovered_block.number() <= best_block_number {
// Remove those blocks that lower than or equal to the highest database
// block.
blocks.pop();
} else {
// If the block is higher than the best block number, stop filtering, as it's
// the first block that's not in the database.
break
}
}
historical = if let Some(block) = blocks.last() {
// If there are any in-memory blocks left after filtering, set the anchor to the
// parent of the oldest block.
(block.recovered_block().number() - 1).into()
} else {
// Otherwise, set the anchor to the original provided parent hash.
parent_hash.into()
};
}
if blocks.is_empty() {
debug!(target: "engine::tree", %parent_hash, "Parent found on disk");
} else {
debug!(target: "engine::tree", %parent_hash, %historical, blocks = blocks.len(), "Parent found in memory");
}
// Convert the historical block to the block number.
let block_number = provider
.convert_hash_or_number(historical)?
.ok_or_else(|| ProviderError::BlockHashNotFound(historical.as_hash().unwrap()))?;
// Retrieve revert state for historical block.
let revert_state = if block_number == best_block_number {
// We do not check against the `last_block_number` here because
// `HashedPostState::from_reverts` only uses the database tables, and not static files.
debug!(target: "engine::tree", block_number, best_block_number, "Empty revert state");
HashedPostState::default()
} else {
let revert_state = HashedPostState::from_reverts::<KeccakKeyHasher>(
provider.tx_ref(),
block_number + 1..,
)
.map_err(ProviderError::from)?;
debug!(
target: "engine::tree",
block_number,
best_block_number,
accounts = revert_state.accounts.len(),
storages = revert_state.storages.len(),
"Non-empty revert state"
);
revert_state
};
input.append(revert_state);
// Extend with contents of parent in-memory blocks.
input.extend_with_blocks(
blocks.iter().rev().map(|block| (block.hashed_state(), block.trie_updates())),
);
Ok(input)
}
/// Handles an error that occurred while inserting a block.
///
/// If this is a validation error this will mark the block as invalid.

View File

@@ -16,9 +16,7 @@ use alloy_consensus::transaction::Either;
use alloy_eips::{eip1898::BlockWithParent, NumHash};
use alloy_evm::Evm;
use alloy_primitives::B256;
use reth_chain_state::{
CanonicalInMemoryState, ExecutedBlock, ExecutedBlockWithTrieUpdates, ExecutedTrieUpdates,
};
use reth_chain_state::{CanonicalInMemoryState, ExecutedBlock};
use reth_consensus::{ConsensusError, FullConsensus};
use reth_engine_primitives::{
ConfigureEngineEvm, ExecutableTxIterator, ExecutionPayload, InvalidBlockHook, PayloadValidator,
@@ -35,12 +33,15 @@ use reth_primitives_traits::{
AlloyBlockHeader, BlockTy, GotExpected, NodePrimitives, RecoveredBlock, SealedHeader,
};
use reth_provider::{
BlockExecutionOutput, BlockHashReader, BlockNumReader, BlockReader, DBProvider,
DatabaseProviderFactory, ExecutionOutcome, HashedPostStateProvider, HeaderProvider,
ProviderError, StateProvider, StateProviderFactory, StateReader, StateRootProvider,
BlockExecutionOutput, BlockNumReader, BlockReader, DBProvider, DatabaseProviderFactory,
ExecutionOutcome, HashedPostStateProvider, ProviderError, StateProvider, StateProviderFactory,
StateReader, StateRootProvider, TrieReader,
};
use reth_revm::db::State;
use reth_trie::{updates::TrieUpdates, HashedPostState, KeccakKeyHasher, TrieInput};
use reth_trie::{
updates::{TrieUpdates, TrieUpdatesSorted},
HashedPostState, KeccakKeyHasher, TrieInput,
};
use reth_trie_db::DatabaseHashedPostState;
use reth_trie_parallel::root::{ParallelStateRoot, ParallelStateRootError};
use revm::context::Block;
@@ -167,7 +168,7 @@ where
impl<N, P, Evm, V> BasicEngineValidator<P, Evm, V>
where
N: NodePrimitives,
P: DatabaseProviderFactory<Provider: BlockReader>
P: DatabaseProviderFactory<Provider: BlockReader + TrieReader>
+ BlockReader<Header = N::BlockHeader>
+ StateProviderFactory
+ StateReader
@@ -283,7 +284,7 @@ where
input: BlockOrPayload<T>,
execution_err: InsertBlockErrorKind,
parent_block: &SealedHeader<N::BlockHeader>,
) -> Result<ExecutedBlockWithTrieUpdates<N>, InsertPayloadError<N::Block>>
) -> Result<ExecutedBlock<N>, InsertPayloadError<N::Block>>
where
V: PayloadValidator<T, Block = N::Block>,
{
@@ -396,15 +397,12 @@ where
// Plan the strategy used for state root computation.
let state_root_plan = self.plan_state_root_computation(&input, &ctx);
let persisting_kind = state_root_plan.persisting_kind;
let has_ancestors_with_missing_trie_updates =
state_root_plan.has_ancestors_with_missing_trie_updates;
let strategy = state_root_plan.strategy;
debug!(
target: "engine::tree",
block=?block_num_hash,
?strategy,
?has_ancestors_with_missing_trie_updates,
"Deciding which state root algorithm to run"
);
@@ -561,38 +559,11 @@ where
// terminate prewarming task with good state output
handle.terminate_caching(Some(&output.state));
// If the block doesn't connect to the database tip, we don't save its trie updates, because
// they may be incorrect as they were calculated on top of the forked block.
//
// We also only save trie updates if all ancestors have trie updates, because otherwise the
// trie updates may be incorrect.
//
// Instead, they will be recomputed on persistence.
let connects_to_last_persisted =
ensure_ok_post_block!(self.block_connects_to_last_persisted(ctx, &block), block);
let should_discard_trie_updates =
!connects_to_last_persisted || has_ancestors_with_missing_trie_updates;
debug!(
target: "engine::tree",
block = ?block_num_hash,
connects_to_last_persisted,
has_ancestors_with_missing_trie_updates,
should_discard_trie_updates,
"Checking if should discard trie updates"
);
let trie_updates = if should_discard_trie_updates {
ExecutedTrieUpdates::Missing
} else {
ExecutedTrieUpdates::Present(Arc::new(trie_output))
};
Ok(ExecutedBlockWithTrieUpdates {
block: ExecutedBlock {
recovered_block: Arc::new(block),
execution_output: Arc::new(ExecutionOutcome::from((output, block_num_hash.number))),
hashed_state: Arc::new(hashed_state),
},
trie: trie_updates,
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),
})
}
@@ -720,51 +691,6 @@ where
ParallelStateRoot::new(consistent_view, input).incremental_root_with_updates()
}
/// Checks if the given block connects to the last persisted block, i.e. if the last persisted
/// block is the ancestor of the given block.
///
/// This checks the database for the actual last persisted block, not [`PersistenceState`].
fn block_connects_to_last_persisted(
&self,
ctx: TreeCtx<'_, N>,
block: &RecoveredBlock<N::Block>,
) -> ProviderResult<bool> {
let provider = self.provider.database_provider_ro()?;
let last_persisted_block = provider.best_block_number()?;
let last_persisted_hash = provider
.block_hash(last_persisted_block)?
.ok_or(ProviderError::HeaderNotFound(last_persisted_block.into()))?;
let last_persisted = NumHash::new(last_persisted_block, last_persisted_hash);
let parent_num_hash = |hash: B256| -> ProviderResult<NumHash> {
let parent_num_hash =
if let Some(header) = ctx.state().tree_state.sealed_header_by_hash(&hash) {
Some(header.parent_num_hash())
} else {
provider.sealed_header_by_hash(hash)?.map(|header| header.parent_num_hash())
};
parent_num_hash.ok_or(ProviderError::BlockHashNotFound(hash))
};
let mut parent_block = block.parent_num_hash();
while parent_block.number > last_persisted.number {
parent_block = parent_num_hash(parent_block.hash)?;
}
let connects = parent_block == last_persisted;
debug!(
target: "engine::tree",
num_hash = ?block.num_hash(),
?last_persisted,
?parent_block,
"Checking if block connects to last persisted block"
);
Ok(connects)
}
/// Validates the block after execution.
///
/// This performs:
@@ -948,27 +874,6 @@ where
}
}
/// Check if the given block has any ancestors with missing trie updates.
fn has_ancestors_with_missing_trie_updates(
&self,
target_header: BlockWithParent,
state: &EngineApiTreeState<N>,
) -> bool {
// Walk back through the chain starting from the parent of the target block
let mut current_hash = target_header.parent;
while let Some(block) = state.tree_state.blocks_by_hash.get(&current_hash) {
// Check if this block is missing trie updates
if block.trie.is_missing() {
return true;
}
// Move to the parent block
current_hash = block.recovered_block().parent_hash();
}
false
}
/// Creates a `StateProviderBuilder` for the given parent hash.
///
/// This method checks if the parent is in the tree state (in-memory) or persisted to disk,
@@ -1019,20 +924,12 @@ where
let can_run_parallel =
persisting_kind.can_run_parallel_state_root() && !self.config.state_root_fallback();
// Check for ancestors with missing trie updates
let has_ancestors_with_missing_trie_updates =
self.has_ancestors_with_missing_trie_updates(input.block_with_parent(), ctx.state());
// Decide on the strategy.
// Use state root task only if:
// 1. No persistence is in progress
// 2. Config allows it
// 3. No ancestors with missing trie updates. If any exist, it will mean that every state
// root task proof calculation will include a lot of unrelated paths in the prefix sets.
// It's cheaper to run a parallel state root that does one walk over trie tables while
// accounting for the prefix sets.
let strategy = if can_run_parallel {
if self.config.use_state_root_task() && !has_ancestors_with_missing_trie_updates {
if self.config.use_state_root_task() {
StateRootStrategy::StateRootTask
} else {
StateRootStrategy::Parallel
@@ -1045,11 +942,10 @@ where
target: "engine::tree",
block=?input.num_hash(),
?strategy,
has_ancestors_with_missing_trie_updates,
"Planned state root computation strategy"
);
StateRootPlan { strategy, has_ancestors_with_missing_trie_updates, persisting_kind }
StateRootPlan { strategy, persisting_kind }
}
/// Called when an invalid block is encountered during validation.
@@ -1083,7 +979,7 @@ where
/// block.
/// 3. Once in-memory blocks are collected and optionally filtered, we compute the
/// [`HashedPostState`] from them.
fn compute_trie_input<TP: DBProvider + BlockNumReader>(
fn compute_trie_input<TP: DBProvider + BlockNumReader + TrieReader>(
&self,
persisting_kind: PersistingKind,
provider: TP,
@@ -1140,17 +1036,19 @@ where
.ok_or_else(|| ProviderError::BlockHashNotFound(historical.as_hash().unwrap()))?;
// Retrieve revert state for historical block.
let revert_state = if block_number == best_block_number {
let (revert_state, revert_trie) = if block_number == best_block_number {
// We do not check against the `last_block_number` here because
// `HashedPostState::from_reverts` only uses the database tables, and not static files.
// `HashedPostState::from_reverts` / `trie_reverts` only use the database tables, and
// not static files.
debug!(target: "engine::tree", block_number, best_block_number, "Empty revert state");
HashedPostState::default()
(HashedPostState::default(), TrieUpdatesSorted::default())
} else {
let revert_state = HashedPostState::from_reverts::<KeccakKeyHasher>(
provider.tx_ref(),
block_number + 1..,
)
.map_err(ProviderError::from)?;
let revert_trie = provider.trie_reverts(block_number + 1)?;
debug!(
target: "engine::tree",
block_number,
@@ -1159,9 +1057,10 @@ where
storages = revert_state.storages.len(),
"Non-empty revert state"
);
revert_state
(revert_state, revert_trie)
};
input.append(revert_state);
input.append_cached(revert_trie.into(), revert_state);
// Extend with contents of parent in-memory blocks.
input.extend_with_blocks(
@@ -1173,8 +1072,7 @@ where
}
/// Output of block or payload validation.
pub type ValidationOutcome<N, E = InsertPayloadError<BlockTy<N>>> =
Result<ExecutedBlockWithTrieUpdates<N>, E>;
pub type ValidationOutcome<N, E = InsertPayloadError<BlockTy<N>>> = Result<ExecutedBlock<N>, E>;
/// Strategy describing how to compute the state root.
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
@@ -1191,8 +1089,6 @@ enum StateRootStrategy {
struct StateRootPlan {
/// Strategy that should be attempted for computing the state root.
strategy: StateRootStrategy,
/// Whether ancestors have missing trie updates.
has_ancestors_with_missing_trie_updates: bool,
/// The persisting kind for this block.
persisting_kind: PersistingKind,
}
@@ -1250,7 +1146,7 @@ pub trait EngineValidator<
impl<N, Types, P, Evm, V> EngineValidator<Types> for BasicEngineValidator<P, Evm, V>
where
P: DatabaseProviderFactory<Provider: BlockReader>
P: DatabaseProviderFactory<Provider: BlockReader + TrieReader>
+ BlockReader<Header = N::BlockHeader>
+ StateProviderFactory
+ StateReader

View File

@@ -1,29 +1,19 @@
//! Functionality related to tree state.
use crate::engine::EngineApiKind;
use alloy_eips::{eip1898::BlockWithParent, merge::EPOCH_SLOTS, BlockNumHash};
use alloy_eips::{eip1898::BlockWithParent, BlockNumHash};
use alloy_primitives::{
map::{HashMap, HashSet},
BlockNumber, B256,
};
use reth_chain_state::{EthPrimitives, ExecutedBlockWithTrieUpdates};
use reth_chain_state::{EthPrimitives, ExecutedBlock};
use reth_primitives_traits::{AlloyBlockHeader, NodePrimitives, SealedHeader};
use reth_trie::updates::TrieUpdates;
use std::{
collections::{btree_map, hash_map, BTreeMap, VecDeque},
ops::Bound,
sync::Arc,
};
use tracing::debug;
/// Default number of blocks to retain persisted trie updates
const DEFAULT_PERSISTED_TRIE_UPDATES_RETENTION: u64 = EPOCH_SLOTS * 2;
/// Number of blocks to retain persisted trie updates for OP Stack chains
/// OP Stack chains only need `EPOCH_SLOTS` as reorgs are relevant only when
/// op-node reorgs to the same chain twice
const OPSTACK_PERSISTED_TRIE_UPDATES_RETENTION: u64 = EPOCH_SLOTS;
/// Keeps track of the state of the tree.
///
/// ## Invariants
@@ -35,19 +25,15 @@ pub struct TreeState<N: NodePrimitives = EthPrimitives> {
/// __All__ unique executed blocks by block hash that are connected to the canonical chain.
///
/// This includes blocks of all forks.
pub(crate) blocks_by_hash: HashMap<B256, ExecutedBlockWithTrieUpdates<N>>,
pub(crate) blocks_by_hash: HashMap<B256, ExecutedBlock<N>>,
/// 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.
pub(crate) blocks_by_number: BTreeMap<BlockNumber, Vec<ExecutedBlockWithTrieUpdates<N>>>,
pub(crate) blocks_by_number: BTreeMap<BlockNumber, Vec<ExecutedBlock<N>>>,
/// Map of any parent block hash to its children.
pub(crate) parent_to_child: HashMap<B256, HashSet<B256>>,
/// Map of hash to trie updates for canonical blocks that are persisted but not finalized.
///
/// Contains the block number for easy removal.
pub(crate) persisted_trie_updates: HashMap<B256, (BlockNumber, Arc<TrieUpdates>)>,
/// Currently tracked canonical head of the chain.
pub(crate) current_canonical_head: BlockNumHash,
/// The engine API variant of this handler
@@ -62,7 +48,6 @@ impl<N: NodePrimitives> TreeState<N> {
blocks_by_number: BTreeMap::new(),
current_canonical_head,
parent_to_child: HashMap::default(),
persisted_trie_updates: HashMap::default(),
engine_kind,
}
}
@@ -77,11 +62,8 @@ impl<N: NodePrimitives> TreeState<N> {
self.blocks_by_hash.len()
}
/// Returns the [`ExecutedBlockWithTrieUpdates`] by hash.
pub(crate) fn executed_block_by_hash(
&self,
hash: B256,
) -> Option<&ExecutedBlockWithTrieUpdates<N>> {
/// Returns the [`ExecutedBlock`] by hash.
pub(crate) fn executed_block_by_hash(&self, hash: B256) -> Option<&ExecutedBlock<N>> {
self.blocks_by_hash.get(&hash)
}
@@ -97,10 +79,7 @@ impl<N: NodePrimitives> TreeState<N> {
/// 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.
pub(crate) fn blocks_by_hash(
&self,
hash: B256,
) -> Option<(B256, Vec<ExecutedBlockWithTrieUpdates<N>>)> {
pub(crate) fn blocks_by_hash(&self, hash: B256) -> Option<(B256, Vec<ExecutedBlock<N>>)> {
let block = self.blocks_by_hash.get(&hash).cloned()?;
let mut parent_hash = block.recovered_block().parent_hash();
let mut blocks = vec![block];
@@ -113,7 +92,7 @@ impl<N: NodePrimitives> TreeState<N> {
}
/// Insert executed block into the state.
pub(crate) fn insert_executed(&mut self, executed: ExecutedBlockWithTrieUpdates<N>) {
pub(crate) fn insert_executed(&mut self, executed: ExecutedBlock<N>) {
let hash = executed.recovered_block().hash();
let parent_hash = executed.recovered_block().parent_hash();
let block_number = executed.recovered_block().number();
@@ -138,10 +117,7 @@ impl<N: NodePrimitives> TreeState<N> {
/// ## Returns
///
/// The removed block and the block hashes of its children.
fn remove_by_hash(
&mut self,
hash: B256,
) -> Option<(ExecutedBlockWithTrieUpdates<N>, HashSet<B256>)> {
fn remove_by_hash(&mut self, hash: B256) -> Option<(ExecutedBlock<N>, HashSet<B256>)> {
let executed = self.blocks_by_hash.remove(&hash)?;
// Remove this block from collection of children of its parent block.
@@ -215,41 +191,12 @@ impl<N: NodePrimitives> TreeState<N> {
if executed.recovered_block().number() <= upper_bound {
let num_hash = executed.recovered_block().num_hash();
debug!(target: "engine::tree", ?num_hash, "Attempting to remove block walking back from the head");
if let Some((mut removed, _)) =
self.remove_by_hash(executed.recovered_block().hash())
{
debug!(target: "engine::tree", ?num_hash, "Removed block walking back from the head");
// finally, move the trie updates
let Some(trie_updates) = removed.trie.take_present() else {
debug!(target: "engine::tree", ?num_hash, "No trie updates found for persisted block");
continue;
};
self.persisted_trie_updates.insert(
removed.recovered_block().hash(),
(removed.recovered_block().number(), trie_updates),
);
}
self.remove_by_hash(executed.recovered_block().hash());
}
}
debug!(target: "engine::tree", ?upper_bound, ?last_persisted_hash, "Removed canonical blocks from the tree");
}
/// Prunes old persisted trie updates based on the current block number
/// and chain type (OP Stack or regular)
pub(crate) fn prune_persisted_trie_updates(&mut self) {
let retention_blocks = if self.engine_kind.is_opstack() {
OPSTACK_PERSISTED_TRIE_UPDATES_RETENTION
} else {
DEFAULT_PERSISTED_TRIE_UPDATES_RETENTION
};
let earliest_block_to_retain =
self.current_canonical_head.number.saturating_sub(retention_blocks);
self.persisted_trie_updates
.retain(|_, (block_number, _)| *block_number > earliest_block_to_retain);
}
/// Removes all blocks that are below the finalized block, as well as removing non-canonical
/// sidechains that fork from below the finalized block.
pub(crate) fn prune_finalized_sidechains(&mut self, finalized_num_hash: BlockNumHash) {
@@ -274,8 +221,6 @@ impl<N: NodePrimitives> TreeState<N> {
}
}
self.prune_persisted_trie_updates();
// The only block that should remain at the `finalized` number now, is the finalized
// block, if it exists.
//

View File

@@ -3,6 +3,7 @@ use crate::{
persistence::PersistenceAction,
tree::{
payload_validator::{BasicEngineValidator, TreeCtx, ValidationOutcome},
persistence_state::CurrentPersistenceAction,
TreeConfig,
},
};
@@ -26,7 +27,7 @@ use reth_ethereum_primitives::{Block, EthPrimitives};
use reth_evm_ethereum::MockEvmConfig;
use reth_primitives_traits::Block as _;
use reth_provider::{test_utils::MockEthProvider, ExecutionOutcome};
use reth_trie::HashedPostState;
use reth_trie::{updates::TrieUpdates, HashedPostState};
use std::{
collections::BTreeMap,
str::FromStr,
@@ -148,7 +149,7 @@ struct TestHarness {
>,
to_tree_tx: Sender<FromEngine<EngineApiRequest<EthEngineTypes, EthPrimitives>, Block>>,
from_tree_rx: UnboundedReceiver<EngineApiEvent>,
blocks: Vec<ExecutedBlockWithTrieUpdates>,
blocks: Vec<ExecutedBlock>,
action_rx: Receiver<PersistenceAction>,
block_builder: TestBlockBuilder,
provider: MockEthProvider,
@@ -228,7 +229,7 @@ impl TestHarness {
}
}
fn with_blocks(mut self, blocks: Vec<ExecutedBlockWithTrieUpdates>) -> Self {
fn with_blocks(mut self, blocks: Vec<ExecutedBlock>) -> Self {
let mut blocks_by_hash = HashMap::default();
let mut blocks_by_number = BTreeMap::new();
let mut state_by_hash = HashMap::default();
@@ -253,7 +254,6 @@ impl TestHarness {
blocks_by_number,
current_canonical_head: blocks.last().unwrap().recovered_block().num_hash(),
parent_to_child,
persisted_trie_updates: HashMap::default(),
engine_kind: EngineApiKind::Ethereum,
};
@@ -405,7 +405,6 @@ impl ValidatorTestHarness {
/// Configure `PersistenceState` for specific `PersistingKind` scenarios
fn start_persistence_operation(&mut self, action: CurrentPersistenceAction) {
use crate::tree::persistence_state::CurrentPersistenceAction;
use tokio::sync::oneshot;
// Create a dummy receiver for testing - it will never receive a value
@@ -828,25 +827,21 @@ fn test_tree_state_on_new_head_deep_fork() {
let chain_b = test_block_builder.create_fork(&last_block, 10);
for block in &chain_a {
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: ExecutedTrieUpdates::empty(),
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()),
});
}
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(ExecutedBlockWithTrieUpdates {
block: ExecutedBlock {
recovered_block: Arc::new(block.clone()),
execution_output: Arc::new(ExecutionOutcome::default()),
hashed_state: Arc::new(HashedPostState::default()),
},
trie: ExecutedTrieUpdates::empty(),
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()),
});
}

View File

@@ -24,7 +24,7 @@ impl<E, P> BackfillJobFactory<E, P> {
Self {
evm_config,
provider,
prune_modes: PruneModes::none(),
prune_modes: PruneModes::default(),
thresholds: ExecutionStageThresholds {
// Default duration for a database transaction to be considered long-lived is
// 60 seconds, so we limit the backfill job to the half of it to be sure we finish

View File

@@ -126,6 +126,7 @@ impl PruningArgs {
storage_history: Some(PruneMode::Distance(MINIMUM_PRUNING_DISTANCE)),
// TODO: set default to pre-merge block if available
bodies_history: None,
merkle_changesets: PruneMode::Distance(MINIMUM_PRUNING_DISTANCE),
receipts_log_filter: Default::default(),
},
}

View File

@@ -38,6 +38,11 @@ pub enum StageEnum {
///
/// Handles Merkle tree-related computations and data processing.
Merkle,
/// The merkle changesets stage within the pipeline.
///
/// Handles Merkle trie changesets for storage and accounts.
#[value(name = "merkle-changesets")]
MerkleChangeSets,
/// The transaction lookup stage within the pipeline.
///
/// Deals with the retrieval and processing of transactions.

View File

@@ -124,6 +124,7 @@ where
recovered_block: block.into(),
execution_output: Arc::new(execution_outcome),
hashed_state: Arc::new(hashed_state),
trie_updates: Arc::default(),
},
);
let pending_flashblock = PendingFlashBlock::new(

View File

@@ -11,7 +11,7 @@ use alloy_primitives::{B256, U256};
use alloy_rpc_types_debug::ExecutionWitness;
use alloy_rpc_types_engine::PayloadId;
use reth_basic_payload_builder::*;
use reth_chain_state::{ExecutedBlock, ExecutedBlockWithTrieUpdates, ExecutedTrieUpdates};
use reth_chain_state::ExecutedBlock;
use reth_chainspec::{ChainSpecProvider, EthChainSpec};
use reth_evm::{
execute::{
@@ -379,13 +379,11 @@ impl<Txs> OpBuilder<'_, Txs> {
);
// create the executed block data
let executed: ExecutedBlockWithTrieUpdates<N> = ExecutedBlockWithTrieUpdates {
block: ExecutedBlock {
recovered_block: Arc::new(block),
execution_output: Arc::new(execution_outcome),
hashed_state: Arc::new(hashed_state),
},
trie: ExecutedTrieUpdates::Present(Arc::new(trie_updates)),
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),
};
let no_tx_pool = ctx.attributes().no_tx_pool();

View File

@@ -16,7 +16,7 @@ use op_alloy_consensus::{encode_holocene_extra_data, encode_jovian_extra_data, E
use op_alloy_rpc_types_engine::{
OpExecutionPayloadEnvelopeV3, OpExecutionPayloadEnvelopeV4, OpExecutionPayloadV4,
};
use reth_chain_state::ExecutedBlockWithTrieUpdates;
use reth_chain_state::ExecutedBlock;
use reth_chainspec::EthChainSpec;
use reth_optimism_evm::OpNextBlockEnvAttributes;
use reth_optimism_forks::OpHardforks;
@@ -176,7 +176,7 @@ pub struct OpBuiltPayload<N: NodePrimitives = OpPrimitives> {
/// Sealed block
pub(crate) block: Arc<SealedBlock<N::Block>>,
/// Block execution data for the payload, if any.
pub(crate) executed_block: Option<ExecutedBlockWithTrieUpdates<N>>,
pub(crate) executed_block: Option<ExecutedBlock<N>>,
/// The fees of the block
pub(crate) fees: U256,
}
@@ -189,7 +189,7 @@ impl<N: NodePrimitives> OpBuiltPayload<N> {
id: PayloadId,
block: Arc<SealedBlock<N::Block>>,
fees: U256,
executed_block: Option<ExecutedBlockWithTrieUpdates<N>>,
executed_block: Option<ExecutedBlock<N>>,
) -> Self {
Self { id, block, fees, executed_block }
}
@@ -226,7 +226,7 @@ impl<N: NodePrimitives> BuiltPayload for OpBuiltPayload<N> {
self.fees
}
fn executed_block(&self) -> Option<ExecutedBlockWithTrieUpdates<N>> {
fn executed_block(&self) -> Option<ExecutedBlock<N>> {
self.executed_block.clone()
}

View File

@@ -9,7 +9,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::ExecutedBlockWithTrieUpdates;
use reth_chain_state::ExecutedBlock;
use reth_primitives_traits::{NodePrimitives, SealedBlock, SealedHeader};
/// Represents a successfully built execution payload (block).
@@ -30,7 +30,7 @@ pub trait BuiltPayload: Send + Sync + fmt::Debug {
/// Returns the complete execution result including state updates.
///
/// Returns `None` if execution data is not available or not tracked.
fn executed_block(&self) -> Option<ExecutedBlockWithTrieUpdates<Self::Primitives>> {
fn executed_block(&self) -> Option<ExecutedBlock<Self::Primitives>> {
None
}

View File

@@ -6,8 +6,8 @@ use reth_db_api::{table::Value, transaction::DbTxMut};
use reth_exex_types::FinishedExExHeight;
use reth_primitives_traits::NodePrimitives;
use reth_provider::{
providers::StaticFileProvider, BlockReader, DBProvider, DatabaseProviderFactory,
NodePrimitivesProvider, PruneCheckpointReader, PruneCheckpointWriter,
providers::StaticFileProvider, BlockReader, ChainStateBlockReader, DBProvider,
DatabaseProviderFactory, NodePrimitivesProvider, PruneCheckpointReader, PruneCheckpointWriter,
StaticFileProviderFactory,
};
use reth_prune_types::PruneModes;
@@ -83,6 +83,7 @@ impl PrunerBuilder {
ProviderRW: PruneCheckpointWriter
+ PruneCheckpointReader
+ BlockReader<Transaction: Encodable2718>
+ ChainStateBlockReader
+ StaticFileProviderFactory<
Primitives: NodePrimitives<SignedTx: Value, Receipt: Value, BlockHeader: Value>,
>,
@@ -113,6 +114,7 @@ impl PrunerBuilder {
Primitives: NodePrimitives<SignedTx: Value, Receipt: Value, BlockHeader: Value>,
> + DBProvider<Tx: DbTxMut>
+ BlockReader<Transaction: Encodable2718>
+ ChainStateBlockReader
+ PruneCheckpointWriter
+ PruneCheckpointReader,
{
@@ -132,7 +134,7 @@ impl Default for PrunerBuilder {
fn default() -> Self {
Self {
block_interval: 5,
segments: PruneModes::none(),
segments: PruneModes::default(),
delete_limit: MAINNET_PRUNE_DELETE_LIMIT,
timeout: None,
finished_exex_height: watch::channel(FinishedExExHeight::NoExExs).1,

View File

@@ -15,8 +15,8 @@ pub use static_file::{
use std::{fmt::Debug, ops::RangeInclusive};
use tracing::error;
pub use user::{
AccountHistory, Receipts as UserReceipts, ReceiptsByLogs, SenderRecovery, StorageHistory,
TransactionLookup,
AccountHistory, MerkleChangeSets, Receipts as UserReceipts, ReceiptsByLogs, SenderRecovery,
StorageHistory, TransactionLookup,
};
/// A segment represents a pruning of some portion of the data.

View File

@@ -1,13 +1,13 @@
use crate::segments::{
AccountHistory, ReceiptsByLogs, Segment, SenderRecovery, StorageHistory, TransactionLookup,
UserReceipts,
AccountHistory, MerkleChangeSets, ReceiptsByLogs, Segment, SenderRecovery, StorageHistory,
TransactionLookup, UserReceipts,
};
use alloy_eips::eip2718::Encodable2718;
use reth_db_api::{table::Value, transaction::DbTxMut};
use reth_primitives_traits::NodePrimitives;
use reth_provider::{
providers::StaticFileProvider, BlockReader, DBProvider, PruneCheckpointReader,
PruneCheckpointWriter, StaticFileProviderFactory,
providers::StaticFileProvider, BlockReader, ChainStateBlockReader, DBProvider,
PruneCheckpointReader, PruneCheckpointWriter, StaticFileProviderFactory,
};
use reth_prune_types::PruneModes;
@@ -52,7 +52,8 @@ where
> + DBProvider<Tx: DbTxMut>
+ PruneCheckpointWriter
+ PruneCheckpointReader
+ BlockReader<Transaction: Encodable2718>,
+ BlockReader<Transaction: Encodable2718>
+ ChainStateBlockReader,
{
/// Creates a [`SegmentSet`] from an existing components, such as [`StaticFileProvider`] and
/// [`PruneModes`].
@@ -67,6 +68,7 @@ where
account_history,
storage_history,
bodies_history: _,
merkle_changesets,
receipts_log_filter,
} = prune_modes;
@@ -77,6 +79,8 @@ where
.segment(StaticFileTransactions::new(static_file_provider.clone()))
// Static file receipts
.segment(StaticFileReceipts::new(static_file_provider))
// Merkle changesets
.segment(MerkleChangeSets::new(merkle_changesets))
// Account history
.segment_opt(account_history.map(AccountHistory::new))
// Storage history

View File

@@ -0,0 +1,116 @@
use crate::{
db_ext::DbTxPruneExt,
segments::{PruneInput, Segment},
PrunerError,
};
use alloy_primitives::B256;
use reth_db_api::{models::BlockNumberHashedAddress, table::Value, tables, transaction::DbTxMut};
use reth_primitives_traits::NodePrimitives;
use reth_provider::{
errors::provider::ProviderResult, BlockReader, ChainStateBlockReader, DBProvider,
NodePrimitivesProvider, PruneCheckpointWriter, TransactionsProvider,
};
use reth_prune_types::{
PruneCheckpoint, PruneMode, PrunePurpose, PruneSegment, SegmentOutput, SegmentOutputCheckpoint,
};
use tracing::{instrument, trace};
#[derive(Debug)]
pub struct MerkleChangeSets {
mode: PruneMode,
}
impl MerkleChangeSets {
pub const fn new(mode: PruneMode) -> Self {
Self { mode }
}
}
impl<Provider> Segment<Provider> for MerkleChangeSets
where
Provider: DBProvider<Tx: DbTxMut>
+ PruneCheckpointWriter
+ TransactionsProvider
+ BlockReader
+ ChainStateBlockReader
+ NodePrimitivesProvider<Primitives: NodePrimitives<Receipt: Value>>,
{
fn segment(&self) -> PruneSegment {
PruneSegment::MerkleChangeSets
}
fn mode(&self) -> Option<PruneMode> {
Some(self.mode)
}
fn purpose(&self) -> PrunePurpose {
PrunePurpose::User
}
#[instrument(level = "trace", target = "pruner", skip(self, provider), ret)]
fn prune(&self, provider: &Provider, input: PruneInput) -> Result<SegmentOutput, PrunerError> {
let Some(block_range) = input.get_next_block_range() else {
trace!(target: "pruner", "No change sets to prune");
return Ok(SegmentOutput::done())
};
let block_range_end = *block_range.end();
let mut limiter = input.limiter;
// Create range for StoragesTrieChangeSets which uses BlockNumberHashedAddress as key
let storage_range_start: BlockNumberHashedAddress =
(*block_range.start(), B256::ZERO).into();
let storage_range_end: BlockNumberHashedAddress =
(*block_range.end() + 1, B256::ZERO).into();
let storage_range = storage_range_start..storage_range_end;
let mut last_storages_pruned_block = None;
let (storages_pruned, done) =
provider.tx_ref().prune_table_with_range::<tables::StoragesTrieChangeSets>(
storage_range,
&mut limiter,
|_| false,
|(BlockNumberHashedAddress((block_number, _)), _)| {
last_storages_pruned_block = Some(block_number);
},
)?;
trace!(target: "pruner", %storages_pruned, %done, "Pruned storages change sets");
let mut last_accounts_pruned_block = block_range_end;
let last_storages_pruned_block = last_storages_pruned_block
// If there's more storage changesets to prune, set the checkpoint block number to
// previous, so we could finish pruning its storage changesets on the next run.
.map(|block_number| if done { block_number } else { block_number.saturating_sub(1) })
.unwrap_or(block_range_end);
let (accounts_pruned, done) =
provider.tx_ref().prune_table_with_range::<tables::AccountsTrieChangeSets>(
block_range,
&mut limiter,
|_| false,
|row| last_accounts_pruned_block = row.0,
)?;
trace!(target: "pruner", %accounts_pruned, %done, "Pruned accounts change sets");
let progress = limiter.progress(done);
Ok(SegmentOutput {
progress,
pruned: accounts_pruned + storages_pruned,
checkpoint: Some(SegmentOutputCheckpoint {
block_number: Some(last_storages_pruned_block.min(last_accounts_pruned_block)),
tx_number: None,
}),
})
}
fn save_checkpoint(
&self,
provider: &Provider,
checkpoint: PruneCheckpoint,
) -> ProviderResult<()> {
provider.save_prune_checkpoint(PruneSegment::MerkleChangeSets, checkpoint)
}
}

View File

@@ -1,5 +1,6 @@
mod account_history;
mod history;
mod merkle_change_sets;
mod receipts;
mod receipts_by_logs;
mod sender_recovery;
@@ -7,6 +8,7 @@ mod storage_history;
mod transaction_lookup;
pub use account_history::AccountHistory;
pub use merkle_change_sets::MerkleChangeSets;
pub use receipts::Receipts;
pub use receipts_by_logs::ReceiptsByLogs;
pub use sender_recovery::SenderRecovery;

View File

@@ -3,6 +3,9 @@ use derive_more::Display;
use thiserror::Error;
/// Segment of the data that can be pruned.
///
/// NOTE new variants must be added to the end of this enum. The variant index is encoded directly
/// when writing to the `PruneCheckpoint` table, so changing the order here will corrupt the table.
#[derive(Debug, Display, Clone, Copy, Eq, PartialEq, Ord, PartialOrd, Hash)]
#[cfg_attr(test, derive(arbitrary::Arbitrary))]
#[cfg_attr(any(test, feature = "reth-codec"), derive(reth_codecs::Compact))]
@@ -26,6 +29,9 @@ pub enum PruneSegment {
Headers,
/// Prune segment responsible for the `Transactions` table.
Transactions,
/// Prune segment responsible for all rows in `AccountsTrieChangeSets` and
/// `StoragesTrieChangeSets` table.
MerkleChangeSets,
}
#[cfg(test)]
@@ -44,9 +50,10 @@ impl PruneSegment {
0
}
Self::Receipts if purpose.is_static_file() => 0,
Self::ContractLogs | Self::AccountHistory | Self::StorageHistory => {
MINIMUM_PRUNING_DISTANCE
}
Self::ContractLogs |
Self::AccountHistory |
Self::StorageHistory |
Self::MerkleChangeSets |
Self::Receipts => MINIMUM_PRUNING_DISTANCE,
}
}

View File

@@ -36,8 +36,13 @@ pub enum HistoryType {
StorageHistory,
}
/// Default pruning mode for merkle changesets
const fn default_merkle_changesets_mode() -> PruneMode {
PruneMode::Distance(MINIMUM_PRUNING_DISTANCE)
}
/// Pruning configuration for every segment of the data that can be pruned.
#[derive(Debug, Clone, Default, Eq, PartialEq)]
#[derive(Debug, Clone, Eq, PartialEq)]
#[cfg_attr(any(test, feature = "serde"), derive(serde::Serialize, serde::Deserialize))]
#[cfg_attr(any(test, feature = "serde"), serde(default))]
pub struct PruneModes {
@@ -84,6 +89,16 @@ pub struct PruneModes {
)
)]
pub bodies_history: Option<PruneMode>,
/// Merkle Changesets pruning configuration for `AccountsTrieChangeSets` and
/// `StoragesTrieChangeSets`.
#[cfg_attr(
any(test, feature = "serde"),
serde(
default = "default_merkle_changesets_mode",
deserialize_with = "deserialize_prune_mode_with_min_blocks::<MINIMUM_PRUNING_DISTANCE, _>"
)
)]
pub merkle_changesets: PruneMode,
/// Receipts pruning configuration by retaining only those receipts that contain logs emitted
/// by the specified addresses, discarding others. This setting is overridden by `receipts`.
///
@@ -92,12 +107,22 @@ pub struct PruneModes {
pub receipts_log_filter: ReceiptsLogPruneConfig,
}
impl PruneModes {
/// Sets pruning to no target.
pub fn none() -> Self {
Self::default()
impl Default for PruneModes {
fn default() -> Self {
Self {
sender_recovery: None,
transaction_lookup: None,
receipts: None,
account_history: None,
storage_history: None,
bodies_history: None,
merkle_changesets: default_merkle_changesets_mode(),
receipts_log_filter: ReceiptsLogPruneConfig::default(),
}
}
}
impl PruneModes {
/// Sets pruning to all targets.
pub fn all() -> Self {
Self {
@@ -107,6 +132,7 @@ impl PruneModes {
account_history: Some(PruneMode::Full),
storage_history: Some(PruneMode::Full),
bodies_history: Some(PruneMode::Full),
merkle_changesets: PruneMode::Full,
receipts_log_filter: Default::default(),
}
}
@@ -116,11 +142,6 @@ impl PruneModes {
self.receipts.is_some() || !self.receipts_log_filter.is_empty()
}
/// Returns true if all prune modes are set to [`None`].
pub fn is_empty(&self) -> bool {
self == &Self::none()
}
/// Returns an error if we can't unwind to the targeted block because the target block is
/// outside the range.
///
@@ -170,6 +191,28 @@ impl PruneModes {
}
}
/// Deserializes [`PruneMode`] and validates that the value is not less than the const
/// generic parameter `MIN_BLOCKS`. This parameter represents the number of blocks that needs to be
/// left in database after the pruning.
///
/// 1. For [`PruneMode::Full`], it fails if `MIN_BLOCKS > 0`.
/// 2. For [`PruneMode::Distance`], it fails if `distance < MIN_BLOCKS + 1`. `+ 1` is needed because
/// `PruneMode::Distance(0)` means that we leave zero blocks from the latest, meaning we have one
/// block in the database.
#[cfg(any(test, feature = "serde"))]
fn deserialize_prune_mode_with_min_blocks<
'de,
const MIN_BLOCKS: u64,
D: serde::Deserializer<'de>,
>(
deserializer: D,
) -> Result<PruneMode, D::Error> {
use serde::Deserialize;
let prune_mode = PruneMode::deserialize(deserializer)?;
serde_deserialize_validate::<MIN_BLOCKS, D>(&prune_mode)?;
Ok(prune_mode)
}
/// Deserializes [`Option<PruneMode>`] and validates that the value is not less than the const
/// generic parameter `MIN_BLOCKS`. This parameter represents the number of blocks that needs to be
/// left in database after the pruning.
@@ -186,12 +229,21 @@ fn deserialize_opt_prune_mode_with_min_blocks<
>(
deserializer: D,
) -> Result<Option<PruneMode>, D::Error> {
use alloc::format;
use serde::Deserialize;
let prune_mode = Option::<PruneMode>::deserialize(deserializer)?;
if let Some(prune_mode) = prune_mode.as_ref() {
serde_deserialize_validate::<MIN_BLOCKS, D>(prune_mode)?;
}
Ok(prune_mode)
}
#[cfg(any(test, feature = "serde"))]
fn serde_deserialize_validate<'a, 'de, const MIN_BLOCKS: u64, D: serde::Deserializer<'de>>(
prune_mode: &'a PruneMode,
) -> Result<(), D::Error> {
use alloc::format;
match prune_mode {
Some(PruneMode::Full) if MIN_BLOCKS > 0 => {
PruneMode::Full if MIN_BLOCKS > 0 => {
Err(serde::de::Error::invalid_value(
serde::de::Unexpected::Str("full"),
// This message should have "expected" wording
@@ -199,15 +251,15 @@ fn deserialize_opt_prune_mode_with_min_blocks<
.as_str(),
))
}
Some(PruneMode::Distance(distance)) if distance < MIN_BLOCKS => {
PruneMode::Distance(distance) if *distance < MIN_BLOCKS => {
Err(serde::de::Error::invalid_value(
serde::de::Unexpected::Unsigned(distance),
serde::de::Unexpected::Unsigned(*distance),
// This message should have "expected" wording
&format!("prune mode that leaves at least {MIN_BLOCKS} blocks in the database")
.as_str(),
))
}
_ => Ok(prune_mode),
_ => Ok(()),
}
}
@@ -240,7 +292,7 @@ mod tests {
#[test]
fn test_unwind_target_unpruned() {
// Test case 1: No pruning configured - should always succeed
let prune_modes = PruneModes::none();
let prune_modes = PruneModes::default();
assert!(prune_modes.ensure_unwind_target_unpruned(1000, 500, &[]).is_ok());
assert!(prune_modes.ensure_unwind_target_unpruned(1000, 0, &[]).is_ok());

View File

@@ -11,9 +11,7 @@
use alloy_consensus::BlockHeader as _;
use alloy_primitives::{Bytes, B256};
use parking_lot::Mutex;
use reth_chain_state::{
ExecutedBlock, ExecutedBlockWithTrieUpdates, ExecutedTrieUpdates, MemoryOverlayStateProvider,
};
use reth_chain_state::{ExecutedBlock, MemoryOverlayStateProvider};
use reth_errors::{ProviderError, ProviderResult};
use reth_ethereum_primitives::{Block, BlockBody, EthPrimitives};
use reth_evm::{execute::Executor, ConfigureEvm};
@@ -125,10 +123,8 @@ where
self.pending_state.invalid_recovered_block(&ancestor_hash)
{
trace!(target: "reth::ress_provider", %block_hash, %ancestor_hash, "Using invalid ancestor block for witness construction");
executed = Some(ExecutedBlockWithTrieUpdates {
block: ExecutedBlock { recovered_block: invalid, ..Default::default() },
trie: ExecutedTrieUpdates::empty(),
});
executed =
Some(ExecutedBlock { recovered_block: invalid, ..Default::default() });
}
let Some(executed) = executed else {
@@ -162,14 +158,8 @@ where
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() {
if let Some(trie_updates) = block.trie.as_ref() {
trie_input.append_cached_ref(trie_updates, &block.hashed_state);
} else {
trace!(target: "reth::ress_provider", ancestor = ?block.recovered_block().num_hash(), "Missing trie updates for ancestor block");
return Err(ProviderError::TrieWitnessError(
"missing trie updates for ancestor".to_owned(),
));
}
let trie_updates = block.trie_updates.as_ref();
trie_input.append_cached_ref(trie_updates, &block.hashed_state);
}
let mut hashed_state = db.into_state();
hashed_state.extend(record.hashed_state);

View File

@@ -5,7 +5,7 @@ use alloy_primitives::{
};
use futures::StreamExt;
use parking_lot::RwLock;
use reth_chain_state::ExecutedBlockWithTrieUpdates;
use reth_chain_state::ExecutedBlock;
use reth_ethereum_primitives::EthPrimitives;
use reth_node_api::{ConsensusEngineEvent, NodePrimitives};
use reth_primitives_traits::{Bytecode, RecoveredBlock};
@@ -20,14 +20,14 @@ pub struct PendingState<N: NodePrimitives>(Arc<RwLock<PendingStateInner<N>>>);
#[derive(Default, Debug)]
struct PendingStateInner<N: NodePrimitives> {
blocks_by_hash: B256Map<ExecutedBlockWithTrieUpdates<N>>,
blocks_by_hash: B256Map<ExecutedBlock<N>>,
invalid_blocks_by_hash: B256Map<Arc<RecoveredBlock<N::Block>>>,
block_hashes_by_number: BTreeMap<BlockNumber, B256HashSet>,
}
impl<N: NodePrimitives> PendingState<N> {
/// Insert executed block with trie updates.
pub fn insert_block(&self, block: ExecutedBlockWithTrieUpdates<N>) {
pub fn insert_block(&self, block: ExecutedBlock<N>) {
let mut this = self.0.write();
let block_hash = block.recovered_block.hash();
this.block_hashes_by_number
@@ -46,13 +46,13 @@ impl<N: NodePrimitives> PendingState<N> {
}
/// Returns only valid executed blocks by hash.
pub fn executed_block(&self, hash: &B256) -> Option<ExecutedBlockWithTrieUpdates<N>> {
pub fn executed_block(&self, hash: &B256) -> Option<ExecutedBlock<N>> {
self.0.read().blocks_by_hash.get(hash).cloned()
}
/// Returns valid recovered block.
pub fn recovered_block(&self, hash: &B256) -> Option<Arc<RecoveredBlock<N::Block>>> {
self.executed_block(hash).map(|b| b.recovered_block.clone())
self.executed_block(hash).map(|b| b.recovered_block)
}
/// Returns invalid recovered block.

View File

@@ -359,7 +359,7 @@ pub trait LoadPendingBlock:
}
}
let BlockBuilderOutcome { execution_result, block, hashed_state, .. } =
let BlockBuilderOutcome { execution_result, block, hashed_state, trie_updates } =
builder.finish(NoopProvider::default()).map_err(Self::Error::from_eth_err)?;
let execution_outcome = ExecutionOutcome::new(
@@ -373,6 +373,7 @@ pub trait LoadPendingBlock:
recovered_block: block.into(),
execution_output: Arc::new(execution_outcome),
hashed_state: Arc::new(hashed_state),
trie_updates: Arc::new(trie_updates),
})
}
}

View File

@@ -9,9 +9,7 @@ use alloy_consensus::BlockHeader;
use alloy_eips::{BlockId, BlockNumberOrTag};
use alloy_primitives::{BlockHash, B256};
use derive_more::Constructor;
use reth_chain_state::{
BlockState, ExecutedBlock, ExecutedBlockWithTrieUpdates, ExecutedTrieUpdates,
};
use reth_chain_state::{BlockState, ExecutedBlock};
use reth_ethereum_primitives::Receipt;
use reth_evm::{ConfigureEvm, EvmEnvFor};
use reth_primitives_traits::{
@@ -135,11 +133,6 @@ impl<N: NodePrimitives> PendingBlock<N> {
impl<N: NodePrimitives> From<PendingBlock<N>> for BlockState<N> {
fn from(pending_block: PendingBlock<N>) -> Self {
Self::new(ExecutedBlockWithTrieUpdates::<N>::new(
pending_block.executed_block.recovered_block,
pending_block.executed_block.execution_output,
pending_block.executed_block.hashed_state,
ExecutedTrieUpdates::Missing,
))
Self::new(pending_block.executed_block)
}
}

View File

@@ -639,14 +639,18 @@ impl<N: ProviderNodeTypes> Pipeline<N> {
// FIXME: When handling errors, we do not commit the database transaction. This
// leads to the Merkle stage not clearing its checkpoint, and restarting from an
// invalid place.
let provider_rw = self.provider_factory.database_provider_rw()?;
provider_rw.save_stage_checkpoint_progress(StageId::MerkleExecute, vec![])?;
provider_rw.save_stage_checkpoint(
StageId::MerkleExecute,
prev_checkpoint.unwrap_or_default(),
)?;
// Only reset MerkleExecute checkpoint if MerkleExecute itself failed
if stage_id == StageId::MerkleExecute {
let provider_rw = self.provider_factory.database_provider_rw()?;
provider_rw
.save_stage_checkpoint_progress(StageId::MerkleExecute, vec![])?;
provider_rw.save_stage_checkpoint(
StageId::MerkleExecute,
prev_checkpoint.unwrap_or_default(),
)?;
provider_rw.commit()?;
provider_rw.commit()?;
}
// We unwind because of a validation error. If the unwind itself
// fails, we bail entirely,

View File

@@ -165,7 +165,7 @@ pub(crate) fn txs_testdata(num_blocks: u64) -> TestStageDB {
db.insert_changesets(transitions, None).unwrap();
let provider_rw = db.factory.provider_rw().unwrap();
provider_rw.write_trie_updates(&updates).unwrap();
provider_rw.write_trie_updates(updates).unwrap();
provider_rw.commit().unwrap();
let (transitions, final_state) = random_changeset_range(

View File

@@ -39,9 +39,9 @@
use crate::{
stages::{
AccountHashingStage, BodyStage, EraImportSource, EraStage, ExecutionStage, FinishStage,
HeaderStage, IndexAccountHistoryStage, IndexStorageHistoryStage, MerkleStage,
PruneSenderRecoveryStage, PruneStage, SenderRecoveryStage, StorageHashingStage,
TransactionLookupStage,
HeaderStage, IndexAccountHistoryStage, IndexStorageHistoryStage, MerkleChangeSets,
MerkleStage, PruneSenderRecoveryStage, PruneStage, SenderRecoveryStage,
StorageHashingStage, TransactionLookupStage,
},
StageSet, StageSetBuilder,
};
@@ -54,7 +54,7 @@ use reth_primitives_traits::{Block, NodePrimitives};
use reth_provider::HeaderSyncGapProvider;
use reth_prune_types::PruneModes;
use reth_stages_api::Stage;
use std::{ops::Not, sync::Arc};
use std::sync::Arc;
use tokio::sync::watch;
/// A set containing all stages to run a fully syncing instance of reth.
@@ -75,6 +75,7 @@ use tokio::sync::watch;
/// - [`AccountHashingStage`]
/// - [`StorageHashingStage`]
/// - [`MerkleStage`] (execute)
/// - [`MerkleChangeSets`]
/// - [`TransactionLookupStage`]
/// - [`IndexStorageHistoryStage`]
/// - [`IndexAccountHistoryStage`]
@@ -336,12 +337,12 @@ where
stages_config: self.stages_config.clone(),
prune_modes: self.prune_modes.clone(),
})
// If any prune modes are set, add the prune stage.
.add_stage_opt(self.prune_modes.is_empty().not().then(|| {
// Prune stage should be added after all hashing stages, because otherwise it will
// delete
PruneStage::new(self.prune_modes.clone(), self.stages_config.prune.commit_threshold)
}))
// Prune stage should be added after all hashing stages, because otherwise it will
// delete
.add_stage(PruneStage::new(
self.prune_modes.clone(),
self.stages_config.prune.commit_threshold,
))
}
}
@@ -387,6 +388,13 @@ where
}
/// A set containing all stages that hash account state.
///
/// This includes:
/// - [`MerkleStage`] (unwind)
/// - [`AccountHashingStage`]
/// - [`StorageHashingStage`]
/// - [`MerkleStage`] (execute)
/// - [`MerkleChangeSets`]
#[derive(Debug, Default)]
#[non_exhaustive]
pub struct HashingStages {
@@ -399,6 +407,7 @@ where
MerkleStage: Stage<Provider>,
AccountHashingStage: Stage<Provider>,
StorageHashingStage: Stage<Provider>,
MerkleChangeSets: Stage<Provider>,
{
fn builder(self) -> StageSetBuilder<Provider> {
StageSetBuilder::default()
@@ -415,6 +424,7 @@ where
self.stages_config.merkle.rebuild_threshold,
self.stages_config.merkle.incremental_threshold,
))
.add_stage(MerkleChangeSets::new())
}
}

View File

@@ -896,7 +896,7 @@ mod tests {
// If there is a pruning configuration, then it's forced to use the database.
// This way we test both cases.
let modes = [None, Some(PruneModes::none())];
let modes = [None, Some(PruneModes::default())];
let random_filter = ReceiptsLogPruneConfig(BTreeMap::from([(
Address::random(),
PruneMode::Distance(100000),
@@ -1033,7 +1033,7 @@ mod tests {
// If there is a pruning configuration, then it's forced to use the database.
// This way we test both cases.
let modes = [None, Some(PruneModes::none())];
let modes = [None, Some(PruneModes::default())];
let random_filter = ReceiptsLogPruneConfig(BTreeMap::from([(
Address::random(),
PruneMode::Before(100000),

View File

@@ -247,7 +247,7 @@ where
})?;
match progress {
StateRootProgress::Progress(state, hashed_entries_walked, updates) => {
provider.write_trie_updates(&updates)?;
provider.write_trie_updates(updates)?;
let mut checkpoint = MerkleCheckpoint::new(
to_block,
@@ -290,7 +290,7 @@ where
})
}
StateRootProgress::Complete(root, hashed_entries_walked, updates) => {
provider.write_trie_updates(&updates)?;
provider.write_trie_updates(updates)?;
entities_checkpoint.processed += hashed_entries_walked as u64;
@@ -317,7 +317,7 @@ where
error!(target: "sync::stages::merkle", %e, ?current_block_number, ?to_block, "Incremental state root failed! {INVALID_STATE_ROOT_ERROR_MESSAGE}");
StageError::Fatal(Box::new(e))
})?;
provider.write_trie_updates(&updates)?;
provider.write_trie_updates(updates)?;
final_root = Some(root);
}
@@ -400,7 +400,7 @@ where
validate_state_root(block_root, SealedHeader::seal_slow(target), input.unwind_to)?;
// Validation passed, apply unwind changes to the database.
provider.write_trie_updates(&updates)?;
provider.write_trie_updates(updates)?;
// Update entities checkpoint to reflect the unwind operation
// Since we're unwinding, we need to recalculate the total entities at the target block

View File

@@ -0,0 +1,380 @@
use crate::stages::merkle::INVALID_STATE_ROOT_ERROR_MESSAGE;
use alloy_consensus::BlockHeader;
use alloy_primitives::BlockNumber;
use reth_consensus::ConsensusError;
use reth_primitives_traits::{GotExpected, SealedHeader};
use reth_provider::{
ChainStateBlockReader, DBProvider, HeaderProvider, ProviderError, StageCheckpointReader,
TrieWriter,
};
use reth_stages_api::{
BlockErrorKind, CheckpointBlockRange, ExecInput, ExecOutput, MerkleChangeSetsCheckpoint, Stage,
StageCheckpoint, StageError, StageId, UnwindInput, UnwindOutput,
};
use reth_trie::{updates::TrieUpdates, HashedPostState, KeccakKeyHasher, StateRoot, TrieInput};
use reth_trie_db::{DatabaseHashedPostState, DatabaseStateRoot};
use std::ops::Range;
use tracing::{debug, error};
/// The `MerkleChangeSets` stage.
///
/// This stage processes and maintains trie changesets from the finalized block to the latest block.
#[derive(Debug, Clone)]
pub struct MerkleChangeSets {
/// The number of blocks to retain changesets for, used as a fallback when the finalized block
/// is not found. Defaults to 64 (2 epochs in beacon chain).
retention_blocks: u64,
}
impl MerkleChangeSets {
/// Creates a new `MerkleChangeSets` stage with default retention blocks of 64.
pub const fn new() -> Self {
Self { retention_blocks: 64 }
}
/// Creates a new `MerkleChangeSets` stage with a custom finalized block height.
pub const fn with_retention_blocks(retention_blocks: u64) -> Self {
Self { retention_blocks }
}
/// Returns the range of blocks which are already computed. Will return an empty range if none
/// have been computed.
fn computed_range(checkpoint: Option<StageCheckpoint>) -> Range<BlockNumber> {
let to = checkpoint.map(|chk| chk.block_number).unwrap_or_default();
let from = checkpoint
.map(|chk| chk.merkle_changesets_stage_checkpoint().unwrap_or_default())
.unwrap_or_default()
.block_range
.to;
from..to + 1
}
/// Determines the target range for changeset computation based on the checkpoint and provider
/// state.
///
/// Returns the target range (exclusive end) to compute changesets for.
fn determine_target_range<Provider>(
&self,
provider: &Provider,
) -> Result<Range<BlockNumber>, StageError>
where
Provider: StageCheckpointReader + ChainStateBlockReader,
{
// Get merkle checkpoint which represents our target end block
let merkle_checkpoint = provider
.get_stage_checkpoint(StageId::MerkleExecute)?
.map(|checkpoint| checkpoint.block_number)
.unwrap_or(0);
let target_end = merkle_checkpoint + 1; // exclusive
// Calculate the target range based on the finalized block and the target block.
// We maintain changesets from the finalized block to the latest block.
let finalized_block = provider.last_finalized_block_number()?;
// Calculate the fallback start position based on retention blocks
let retention_based_start = merkle_checkpoint.saturating_sub(self.retention_blocks);
// If the finalized block was way in the past then we don't want to generate changesets for
// all of those past blocks; we only care about the recent history.
//
// Use maximum of finalized_block and retention_based_start if finalized_block exists,
// otherwise just use retention_based_start.
let mut target_start = finalized_block
.map(|finalized| finalized.saturating_add(1).max(retention_based_start))
.unwrap_or(retention_based_start);
// We cannot revert the genesis block; target_start must be >0
target_start = target_start.max(1);
Ok(target_start..target_end)
}
/// Calculates the trie updates given a [`TrieInput`], asserting that the resulting state root
/// matches the expected one for the block.
fn calculate_block_trie_updates<Provider: DBProvider + HeaderProvider>(
provider: &Provider,
block_number: BlockNumber,
input: TrieInput,
) -> Result<TrieUpdates, StageError> {
let (root, trie_updates) =
StateRoot::overlay_root_from_nodes_with_updates(provider.tx_ref(), input).map_err(
|e| {
error!(
target: "sync::stages::merkle_changesets",
%e,
?block_number,
"Incremental state root failed! {INVALID_STATE_ROOT_ERROR_MESSAGE}");
StageError::Fatal(Box::new(e))
},
)?;
let block = provider
.header_by_number(block_number)?
.ok_or_else(|| ProviderError::HeaderNotFound(block_number.into()))?;
let (got, expected) = (root, block.state_root());
if got != expected {
// Only seal the header when we need it for the error
let header = SealedHeader::seal_slow(block);
error!(
target: "sync::stages::merkle_changesets",
?block_number,
?got,
?expected,
"Failed to verify block state root! {INVALID_STATE_ROOT_ERROR_MESSAGE}",
);
return Err(StageError::Block {
error: BlockErrorKind::Validation(ConsensusError::BodyStateRootDiff(
GotExpected { got, expected }.into(),
)),
block: Box::new(header.block_with_parent()),
})
}
Ok(trie_updates)
}
fn populate_range<Provider>(
provider: &Provider,
target_range: Range<BlockNumber>,
) -> Result<(), StageError>
where
Provider: StageCheckpointReader
+ TrieWriter
+ DBProvider
+ HeaderProvider
+ ChainStateBlockReader,
{
let target_start = target_range.start;
let target_end = target_range.end;
debug!(
target: "sync::stages::merkle_changesets",
?target_range,
"Starting trie changeset computation",
);
// We need to distinguish a cumulative revert and a per-block revert. A cumulative revert
// reverts changes starting at db tip all the way to a block. A per-block revert only
// reverts a block's changes.
//
// We need to calculate the cumulative HashedPostState reverts for every block in the
// target range. The cumulative HashedPostState revert for block N can be calculated as:
//
//
// ```
// // where `extend` overwrites any shared keys
// cumulative_state_revert(N) = cumulative_state_revert(N + 1).extend(get_block_state_revert(N))
// ```
//
// We need per-block reverts to calculate the prefix set for each individual block. By
// using the per-block reverts to calculate cumulative reverts on-the-fly we can save a
// bunch of memory.
debug!(
target: "sync::stages::merkle_changesets",
?target_range,
"Computing per-block state reverts",
);
let mut per_block_state_reverts = Vec::new();
for block_number in target_range.clone() {
per_block_state_reverts.push(HashedPostState::from_reverts::<KeccakKeyHasher>(
provider.tx_ref(),
block_number..=block_number,
)?);
}
// Helper to retrieve state revert data for a specific block from the pre-computed array
let get_block_state_revert = |block_number: BlockNumber| -> &HashedPostState {
let index = (block_number - target_start) as usize;
&per_block_state_reverts[index]
};
// Helper to accumulate state reverts from a given block to the target end
let compute_cumulative_state_revert = |block_number: BlockNumber| -> HashedPostState {
let mut cumulative_revert = HashedPostState::default();
for n in (block_number..target_end).rev() {
cumulative_revert.extend_ref(get_block_state_revert(n))
}
cumulative_revert
};
// To calculate the changeset for a block, we first need the TrieUpdates which are
// generated as a result of processing the block. To get these we need:
// 1) The TrieUpdates which revert the db's trie to _prior_ to the block
// 2) The HashedPostState to revert the db's state to _after_ the block
//
// To get (1) for `target_start` we need to do a big state root calculation which takes
// into account all changes between that block and db tip. For each block after the
// `target_start` we can update (1) using the TrieUpdates which were output by the previous
// block, only targeting the state changes of that block.
debug!(
target: "sync::stages::merkle_changesets",
?target_start,
"Computing trie state at starting block",
);
let mut input = TrieInput::default();
input.state = compute_cumulative_state_revert(target_start);
input.prefix_sets = input.state.construct_prefix_sets();
// target_start will be >= 1, see `determine_target_range`.
input.nodes =
Self::calculate_block_trie_updates(provider, target_start - 1, input.clone())?;
for block_number in target_range {
debug!(
target: "sync::stages::merkle_changesets",
?block_number,
"Computing trie updates for block",
);
// Revert the state so that this block has been just processed, meaning we take the
// cumulative revert of the subsequent block.
input.state = compute_cumulative_state_revert(block_number + 1);
// Construct prefix sets from only this block's `HashedPostState`, because we only care
// about trie updates which occurred as a result of this block being processed.
input.prefix_sets = get_block_state_revert(block_number).construct_prefix_sets();
// Calculate the trie updates for this block, then apply those updates to the reverts.
// We calculate the overlay which will be passed into the next step using the trie
// reverts prior to them being updated.
let this_trie_updates =
Self::calculate_block_trie_updates(provider, block_number, input.clone())?;
let trie_overlay = input.nodes.clone().into_sorted();
input.nodes.extend_ref(&this_trie_updates);
let this_trie_updates = this_trie_updates.into_sorted();
// Write the changesets to the DB using the trie updates produced by the block, and the
// trie reverts as the overlay.
debug!(
target: "sync::stages::merkle_changesets",
?block_number,
"Writing trie changesets for block",
);
provider.write_trie_changesets(
block_number,
&this_trie_updates,
Some(&trie_overlay),
)?;
}
Ok(())
}
}
impl Default for MerkleChangeSets {
fn default() -> Self {
Self::new()
}
}
impl<Provider> Stage<Provider> for MerkleChangeSets
where
Provider:
StageCheckpointReader + TrieWriter + DBProvider + HeaderProvider + ChainStateBlockReader,
{
fn id(&self) -> StageId {
StageId::MerkleChangeSets
}
fn execute(&mut self, provider: &Provider, input: ExecInput) -> Result<ExecOutput, StageError> {
// Get merkle checkpoint and assert that the target is the same.
let merkle_checkpoint = provider
.get_stage_checkpoint(StageId::MerkleExecute)?
.map(|checkpoint| checkpoint.block_number)
.unwrap_or(0);
if input.target.is_none_or(|target| merkle_checkpoint != target) {
return Err(StageError::Fatal(eyre::eyre!("Cannot sync stage to block {:?} when MerkleExecute is at block {merkle_checkpoint:?}", input.target).into()))
}
let mut target_range = self.determine_target_range(provider)?;
// Get the previously computed range. This will be updated to reflect the populating of the
// target range.
let mut computed_range = Self::computed_range(input.checkpoint);
// We want the target range to not include any data already computed previously, if
// possible, so we start the target range from the end of the computed range if that is
// greater.
//
// ------------------------------> Block #
// |------computed-----|
// |-----target-----|
// |--actual--|
//
// However, if the target start is less than the previously computed start, we don't want to
// do this, as it would leave a gap of data at `target_range.start..=computed_range.start`.
//
// ------------------------------> Block #
// |---computed---|
// |-------target-------|
// |-------actual-------|
//
if target_range.start >= computed_range.start {
target_range.start = target_range.start.max(computed_range.end);
}
// If target range is empty (target_start >= target_end), stage is already successfully
// executed
if target_range.start >= target_range.end {
return Ok(ExecOutput::done(input.checkpoint.unwrap_or_default()));
}
// If our target range is a continuation of the already computed range then we can keep the
// already computed data.
if target_range.start == computed_range.end {
// Clear from target_start onwards to ensure no stale data exists
provider.clear_trie_changesets_from(target_range.start)?;
computed_range.end = target_range.end;
} else {
// If our target range is not a continuation of the already computed range then we
// simply clear the computed data, to make sure there's no gaps or conflicts.
provider.clear_trie_changesets()?;
computed_range = target_range.clone();
}
// Populate the target range with changesets
Self::populate_range(provider, target_range)?;
let checkpoint_block_range = CheckpointBlockRange {
from: computed_range.start,
// CheckpointBlockRange is inclusive
to: computed_range.end.saturating_sub(1),
};
let checkpoint = StageCheckpoint::new(checkpoint_block_range.to)
.with_merkle_changesets_stage_checkpoint(MerkleChangeSetsCheckpoint {
block_range: checkpoint_block_range,
});
Ok(ExecOutput::done(checkpoint))
}
fn unwind(
&mut self,
provider: &Provider,
input: UnwindInput,
) -> Result<UnwindOutput, StageError> {
// Unwinding is trivial; just clear everything after the target block.
provider.clear_trie_changesets_from(input.unwind_to + 1)?;
let mut computed_range = Self::computed_range(Some(input.checkpoint));
computed_range.end = input.unwind_to + 1;
if computed_range.start > computed_range.end {
computed_range.start = computed_range.end;
}
let checkpoint_block_range = CheckpointBlockRange {
from: computed_range.start,
// computed_range.end is exclusive
to: computed_range.end.saturating_sub(1),
};
let checkpoint = StageCheckpoint::new(input.unwind_to)
.with_merkle_changesets_stage_checkpoint(MerkleChangeSetsCheckpoint {
block_range: checkpoint_block_range,
});
Ok(UnwindOutput { checkpoint })
}
}

View File

@@ -16,6 +16,8 @@ mod index_account_history;
mod index_storage_history;
/// Stage for computing state root.
mod merkle;
/// Stage for computing merkle changesets.
mod merkle_changesets;
mod prune;
/// The sender recovery stage.
mod sender_recovery;
@@ -32,6 +34,7 @@ pub use headers::*;
pub use index_account_history::*;
pub use index_storage_history::*;
pub use merkle::*;
pub use merkle_changesets::*;
pub use prune::*;
pub use sender_recovery::*;
pub use tx_lookup::*;
@@ -223,7 +226,7 @@ mod tests {
// In an unpruned configuration there is 1 receipt, 3 changed accounts and 1 changed
// storage.
let mut prune = PruneModes::none();
let mut prune = PruneModes::default();
check_pruning(test_db.factory.clone(), prune.clone(), 1, 3, 1).await;
prune.receipts = Some(PruneMode::Full);

View File

@@ -1,7 +1,7 @@
use reth_db_api::{table::Value, transaction::DbTxMut};
use reth_primitives_traits::NodePrimitives;
use reth_provider::{
BlockReader, DBProvider, PruneCheckpointReader, PruneCheckpointWriter,
BlockReader, ChainStateBlockReader, DBProvider, PruneCheckpointReader, PruneCheckpointWriter,
StaticFileProviderFactory,
};
use reth_prune::{
@@ -42,6 +42,7 @@ where
+ PruneCheckpointReader
+ PruneCheckpointWriter
+ BlockReader
+ ChainStateBlockReader
+ StaticFileProviderFactory<
Primitives: NodePrimitives<SignedTx: Value, Receipt: Value, BlockHeader: Value>,
>,
@@ -121,7 +122,7 @@ impl PruneSenderRecoveryStage {
/// Create new prune sender recovery stage with the given prune mode and commit threshold.
pub fn new(prune_mode: PruneMode, commit_threshold: usize) -> Self {
Self(PruneStage::new(
PruneModes { sender_recovery: Some(prune_mode), ..PruneModes::none() },
PruneModes { sender_recovery: Some(prune_mode), ..PruneModes::default() },
commit_threshold,
))
}
@@ -133,6 +134,7 @@ where
+ PruneCheckpointReader
+ PruneCheckpointWriter
+ BlockReader
+ ChainStateBlockReader
+ StaticFileProviderFactory<
Primitives: NodePrimitives<SignedTx: Value, Receipt: Value, BlockHeader: Value>,
>,

View File

@@ -287,6 +287,17 @@ pub struct IndexHistoryCheckpoint {
pub progress: EntitiesCheckpoint,
}
/// Saves the progress of `MerkleChangeSets` stage.
#[derive(Default, Debug, Copy, Clone, PartialEq, Eq)]
#[cfg_attr(any(test, feature = "test-utils"), derive(arbitrary::Arbitrary))]
#[cfg_attr(any(test, feature = "reth-codec"), derive(reth_codecs::Compact))]
#[cfg_attr(any(test, feature = "reth-codec"), reth_codecs::add_arbitrary_tests(compact))]
#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))]
pub struct MerkleChangeSetsCheckpoint {
/// Block range which this checkpoint is valid for.
pub block_range: CheckpointBlockRange,
}
/// Saves the progress of abstract stage iterating over or downloading entities.
#[derive(Debug, Default, PartialEq, Eq, Clone, Copy)]
#[cfg_attr(any(test, feature = "test-utils"), derive(arbitrary::Arbitrary))]
@@ -386,6 +397,9 @@ impl StageCheckpoint {
StageId::IndexStorageHistory | StageId::IndexAccountHistory => {
StageUnitCheckpoint::IndexHistory(IndexHistoryCheckpoint::default())
}
StageId::MerkleChangeSets => {
StageUnitCheckpoint::MerkleChangeSets(MerkleChangeSetsCheckpoint::default())
}
_ => return self,
});
_ = self.stage_checkpoint.map(|mut checkpoint| checkpoint.set_block_range(from, to));
@@ -411,6 +425,7 @@ impl StageCheckpoint {
progress: entities,
..
}) => Some(entities),
StageUnitCheckpoint::MerkleChangeSets(_) => None,
}
}
}
@@ -436,6 +451,8 @@ pub enum StageUnitCheckpoint {
Headers(HeadersCheckpoint),
/// Saves the progress of Index History stage.
IndexHistory(IndexHistoryCheckpoint),
/// Saves the progress of `MerkleChangeSets` stage.
MerkleChangeSets(MerkleChangeSetsCheckpoint),
}
impl StageUnitCheckpoint {
@@ -446,7 +463,8 @@ impl StageUnitCheckpoint {
Self::Account(AccountHashingCheckpoint { block_range, .. }) |
Self::Storage(StorageHashingCheckpoint { block_range, .. }) |
Self::Execution(ExecutionCheckpoint { block_range, .. }) |
Self::IndexHistory(IndexHistoryCheckpoint { block_range, .. }) => {
Self::IndexHistory(IndexHistoryCheckpoint { block_range, .. }) |
Self::MerkleChangeSets(MerkleChangeSetsCheckpoint { block_range, .. }) => {
let old_range = *block_range;
*block_range = CheckpointBlockRange { from, to };
@@ -544,6 +562,15 @@ stage_unit_checkpoints!(
index_history_stage_checkpoint,
/// Sets the stage checkpoint to index history.
with_index_history_stage_checkpoint
),
(
6,
MerkleChangeSets,
MerkleChangeSetsCheckpoint,
/// Returns the merkle changesets stage checkpoint, if any.
merkle_changesets_stage_checkpoint,
/// Sets the stage checkpoint to merkle changesets.
with_merkle_changesets_stage_checkpoint
)
);

View File

@@ -25,6 +25,7 @@ pub enum StageId {
TransactionLookup,
IndexStorageHistory,
IndexAccountHistory,
MerkleChangeSets,
Prune,
Finish,
/// Other custom stage with a provided string identifier.
@@ -39,7 +40,7 @@ static ENCODED_STAGE_IDS: OnceLock<HashMap<StageId, Vec<u8>>> = OnceLock::new();
impl StageId {
/// All supported Stages
pub const ALL: [Self; 15] = [
pub const ALL: [Self; 16] = [
Self::Era,
Self::Headers,
Self::Bodies,
@@ -53,6 +54,7 @@ impl StageId {
Self::TransactionLookup,
Self::IndexStorageHistory,
Self::IndexAccountHistory,
Self::MerkleChangeSets,
Self::Prune,
Self::Finish,
];
@@ -88,6 +90,7 @@ impl StageId {
Self::TransactionLookup => "TransactionLookup",
Self::IndexAccountHistory => "IndexAccountHistory",
Self::IndexStorageHistory => "IndexStorageHistory",
Self::MerkleChangeSets => "MerkleChangeSets",
Self::Prune => "Prune",
Self::Finish => "Finish",
Self::Other(s) => s,

View File

@@ -18,8 +18,8 @@ pub use id::StageId;
mod checkpoints;
pub use checkpoints::{
AccountHashingCheckpoint, CheckpointBlockRange, EntitiesCheckpoint, ExecutionCheckpoint,
HeadersCheckpoint, IndexHistoryCheckpoint, MerkleCheckpoint, StageCheckpoint,
StageUnitCheckpoint, StorageHashingCheckpoint, StorageRootMerkleCheckpoint,
HeadersCheckpoint, IndexHistoryCheckpoint, MerkleChangeSetsCheckpoint, MerkleCheckpoint,
StageCheckpoint, StageUnitCheckpoint, StorageHashingCheckpoint, StorageRootMerkleCheckpoint,
};
mod execution;

View File

@@ -87,7 +87,7 @@ pub trait DbDupCursorRO<T: DupSort> {
/// | `key` | `subkey` | **Equivalent starting position** |
/// |--------|----------|-----------------------------------------|
/// | `None` | `None` | [`DbCursorRO::first()`] |
/// | `Some` | `None` | [`DbCursorRO::seek()`] |
/// | `Some` | `None` | [`DbCursorRO::seek_exact()`] |
/// | `None` | `Some` | [`DbDupCursorRO::seek_by_key_subkey()`] |
/// | `Some` | `Some` | [`DbDupCursorRO::seek_by_key_subkey()`] |
fn walk_dup(

View File

@@ -176,7 +176,11 @@ impl Decode for AddressStorageKey {
}
}
impl_fixed_arbitrary!((BlockNumberAddress, 28), (AddressStorageKey, 52));
impl_fixed_arbitrary!(
(BlockNumberAddress, 28),
(BlockNumberHashedAddress, 40),
(AddressStorageKey, 52)
);
#[cfg(test)]
mod tests {
@@ -209,6 +213,31 @@ mod tests {
assert_eq!(bytes, Encode::encode(key));
}
#[test]
fn test_block_number_hashed_address() {
let num = 1u64;
let hash = B256::from_slice(&[0xba; 32]);
let key = BlockNumberHashedAddress((num, hash));
let mut bytes = [0u8; 40];
bytes[..8].copy_from_slice(&num.to_be_bytes());
bytes[8..].copy_from_slice(hash.as_slice());
let encoded = Encode::encode(key);
assert_eq!(encoded, bytes);
let decoded: BlockNumberHashedAddress = Decode::decode(&encoded).unwrap();
assert_eq!(decoded, key);
}
#[test]
fn test_block_number_hashed_address_rand() {
let mut bytes = [0u8; 40];
rng().fill(bytes.as_mut_slice());
let key = BlockNumberHashedAddress::arbitrary(&mut Unstructured::new(&bytes)).unwrap();
assert_eq!(bytes, Encode::encode(key));
}
#[test]
fn test_address_storage_key() {
let storage_key = StorageKey::random();

View File

@@ -12,7 +12,9 @@ use reth_ethereum_primitives::{Receipt, TransactionSigned, TxType};
use reth_primitives_traits::{Account, Bytecode, StorageEntry};
use reth_prune_types::{PruneCheckpoint, PruneSegment};
use reth_stages_types::StageCheckpoint;
use reth_trie_common::{StoredNibbles, StoredNibblesSubKey, *};
use reth_trie_common::{
StorageTrieEntry, StoredNibbles, StoredNibblesSubKey, TrieChangeSetsEntry, *,
};
use serde::{Deserialize, Serialize};
pub mod accounts;
@@ -219,6 +221,7 @@ impl_compression_for_compact!(
TxType,
StorageEntry,
BranchNodeCompact,
TrieChangeSetsEntry,
StoredNibbles,
StoredNibblesSubKey,
StorageTrieEntry,

View File

@@ -21,8 +21,8 @@ use crate::{
accounts::BlockNumberAddress,
blocks::{HeaderHash, StoredBlockOmmers},
storage_sharded_key::StorageShardedKey,
AccountBeforeTx, ClientVersion, CompactU256, IntegerList, ShardedKey,
StoredBlockBodyIndices, StoredBlockWithdrawals,
AccountBeforeTx, BlockNumberHashedAddress, ClientVersion, CompactU256, IntegerList,
ShardedKey, StoredBlockBodyIndices, StoredBlockWithdrawals,
},
table::{Decode, DupSort, Encode, Table, TableInfo},
};
@@ -32,7 +32,9 @@ use reth_ethereum_primitives::{Receipt, TransactionSigned};
use reth_primitives_traits::{Account, Bytecode, StorageEntry};
use reth_prune_types::{PruneCheckpoint, PruneSegment};
use reth_stages_types::StageCheckpoint;
use reth_trie_common::{BranchNodeCompact, StorageTrieEntry, StoredNibbles, StoredNibblesSubKey};
use reth_trie_common::{
BranchNodeCompact, StorageTrieEntry, StoredNibbles, StoredNibblesSubKey, TrieChangeSetsEntry,
};
use serde::{Deserialize, Serialize};
use std::fmt;
@@ -486,6 +488,20 @@ tables! {
type SubKey = StoredNibblesSubKey;
}
/// Stores the state of a node in the accounts trie prior to a particular block being executed.
table AccountsTrieChangeSets {
type Key = BlockNumber;
type Value = TrieChangeSetsEntry;
type SubKey = StoredNibblesSubKey;
}
/// Stores the state of a node in a storage trie prior to a particular block being executed.
table StoragesTrieChangeSets {
type Key = BlockNumberHashedAddress;
type Value = TrieChangeSetsEntry;
type SubKey = StoredNibblesSubKey;
}
/// Stores the transaction sender for each canonical transaction.
/// It is needed to speed up execution stage and allows fetching signer without doing
/// transaction signed recovery

View File

@@ -602,7 +602,7 @@ where
match state_root.root_with_progress()? {
StateRootProgress::Progress(state, _, updates) => {
let updated_len = provider.write_trie_updates(&updates)?;
let updated_len = provider.write_trie_updates(updates)?;
total_flushed_updates += updated_len;
trace!(target: "reth::cli",
@@ -622,7 +622,7 @@ where
}
}
StateRootProgress::Complete(root, _, updates) => {
let updated_len = provider.write_trie_updates(&updates)?;
let updated_len = provider.write_trie_updates(updates)?;
total_flushed_updates += updated_len;
trace!(target: "reth::cli",

View File

@@ -137,6 +137,14 @@ pub enum ProviderError {
/// Missing trie updates.
#[error("missing trie updates for block {0}")]
MissingTrieUpdates(B256),
/// Insufficient changesets to revert to the requested block.
#[error("insufficient changesets to revert to block #{requested}. Available changeset range: {available:?}")]
InsufficientChangesets {
/// The block number requested for reversion
requested: BlockNumber,
/// The available range of blocks with changesets
available: core::ops::RangeInclusive<BlockNumber>,
},
/// Any other error type wrapped into a cloneable [`AnyError`].
#[error(transparent)]
Other(#[from] AnyError),

View File

@@ -1,5 +0,0 @@
//! Bundle state module.
//! This module contains all the logic related to bundle state.
mod state_reverts;
pub use state_reverts::StorageRevertsIter;

View File

@@ -0,0 +1,7 @@
//! This module contains helpful utilities related to populating changesets tables.
mod state_reverts;
pub use state_reverts::StorageRevertsIter;
mod trie;
pub use trie::*;

View File

@@ -0,0 +1,147 @@
use itertools::{merge_join_by, EitherOrBoth};
use reth_db_api::DatabaseError;
use reth_trie::{trie_cursor::TrieCursor, BranchNodeCompact, Nibbles};
use std::cmp::{Ord, Ordering};
/// Combines a sorted iterator of trie node paths and a storage trie cursor into a new
/// iterator which produces the current values of all given paths in the same order.
#[derive(Debug)]
pub struct StorageTrieCurrentValuesIter<'cursor, P, C> {
/// Sorted iterator of node paths which we want the values of.
paths: P,
/// Storage trie cursor.
cursor: &'cursor mut C,
/// Current value at the cursor, allows us to treat the cursor as a peekable iterator.
cursor_current: Option<(Nibbles, BranchNodeCompact)>,
}
impl<'cursor, P, C> StorageTrieCurrentValuesIter<'cursor, P, C>
where
P: Iterator<Item = Nibbles>,
C: TrieCursor,
{
/// Instantiate a [`StorageTrieCurrentValuesIter`] from a sorted paths iterator and a cursor.
pub fn new(paths: P, cursor: &'cursor mut C) -> Result<Self, DatabaseError> {
let mut new_self = Self { paths, cursor, cursor_current: None };
new_self.seek_cursor(Nibbles::default())?;
Ok(new_self)
}
fn seek_cursor(&mut self, path: Nibbles) -> Result<(), DatabaseError> {
self.cursor_current = self.cursor.seek(path)?;
Ok(())
}
}
impl<'cursor, P, C> Iterator for StorageTrieCurrentValuesIter<'cursor, P, C>
where
P: Iterator<Item = Nibbles>,
C: TrieCursor,
{
type Item = Result<(Nibbles, Option<BranchNodeCompact>), DatabaseError>;
fn next(&mut self) -> Option<Self::Item> {
let Some(curr_path) = self.paths.next() else {
// If there are no more paths then there is no further possible output.
return None
};
// If the path is ahead of the cursor then seek the cursor forward to catch up. The cursor
// will seek either to `curr_path` or beyond it.
if self.cursor_current.as_ref().is_some_and(|(cursor_path, _)| curr_path > *cursor_path) &&
let Err(err) = self.seek_cursor(curr_path)
{
return Some(Err(err))
}
// If there is a path but the cursor is empty then that path has no node.
if self.cursor_current.is_none() {
return Some(Ok((curr_path, None)))
}
let (cursor_path, cursor_node) =
self.cursor_current.as_mut().expect("already checked for None");
// There is both a path and a cursor value, compare their paths.
match curr_path.cmp(cursor_path) {
Ordering::Less => {
// If the path is behind the cursor then there is no value for that
// path, produce None.
Some(Ok((curr_path, None)))
}
Ordering::Equal => {
// If the target path and cursor's path match then there is a value for that path,
// return the value. We don't seek the cursor here, that will be handled on the
// next call to `next` after checking that `paths` isn't None.
let cursor_node = core::mem::take(cursor_node);
Some(Ok((*cursor_path, Some(cursor_node))))
}
Ordering::Greater => {
panic!("cursor was seeked to {curr_path:?}, but produced a node at a lower path {cursor_path:?}")
}
}
}
}
/// Returns an iterator which produces the values to be inserted into the `StoragesTrieChangeSets`
/// table for an account whose storage was wiped during a block. It is expected that this is called
/// prior to inserting the block's trie updates.
///
/// ## Arguments
///
/// - `curr_values_of_changed` is an iterator over the current values of all trie nodes modified by
/// the block, ordered by path.
/// - `all_nodes` is an iterator over all existing trie nodes for the account, ordered by path.
///
/// ## Returns
///
/// An iterator of trie node paths and a `Some(node)` (indicating the node was wiped) or a `None`
/// (indicating the node was modified in the block but didn't previously exist. The iterator's
/// results will be ordered by path.
pub fn storage_trie_wiped_changeset_iter(
curr_values_of_changed: impl Iterator<
Item = Result<(Nibbles, Option<BranchNodeCompact>), DatabaseError>,
>,
all_nodes: impl Iterator<Item = Result<(Nibbles, BranchNodeCompact), DatabaseError>>,
) -> Result<
impl Iterator<Item = Result<(Nibbles, Option<BranchNodeCompact>), DatabaseError>>,
DatabaseError,
> {
let all_nodes = all_nodes.map(|e| e.map(|(nibbles, node)| (nibbles, Some(node))));
let merged = merge_join_by(curr_values_of_changed, all_nodes, |a, b| match (a, b) {
(Err(_), _) => Ordering::Less,
(_, Err(_)) => Ordering::Greater,
(Ok(a), Ok(b)) => a.0.cmp(&b.0),
});
Ok(merged.map(|either_or| match either_or {
EitherOrBoth::Left(changed) => {
// A path of a changed node (given in `paths`) which was not found in the database (or
// there's an error). The current value of this path must be None, otherwise it would
// have also been returned by the `all_nodes` iter.
debug_assert!(
changed.as_ref().is_err() || changed.as_ref().is_ok_and(|(_, node)| node.is_none()),
"changed node is Some but wasn't returned by `all_nodes` iterator: {changed:?}",
);
changed
}
EitherOrBoth::Right(wiped) => {
// A node was found in the db (indicating it was wiped) but was not given in `paths`.
// Return it as-is.
wiped
}
EitherOrBoth::Both(changed, _wiped) => {
// A path of a changed node (given in `paths`) was found with a previous value in the
// database. The changed node must have a value which is equal to the one found by the
// `all_nodes` iterator. If the changed node had no previous value (None) it wouldn't
// be returned by `all_nodes` and so would be in the Left branch.
//
// Due to the ordering closure passed to `merge_join_by` it's not possible for either
// value to be an error here.
debug_assert!(changed.is_ok(), "unreachable error condition: {changed:?}");
debug_assert_eq!(changed, _wiped);
changed
}
}))
}

View File

@@ -35,7 +35,7 @@ pub use static_file::StaticFileSegment;
pub use reth_execution_types::*;
pub mod bundle_state;
pub mod changesets_utils;
/// Re-export `OriginalValuesKnown`
pub use revm_database::states::OriginalValuesKnown;

View File

@@ -6,7 +6,7 @@ use crate::{
HashedPostStateProvider, HeaderProvider, ProviderError, ProviderFactory, PruneCheckpointReader,
ReceiptProvider, ReceiptProviderIdExt, StageCheckpointReader, StateProviderBox,
StateProviderFactory, StateReader, StaticFileProviderFactory, TransactionVariant,
TransactionsProvider,
TransactionsProvider, TrieReader,
};
use alloy_consensus::transaction::TransactionMeta;
use alloy_eips::{BlockHashOrNumber, BlockId, BlockNumHash, BlockNumberOrTag};
@@ -25,7 +25,7 @@ use reth_prune_types::{PruneCheckpoint, PruneSegment};
use reth_stages_types::{StageCheckpoint, StageId};
use reth_storage_api::{BlockBodyIndicesProvider, NodePrimitivesProvider, StorageChangeSetReader};
use reth_storage_errors::provider::ProviderResult;
use reth_trie::{HashedPostState, KeccakKeyHasher};
use reth_trie::{updates::TrieUpdatesSorted, HashedPostState, KeccakKeyHasher};
use revm_database::BundleState;
use std::{
ops::{RangeBounds, RangeInclusive},
@@ -739,6 +739,19 @@ impl<N: ProviderNodeTypes> StateReader for BlockchainProvider<N> {
}
}
impl<N: ProviderNodeTypes> TrieReader for BlockchainProvider<N> {
fn trie_reverts(&self, from: BlockNumber) -> ProviderResult<TrieUpdatesSorted> {
self.consistent_provider()?.trie_reverts(from)
}
fn get_block_trie_updates(
&self,
block_number: BlockNumber,
) -> ProviderResult<TrieUpdatesSorted> {
self.consistent_provider()?.get_block_trie_updates(block_number)
}
}
#[cfg(test)]
mod tests {
use crate::{
@@ -755,8 +768,7 @@ mod tests {
use rand::Rng;
use reth_chain_state::{
test_utils::TestBlockBuilder, CanonStateNotification, CanonStateSubscriptions,
CanonicalInMemoryState, ExecutedBlock, ExecutedBlockWithTrieUpdates, ExecutedTrieUpdates,
NewCanonicalChain,
CanonicalInMemoryState, ExecutedBlock, NewCanonicalChain,
};
use reth_chainspec::{ChainSpec, MAINNET};
use reth_db_api::models::{AccountBeforeTx, StoredBlockBodyIndices};
@@ -882,12 +894,14 @@ mod tests {
let execution_outcome =
ExecutionOutcome { receipts: vec![block_receipts], ..Default::default() };
ExecutedBlockWithTrieUpdates::new(
Arc::new(RecoveredBlock::new_sealed(block.clone(), senders)),
execution_outcome.into(),
Default::default(),
ExecutedTrieUpdates::empty(),
)
ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(
block.clone(),
senders,
)),
execution_output: execution_outcome.into(),
..Default::default()
}
})
.collect(),
};
@@ -1009,15 +1023,13 @@ mod tests {
let in_memory_block_senders =
first_in_mem_block.senders().expect("failed to recover senders");
let chain = NewCanonicalChain::Commit {
new: vec![ExecutedBlockWithTrieUpdates::new(
Arc::new(RecoveredBlock::new_sealed(
new: vec![ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(
first_in_mem_block.clone(),
in_memory_block_senders,
)),
Default::default(),
Default::default(),
ExecutedTrieUpdates::empty(),
)],
..Default::default()
}],
};
provider.canonical_in_memory_state.update_chain(chain);
@@ -1045,16 +1057,12 @@ 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(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: ExecutedTrieUpdates::empty(),
provider.canonical_in_memory_state.set_pending_block(ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(
last_in_mem_block.clone(),
Default::default(),
)),
..Default::default()
});
// Now the last block should be found in memory
@@ -1105,15 +1113,13 @@ mod tests {
let in_memory_block_senders =
first_in_mem_block.senders().expect("failed to recover senders");
let chain = NewCanonicalChain::Commit {
new: vec![ExecutedBlockWithTrieUpdates::new(
Arc::new(RecoveredBlock::new_sealed(
new: vec![ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(
first_in_mem_block.clone(),
in_memory_block_senders,
)),
Default::default(),
Default::default(),
ExecutedTrieUpdates::empty(),
)],
..Default::default()
}],
};
provider.canonical_in_memory_state.update_chain(chain);
@@ -1159,16 +1165,12 @@ mod tests {
);
// Set the block as pending
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: ExecutedTrieUpdates::empty(),
provider.canonical_in_memory_state.set_pending_block(ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(
block.clone(),
block.senders().unwrap(),
)),
..Default::default()
});
// Assertions related to the pending block
@@ -1206,15 +1208,13 @@ mod tests {
let in_memory_block_senders =
first_in_mem_block.senders().expect("failed to recover senders");
let chain = NewCanonicalChain::Commit {
new: vec![ExecutedBlockWithTrieUpdates::new(
Arc::new(RecoveredBlock::new_sealed(
new: vec![ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(
first_in_mem_block.clone(),
in_memory_block_senders,
)),
Default::default(),
Default::default(),
ExecutedTrieUpdates::empty(),
)],
..Default::default()
}],
};
provider.canonical_in_memory_state.update_chain(chain);
@@ -1686,9 +1686,12 @@ mod tests {
.first()
.map(|block| {
let senders = block.senders().expect("failed to recover senders");
ExecutedBlockWithTrieUpdates::new(
Arc::new(RecoveredBlock::new_sealed(block.clone(), senders)),
Arc::new(ExecutionOutcome {
ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(
block.clone(),
senders,
)),
execution_output: Arc::new(ExecutionOutcome {
bundle: BundleState::new(
in_memory_state.into_iter().map(|(address, (account, _))| {
(address, None, Some(account.into()), Default::default())
@@ -1701,9 +1704,8 @@ mod tests {
first_block: first_in_memory_block,
..Default::default()
}),
Default::default(),
ExecutedTrieUpdates::empty(),
)
..Default::default()
}
})
.unwrap()],
};
@@ -1821,19 +1823,13 @@ 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(
ExecutedBlockWithTrieUpdates {
block: ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(
pending_block.clone(),
Default::default(),
)),
execution_output: Default::default(),
hashed_state: Default::default(),
},
trie: ExecutedTrieUpdates::empty(),
},
);
only_database_provider.canonical_in_memory_state.set_pending_block(ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(
pending_block.clone(),
Default::default(),
)),
..Default::default()
});
assert_eq!(
pending_block.hash(),
@@ -1919,16 +1915,12 @@ mod tests {
// Set the pending block in memory
let pending_block = in_memory_blocks.last().unwrap();
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: ExecutedTrieUpdates::empty(),
provider.canonical_in_memory_state.set_pending_block(ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(
pending_block.clone(),
Default::default(),
)),
..Default::default()
});
// Set the safe block in memory

View File

@@ -4,7 +4,7 @@ use crate::{
BlockReader, BlockReaderIdExt, BlockSource, ChainSpecProvider, ChangeSetReader, HeaderProvider,
ProviderError, PruneCheckpointReader, ReceiptProvider, ReceiptProviderIdExt,
StageCheckpointReader, StateReader, StaticFileProviderFactory, TransactionVariant,
TransactionsProvider,
TransactionsProvider, TrieReader,
};
use alloy_consensus::{transaction::TransactionMeta, BlockHeader};
use alloy_eips::{
@@ -28,6 +28,7 @@ use reth_storage_api::{
StorageChangeSetReader, TryIntoHistoricalStateProvider,
};
use reth_storage_errors::provider::ProviderResult;
use reth_trie::updates::TrieUpdatesSorted;
use revm_database::states::PlainStorageRevert;
use std::{
ops::{Add, Bound, RangeBounds, RangeInclusive, Sub},
@@ -1504,6 +1505,19 @@ impl<N: ProviderNodeTypes> StateReader for ConsistentProvider<N> {
}
}
impl<N: ProviderNodeTypes> TrieReader for ConsistentProvider<N> {
fn trie_reverts(&self, from: BlockNumber) -> ProviderResult<TrieUpdatesSorted> {
self.storage_provider.trie_reverts(from)
}
fn get_block_trie_updates(
&self,
block_number: BlockNumber,
) -> ProviderResult<TrieUpdatesSorted> {
self.storage_provider.get_block_trie_updates(block_number)
}
}
#[cfg(test)]
mod tests {
use crate::{
@@ -1514,9 +1528,7 @@ mod tests {
use alloy_primitives::B256;
use itertools::Itertools;
use rand::Rng;
use reth_chain_state::{
ExecutedBlock, ExecutedBlockWithTrieUpdates, ExecutedTrieUpdates, NewCanonicalChain,
};
use reth_chain_state::{ExecutedBlock, NewCanonicalChain};
use reth_db_api::models::AccountBeforeTx;
use reth_ethereum_primitives::Block;
use reth_execution_types::ExecutionOutcome;
@@ -1619,15 +1631,13 @@ mod tests {
let in_memory_block_senders =
first_in_mem_block.senders().expect("failed to recover senders");
let chain = NewCanonicalChain::Commit {
new: vec![ExecutedBlockWithTrieUpdates::new(
Arc::new(RecoveredBlock::new_sealed(
new: vec![ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(
first_in_mem_block.clone(),
in_memory_block_senders,
)),
Default::default(),
Default::default(),
ExecutedTrieUpdates::empty(),
)],
..Default::default()
}],
};
consistent_provider.canonical_in_memory_state.update_chain(chain);
let consistent_provider = provider.consistent_provider()?;
@@ -1661,16 +1671,12 @@ mod tests {
);
// Insert the last block into the pending state
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: ExecutedTrieUpdates::empty(),
provider.canonical_in_memory_state.set_pending_block(ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(
last_in_mem_block.clone(),
Default::default(),
)),
..Default::default()
});
// Now the last block should be found in memory
@@ -1729,15 +1735,13 @@ mod tests {
let in_memory_block_senders =
first_in_mem_block.senders().expect("failed to recover senders");
let chain = NewCanonicalChain::Commit {
new: vec![ExecutedBlockWithTrieUpdates::new(
Arc::new(RecoveredBlock::new_sealed(
new: vec![ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(
first_in_mem_block.clone(),
in_memory_block_senders,
)),
Default::default(),
Default::default(),
ExecutedTrieUpdates::empty(),
)],
..Default::default()
}],
};
consistent_provider.canonical_in_memory_state.update_chain(chain);
@@ -1834,9 +1838,12 @@ mod tests {
.first()
.map(|block| {
let senders = block.senders().expect("failed to recover senders");
ExecutedBlockWithTrieUpdates::new(
Arc::new(RecoveredBlock::new_sealed(block.clone(), senders)),
Arc::new(ExecutionOutcome {
ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(
block.clone(),
senders,
)),
execution_output: Arc::new(ExecutionOutcome {
bundle: BundleState::new(
in_memory_state.into_iter().map(|(address, (account, _))| {
(address, None, Some(account.into()), Default::default())
@@ -1849,9 +1856,8 @@ mod tests {
first_block: first_in_memory_block,
..Default::default()
}),
Default::default(),
ExecutedTrieUpdates::empty(),
)
..Default::default()
}
})
.unwrap()],
};

View File

@@ -84,7 +84,7 @@ impl<N: NodeTypesWithDB> ProviderFactory<N> {
db,
chain_spec,
static_file_provider,
prune_modes: PruneModes::none(),
prune_modes: PruneModes::default(),
storage: Default::default(),
}
}
@@ -131,7 +131,7 @@ impl<N: NodeTypesWithDB<DB = Arc<DatabaseEnv>>> ProviderFactory<N> {
db: Arc::new(init_db(path, args).map_err(RethError::msg)?),
chain_spec,
static_file_provider,
prune_modes: PruneModes::none(),
prune_modes: PruneModes::default(),
storage: Default::default(),
})
}
@@ -670,7 +670,7 @@ mod tests {
let prune_modes = PruneModes {
sender_recovery: Some(PruneMode::Full),
transaction_lookup: Some(PruneMode::Full),
..PruneModes::none()
..PruneModes::default()
};
let factory = create_test_provider_factory();
let provider = factory.with_prune_modes(prune_modes).provider_rw().unwrap();

File diff suppressed because it is too large Load Diff

View File

@@ -17,7 +17,7 @@ mod state;
pub use state::{
historical::{HistoricalStateProvider, HistoricalStateProviderRef, LowestAvailableBlocks},
latest::{LatestStateProvider, LatestStateProviderRef},
overlay::OverlayStateProvider,
overlay::{OverlayStateProvider, OverlayStateProviderFactory},
};
mod consistent_view;

View File

@@ -1,15 +1,143 @@
use alloy_primitives::B256;
use alloy_primitives::{BlockNumber, B256};
use reth_db_api::DatabaseError;
use reth_storage_api::DBProvider;
use reth_errors::ProviderError;
use reth_stages_types::StageId;
use reth_storage_api::{DBProvider, DatabaseProviderFactory, StageCheckpointReader, TrieReader};
use reth_trie::{
hashed_cursor::{HashedCursorFactory, HashedPostStateCursorFactory},
trie_cursor::{InMemoryTrieCursorFactory, TrieCursorFactory},
updates::TrieUpdatesSorted,
HashedPostStateSorted,
HashedPostState, HashedPostStateSorted, KeccakKeyHasher,
};
use reth_trie_db::{
DatabaseHashedCursorFactory, DatabaseHashedPostState, DatabaseTrieCursorFactory,
};
use reth_trie_db::{DatabaseHashedCursorFactory, DatabaseTrieCursorFactory};
use std::sync::Arc;
/// Factory for creating overlay state providers with optional reverts and overlays.
///
/// This factory allows building an `OverlayStateProvider` whose DB state has been reverted to a
/// particular block, and/or with additional overlay information added on top.
#[derive(Debug, Clone)]
pub struct OverlayStateProviderFactory<F> {
/// The underlying database provider factory
factory: F,
/// Optional block number for collecting reverts
block_number: Option<BlockNumber>,
/// Optional trie overlay
trie_overlay: Option<Arc<TrieUpdatesSorted>>,
/// Optional hashed state overlay
hashed_state_overlay: Option<Arc<HashedPostStateSorted>>,
}
impl<F> OverlayStateProviderFactory<F>
where
F: DatabaseProviderFactory,
F::Provider: Clone + TrieReader + StageCheckpointReader,
{
/// Create a new overlay state provider factory
pub const fn new(factory: F) -> Self {
Self { factory, block_number: None, trie_overlay: None, hashed_state_overlay: None }
}
/// Set the block number for collecting reverts
pub const fn with_block_number(mut self, block_number: Option<BlockNumber>) -> Self {
self.block_number = block_number;
self
}
/// Set the trie overlay
pub fn with_trie_overlay(mut self, trie_overlay: Option<Arc<TrieUpdatesSorted>>) -> Self {
self.trie_overlay = trie_overlay;
self
}
/// Set the hashed state overlay
pub fn with_hashed_state_overlay(
mut self,
hashed_state_overlay: Option<Arc<HashedPostStateSorted>>,
) -> Self {
self.hashed_state_overlay = hashed_state_overlay;
self
}
/// Validates that there are sufficient changesets to revert to the requested block number.
///
/// Returns an error if the `MerkleChangeSets` checkpoint doesn't cover the requested block.
fn validate_changesets_availability(
&self,
provider: &F::Provider,
requested_block: BlockNumber,
) -> Result<(), ProviderError> {
// Get the MerkleChangeSets stage checkpoint - let errors propagate as-is
let checkpoint = provider.get_stage_checkpoint(StageId::MerkleChangeSets)?;
// If there's no checkpoint at all or block range details are missing, we can't revert
let available_range = checkpoint
.and_then(|chk| {
chk.merkle_changesets_stage_checkpoint()
.map(|stage_chk| stage_chk.block_range.from..=chk.block_number)
})
.ok_or_else(|| ProviderError::InsufficientChangesets {
requested: requested_block,
available: 0..=0,
})?;
// Check if the requested block is within the available range
if !available_range.contains(&requested_block) {
return Err(ProviderError::InsufficientChangesets {
requested: requested_block,
available: available_range,
});
}
Ok(())
}
/// Create a read-only [`OverlayStateProvider`].
pub fn provider_ro(&self) -> Result<OverlayStateProvider<F::Provider>, ProviderError> {
// Get a read-only provider
let provider = self.factory.database_provider_ro()?;
// If block_number is provided, collect reverts
let (trie_updates, hashed_state) = if let Some(from_block) = self.block_number {
// Validate that we have sufficient changesets for the requested block
self.validate_changesets_availability(&provider, from_block)?;
// Collect trie reverts
let mut trie_updates_mut = provider.trie_reverts(from_block)?;
// Collect state reverts using HashedPostState::from_reverts
let reverted_state =
HashedPostState::from_reverts::<KeccakKeyHasher>(provider.tx_ref(), from_block..)?;
let mut hashed_state_mut = reverted_state.into_sorted();
// Extend with overlays if provided
if let Some(trie_overlay) = &self.trie_overlay {
trie_updates_mut.extend_ref(trie_overlay);
}
if let Some(hashed_state_overlay) = &self.hashed_state_overlay {
hashed_state_mut.extend_ref(hashed_state_overlay);
}
(Arc::new(trie_updates_mut), Arc::new(hashed_state_mut))
} else {
// If no block_number, use overlays directly or defaults
let trie_updates =
self.trie_overlay.clone().unwrap_or_else(|| Arc::new(TrieUpdatesSorted::default()));
let hashed_state = self
.hashed_state_overlay
.clone()
.unwrap_or_else(|| Arc::new(HashedPostStateSorted::default()));
(trie_updates, hashed_state)
};
Ok(OverlayStateProvider::new(provider, trie_updates, hashed_state))
}
}
/// State provider with in-memory overlay from trie updates and hashed post state.
///
/// This provider uses in-memory trie updates and hashed post state as an overlay

View File

@@ -34,12 +34,13 @@ use reth_stages_types::{StageCheckpoint, StageId};
use reth_storage_api::{
BlockBodyIndicesProvider, BytecodeReader, DBProvider, DatabaseProviderFactory,
HashedPostStateProvider, NodePrimitivesProvider, StageCheckpointReader, StateProofProvider,
StorageRootProvider,
StorageRootProvider, TrieReader,
};
use reth_storage_errors::provider::{ConsistentViewError, ProviderError, ProviderResult};
use reth_trie::{
updates::TrieUpdates, AccountProof, HashedPostState, HashedStorage, MultiProof,
MultiProofTargets, StorageMultiProof, StorageProof, TrieInput,
updates::{TrieUpdates, TrieUpdatesSorted},
AccountProof, HashedPostState, HashedStorage, MultiProof, MultiProofTargets, StorageMultiProof,
StorageProof, TrieInput,
};
use std::{
collections::BTreeMap,
@@ -1005,6 +1006,19 @@ impl<T: NodePrimitives, ChainSpec: Send + Sync> StateReader for MockEthProvider<
}
}
impl<T: NodePrimitives, ChainSpec: Send + Sync> TrieReader for MockEthProvider<T, ChainSpec> {
fn trie_reverts(&self, _from: BlockNumber) -> ProviderResult<TrieUpdatesSorted> {
Ok(TrieUpdatesSorted::default())
}
fn get_block_trie_updates(
&self,
_block_number: BlockNumber,
) -> ProviderResult<TrieUpdatesSorted> {
Ok(TrieUpdatesSorted::default())
}
}
impl<T: NodePrimitives, ChainSpec: Send + Sync> CanonStateSubscriptions
for MockEthProvider<T, ChainSpec>
{

View File

@@ -89,7 +89,7 @@ pub fn insert_genesis<N: ProviderNodeTypes<ChainSpec = ChainSpec>>(
let (root, updates) = StateRoot::from_tx(provider.tx_ref())
.root_with_updates()
.map_err(reth_db::DatabaseError::from)?;
provider.write_trie_updates(&updates).unwrap();
provider.write_trie_updates(updates).unwrap();
provider.commit()?;

View File

@@ -3,7 +3,7 @@
use crate::{
AccountReader, BlockReader, BlockReaderIdExt, ChainSpecProvider, ChangeSetReader,
DatabaseProviderFactory, HashedPostStateProvider, StageCheckpointReader, StateProviderFactory,
StateReader, StaticFileProviderFactory,
StateReader, StaticFileProviderFactory, TrieReader,
};
use reth_chain_state::{CanonStateSubscriptions, ForkChoiceSubscriptions};
use reth_node_types::{BlockTy, HeaderTy, NodeTypesWithDB, ReceiptTy, TxTy};
@@ -12,7 +12,7 @@ use std::fmt::Debug;
/// Helper trait to unify all provider traits for simplicity.
pub trait FullProvider<N: NodeTypesWithDB>:
DatabaseProviderFactory<DB = N::DB, Provider: BlockReader>
DatabaseProviderFactory<DB = N::DB, Provider: BlockReader + TrieReader>
+ NodePrimitivesProvider<Primitives = N::Primitives>
+ StaticFileProviderFactory<Primitives = N::Primitives>
+ BlockReaderIdExt<
@@ -37,7 +37,7 @@ pub trait FullProvider<N: NodeTypesWithDB>:
}
impl<T, N: NodeTypesWithDB> FullProvider<N> for T where
T: DatabaseProviderFactory<DB = N::DB, Provider: BlockReader>
T: DatabaseProviderFactory<DB = N::DB, Provider: BlockReader + TrieReader>
+ NodePrimitivesProvider<Primitives = N::Primitives>
+ StaticFileProviderFactory<Primitives = N::Primitives>
+ BlockReaderIdExt<

View File

@@ -909,7 +909,7 @@ mod tests {
}
let (_, updates) = StateRoot::from_tx(tx).root_with_updates().unwrap();
provider_rw.write_trie_updates(&updates).unwrap();
provider_rw.write_trie_updates(updates).unwrap();
let mut state = State::builder().with_bundle_update().build();
@@ -1127,7 +1127,10 @@ mod tests {
assert_eq!(storage_root, storage_root_prehashed(init_storage.storage));
assert!(!storage_updates.is_empty());
provider_rw
.write_storage_trie_updates(core::iter::once((&hashed_address, &storage_updates)))
.write_storage_trie_updates_sorted(core::iter::once((
&hashed_address,
&storage_updates.into_sorted(),
)))
.unwrap();
// destroy the storage and re-create with new slots

View File

@@ -6,7 +6,7 @@ use crate::{
HashedPostStateProvider, HeaderProvider, NodePrimitivesProvider, PruneCheckpointReader,
ReceiptProvider, ReceiptProviderIdExt, StageCheckpointReader, StateProofProvider,
StateProvider, StateProviderBox, StateProviderFactory, StateReader, StateRootProvider,
StorageRootProvider, TransactionVariant, TransactionsProvider,
StorageRootProvider, TransactionVariant, TransactionsProvider, TrieReader,
};
#[cfg(feature = "db-api")]
@@ -35,8 +35,9 @@ use reth_prune_types::{PruneCheckpoint, PruneSegment};
use reth_stages_types::{StageCheckpoint, StageId};
use reth_storage_errors::provider::{ProviderError, ProviderResult};
use reth_trie_common::{
updates::TrieUpdates, AccountProof, HashedPostState, HashedStorage, MultiProof,
MultiProofTargets, StorageMultiProof, StorageProof, TrieInput,
updates::{TrieUpdates, TrieUpdatesSorted},
AccountProof, HashedPostState, HashedStorage, MultiProof, MultiProofTargets, StorageMultiProof,
StorageProof, TrieInput,
};
/// Supports various api interfaces for testing purposes.
@@ -59,7 +60,7 @@ impl<ChainSpec, N> NoopProvider<ChainSpec, N> {
#[cfg(feature = "db-api")]
tx: TxMock::default(),
#[cfg(feature = "db-api")]
prune_modes: PruneModes::none(),
prune_modes: PruneModes::default(),
_phantom: Default::default(),
}
}
@@ -73,7 +74,7 @@ impl<ChainSpec> NoopProvider<ChainSpec> {
#[cfg(feature = "db-api")]
tx: TxMock::default(),
#[cfg(feature = "db-api")]
prune_modes: PruneModes::none(),
prune_modes: PruneModes::default(),
_phantom: Default::default(),
}
}
@@ -646,6 +647,19 @@ impl<ChainSpec: Send + Sync, N: NodePrimitives> DBProvider for NoopProvider<Chai
}
}
impl<C: Send + Sync, N: NodePrimitives> TrieReader for NoopProvider<C, N> {
fn trie_reverts(&self, _from: BlockNumber) -> ProviderResult<TrieUpdatesSorted> {
Ok(TrieUpdatesSorted::default())
}
fn get_block_trie_updates(
&self,
_block_number: BlockNumber,
) -> ProviderResult<TrieUpdatesSorted> {
Ok(TrieUpdatesSorted::default())
}
}
#[cfg(feature = "db-api")]
impl<ChainSpec: Send + Sync, N: NodePrimitives> DatabaseProviderFactory
for NoopProvider<ChainSpec, N>

View File

@@ -1,8 +1,8 @@
use alloc::vec::Vec;
use alloy_primitives::{Address, Bytes, B256};
use alloy_primitives::{Address, BlockNumber, Bytes, B256};
use reth_storage_errors::provider::ProviderResult;
use reth_trie_common::{
updates::{StorageTrieUpdates, TrieUpdates},
updates::{StorageTrieUpdatesSorted, TrieUpdates, TrieUpdatesSorted},
AccountProof, HashedPostState, HashedStorage, MultiProof, MultiProofTargets, StorageMultiProof,
StorageProof, TrieInput,
};
@@ -89,25 +89,93 @@ pub trait StateProofProvider: Send + Sync {
fn witness(&self, input: TrieInput, target: HashedPostState) -> ProviderResult<Vec<Bytes>>;
}
/// Trie Reader
#[auto_impl::auto_impl(&, Arc, Box)]
pub trait TrieReader: Send + Sync {
/// Returns the [`TrieUpdatesSorted`] for reverting the trie database to its state prior to the
/// given block and onwards having been processed.
fn trie_reverts(&self, from: BlockNumber) -> ProviderResult<TrieUpdatesSorted>;
/// Returns the trie updates that were applied by the specified block.
fn get_block_trie_updates(
&self,
block_number: BlockNumber,
) -> ProviderResult<TrieUpdatesSorted>;
}
/// Trie Writer
#[auto_impl::auto_impl(&, Arc, Box)]
pub trait TrieWriter: Send + Sync {
/// Writes trie updates to the database.
///
/// Returns the number of entries modified.
fn write_trie_updates(&self, trie_updates: &TrieUpdates) -> ProviderResult<usize>;
fn write_trie_updates(&self, trie_updates: TrieUpdates) -> ProviderResult<usize> {
self.write_trie_updates_sorted(&trie_updates.into_sorted())
}
/// Writes trie updates to the database with already sorted updates.
///
/// Returns the number of entries modified.
fn write_trie_updates_sorted(&self, trie_updates: &TrieUpdatesSorted) -> ProviderResult<usize>;
/// Records the current values of all trie nodes which will be updated using the [`TrieUpdates`]
/// into the trie changesets tables.
///
/// The intended usage of this method is to call it _prior_ to calling `write_trie_updates` with
/// the same [`TrieUpdates`].
///
/// The `updates_overlay` parameter allows providing additional in-memory trie updates that
/// should be considered when looking up current node values. When provided, these overlay
/// updates are applied on top of the database state, allowing the method to see a view that
/// includes both committed database values and pending in-memory changes. This is useful
/// when writing changesets for updates that depend on previous uncommitted trie changes.
///
/// Returns the number of keys written.
fn write_trie_changesets(
&self,
block_number: BlockNumber,
trie_updates: &TrieUpdatesSorted,
updates_overlay: Option<&TrieUpdatesSorted>,
) -> ProviderResult<usize>;
/// Clears contents of trie changesets completely
fn clear_trie_changesets(&self) -> ProviderResult<()>;
/// Clears contents of trie changesets starting from the given block number (inclusive) onwards.
fn clear_trie_changesets_from(&self, from: BlockNumber) -> ProviderResult<()>;
}
/// Storage Trie Writer
#[auto_impl::auto_impl(&, Arc, Box)]
pub trait StorageTrieWriter: Send + Sync {
/// Writes storage trie updates from the given storage trie map.
/// Writes storage trie updates from the given storage trie map with already sorted updates.
///
/// First sorts the storage trie updates by the hashed address key, writing in sorted order.
/// Expects the storage trie updates to already be sorted by the hashed address key.
///
/// Returns the number of entries modified.
fn write_storage_trie_updates<'a>(
fn write_storage_trie_updates_sorted<'a>(
&self,
storage_tries: impl Iterator<Item = (&'a B256, &'a StorageTrieUpdates)>,
storage_tries: impl Iterator<Item = (&'a B256, &'a StorageTrieUpdatesSorted)>,
) -> ProviderResult<usize>;
/// Records the current values of all trie nodes which will be updated using the
/// [`StorageTrieUpdatesSorted`] into the storage trie changesets table.
///
/// The intended usage of this method is to call it _prior_ to calling
/// `write_storage_trie_updates` with the same set of [`StorageTrieUpdatesSorted`].
///
/// The `updates_overlay` parameter allows providing additional in-memory trie updates that
/// should be considered when looking up current node values. When provided, these overlay
/// updates are applied on top of the database state for each storage trie, allowing the
/// method to see a view that includes both committed database values and pending in-memory
/// changes. This is useful when writing changesets for storage updates that depend on
/// previous uncommitted trie changes.
///
/// Returns the number of keys written.
fn write_storage_trie_changesets<'a>(
&self,
block_number: BlockNumber,
storage_tries: impl Iterator<Item = (&'a B256, &'a StorageTrieUpdatesSorted)>,
updates_overlay: Option<&TrieUpdatesSorted>,
) -> ProviderResult<usize>;
}

View File

@@ -3,6 +3,7 @@ use core::ops::Not;
use crate::{
added_removed_keys::MultiAddedRemovedKeys,
prefix_set::{PrefixSetMut, TriePrefixSetsMut},
utils::extend_sorted_vec,
KeyHasher, MultiProofTargets, Nibbles,
};
use alloc::{borrow::Cow, vec::Vec};
@@ -484,6 +485,21 @@ impl HashedPostStateSorted {
pub const fn account_storages(&self) -> &B256Map<HashedStorageSorted> {
&self.storages
}
/// Extends this state with contents of another sorted state.
/// Entries in `other` take precedence for duplicate keys.
pub fn extend_ref(&mut self, other: &Self) {
// Extend accounts
self.accounts.extend_ref(&other.accounts);
// Extend storages
for (hashed_address, other_storage) in &other.storages {
self.storages
.entry(*hashed_address)
.and_modify(|existing| existing.extend_ref(other_storage))
.or_insert_with(|| other_storage.clone());
}
}
}
impl AsRef<Self> for HashedPostStateSorted {
@@ -510,6 +526,20 @@ impl HashedAccountsSorted {
.chain(self.destroyed_accounts.iter().map(|address| (*address, None)))
.sorted_by_key(|entry| *entry.0)
}
/// Extends this collection with contents of another sorted collection.
/// Entries in `other` take precedence for duplicate keys.
pub fn extend_ref(&mut self, other: &Self) {
// Updates take precedence over removals, so we want removals from `other` to only apply to
// the previous accounts.
self.accounts.retain(|(addr, _)| !other.destroyed_accounts.contains(addr));
// Extend the sorted accounts vector
extend_sorted_vec(&mut self.accounts, &other.accounts);
// Merge destroyed accounts sets
self.destroyed_accounts.extend(&other.destroyed_accounts);
}
}
/// Sorted hashed storage optimized for iterating during state trie calculation.
@@ -537,6 +567,28 @@ impl HashedStorageSorted {
.chain(self.zero_valued_slots.iter().map(|hashed_slot| (*hashed_slot, U256::ZERO)))
.sorted_by_key(|entry| *entry.0)
}
/// Extends this storage with contents of another sorted storage.
/// Entries in `other` take precedence for duplicate keys.
pub fn extend_ref(&mut self, other: &Self) {
if other.wiped {
// If other is wiped, clear everything and copy from other
self.wiped = true;
self.non_zero_valued_slots.clear();
self.zero_valued_slots.clear();
self.non_zero_valued_slots.extend_from_slice(&other.non_zero_valued_slots);
self.zero_valued_slots.extend(&other.zero_valued_slots);
return;
}
self.non_zero_valued_slots.retain(|(slot, _)| !other.zero_valued_slots.contains(slot));
// Extend the sorted non-zero valued slots
extend_sorted_vec(&mut self.non_zero_valued_slots, &other.non_zero_valued_slots);
// Merge zero valued slots sets
self.zero_valued_slots.extend(&other.zero_valued_slots);
}
}
/// An iterator that yields chunks of the state updates of at most `size` account and storage
@@ -1072,4 +1124,102 @@ mod tests {
);
assert_eq!(chunks.next(), None);
}
#[test]
fn test_hashed_post_state_sorted_extend_ref() {
// Test extending accounts
let mut state1 = HashedPostStateSorted {
accounts: HashedAccountsSorted {
accounts: vec![
(B256::from([1; 32]), Account::default()),
(B256::from([3; 32]), Account::default()),
],
destroyed_accounts: B256Set::from_iter([B256::from([5; 32])]),
},
storages: B256Map::default(),
};
let state2 = HashedPostStateSorted {
accounts: HashedAccountsSorted {
accounts: vec![
(B256::from([2; 32]), Account::default()),
(B256::from([3; 32]), Account { nonce: 1, ..Default::default() }), // Override
(B256::from([4; 32]), Account::default()),
],
destroyed_accounts: B256Set::from_iter([B256::from([6; 32])]),
},
storages: B256Map::default(),
};
state1.extend_ref(&state2);
// Check accounts are merged and sorted
assert_eq!(state1.accounts.accounts.len(), 4);
assert_eq!(state1.accounts.accounts[0].0, B256::from([1; 32]));
assert_eq!(state1.accounts.accounts[1].0, B256::from([2; 32]));
assert_eq!(state1.accounts.accounts[2].0, B256::from([3; 32]));
assert_eq!(state1.accounts.accounts[2].1.nonce, 1); // Should have state2's value
assert_eq!(state1.accounts.accounts[3].0, B256::from([4; 32]));
// Check destroyed accounts are merged
assert!(state1.accounts.destroyed_accounts.contains(&B256::from([5; 32])));
assert!(state1.accounts.destroyed_accounts.contains(&B256::from([6; 32])));
}
#[test]
fn test_hashed_storage_sorted_extend_ref() {
// Test normal extension
let mut storage1 = HashedStorageSorted {
non_zero_valued_slots: vec![
(B256::from([1; 32]), U256::from(10)),
(B256::from([3; 32]), U256::from(30)),
],
zero_valued_slots: B256Set::from_iter([B256::from([5; 32])]),
wiped: false,
};
let storage2 = HashedStorageSorted {
non_zero_valued_slots: vec![
(B256::from([2; 32]), U256::from(20)),
(B256::from([3; 32]), U256::from(300)), // Override
(B256::from([4; 32]), U256::from(40)),
],
zero_valued_slots: B256Set::from_iter([B256::from([6; 32])]),
wiped: false,
};
storage1.extend_ref(&storage2);
assert_eq!(storage1.non_zero_valued_slots.len(), 4);
assert_eq!(storage1.non_zero_valued_slots[0].0, B256::from([1; 32]));
assert_eq!(storage1.non_zero_valued_slots[1].0, B256::from([2; 32]));
assert_eq!(storage1.non_zero_valued_slots[2].0, B256::from([3; 32]));
assert_eq!(storage1.non_zero_valued_slots[2].1, U256::from(300)); // Should have storage2's value
assert_eq!(storage1.non_zero_valued_slots[3].0, B256::from([4; 32]));
assert!(storage1.zero_valued_slots.contains(&B256::from([5; 32])));
assert!(storage1.zero_valued_slots.contains(&B256::from([6; 32])));
assert!(!storage1.wiped);
// Test wiped storage
let mut storage3 = HashedStorageSorted {
non_zero_valued_slots: vec![(B256::from([1; 32]), U256::from(10))],
zero_valued_slots: B256Set::from_iter([B256::from([2; 32])]),
wiped: false,
};
let storage4 = HashedStorageSorted {
non_zero_valued_slots: vec![(B256::from([3; 32]), U256::from(30))],
zero_valued_slots: B256Set::from_iter([B256::from([4; 32])]),
wiped: true,
};
storage3.extend_ref(&storage4);
assert!(storage3.wiped);
// When wiped, should only have storage4's values
assert_eq!(storage3.non_zero_valued_slots.len(), 1);
assert_eq!(storage3.non_zero_valued_slots[0].0, B256::from([3; 32]));
assert_eq!(storage3.zero_valued_slots.len(), 1);
assert!(storage3.zero_valued_slots.contains(&B256::from([4; 32])));
}
}

View File

@@ -34,7 +34,7 @@ impl TrieInput {
/// Create new trie input from the provided blocks, from oldest to newest. See the documentation
/// for [`Self::extend_with_blocks`] for details.
pub fn from_blocks<'a>(
blocks: impl IntoIterator<Item = (&'a HashedPostState, Option<&'a TrieUpdates>)>,
blocks: impl IntoIterator<Item = (&'a HashedPostState, &'a TrieUpdates)>,
) -> Self {
let mut input = Self::default();
input.extend_with_blocks(blocks);
@@ -47,14 +47,10 @@ impl TrieInput {
/// constructed from the state of this block and the state itself, **without** trie updates.
pub fn extend_with_blocks<'a>(
&mut self,
blocks: impl IntoIterator<Item = (&'a HashedPostState, Option<&'a TrieUpdates>)>,
blocks: impl IntoIterator<Item = (&'a HashedPostState, &'a TrieUpdates)>,
) {
for (hashed_state, trie_updates) in blocks {
if let Some(nodes) = trie_updates.as_ref() {
self.append_cached_ref(nodes, hashed_state);
} else {
self.append_ref(hashed_state);
}
self.append_cached_ref(trie_updates, hashed_state);
}
}

View File

@@ -36,7 +36,7 @@ mod nibbles;
pub use nibbles::{Nibbles, StoredNibbles, StoredNibblesSubKey};
mod storage;
pub use storage::StorageTrieEntry;
pub use storage::{StorageTrieEntry, TrieChangeSetsEntry};
mod subnode;
pub use subnode::StoredSubNode;
@@ -57,6 +57,9 @@ pub mod updates;
pub mod added_removed_keys;
/// Utilities used by other modules in this crate.
mod utils;
/// Bincode-compatible serde implementations for trie types.
///
/// `bincode` crate allows for more efficient serialization of trie types, because it allows

View File

@@ -1,6 +1,8 @@
use super::{BranchNodeCompact, StoredNibblesSubKey};
use super::{BranchNodeCompact, Nibbles, StoredNibblesSubKey};
/// Account storage trie node.
///
/// `nibbles` is the subkey when used as a value in the `StorageTrie` table.
#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord)]
#[cfg_attr(any(test, feature = "serde"), derive(serde::Serialize, serde::Deserialize))]
pub struct StorageTrieEntry {
@@ -31,3 +33,173 @@ impl reth_codecs::Compact for StorageTrieEntry {
(this, buf)
}
}
/// Trie changeset entry representing the state of a trie node before a block.
///
/// `nibbles` is the subkey when used as a value in the changeset tables.
#[derive(Debug, Clone, PartialEq, Eq)]
#[cfg_attr(any(test, feature = "serde"), derive(serde::Serialize, serde::Deserialize))]
pub struct TrieChangeSetsEntry {
/// The nibbles of the intermediate node
pub nibbles: StoredNibblesSubKey,
/// Node value prior to the block being processed, None indicating it didn't exist.
pub node: Option<BranchNodeCompact>,
}
#[cfg(any(test, feature = "reth-codec"))]
impl reth_codecs::Compact for TrieChangeSetsEntry {
fn to_compact<B>(&self, buf: &mut B) -> usize
where
B: bytes::BufMut + AsMut<[u8]>,
{
let nibbles_len = self.nibbles.to_compact(buf);
let node_len = self.node.as_ref().map(|node| node.to_compact(buf)).unwrap_or(0);
nibbles_len + node_len
}
fn from_compact(buf: &[u8], len: usize) -> (Self, &[u8]) {
if len == 0 {
// Return an empty entry without trying to parse anything
return (
Self { nibbles: StoredNibblesSubKey::from(Nibbles::default()), node: None },
buf,
)
}
let (nibbles, buf) = StoredNibblesSubKey::from_compact(buf, 65);
if len <= 65 {
return (Self { nibbles, node: None }, buf)
}
let (node, buf) = BranchNodeCompact::from_compact(buf, len - 65);
(Self { nibbles, node: Some(node) }, buf)
}
}
#[cfg(test)]
mod tests {
use super::*;
use bytes::BytesMut;
use reth_codecs::Compact;
#[test]
fn test_trie_changesets_entry_full_empty() {
// Test a fully empty entry (empty nibbles, None node)
let entry = TrieChangeSetsEntry { nibbles: StoredNibblesSubKey::from(vec![]), node: None };
let mut buf = BytesMut::new();
let len = entry.to_compact(&mut buf);
// Empty nibbles takes 65 bytes (64 for padding + 1 for length)
// None node adds 0 bytes
assert_eq!(len, 65);
assert_eq!(buf.len(), 65);
// Deserialize and verify
let (decoded, remaining) = TrieChangeSetsEntry::from_compact(&buf, len);
assert_eq!(decoded.nibbles.0.to_vec(), Vec::<u8>::new());
assert_eq!(decoded.node, None);
assert_eq!(remaining.len(), 0);
}
#[test]
fn test_trie_changesets_entry_none_node() {
// Test non-empty nibbles with None node
let nibbles_data = vec![0x01, 0x02, 0x03, 0x04];
let entry = TrieChangeSetsEntry {
nibbles: StoredNibblesSubKey::from(nibbles_data.clone()),
node: None,
};
let mut buf = BytesMut::new();
let len = entry.to_compact(&mut buf);
// Nibbles takes 65 bytes regardless of content
assert_eq!(len, 65);
// Deserialize and verify
let (decoded, remaining) = TrieChangeSetsEntry::from_compact(&buf, len);
assert_eq!(decoded.nibbles.0.to_vec(), nibbles_data);
assert_eq!(decoded.node, None);
assert_eq!(remaining.len(), 0);
}
#[test]
fn test_trie_changesets_entry_empty_path_with_node() {
// Test empty path with Some node
// Using the same signature as in the codebase: (state_mask, hash_mask, tree_mask, hashes,
// value)
let test_node = BranchNodeCompact::new(
0b1111_1111_1111_1111, // state_mask: all children present
0b1111_1111_1111_1111, // hash_mask: all have hashes
0b0000_0000_0000_0000, // tree_mask: no embedded trees
vec![], // hashes
None, // value
);
let entry = TrieChangeSetsEntry {
nibbles: StoredNibblesSubKey::from(vec![]),
node: Some(test_node.clone()),
};
let mut buf = BytesMut::new();
let len = entry.to_compact(&mut buf);
// Calculate expected length
let mut temp_buf = BytesMut::new();
let node_len = test_node.to_compact(&mut temp_buf);
assert_eq!(len, 65 + node_len);
// Deserialize and verify
let (decoded, remaining) = TrieChangeSetsEntry::from_compact(&buf, len);
assert_eq!(decoded.nibbles.0.to_vec(), Vec::<u8>::new());
assert_eq!(decoded.node, Some(test_node));
assert_eq!(remaining.len(), 0);
}
#[test]
fn test_trie_changesets_entry_normal() {
// Test normal case: non-empty path with Some node
let nibbles_data = vec![0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f];
// Using the same signature as in the codebase
let test_node = BranchNodeCompact::new(
0b0000_0000_1111_0000, // state_mask: some children present
0b0000_0000_0011_0000, // hash_mask: some have hashes
0b0000_0000_0000_0000, // tree_mask: no embedded trees
vec![], // hashes (empty for this test)
None, // value
);
let entry = TrieChangeSetsEntry {
nibbles: StoredNibblesSubKey::from(nibbles_data.clone()),
node: Some(test_node.clone()),
};
let mut buf = BytesMut::new();
let len = entry.to_compact(&mut buf);
// Verify serialization length
let mut temp_buf = BytesMut::new();
let node_len = test_node.to_compact(&mut temp_buf);
assert_eq!(len, 65 + node_len);
// Deserialize and verify
let (decoded, remaining) = TrieChangeSetsEntry::from_compact(&buf, len);
assert_eq!(decoded.nibbles.0.to_vec(), nibbles_data);
assert_eq!(decoded.node, Some(test_node));
assert_eq!(remaining.len(), 0);
}
#[test]
fn test_trie_changesets_entry_from_compact_zero_len() {
// Test from_compact with zero length
let buf = vec![0x01, 0x02, 0x03];
let (decoded, remaining) = TrieChangeSetsEntry::from_compact(&buf, 0);
// Should return empty nibbles and None node
assert_eq!(decoded.nibbles.0.to_vec(), Vec::<u8>::new());
assert_eq!(decoded.node, None);
assert_eq!(remaining, &buf[..]); // Buffer should be unchanged
}
}

View File

@@ -1,4 +1,4 @@
use crate::{BranchNodeCompact, HashBuilder, Nibbles};
use crate::{utils::extend_sorted_vec, BranchNodeCompact, HashBuilder, Nibbles};
use alloc::{
collections::{btree_map::BTreeMap, btree_set::BTreeSet},
vec::Vec,
@@ -438,6 +438,11 @@ pub struct TrieUpdatesSorted {
}
impl TrieUpdatesSorted {
/// Returns `true` if the updates are empty.
pub fn is_empty(&self) -> bool {
self.account_nodes.is_empty() && self.storage_tries.is_empty()
}
/// Returns reference to updated account nodes.
pub fn account_nodes_ref(&self) -> &[(Nibbles, Option<BranchNodeCompact>)] {
&self.account_nodes
@@ -447,6 +452,24 @@ impl TrieUpdatesSorted {
pub const fn storage_tries_ref(&self) -> &B256Map<StorageTrieUpdatesSorted> {
&self.storage_tries
}
/// Extends the trie updates with another set of sorted updates.
///
/// This merges the account nodes and storage tries from `other` into `self`.
/// Account nodes are merged and re-sorted, with `other`'s values taking precedence
/// for duplicate keys.
pub fn extend_ref(&mut self, other: &Self) {
// Extend account nodes
extend_sorted_vec(&mut self.account_nodes, &other.account_nodes);
// Merge storage tries
for (hashed_address, storage_trie) in &other.storage_tries {
self.storage_tries
.entry(*hashed_address)
.and_modify(|existing| existing.extend_ref(storage_trie))
.or_insert_with(|| storage_trie.clone());
}
}
}
impl AsRef<Self> for TrieUpdatesSorted {
@@ -455,6 +478,29 @@ impl AsRef<Self> for TrieUpdatesSorted {
}
}
impl From<TrieUpdatesSorted> for TrieUpdates {
fn from(sorted: TrieUpdatesSorted) -> Self {
let mut account_nodes = HashMap::default();
let mut removed_nodes = HashSet::default();
for (nibbles, node) in sorted.account_nodes {
if let Some(node) = node {
account_nodes.insert(nibbles, node);
} else {
removed_nodes.insert(nibbles);
}
}
let storage_tries = sorted
.storage_tries
.into_iter()
.map(|(address, storage)| (address, storage.into()))
.collect();
Self { account_nodes, removed_nodes, storage_tries }
}
}
/// Sorted storage trie updates reference used for serializing to file.
#[derive(PartialEq, Eq, Clone, Default, Debug)]
#[cfg_attr(any(test, feature = "serde"), derive(serde::Serialize))]
@@ -488,6 +534,23 @@ impl StorageTrieUpdatesSorted {
pub fn storage_nodes_ref(&self) -> &[(Nibbles, Option<BranchNodeCompact>)] {
&self.storage_nodes
}
/// Extends the storage trie updates with another set of sorted updates.
///
/// If `other` is marked as deleted, this will be marked as deleted and all nodes cleared.
/// Otherwise, nodes are merged with `other`'s values taking precedence for duplicates.
pub fn extend_ref(&mut self, other: &Self) {
if other.is_deleted {
self.is_deleted = true;
self.storage_nodes.clear();
self.storage_nodes.extend(other.storage_nodes.iter().cloned());
return;
}
// Extend storage nodes
extend_sorted_vec(&mut self.storage_nodes, &other.storage_nodes);
self.is_deleted = self.is_deleted || other.is_deleted;
}
}
/// Excludes empty nibbles from the given iterator.
@@ -502,6 +565,153 @@ fn exclude_empty_from_pair<V>(
iter.into_iter().filter(|(n, _)| !n.is_empty())
}
impl From<StorageTrieUpdatesSorted> for StorageTrieUpdates {
fn from(sorted: StorageTrieUpdatesSorted) -> Self {
let mut storage_nodes = HashMap::default();
let mut removed_nodes = HashSet::default();
for (nibbles, node) in sorted.storage_nodes {
if let Some(node) = node {
storage_nodes.insert(nibbles, node);
} else {
removed_nodes.insert(nibbles);
}
}
Self { is_deleted: sorted.is_deleted, storage_nodes, removed_nodes }
}
}
#[cfg(test)]
mod tests {
use super::*;
use alloy_primitives::B256;
#[test]
fn test_trie_updates_sorted_extend_ref() {
// Test extending with empty updates
let mut updates1 = TrieUpdatesSorted::default();
let updates2 = TrieUpdatesSorted::default();
updates1.extend_ref(&updates2);
assert_eq!(updates1.account_nodes.len(), 0);
assert_eq!(updates1.storage_tries.len(), 0);
// Test extending account nodes
let mut updates1 = TrieUpdatesSorted {
account_nodes: vec![
(Nibbles::from_nibbles_unchecked([0x01]), Some(BranchNodeCompact::default())),
(Nibbles::from_nibbles_unchecked([0x03]), None),
],
storage_tries: B256Map::default(),
};
let updates2 = TrieUpdatesSorted {
account_nodes: vec![
(Nibbles::from_nibbles_unchecked([0x02]), Some(BranchNodeCompact::default())),
(Nibbles::from_nibbles_unchecked([0x03]), Some(BranchNodeCompact::default())), /* Override */
],
storage_tries: B256Map::default(),
};
updates1.extend_ref(&updates2);
assert_eq!(updates1.account_nodes.len(), 3);
// Should be sorted: 0x01, 0x02, 0x03
assert_eq!(updates1.account_nodes[0].0, Nibbles::from_nibbles_unchecked([0x01]));
assert_eq!(updates1.account_nodes[1].0, Nibbles::from_nibbles_unchecked([0x02]));
assert_eq!(updates1.account_nodes[2].0, Nibbles::from_nibbles_unchecked([0x03]));
// 0x03 should have Some value from updates2 (override)
assert!(updates1.account_nodes[2].1.is_some());
// Test extending storage tries
let storage_trie1 = StorageTrieUpdatesSorted {
is_deleted: false,
storage_nodes: vec![(
Nibbles::from_nibbles_unchecked([0x0a]),
Some(BranchNodeCompact::default()),
)],
};
let storage_trie2 = StorageTrieUpdatesSorted {
is_deleted: false,
storage_nodes: vec![(Nibbles::from_nibbles_unchecked([0x0b]), None)],
};
let hashed_address1 = B256::from([1; 32]);
let hashed_address2 = B256::from([2; 32]);
let mut updates1 = TrieUpdatesSorted {
account_nodes: vec![],
storage_tries: B256Map::from_iter([(hashed_address1, storage_trie1.clone())]),
};
let updates2 = TrieUpdatesSorted {
account_nodes: vec![],
storage_tries: B256Map::from_iter([
(hashed_address1, storage_trie2),
(hashed_address2, storage_trie1),
]),
};
updates1.extend_ref(&updates2);
assert_eq!(updates1.storage_tries.len(), 2);
assert!(updates1.storage_tries.contains_key(&hashed_address1));
assert!(updates1.storage_tries.contains_key(&hashed_address2));
// Check that storage trie for hashed_address1 was extended
let merged_storage = &updates1.storage_tries[&hashed_address1];
assert_eq!(merged_storage.storage_nodes.len(), 2);
}
#[test]
fn test_storage_trie_updates_sorted_extend_ref_deleted() {
// Test case 1: Extending with a deleted storage trie that has nodes
let mut storage1 = StorageTrieUpdatesSorted {
is_deleted: false,
storage_nodes: vec![
(Nibbles::from_nibbles_unchecked([0x01]), Some(BranchNodeCompact::default())),
(Nibbles::from_nibbles_unchecked([0x02]), None),
],
};
let storage2 = StorageTrieUpdatesSorted {
is_deleted: true,
storage_nodes: vec![
(Nibbles::from_nibbles_unchecked([0x03]), Some(BranchNodeCompact::default())),
(Nibbles::from_nibbles_unchecked([0x04]), None),
],
};
storage1.extend_ref(&storage2);
// Should be marked as deleted
assert!(storage1.is_deleted);
// Original nodes should be cleared, but other's nodes should be added
assert_eq!(storage1.storage_nodes.len(), 2);
assert_eq!(storage1.storage_nodes[0].0, Nibbles::from_nibbles_unchecked([0x03]));
assert_eq!(storage1.storage_nodes[1].0, Nibbles::from_nibbles_unchecked([0x04]));
// Test case 2: Extending a deleted storage trie with more nodes
let mut storage3 = StorageTrieUpdatesSorted {
is_deleted: true,
storage_nodes: vec![(
Nibbles::from_nibbles_unchecked([0x05]),
Some(BranchNodeCompact::default()),
)],
};
let storage4 = StorageTrieUpdatesSorted {
is_deleted: true,
storage_nodes: vec![
(Nibbles::from_nibbles_unchecked([0x06]), Some(BranchNodeCompact::default())),
(Nibbles::from_nibbles_unchecked([0x07]), None),
],
};
storage3.extend_ref(&storage4);
// Should remain deleted
assert!(storage3.is_deleted);
// Should have nodes from other (original cleared then extended)
assert_eq!(storage3.storage_nodes.len(), 2);
assert_eq!(storage3.storage_nodes[0].0, Nibbles::from_nibbles_unchecked([0x06]));
assert_eq!(storage3.storage_nodes[1].0, Nibbles::from_nibbles_unchecked([0x07]));
}
}
/// Bincode-compatible trie updates type serde implementations.
#[cfg(feature = "serde-bincode-compat")]
pub mod serde_bincode_compat {
@@ -717,7 +927,7 @@ pub mod serde_bincode_compat {
}
#[cfg(all(test, feature = "serde"))]
mod tests {
mod serde_tests {
use super::*;
#[test]

View File

@@ -0,0 +1,53 @@
use alloc::vec::Vec;
/// Helper function to extend a sorted vector with another sorted vector.
/// Values from `other` take precedence for duplicate keys.
///
/// This function efficiently merges two sorted vectors by:
/// 1. Iterating through the target vector with mutable references
/// 2. Using a peekable iterator for the other vector
/// 3. For each target item, processing other items that come before or equal to it
/// 4. Collecting items from other that need to be inserted
/// 5. Appending and re-sorting only if new items were added
pub(crate) fn extend_sorted_vec<K, V>(target: &mut Vec<(K, V)>, other: &[(K, V)])
where
K: Clone + Ord + core::hash::Hash + Eq,
V: Clone,
{
if other.is_empty() {
return;
}
let mut other_iter = other.iter().peekable();
let mut to_insert = Vec::new();
// Iterate through target and update/collect items from other
for target_item in target.iter_mut() {
while let Some(other_item) = other_iter.peek() {
use core::cmp::Ordering;
match other_item.0.cmp(&target_item.0) {
Ordering::Less => {
// Other item comes before current target item, collect it
to_insert.push(other_iter.next().unwrap().clone());
}
Ordering::Equal => {
// Same key, update target with other's value
target_item.1 = other_iter.next().unwrap().1.clone();
break;
}
Ordering::Greater => {
// Other item comes after current target item, keep target unchanged
break;
}
}
}
}
// Append collected new items, as well as any remaining from `other` which are necessarily also
// new, and sort if needed
if !to_insert.is_empty() || other_iter.peek().is_some() {
target.extend(to_insert);
target.extend(other_iter.cloned());
target.sort_unstable_by(|a, b| a.0.cmp(&b.0));
}
}

View File

@@ -7,7 +7,7 @@ use reth_db_api::{
};
use reth_trie::{
trie_cursor::{TrieCursor, TrieCursorFactory},
updates::StorageTrieUpdates,
updates::StorageTrieUpdatesSorted,
BranchNodeCompact, Nibbles, StorageTrieEntry, StoredNibbles, StoredNibblesSubKey,
};
@@ -110,31 +110,19 @@ where
+ DbDupCursorRO<tables::StoragesTrie>
+ DbDupCursorRW<tables::StoragesTrie>,
{
/// Writes storage updates
pub fn write_storage_trie_updates(
/// Writes storage updates that are already sorted
pub fn write_storage_trie_updates_sorted(
&mut self,
updates: &StorageTrieUpdates,
updates: &StorageTrieUpdatesSorted,
) -> Result<usize, DatabaseError> {
// The storage trie for this account has to be deleted.
if updates.is_deleted() && self.cursor.seek_exact(self.hashed_address)?.is_some() {
self.cursor.delete_current_duplicates()?;
}
// Merge updated and removed nodes. Updated nodes must take precedence.
let mut storage_updates = updates
.removed_nodes_ref()
.iter()
.filter_map(|n| (!updates.storage_nodes_ref().contains_key(n)).then_some((n, None)))
.collect::<Vec<_>>();
storage_updates.extend(
updates.storage_nodes_ref().iter().map(|(nibbles, node)| (nibbles, Some(node))),
);
// Sort trie node updates.
storage_updates.sort_unstable_by(|a, b| a.0.cmp(b.0));
let mut num_entries = 0;
for (nibbles, maybe_updated) in storage_updates.into_iter().filter(|(n, _)| !n.is_empty()) {
for (nibbles, maybe_updated) in updates.storage_nodes.iter().filter(|(n, _)| !n.is_empty())
{
num_entries += 1;
let nibbles = StoredNibblesSubKey(*nibbles);
// Delete the old entry if it exists.

View File

@@ -81,7 +81,11 @@ fn incremental_vs_full_root(inputs: &[&str], modified: &str) {
let modified_root = loader.root().unwrap();
// Update the intermediate roots table so that we can run the incremental verification
tx.write_storage_trie_updates(core::iter::once((&hashed_address, &trie_updates))).unwrap();
tx.write_storage_trie_updates_sorted(core::iter::once((
&hashed_address,
&trie_updates.into_sorted(),
)))
.unwrap();
// 3. Calculate the incremental root
let mut storage_changes = PrefixSetMut::default();
@@ -620,7 +624,7 @@ fn account_trie_around_extension_node_with_dbtrie() {
let (got, updates) = StateRoot::from_tx(tx.tx_ref()).root_with_updates().unwrap();
assert_eq!(expected, got);
tx.write_trie_updates(&updates).unwrap();
tx.write_trie_updates(updates).unwrap();
// read the account updates from the db
let mut accounts_trie = tx.tx_ref().cursor_read::<tables::AccountsTrie>().unwrap();
@@ -667,7 +671,7 @@ proptest! {
state.iter().map(|(&key, &balance)| (key, (Account { balance, ..Default::default() }, std::iter::empty())))
);
assert_eq!(expected_root, state_root);
tx.write_trie_updates(&trie_updates).unwrap();
tx.write_trie_updates(trie_updates).unwrap();
}
}
}

View File

@@ -33,7 +33,7 @@ pub fn calculate_state_root(c: &mut Criterion) {
provider_rw.write_hashed_state(&db_state.into_sorted()).unwrap();
let (_, updates) =
StateRoot::from_tx(provider_rw.tx_ref()).root_with_updates().unwrap();
provider_rw.write_trie_updates(&updates).unwrap();
provider_rw.write_trie_updates(updates).unwrap();
provider_rw.commit().unwrap();
}

View File

@@ -4998,9 +4998,12 @@ mod tests {
state.keys().copied(),
);
// Extract account nodes before moving hash_builder_updates
let hash_builder_account_nodes = hash_builder_updates.account_nodes.clone();
// Write trie updates to the database
let provider_rw = provider_factory.provider_rw().unwrap();
provider_rw.write_trie_updates(&hash_builder_updates).unwrap();
provider_rw.write_trie_updates(hash_builder_updates).unwrap();
provider_rw.commit().unwrap();
// Assert that the sparse trie root matches the hash builder root
@@ -5008,7 +5011,7 @@ mod tests {
// Assert that the sparse trie updates match the hash builder updates
pretty_assertions::assert_eq!(
BTreeMap::from_iter(sparse_updates.updated_nodes),
BTreeMap::from_iter(hash_builder_updates.account_nodes)
BTreeMap::from_iter(hash_builder_account_nodes)
);
// Assert that the sparse trie nodes match the hash builder proof nodes
assert_eq_parallel_sparse_trie_proof_nodes(
@@ -5043,9 +5046,12 @@ mod tests {
state.keys().copied(),
);
// Extract account nodes before moving hash_builder_updates
let hash_builder_account_nodes = hash_builder_updates.account_nodes.clone();
// Write trie updates to the database
let provider_rw = provider_factory.provider_rw().unwrap();
provider_rw.write_trie_updates(&hash_builder_updates).unwrap();
provider_rw.write_trie_updates(hash_builder_updates).unwrap();
provider_rw.commit().unwrap();
// Assert that the sparse trie root matches the hash builder root
@@ -5053,7 +5059,7 @@ mod tests {
// Assert that the sparse trie updates match the hash builder updates
pretty_assertions::assert_eq!(
BTreeMap::from_iter(sparse_updates.updated_nodes),
BTreeMap::from_iter(hash_builder_updates.account_nodes)
BTreeMap::from_iter(hash_builder_account_nodes)
);
// Assert that the sparse trie nodes match the hash builder proof nodes
assert_eq_parallel_sparse_trie_proof_nodes(

View File

@@ -3034,9 +3034,12 @@ mod tests {
state.keys().copied(),
);
// Extract account nodes before moving hash_builder_updates
let hash_builder_account_nodes = hash_builder_updates.account_nodes.clone();
// Write trie updates to the database
let provider_rw = provider_factory.provider_rw().unwrap();
provider_rw.write_trie_updates(&hash_builder_updates).unwrap();
provider_rw.write_trie_updates(hash_builder_updates).unwrap();
provider_rw.commit().unwrap();
// Assert that the sparse trie root matches the hash builder root
@@ -3044,7 +3047,7 @@ mod tests {
// Assert that the sparse trie updates match the hash builder updates
pretty_assertions::assert_eq!(
BTreeMap::from_iter(sparse_updates.updated_nodes),
BTreeMap::from_iter(hash_builder_updates.account_nodes)
BTreeMap::from_iter(hash_builder_account_nodes)
);
// Assert that the sparse trie nodes match the hash builder proof nodes
assert_eq_sparse_trie_proof_nodes(&updated_sparse, hash_builder_proof_nodes);
@@ -3076,9 +3079,12 @@ mod tests {
state.keys().copied(),
);
// Extract account nodes before moving hash_builder_updates
let hash_builder_account_nodes = hash_builder_updates.account_nodes.clone();
// Write trie updates to the database
let provider_rw = provider_factory.provider_rw().unwrap();
provider_rw.write_trie_updates(&hash_builder_updates).unwrap();
provider_rw.write_trie_updates(hash_builder_updates).unwrap();
provider_rw.commit().unwrap();
// Assert that the sparse trie root matches the hash builder root
@@ -3086,7 +3092,7 @@ mod tests {
// Assert that the sparse trie updates match the hash builder updates
pretty_assertions::assert_eq!(
BTreeMap::from_iter(sparse_updates.updated_nodes),
BTreeMap::from_iter(hash_builder_updates.account_nodes)
BTreeMap::from_iter(hash_builder_account_nodes)
);
// Assert that the sparse trie nodes match the hash builder proof nodes
assert_eq_sparse_trie_proof_nodes(&updated_sparse, hash_builder_proof_nodes);

View File

@@ -58,3 +58,48 @@ pub trait TrieCursor: Send + Sync {
/// Get the current entry.
fn current(&mut self) -> Result<Option<Nibbles>, DatabaseError>;
}
/// Iterator wrapper for `TrieCursor` types
#[derive(Debug)]
pub struct TrieCursorIter<'a, C> {
cursor: &'a mut C,
/// The initial value from seek, if any
initial: Option<Result<(Nibbles, BranchNodeCompact), DatabaseError>>,
}
impl<'a, C> TrieCursorIter<'a, C> {
/// Create a new iterator from a mutable reference to a cursor. The Iterator will start from the
/// empty path.
pub fn new(cursor: &'a mut C) -> Self
where
C: TrieCursor,
{
let initial = cursor.seek(Nibbles::default()).transpose();
Self { cursor, initial }
}
}
impl<'a, C> From<&'a mut C> for TrieCursorIter<'a, C>
where
C: TrieCursor,
{
fn from(cursor: &'a mut C) -> Self {
Self::new(cursor)
}
}
impl<'a, C> Iterator for TrieCursorIter<'a, C>
where
C: TrieCursor,
{
type Item = Result<(Nibbles, BranchNodeCompact), DatabaseError>;
fn next(&mut self) -> Option<Self::Item> {
// If we have an initial value from seek, return it first
if let Some(initial) = self.initial.take() {
return Some(initial);
}
self.cursor.next().transpose()
}
}

View File

@@ -72,17 +72,18 @@ Database:
<STAGE>
Possible values:
- headers: The headers stage within the pipeline
- bodies: The bodies stage within the pipeline
- senders: The senders stage within the pipeline
- execution: The execution stage within the pipeline
- account-hashing: The account hashing stage within the pipeline
- storage-hashing: The storage hashing stage within the pipeline
- hashing: The account and storage hashing stages within the pipeline
- merkle: The merkle stage within the pipeline
- tx-lookup: The transaction lookup stage within the pipeline
- account-history: The account history stage within the pipeline
- storage-history: The storage history stage within the pipeline
- headers: The headers stage within the pipeline
- bodies: The bodies stage within the pipeline
- senders: The senders stage within the pipeline
- execution: The execution stage within the pipeline
- account-hashing: The account hashing stage within the pipeline
- storage-hashing: The storage hashing stage within the pipeline
- hashing: The account and storage hashing stages within the pipeline
- merkle: The merkle stage within the pipeline
- merkle-changesets: The merkle changesets stage within the pipeline
- tx-lookup: The transaction lookup stage within the pipeline
- account-history: The account history stage within the pipeline
- storage-history: The storage history stage within the pipeline
Logging:
--log.stdout.format <FORMAT>

View File

@@ -101,17 +101,18 @@ Database:
The name of the stage to run
Possible values:
- headers: The headers stage within the pipeline
- bodies: The bodies stage within the pipeline
- senders: The senders stage within the pipeline
- execution: The execution stage within the pipeline
- account-hashing: The account hashing stage within the pipeline
- storage-hashing: The storage hashing stage within the pipeline
- hashing: The account and storage hashing stages within the pipeline
- merkle: The merkle stage within the pipeline
- tx-lookup: The transaction lookup stage within the pipeline
- account-history: The account history stage within the pipeline
- storage-history: The storage history stage within the pipeline
- headers: The headers stage within the pipeline
- bodies: The bodies stage within the pipeline
- senders: The senders stage within the pipeline
- execution: The execution stage within the pipeline
- account-hashing: The account hashing stage within the pipeline
- storage-hashing: The storage hashing stage within the pipeline
- hashing: The account and storage hashing stages within the pipeline
- merkle: The merkle stage within the pipeline
- merkle-changesets: The merkle changesets stage within the pipeline
- tx-lookup: The transaction lookup stage within the pipeline
- account-history: The account history stage within the pipeline
- storage-history: The storage history stage within the pipeline
Networking:
-d, --disable-discovery

View File

@@ -6,7 +6,7 @@ use crate::{
};
use alloy_eips::eip2718::WithEncoded;
use op_alloy_rpc_types_engine::{OpExecutionData, OpExecutionPayload};
use reth_chain_state::ExecutedBlockWithTrieUpdates;
use reth_chain_state::ExecutedBlock;
use reth_engine_primitives::EngineApiValidator;
use reth_ethereum::{
node::api::{
@@ -167,7 +167,7 @@ impl BuiltPayload for CustomBuiltPayload {
self.0.fees()
}
fn executed_block(&self) -> Option<ExecutedBlockWithTrieUpdates<Self::Primitives>> {
fn executed_block(&self) -> Option<ExecutedBlock<Self::Primitives>> {
self.0.executed_block()
}