reth_provider/providers/state/
historical.rs

1use crate::{
2    AccountReader, BlockHashReader, ChangeSetReader, EitherReader, HashedPostStateProvider,
3    ProviderError, RocksDBProviderFactory, StateProvider, StateRootProvider,
4};
5use alloy_eips::merge::EPOCH_SLOTS;
6use alloy_primitives::{Address, BlockNumber, Bytes, StorageKey, StorageValue, B256};
7use reth_db_api::{
8    cursor::{DbCursorRO, DbDupCursorRO},
9    table::Table,
10    tables,
11    transaction::DbTx,
12    BlockNumberList,
13};
14use reth_primitives_traits::{Account, Bytecode};
15use reth_storage_api::{
16    BlockNumReader, BytecodeReader, DBProvider, NodePrimitivesProvider, StateProofProvider,
17    StorageChangeSetReader, StorageRootProvider, StorageSettingsCache,
18};
19use reth_storage_errors::provider::ProviderResult;
20use reth_trie::{
21    proof::{Proof, StorageProof},
22    updates::TrieUpdates,
23    witness::TrieWitness,
24    AccountProof, HashedPostState, HashedPostStateSorted, HashedStorage, KeccakKeyHasher,
25    MultiProof, MultiProofTargets, StateRoot, StorageMultiProof, StorageRoot, TrieInput,
26    TrieInputSorted,
27};
28use reth_trie_db::{
29    hashed_storage_from_reverts_with_provider, DatabaseHashedPostState, DatabaseProof,
30    DatabaseStateRoot, DatabaseStorageProof, DatabaseStorageRoot, DatabaseTrieWitness,
31};
32
33use std::fmt::Debug;
34
35/// Result of a history lookup for an account or storage slot.
36///
37/// Indicates where to find the historical value for a given key at a specific block.
38#[derive(Debug, Eq, PartialEq)]
39pub enum HistoryInfo {
40    /// The key is written to, but only after our block (not yet written at the target block). Or
41    /// it has never been written.
42    NotYetWritten,
43    /// The chunk contains an entry for a write after our block at the given block number.
44    /// The value should be looked up in the changeset at this block.
45    InChangeset(u64),
46    /// The chunk does not contain an entry for a write after our block. This can only
47    /// happen if this is the last chunk, so we need to look in the plain state.
48    InPlainState,
49    /// The key may have been written, but due to pruning we may not have changesets and
50    /// history, so we need to make a plain state lookup.
51    MaybeInPlainState,
52}
53
54impl HistoryInfo {
55    /// Determines where to find the historical value based on computed shard lookup results.
56    ///
57    /// This is a pure function shared by both MDBX and `RocksDB` backends.
58    ///
59    /// # Arguments
60    /// * `found_block` - The block number from the shard lookup
61    /// * `is_before_first_write` - True if the target block is before the first write to this key.
62    ///   This should be computed as: `rank == 0 && found_block != Some(block_number) &&
63    ///   !has_previous_shard` where `has_previous_shard` comes from a lazy `cursor.prev()` check.
64    /// * `lowest_available` - Lowest block where history is available (pruning boundary)
65    pub const fn from_lookup(
66        found_block: Option<u64>,
67        is_before_first_write: bool,
68        lowest_available: Option<BlockNumber>,
69    ) -> Self {
70        if is_before_first_write {
71            if let (Some(_), Some(block_number)) = (lowest_available, found_block) {
72                // The key may have been written, but due to pruning we may not have changesets
73                // and history, so we need to make a changeset lookup.
74                return Self::InChangeset(block_number)
75            }
76            // The key is written to, but only after our block.
77            return Self::NotYetWritten
78        }
79
80        if let Some(block_number) = found_block {
81            // The chunk contains an entry for a write after our block, return it.
82            Self::InChangeset(block_number)
83        } else {
84            // The chunk does not contain an entry for a write after our block. This can only
85            // happen if this is the last chunk and so we need to look in the plain state.
86            Self::InPlainState
87        }
88    }
89}
90
91/// State provider for a given block number which takes a tx reference.
92///
93/// Historical state provider accesses the state at the start of the provided block number.
94/// It means that all changes made in the provided block number are not included.
95///
96/// Historical state provider reads the following tables:
97/// - [`tables::AccountsHistory`]
98/// - [`tables::Bytecodes`]
99/// - [`tables::StoragesHistory`]
100/// - [`tables::AccountChangeSets`]
101/// - [`tables::StorageChangeSets`]
102#[derive(Debug)]
103pub struct HistoricalStateProviderRef<'b, Provider> {
104    /// Database provider
105    provider: &'b Provider,
106    /// Block number is main index for the history state of accounts and storages.
107    block_number: BlockNumber,
108    /// Lowest blocks at which different parts of the state are available.
109    lowest_available_blocks: LowestAvailableBlocks,
110}
111
112impl<'b, Provider: DBProvider + ChangeSetReader + StorageChangeSetReader + BlockNumReader>
113    HistoricalStateProviderRef<'b, Provider>
114{
115    /// Create new `StateProvider` for historical block number
116    pub fn new(provider: &'b Provider, block_number: BlockNumber) -> Self {
117        Self { provider, block_number, lowest_available_blocks: Default::default() }
118    }
119
120    /// Create new `StateProvider` for historical block number and lowest block numbers at which
121    /// account & storage histories are available.
122    pub const fn new_with_lowest_available_blocks(
123        provider: &'b Provider,
124        block_number: BlockNumber,
125        lowest_available_blocks: LowestAvailableBlocks,
126    ) -> Self {
127        Self { provider, block_number, lowest_available_blocks }
128    }
129
130    /// Lookup an account in the `AccountsHistory` table using `EitherReader`.
131    pub fn account_history_lookup(&self, address: Address) -> ProviderResult<HistoryInfo>
132    where
133        Provider: StorageSettingsCache + RocksDBProviderFactory + NodePrimitivesProvider,
134    {
135        if !self.lowest_available_blocks.is_account_history_available(self.block_number) {
136            return Err(ProviderError::StateAtBlockPruned(self.block_number))
137        }
138
139        self.provider.with_rocksdb_tx(|rocks_tx_ref| {
140            let mut reader = EitherReader::new_accounts_history(self.provider, rocks_tx_ref)?;
141            reader.account_history_info(
142                address,
143                self.block_number,
144                self.lowest_available_blocks.account_history_block_number,
145            )
146        })
147    }
148
149    /// Lookup a storage key in the `StoragesHistory` table using `EitherReader`.
150    pub fn storage_history_lookup(
151        &self,
152        address: Address,
153        storage_key: StorageKey,
154    ) -> ProviderResult<HistoryInfo>
155    where
156        Provider: StorageSettingsCache + RocksDBProviderFactory + NodePrimitivesProvider,
157    {
158        if !self.lowest_available_blocks.is_storage_history_available(self.block_number) {
159            return Err(ProviderError::StateAtBlockPruned(self.block_number))
160        }
161
162        self.provider.with_rocksdb_tx(|rocks_tx_ref| {
163            let mut reader = EitherReader::new_storages_history(self.provider, rocks_tx_ref)?;
164            reader.storage_history_info(
165                address,
166                storage_key,
167                self.block_number,
168                self.lowest_available_blocks.storage_history_block_number,
169            )
170        })
171    }
172
173    /// Checks and returns `true` if distance to historical block exceeds the provided limit.
174    fn check_distance_against_limit(&self, limit: u64) -> ProviderResult<bool> {
175        let tip = self.provider.last_block_number()?;
176
177        Ok(tip.saturating_sub(self.block_number) > limit)
178    }
179
180    /// Retrieve revert hashed state for this history provider.
181    fn revert_state(&self) -> ProviderResult<HashedPostStateSorted> {
182        if !self.lowest_available_blocks.is_account_history_available(self.block_number) ||
183            !self.lowest_available_blocks.is_storage_history_available(self.block_number)
184        {
185            return Err(ProviderError::StateAtBlockPruned(self.block_number))
186        }
187
188        if self.check_distance_against_limit(EPOCH_SLOTS)? {
189            tracing::warn!(
190                target: "provider::historical_sp",
191                target = self.block_number,
192                "Attempt to calculate state root for an old block might result in OOM"
193            );
194        }
195
196        HashedPostStateSorted::from_reverts::<KeccakKeyHasher>(self.provider, self.block_number..)
197    }
198
199    /// Retrieve revert hashed storage for this history provider and target address.
200    fn revert_storage(&self, address: Address) -> ProviderResult<HashedStorage> {
201        if !self.lowest_available_blocks.is_storage_history_available(self.block_number) {
202            return Err(ProviderError::StateAtBlockPruned(self.block_number))
203        }
204
205        if self.check_distance_against_limit(EPOCH_SLOTS * 10)? {
206            tracing::warn!(
207                target: "provider::historical_sp",
208                target = self.block_number,
209                "Attempt to calculate storage root for an old block might result in OOM"
210            );
211        }
212
213        hashed_storage_from_reverts_with_provider(self.provider, address, self.block_number)
214    }
215
216    /// Set the lowest block number at which the account history is available.
217    pub const fn with_lowest_available_account_history_block_number(
218        mut self,
219        block_number: BlockNumber,
220    ) -> Self {
221        self.lowest_available_blocks.account_history_block_number = Some(block_number);
222        self
223    }
224
225    /// Set the lowest block number at which the storage history is available.
226    pub const fn with_lowest_available_storage_history_block_number(
227        mut self,
228        block_number: BlockNumber,
229    ) -> Self {
230        self.lowest_available_blocks.storage_history_block_number = Some(block_number);
231        self
232    }
233}
234
235impl<Provider: DBProvider + BlockNumReader> HistoricalStateProviderRef<'_, Provider> {
236    fn tx(&self) -> &Provider::Tx {
237        self.provider.tx_ref()
238    }
239}
240
241impl<
242        Provider: DBProvider
243            + BlockNumReader
244            + ChangeSetReader
245            + StorageChangeSetReader
246            + StorageSettingsCache
247            + RocksDBProviderFactory
248            + NodePrimitivesProvider,
249    > AccountReader for HistoricalStateProviderRef<'_, Provider>
250{
251    /// Get basic account information.
252    fn basic_account(&self, address: &Address) -> ProviderResult<Option<Account>> {
253        match self.account_history_lookup(*address)? {
254            HistoryInfo::NotYetWritten => Ok(None),
255            HistoryInfo::InChangeset(changeset_block_number) => {
256                // Use ChangeSetReader trait method to get the account from changesets
257                self.provider
258                    .get_account_before_block(changeset_block_number, *address)?
259                    .ok_or(ProviderError::AccountChangesetNotFound {
260                        block_number: changeset_block_number,
261                        address: *address,
262                    })
263                    .map(|account_before| account_before.info)
264            }
265            HistoryInfo::InPlainState | HistoryInfo::MaybeInPlainState => {
266                Ok(self.tx().get_by_encoded_key::<tables::PlainAccountState>(address)?)
267            }
268        }
269    }
270}
271
272impl<Provider: DBProvider + BlockNumReader + BlockHashReader> BlockHashReader
273    for HistoricalStateProviderRef<'_, Provider>
274{
275    /// Get block hash by number.
276    fn block_hash(&self, number: u64) -> ProviderResult<Option<B256>> {
277        self.provider.block_hash(number)
278    }
279
280    fn canonical_hashes_range(
281        &self,
282        start: BlockNumber,
283        end: BlockNumber,
284    ) -> ProviderResult<Vec<B256>> {
285        self.provider.canonical_hashes_range(start, end)
286    }
287}
288
289impl<Provider: DBProvider + ChangeSetReader + StorageChangeSetReader + BlockNumReader>
290    StateRootProvider for HistoricalStateProviderRef<'_, Provider>
291{
292    fn state_root(&self, hashed_state: HashedPostState) -> ProviderResult<B256> {
293        let mut revert_state = self.revert_state()?;
294        let hashed_state_sorted = hashed_state.into_sorted();
295        revert_state.extend_ref_and_sort(&hashed_state_sorted);
296        Ok(StateRoot::overlay_root(self.tx(), &revert_state)?)
297    }
298
299    fn state_root_from_nodes(&self, mut input: TrieInput) -> ProviderResult<B256> {
300        input.prepend(self.revert_state()?.into());
301        Ok(StateRoot::overlay_root_from_nodes(self.tx(), TrieInputSorted::from_unsorted(input))?)
302    }
303
304    fn state_root_with_updates(
305        &self,
306        hashed_state: HashedPostState,
307    ) -> ProviderResult<(B256, TrieUpdates)> {
308        let mut revert_state = self.revert_state()?;
309        let hashed_state_sorted = hashed_state.into_sorted();
310        revert_state.extend_ref_and_sort(&hashed_state_sorted);
311        Ok(StateRoot::overlay_root_with_updates(self.tx(), &revert_state)?)
312    }
313
314    fn state_root_from_nodes_with_updates(
315        &self,
316        mut input: TrieInput,
317    ) -> ProviderResult<(B256, TrieUpdates)> {
318        input.prepend(self.revert_state()?.into());
319        Ok(StateRoot::overlay_root_from_nodes_with_updates(
320            self.tx(),
321            TrieInputSorted::from_unsorted(input),
322        )?)
323    }
324}
325
326impl<Provider: DBProvider + ChangeSetReader + StorageChangeSetReader + BlockNumReader>
327    StorageRootProvider for HistoricalStateProviderRef<'_, Provider>
328{
329    fn storage_root(
330        &self,
331        address: Address,
332        hashed_storage: HashedStorage,
333    ) -> ProviderResult<B256> {
334        let mut revert_storage = self.revert_storage(address)?;
335        revert_storage.extend(&hashed_storage);
336        StorageRoot::overlay_root(self.tx(), address, revert_storage)
337            .map_err(|err| ProviderError::Database(err.into()))
338    }
339
340    fn storage_proof(
341        &self,
342        address: Address,
343        slot: B256,
344        hashed_storage: HashedStorage,
345    ) -> ProviderResult<reth_trie::StorageProof> {
346        let mut revert_storage = self.revert_storage(address)?;
347        revert_storage.extend(&hashed_storage);
348        StorageProof::overlay_storage_proof(self.tx(), address, slot, revert_storage)
349            .map_err(ProviderError::from)
350    }
351
352    fn storage_multiproof(
353        &self,
354        address: Address,
355        slots: &[B256],
356        hashed_storage: HashedStorage,
357    ) -> ProviderResult<StorageMultiProof> {
358        let mut revert_storage = self.revert_storage(address)?;
359        revert_storage.extend(&hashed_storage);
360        StorageProof::overlay_storage_multiproof(self.tx(), address, slots, revert_storage)
361            .map_err(ProviderError::from)
362    }
363}
364
365impl<Provider: DBProvider + ChangeSetReader + StorageChangeSetReader + BlockNumReader>
366    StateProofProvider for HistoricalStateProviderRef<'_, Provider>
367{
368    /// Get account and storage proofs.
369    fn proof(
370        &self,
371        mut input: TrieInput,
372        address: Address,
373        slots: &[B256],
374    ) -> ProviderResult<AccountProof> {
375        input.prepend(self.revert_state()?.into());
376        let proof = <Proof<_, _> as DatabaseProof>::from_tx(self.tx());
377        proof.overlay_account_proof(input, address, slots).map_err(ProviderError::from)
378    }
379
380    fn multiproof(
381        &self,
382        mut input: TrieInput,
383        targets: MultiProofTargets,
384    ) -> ProviderResult<MultiProof> {
385        input.prepend(self.revert_state()?.into());
386        let proof = <Proof<_, _> as DatabaseProof>::from_tx(self.tx());
387        proof.overlay_multiproof(input, targets).map_err(ProviderError::from)
388    }
389
390    fn witness(&self, mut input: TrieInput, target: HashedPostState) -> ProviderResult<Vec<Bytes>> {
391        input.prepend(self.revert_state()?.into());
392        TrieWitness::overlay_witness(self.tx(), input, target)
393            .map_err(ProviderError::from)
394            .map(|hm| hm.into_values().collect())
395    }
396}
397
398impl<Provider> HashedPostStateProvider for HistoricalStateProviderRef<'_, Provider> {
399    fn hashed_post_state(&self, bundle_state: &revm_database::BundleState) -> HashedPostState {
400        HashedPostState::from_bundle_state::<KeccakKeyHasher>(bundle_state.state())
401    }
402}
403
404impl<
405        Provider: DBProvider
406            + BlockNumReader
407            + BlockHashReader
408            + ChangeSetReader
409            + StorageChangeSetReader
410            + StorageSettingsCache
411            + RocksDBProviderFactory
412            + NodePrimitivesProvider,
413    > StateProvider for HistoricalStateProviderRef<'_, Provider>
414{
415    /// Get storage.
416    fn storage(
417        &self,
418        address: Address,
419        storage_key: StorageKey,
420    ) -> ProviderResult<Option<StorageValue>> {
421        match self.storage_history_lookup(address, storage_key)? {
422            HistoryInfo::NotYetWritten => Ok(None),
423            HistoryInfo::InChangeset(changeset_block_number) => self
424                .provider
425                .get_storage_before_block(changeset_block_number, address, storage_key)?
426                .ok_or_else(|| ProviderError::StorageChangesetNotFound {
427                    block_number: changeset_block_number,
428                    address,
429                    storage_key: Box::new(storage_key),
430                })
431                .map(|entry| entry.value)
432                .map(Some),
433            HistoryInfo::InPlainState | HistoryInfo::MaybeInPlainState => Ok(self
434                .tx()
435                .cursor_dup_read::<tables::PlainStorageState>()?
436                .seek_by_key_subkey(address, storage_key)?
437                .filter(|entry| entry.key == storage_key)
438                .map(|entry| entry.value)
439                .or(Some(StorageValue::ZERO))),
440        }
441    }
442}
443
444impl<Provider: DBProvider + BlockNumReader> BytecodeReader
445    for HistoricalStateProviderRef<'_, Provider>
446{
447    /// Get account code by its hash
448    fn bytecode_by_hash(&self, code_hash: &B256) -> ProviderResult<Option<Bytecode>> {
449        self.tx().get_by_encoded_key::<tables::Bytecodes>(code_hash).map_err(Into::into)
450    }
451}
452
453/// State provider for a given block number.
454/// For more detailed description, see [`HistoricalStateProviderRef`].
455#[derive(Debug)]
456pub struct HistoricalStateProvider<Provider> {
457    /// Database provider.
458    provider: Provider,
459    /// State at the block number is the main indexer of the state.
460    block_number: BlockNumber,
461    /// Lowest blocks at which different parts of the state are available.
462    lowest_available_blocks: LowestAvailableBlocks,
463}
464
465impl<Provider: DBProvider + ChangeSetReader + StorageChangeSetReader + BlockNumReader>
466    HistoricalStateProvider<Provider>
467{
468    /// Create new `StateProvider` for historical block number
469    pub fn new(provider: Provider, block_number: BlockNumber) -> Self {
470        Self { provider, block_number, lowest_available_blocks: Default::default() }
471    }
472
473    /// Set the lowest block number at which the account history is available.
474    pub const fn with_lowest_available_account_history_block_number(
475        mut self,
476        block_number: BlockNumber,
477    ) -> Self {
478        self.lowest_available_blocks.account_history_block_number = Some(block_number);
479        self
480    }
481
482    /// Set the lowest block number at which the storage history is available.
483    pub const fn with_lowest_available_storage_history_block_number(
484        mut self,
485        block_number: BlockNumber,
486    ) -> Self {
487        self.lowest_available_blocks.storage_history_block_number = Some(block_number);
488        self
489    }
490
491    /// Returns a new provider that takes the `TX` as reference
492    #[inline(always)]
493    const fn as_ref(&self) -> HistoricalStateProviderRef<'_, Provider> {
494        HistoricalStateProviderRef::new_with_lowest_available_blocks(
495            &self.provider,
496            self.block_number,
497            self.lowest_available_blocks,
498        )
499    }
500}
501
502// Delegates all provider impls to [HistoricalStateProviderRef]
503reth_storage_api::macros::delegate_provider_impls!(HistoricalStateProvider<Provider> where [Provider: DBProvider + BlockNumReader + BlockHashReader + ChangeSetReader + StorageChangeSetReader + StorageSettingsCache + RocksDBProviderFactory + NodePrimitivesProvider]);
504
505/// Lowest blocks at which different parts of the state are available.
506/// They may be [Some] if pruning is enabled.
507#[derive(Clone, Copy, Debug, Default)]
508pub struct LowestAvailableBlocks {
509    /// Lowest block number at which the account history is available. It may not be available if
510    /// [`reth_prune_types::PruneSegment::AccountHistory`] was pruned.
511    /// [`Option::None`] means all history is available.
512    pub account_history_block_number: Option<BlockNumber>,
513    /// Lowest block number at which the storage history is available. It may not be available if
514    /// [`reth_prune_types::PruneSegment::StorageHistory`] was pruned.
515    /// [`Option::None`] means all history is available.
516    pub storage_history_block_number: Option<BlockNumber>,
517}
518
519impl LowestAvailableBlocks {
520    /// Check if account history is available at the provided block number, i.e. lowest available
521    /// block number for account history is less than or equal to the provided block number.
522    pub fn is_account_history_available(&self, at: BlockNumber) -> bool {
523        self.account_history_block_number.map(|block_number| block_number <= at).unwrap_or(true)
524    }
525
526    /// Check if storage history is available at the provided block number, i.e. lowest available
527    /// block number for storage history is less than or equal to the provided block number.
528    pub fn is_storage_history_available(&self, at: BlockNumber) -> bool {
529        self.storage_history_block_number.map(|block_number| block_number <= at).unwrap_or(true)
530    }
531}
532
533/// Computes the rank and finds the next modification block in a history shard.
534///
535/// Given a `block_number`, this function returns:
536/// - `rank`: The number of entries strictly before `block_number` in the shard
537/// - `found_block`: The block number at position `rank` (i.e., the first block >= `block_number`
538///   where a modification occurred), or `None` if `rank` is out of bounds
539///
540/// The rank is adjusted when `block_number` exactly matches an entry in the shard,
541/// so that `found_block` always returns the modification at or after the target.
542///
543/// This logic is shared between MDBX cursor-based lookups and `RocksDB` iterator lookups.
544#[inline]
545pub fn compute_history_rank(
546    chunk: &reth_db_api::BlockNumberList,
547    block_number: BlockNumber,
548) -> (u64, Option<u64>) {
549    let mut rank = chunk.rank(block_number);
550    // `rank(block_number)` returns count of entries <= block_number.
551    // We want the first entry >= block_number, so if block_number is in the shard,
552    // we need to step back one position to point at it (not past it).
553    if rank.checked_sub(1).and_then(|r| chunk.select(r)) == Some(block_number) {
554        rank -= 1;
555    }
556    (rank, chunk.select(rank))
557}
558
559/// Checks if a previous shard lookup is needed to determine if we're before the first write.
560///
561/// Returns `true` when `rank == 0` (first entry in shard) and the found block doesn't match
562/// the target block number. In this case, we need to check if there's a previous shard.
563#[inline]
564pub fn needs_prev_shard_check(
565    rank: u64,
566    found_block: Option<u64>,
567    block_number: BlockNumber,
568) -> bool {
569    rank == 0 && found_block != Some(block_number)
570}
571
572/// Generic history lookup for sharded history tables.
573///
574/// Seeks to the shard containing `block_number`, verifies the key via `key_filter`,
575/// and checks previous shard to detect if we're before the first write.
576pub fn history_info<T, K, C>(
577    cursor: &mut C,
578    key: K,
579    block_number: BlockNumber,
580    key_filter: impl Fn(&K) -> bool,
581    lowest_available_block_number: Option<BlockNumber>,
582) -> ProviderResult<HistoryInfo>
583where
584    T: Table<Key = K, Value = BlockNumberList>,
585    C: DbCursorRO<T>,
586{
587    // Lookup the history chunk in the history index. If the key does not appear in the
588    // index, the first chunk for the next key will be returned so we filter out chunks that
589    // have a different key.
590    if let Some(chunk) = cursor.seek(key)?.filter(|(k, _)| key_filter(k)).map(|x| x.1) {
591        let (rank, found_block) = compute_history_rank(&chunk, block_number);
592
593        // If our block is before the first entry in the index chunk and this first entry
594        // doesn't equal to our block, it might be before the first write ever. To check, we
595        // look at the previous entry and check if the key is the same.
596        // This check is worth it, the `cursor.prev()` check is rarely triggered (the if will
597        // short-circuit) and when it passes we save a full seek into the changeset/plain state
598        // table.
599        let is_before_first_write = needs_prev_shard_check(rank, found_block, block_number) &&
600            !cursor.prev()?.is_some_and(|(k, _)| key_filter(&k));
601
602        Ok(HistoryInfo::from_lookup(
603            found_block,
604            is_before_first_write,
605            lowest_available_block_number,
606        ))
607    } else if lowest_available_block_number.is_some() {
608        // The key may have been written, but due to pruning we may not have changesets and
609        // history, so we need to make a plain state lookup.
610        Ok(HistoryInfo::MaybeInPlainState)
611    } else {
612        // The key has not been written to at all.
613        Ok(HistoryInfo::NotYetWritten)
614    }
615}
616
617#[cfg(test)]
618mod tests {
619    use super::needs_prev_shard_check;
620    use crate::{
621        providers::state::historical::{HistoryInfo, LowestAvailableBlocks},
622        test_utils::create_test_provider_factory,
623        AccountReader, HistoricalStateProvider, HistoricalStateProviderRef, RocksDBProviderFactory,
624        StateProvider,
625    };
626    use alloy_primitives::{address, b256, Address, B256, U256};
627    use reth_db_api::{
628        models::{storage_sharded_key::StorageShardedKey, AccountBeforeTx, ShardedKey},
629        tables,
630        transaction::{DbTx, DbTxMut},
631        BlockNumberList,
632    };
633    use reth_primitives_traits::{Account, StorageEntry};
634    use reth_storage_api::{
635        BlockHashReader, BlockNumReader, ChangeSetReader, DBProvider, DatabaseProviderFactory,
636        NodePrimitivesProvider, StorageChangeSetReader, StorageSettingsCache,
637    };
638    use reth_storage_errors::provider::ProviderError;
639
640    const ADDRESS: Address = address!("0x0000000000000000000000000000000000000001");
641    const HIGHER_ADDRESS: Address = address!("0x0000000000000000000000000000000000000005");
642    const STORAGE: B256 =
643        b256!("0x0000000000000000000000000000000000000000000000000000000000000001");
644
645    const fn assert_state_provider<T: StateProvider>() {}
646    #[expect(dead_code)]
647    const fn assert_historical_state_provider<
648        T: DBProvider
649            + BlockNumReader
650            + BlockHashReader
651            + ChangeSetReader
652            + StorageChangeSetReader
653            + StorageSettingsCache
654            + RocksDBProviderFactory
655            + NodePrimitivesProvider,
656    >() {
657        assert_state_provider::<HistoricalStateProvider<T>>();
658    }
659
660    #[test]
661    fn history_provider_get_account() {
662        let factory = create_test_provider_factory();
663        let tx = factory.provider_rw().unwrap().into_tx();
664
665        tx.put::<tables::AccountsHistory>(
666            ShardedKey { key: ADDRESS, highest_block_number: 7 },
667            BlockNumberList::new([1, 3, 7]).unwrap(),
668        )
669        .unwrap();
670        tx.put::<tables::AccountsHistory>(
671            ShardedKey { key: ADDRESS, highest_block_number: u64::MAX },
672            BlockNumberList::new([10, 15]).unwrap(),
673        )
674        .unwrap();
675        tx.put::<tables::AccountsHistory>(
676            ShardedKey { key: HIGHER_ADDRESS, highest_block_number: u64::MAX },
677            BlockNumberList::new([4]).unwrap(),
678        )
679        .unwrap();
680
681        let acc_plain = Account { nonce: 100, balance: U256::ZERO, bytecode_hash: None };
682        let acc_at15 = Account { nonce: 15, balance: U256::ZERO, bytecode_hash: None };
683        let acc_at10 = Account { nonce: 10, balance: U256::ZERO, bytecode_hash: None };
684        let acc_at7 = Account { nonce: 7, balance: U256::ZERO, bytecode_hash: None };
685        let acc_at3 = Account { nonce: 3, balance: U256::ZERO, bytecode_hash: None };
686
687        let higher_acc_plain = Account { nonce: 4, balance: U256::ZERO, bytecode_hash: None };
688
689        // setup
690        tx.put::<tables::AccountChangeSets>(1, AccountBeforeTx { address: ADDRESS, info: None })
691            .unwrap();
692        tx.put::<tables::AccountChangeSets>(
693            3,
694            AccountBeforeTx { address: ADDRESS, info: Some(acc_at3) },
695        )
696        .unwrap();
697        tx.put::<tables::AccountChangeSets>(
698            4,
699            AccountBeforeTx { address: HIGHER_ADDRESS, info: None },
700        )
701        .unwrap();
702        tx.put::<tables::AccountChangeSets>(
703            7,
704            AccountBeforeTx { address: ADDRESS, info: Some(acc_at7) },
705        )
706        .unwrap();
707        tx.put::<tables::AccountChangeSets>(
708            10,
709            AccountBeforeTx { address: ADDRESS, info: Some(acc_at10) },
710        )
711        .unwrap();
712        tx.put::<tables::AccountChangeSets>(
713            15,
714            AccountBeforeTx { address: ADDRESS, info: Some(acc_at15) },
715        )
716        .unwrap();
717
718        // setup plain state
719        tx.put::<tables::PlainAccountState>(ADDRESS, acc_plain).unwrap();
720        tx.put::<tables::PlainAccountState>(HIGHER_ADDRESS, higher_acc_plain).unwrap();
721        tx.commit().unwrap();
722
723        let db = factory.provider().unwrap();
724
725        // run
726        assert!(matches!(
727            HistoricalStateProviderRef::new(&db, 1).basic_account(&ADDRESS),
728            Ok(None)
729        ));
730        assert!(matches!(
731            HistoricalStateProviderRef::new(&db, 2).basic_account(&ADDRESS),
732            Ok(Some(acc)) if acc == acc_at3
733        ));
734        assert!(matches!(
735            HistoricalStateProviderRef::new(&db, 3).basic_account(&ADDRESS),
736            Ok(Some(acc)) if acc == acc_at3
737        ));
738        assert!(matches!(
739            HistoricalStateProviderRef::new(&db, 4).basic_account(&ADDRESS),
740            Ok(Some(acc)) if acc == acc_at7
741        ));
742        assert!(matches!(
743            HistoricalStateProviderRef::new(&db, 7).basic_account(&ADDRESS),
744            Ok(Some(acc)) if acc == acc_at7
745        ));
746        assert!(matches!(
747            HistoricalStateProviderRef::new(&db, 9).basic_account(&ADDRESS),
748            Ok(Some(acc)) if acc == acc_at10
749        ));
750        assert!(matches!(
751            HistoricalStateProviderRef::new(&db, 10).basic_account(&ADDRESS),
752            Ok(Some(acc)) if acc == acc_at10
753        ));
754        assert!(matches!(
755            HistoricalStateProviderRef::new(&db, 11).basic_account(&ADDRESS),
756            Ok(Some(acc)) if acc == acc_at15
757        ));
758        assert!(matches!(
759            HistoricalStateProviderRef::new(&db, 16).basic_account(&ADDRESS),
760            Ok(Some(acc)) if acc == acc_plain
761        ));
762
763        assert!(matches!(
764            HistoricalStateProviderRef::new(&db, 1).basic_account(&HIGHER_ADDRESS),
765            Ok(None)
766        ));
767        assert!(matches!(
768            HistoricalStateProviderRef::new(&db, 1000).basic_account(&HIGHER_ADDRESS),
769            Ok(Some(acc)) if acc == higher_acc_plain
770        ));
771    }
772
773    #[test]
774    fn history_provider_get_storage() {
775        let factory = create_test_provider_factory();
776        let tx = factory.provider_rw().unwrap().into_tx();
777
778        tx.put::<tables::StoragesHistory>(
779            StorageShardedKey {
780                address: ADDRESS,
781                sharded_key: ShardedKey { key: STORAGE, highest_block_number: 7 },
782            },
783            BlockNumberList::new([3, 7]).unwrap(),
784        )
785        .unwrap();
786        tx.put::<tables::StoragesHistory>(
787            StorageShardedKey {
788                address: ADDRESS,
789                sharded_key: ShardedKey { key: STORAGE, highest_block_number: u64::MAX },
790            },
791            BlockNumberList::new([10, 15]).unwrap(),
792        )
793        .unwrap();
794        tx.put::<tables::StoragesHistory>(
795            StorageShardedKey {
796                address: HIGHER_ADDRESS,
797                sharded_key: ShardedKey { key: STORAGE, highest_block_number: u64::MAX },
798            },
799            BlockNumberList::new([4]).unwrap(),
800        )
801        .unwrap();
802
803        let higher_entry_plain = StorageEntry { key: STORAGE, value: U256::from(1000) };
804        let higher_entry_at4 = StorageEntry { key: STORAGE, value: U256::from(0) };
805        let entry_plain = StorageEntry { key: STORAGE, value: U256::from(100) };
806        let entry_at15 = StorageEntry { key: STORAGE, value: U256::from(15) };
807        let entry_at10 = StorageEntry { key: STORAGE, value: U256::from(10) };
808        let entry_at7 = StorageEntry { key: STORAGE, value: U256::from(7) };
809        let entry_at3 = StorageEntry { key: STORAGE, value: U256::from(0) };
810
811        // setup
812        tx.put::<tables::StorageChangeSets>((3, ADDRESS).into(), entry_at3).unwrap();
813        tx.put::<tables::StorageChangeSets>((4, HIGHER_ADDRESS).into(), higher_entry_at4).unwrap();
814        tx.put::<tables::StorageChangeSets>((7, ADDRESS).into(), entry_at7).unwrap();
815        tx.put::<tables::StorageChangeSets>((10, ADDRESS).into(), entry_at10).unwrap();
816        tx.put::<tables::StorageChangeSets>((15, ADDRESS).into(), entry_at15).unwrap();
817
818        // setup plain state
819        tx.put::<tables::PlainStorageState>(ADDRESS, entry_plain).unwrap();
820        tx.put::<tables::PlainStorageState>(HIGHER_ADDRESS, higher_entry_plain).unwrap();
821        tx.commit().unwrap();
822
823        let db = factory.provider().unwrap();
824
825        // run
826        assert!(matches!(
827            HistoricalStateProviderRef::new(&db, 0).storage(ADDRESS, STORAGE),
828            Ok(None)
829        ));
830        assert!(matches!(
831            HistoricalStateProviderRef::new(&db, 3).storage(ADDRESS, STORAGE),
832            Ok(Some(U256::ZERO))
833        ));
834        assert!(matches!(
835            HistoricalStateProviderRef::new(&db, 4).storage(ADDRESS, STORAGE),
836            Ok(Some(expected_value)) if expected_value == entry_at7.value
837        ));
838        assert!(matches!(
839            HistoricalStateProviderRef::new(&db, 7).storage(ADDRESS, STORAGE),
840            Ok(Some(expected_value)) if expected_value == entry_at7.value
841        ));
842        assert!(matches!(
843            HistoricalStateProviderRef::new(&db, 9).storage(ADDRESS, STORAGE),
844            Ok(Some(expected_value)) if expected_value == entry_at10.value
845        ));
846        assert!(matches!(
847            HistoricalStateProviderRef::new(&db, 10).storage(ADDRESS, STORAGE),
848            Ok(Some(expected_value)) if expected_value == entry_at10.value
849        ));
850        assert!(matches!(
851            HistoricalStateProviderRef::new(&db, 11).storage(ADDRESS, STORAGE),
852            Ok(Some(expected_value)) if expected_value == entry_at15.value
853        ));
854        assert!(matches!(
855            HistoricalStateProviderRef::new(&db, 16).storage(ADDRESS, STORAGE),
856            Ok(Some(expected_value)) if expected_value == entry_plain.value
857        ));
858        assert!(matches!(
859            HistoricalStateProviderRef::new(&db, 1).storage(HIGHER_ADDRESS, STORAGE),
860            Ok(None)
861        ));
862        assert!(matches!(
863            HistoricalStateProviderRef::new(&db, 1000).storage(HIGHER_ADDRESS, STORAGE),
864            Ok(Some(expected_value)) if expected_value == higher_entry_plain.value
865        ));
866    }
867
868    #[test]
869    fn history_provider_unavailable() {
870        let factory = create_test_provider_factory();
871        let db = factory.database_provider_rw().unwrap();
872
873        // provider block_number < lowest available block number,
874        // i.e. state at provider block is pruned
875        let provider = HistoricalStateProviderRef::new_with_lowest_available_blocks(
876            &db,
877            2,
878            LowestAvailableBlocks {
879                account_history_block_number: Some(3),
880                storage_history_block_number: Some(3),
881            },
882        );
883        assert!(matches!(
884            provider.account_history_lookup(ADDRESS),
885            Err(ProviderError::StateAtBlockPruned(number)) if number == provider.block_number
886        ));
887        assert!(matches!(
888            provider.storage_history_lookup(ADDRESS, STORAGE),
889            Err(ProviderError::StateAtBlockPruned(number)) if number == provider.block_number
890        ));
891
892        // provider block_number == lowest available block number,
893        // i.e. state at provider block is available
894        let provider = HistoricalStateProviderRef::new_with_lowest_available_blocks(
895            &db,
896            2,
897            LowestAvailableBlocks {
898                account_history_block_number: Some(2),
899                storage_history_block_number: Some(2),
900            },
901        );
902        assert!(matches!(
903            provider.account_history_lookup(ADDRESS),
904            Ok(HistoryInfo::MaybeInPlainState)
905        ));
906        assert!(matches!(
907            provider.storage_history_lookup(ADDRESS, STORAGE),
908            Ok(HistoryInfo::MaybeInPlainState)
909        ));
910
911        // provider block_number == lowest available block number,
912        // i.e. state at provider block is available
913        let provider = HistoricalStateProviderRef::new_with_lowest_available_blocks(
914            &db,
915            2,
916            LowestAvailableBlocks {
917                account_history_block_number: Some(1),
918                storage_history_block_number: Some(1),
919            },
920        );
921        assert!(matches!(
922            provider.account_history_lookup(ADDRESS),
923            Ok(HistoryInfo::MaybeInPlainState)
924        ));
925        assert!(matches!(
926            provider.storage_history_lookup(ADDRESS, STORAGE),
927            Ok(HistoryInfo::MaybeInPlainState)
928        ));
929    }
930
931    #[test]
932    fn test_history_info_from_lookup() {
933        // Before first write, no pruning → not yet written
934        assert_eq!(HistoryInfo::from_lookup(Some(10), true, None), HistoryInfo::NotYetWritten);
935        assert_eq!(HistoryInfo::from_lookup(None, true, None), HistoryInfo::NotYetWritten);
936
937        // Before first write WITH pruning → check changeset (pruning may have removed history)
938        assert_eq!(HistoryInfo::from_lookup(Some(10), true, Some(5)), HistoryInfo::InChangeset(10));
939        assert_eq!(HistoryInfo::from_lookup(None, true, Some(5)), HistoryInfo::NotYetWritten);
940
941        // Not before first write → check changeset or plain state
942        assert_eq!(HistoryInfo::from_lookup(Some(10), false, None), HistoryInfo::InChangeset(10));
943        assert_eq!(HistoryInfo::from_lookup(None, false, None), HistoryInfo::InPlainState);
944    }
945
946    #[test]
947    fn test_needs_prev_shard_check() {
948        // Only needs check when rank == 0 and found_block != block_number
949        assert!(needs_prev_shard_check(0, Some(10), 5));
950        assert!(needs_prev_shard_check(0, None, 5));
951        assert!(!needs_prev_shard_check(0, Some(5), 5)); // found_block == block_number
952        assert!(!needs_prev_shard_check(1, Some(10), 5)); // rank > 0
953    }
954}