mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: extract storage traits to storage-api (#8427)
This commit is contained in:
14
Cargo.lock
generated
14
Cargo.lock
generated
@ -7677,6 +7677,7 @@ dependencies = [
|
||||
"reth-metrics",
|
||||
"reth-nippy-jar",
|
||||
"reth-primitives",
|
||||
"reth-storage-api",
|
||||
"reth-storage-errors",
|
||||
"reth-trie",
|
||||
"revm",
|
||||
@ -8018,6 +8019,19 @@ dependencies = [
|
||||
"strum",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "reth-storage-api"
|
||||
version = "0.2.0-beta.7"
|
||||
dependencies = [
|
||||
"auto_impl",
|
||||
"reth-db",
|
||||
"reth-execution-types",
|
||||
"reth-primitives",
|
||||
"reth-storage-errors",
|
||||
"reth-trie",
|
||||
"revm",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "reth-storage-errors"
|
||||
version = "0.2.0-beta.7"
|
||||
|
||||
@ -74,6 +74,7 @@ members = [
|
||||
"crates/storage/libmdbx-rs/mdbx-sys/",
|
||||
"crates/storage/nippy-jar/",
|
||||
"crates/storage/provider/",
|
||||
"crates/storage/storage-api/",
|
||||
"crates/tasks/",
|
||||
"crates/tokio-util/",
|
||||
"crates/tracing/",
|
||||
@ -288,6 +289,7 @@ reth-stages = { path = "crates/stages" }
|
||||
reth-stages-api = { path = "crates/stages-api" }
|
||||
reth-static-file = { path = "crates/static-file" }
|
||||
reth-static-file-types = { path = "crates/static-file-types" }
|
||||
reth-storage-api = { path = "crates/storage/storage-api" }
|
||||
reth-storage-errors = { path = "crates/storage/errors" }
|
||||
reth-tasks = { path = "crates/tasks" }
|
||||
reth-testing-utils = { path = "testing/testing-utils" }
|
||||
|
||||
@ -20,6 +20,7 @@ reth-primitives.workspace = true
|
||||
reth-fs-util.workspace = true
|
||||
reth-storage-errors.workspace = true
|
||||
reth-interfaces.workspace = true
|
||||
reth-storage-api.workspace = true
|
||||
reth-db.workspace = true
|
||||
reth-trie = { workspace = true, features = ["metrics"] }
|
||||
reth-nippy-jar.workspace = true
|
||||
|
||||
@ -1,16 +1,12 @@
|
||||
use crate::{
|
||||
providers::StaticFileProviderRWRefMut, BundleStateDataProvider, StateChanges, StateReverts,
|
||||
StateWriter,
|
||||
};
|
||||
use crate::{providers::StaticFileProviderRWRefMut, StateChanges, StateReverts, StateWriter};
|
||||
use reth_db::{
|
||||
cursor::{DbCursorRO, DbCursorRW},
|
||||
tables,
|
||||
transaction::{DbTx, DbTxMut},
|
||||
};
|
||||
use reth_primitives::{BlockHash, BlockNumber, StaticFileSegment};
|
||||
use reth_storage_errors::provider::{ProviderError, ProviderResult};
|
||||
|
||||
pub use reth_execution_types::*;
|
||||
use reth_primitives::StaticFileSegment;
|
||||
use reth_storage_errors::provider::{ProviderError, ProviderResult};
|
||||
pub use revm::db::states::OriginalValuesKnown;
|
||||
|
||||
impl StateWriter for BundleStateWithReceipts {
|
||||
@ -67,17 +63,6 @@ impl StateWriter for BundleStateWithReceipts {
|
||||
}
|
||||
}
|
||||
|
||||
impl BundleStateDataProvider for BundleStateWithReceipts {
|
||||
fn state(&self) -> &BundleStateWithReceipts {
|
||||
self
|
||||
}
|
||||
|
||||
/// Always returns [None] because we don't have any information about the block header.
|
||||
fn block_hash(&self, _block_number: BlockNumber) -> Option<BlockHash> {
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::collections::{BTreeMap, HashMap};
|
||||
|
||||
@ -1,269 +1,13 @@
|
||||
use crate::{
|
||||
BlockIdReader, BlockNumReader, BundleStateWithReceipts, Chain, HeaderProvider, ReceiptProvider,
|
||||
ReceiptProviderIdExt, TransactionsProvider, WithdrawalsProvider,
|
||||
};
|
||||
use auto_impl::auto_impl;
|
||||
use crate::{BundleStateWithReceipts, Chain};
|
||||
use reth_db::models::StoredBlockBodyIndices;
|
||||
use reth_primitives::{
|
||||
Block, BlockHashOrNumber, BlockId, BlockNumber, BlockNumberOrTag, BlockWithSenders, Header,
|
||||
PruneModes, Receipt, SealedBlock, SealedBlockWithSenders, SealedHeader, B256,
|
||||
};
|
||||
use reth_primitives::{BlockNumber, PruneModes, SealedBlockWithSenders};
|
||||
use reth_storage_api::BlockReader;
|
||||
use reth_storage_errors::provider::ProviderResult;
|
||||
use reth_trie::{updates::TrieUpdates, HashedPostState};
|
||||
use std::ops::RangeInclusive;
|
||||
|
||||
/// Enum to control transaction hash inclusion.
|
||||
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Default)]
|
||||
pub enum TransactionVariant {
|
||||
/// Indicates that transactions should be processed without including their hashes.
|
||||
NoHash,
|
||||
/// Indicates that transactions should be processed along with their hashes.
|
||||
#[default]
|
||||
WithHash,
|
||||
}
|
||||
|
||||
/// A helper enum that represents the origin of the requested block.
|
||||
///
|
||||
/// This helper type's sole purpose is to give the caller more control over from where blocks can be
|
||||
/// fetched.
|
||||
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Default)]
|
||||
pub enum BlockSource {
|
||||
/// Check all available sources.
|
||||
///
|
||||
/// Note: it's expected that looking up pending blocks is faster than looking up blocks in the
|
||||
/// database so this prioritizes Pending > Database.
|
||||
#[default]
|
||||
Any,
|
||||
/// The block was fetched from the pending block source, the blockchain tree that buffers
|
||||
/// blocks that are not yet finalized.
|
||||
Pending,
|
||||
/// The block was fetched from the database.
|
||||
Database,
|
||||
}
|
||||
|
||||
impl BlockSource {
|
||||
/// Returns `true` if the block source is `Pending` or `Any`.
|
||||
pub fn is_pending(&self) -> bool {
|
||||
matches!(self, BlockSource::Pending | BlockSource::Any)
|
||||
}
|
||||
|
||||
/// Returns `true` if the block source is `Database` or `Any`.
|
||||
pub fn is_database(&self) -> bool {
|
||||
matches!(self, BlockSource::Database | BlockSource::Any)
|
||||
}
|
||||
}
|
||||
|
||||
/// Api trait for fetching `Block` related data.
|
||||
///
|
||||
/// If not requested otherwise, implementers of this trait should prioritize fetching blocks from
|
||||
/// the database.
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait BlockReader:
|
||||
BlockNumReader
|
||||
+ HeaderProvider
|
||||
+ TransactionsProvider
|
||||
+ ReceiptProvider
|
||||
+ WithdrawalsProvider
|
||||
+ Send
|
||||
+ Sync
|
||||
{
|
||||
/// Tries to find in the given block source.
|
||||
///
|
||||
/// Note: this only operates on the hash because the number might be ambiguous.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn find_block_by_hash(&self, hash: B256, source: BlockSource) -> ProviderResult<Option<Block>>;
|
||||
|
||||
/// Returns the block with given id from the database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block(&self, id: BlockHashOrNumber) -> ProviderResult<Option<Block>>;
|
||||
|
||||
/// Returns the pending block if available
|
||||
///
|
||||
/// Note: This returns a [SealedBlock] because it's expected that this is sealed by the provider
|
||||
/// and the caller does not know the hash.
|
||||
fn pending_block(&self) -> ProviderResult<Option<SealedBlock>>;
|
||||
|
||||
/// Returns the pending block if available
|
||||
///
|
||||
/// Note: This returns a [SealedBlockWithSenders] because it's expected that this is sealed by
|
||||
/// the provider and the caller does not know the hash.
|
||||
fn pending_block_with_senders(&self) -> ProviderResult<Option<SealedBlockWithSenders>>;
|
||||
|
||||
/// Returns the pending block and receipts if available.
|
||||
fn pending_block_and_receipts(&self) -> ProviderResult<Option<(SealedBlock, Vec<Receipt>)>>;
|
||||
|
||||
/// Returns the ommers/uncle headers of the given block from the database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn ommers(&self, id: BlockHashOrNumber) -> ProviderResult<Option<Vec<Header>>>;
|
||||
|
||||
/// Returns the block with matching hash from the database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_by_hash(&self, hash: B256) -> ProviderResult<Option<Block>> {
|
||||
self.block(hash.into())
|
||||
}
|
||||
|
||||
/// Returns the block with matching number from database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_by_number(&self, num: u64) -> ProviderResult<Option<Block>> {
|
||||
self.block(num.into())
|
||||
}
|
||||
|
||||
/// Returns the block body indices with matching number from database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_body_indices(&self, num: u64) -> ProviderResult<Option<StoredBlockBodyIndices>>;
|
||||
|
||||
/// Returns the block with senders with matching number or hash from database.
|
||||
///
|
||||
/// Returns the block's transactions in the requested variant.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_with_senders(
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
transaction_kind: TransactionVariant,
|
||||
) -> ProviderResult<Option<BlockWithSenders>>;
|
||||
|
||||
/// Returns all blocks in the given inclusive range.
|
||||
///
|
||||
/// Note: returns only available blocks
|
||||
fn block_range(&self, range: RangeInclusive<BlockNumber>) -> ProviderResult<Vec<Block>>;
|
||||
|
||||
/// retrieves a range of blocks from the database, along with the senders of each
|
||||
/// transaction in the blocks.
|
||||
///
|
||||
/// The `transaction_kind` parameter determines whether to return its hash
|
||||
fn block_with_senders_range(
|
||||
&self,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> ProviderResult<Vec<BlockWithSenders>>;
|
||||
}
|
||||
|
||||
/// Trait extension for `BlockReader`, for types that implement `BlockId` conversion.
|
||||
///
|
||||
/// The `BlockReader` trait should be implemented on types that can retrieve a block from either
|
||||
/// a block number or hash. However, it might be desirable to fetch a block from a `BlockId` type,
|
||||
/// which can be a number, hash, or tag such as `BlockNumberOrTag::Safe`.
|
||||
///
|
||||
/// Resolving tags requires keeping track of block hashes or block numbers associated with the tag,
|
||||
/// so this trait can only be implemented for types that implement `BlockIdReader`. The
|
||||
/// `BlockIdReader` methods should be used to resolve `BlockId`s to block numbers or hashes, and
|
||||
/// retrieving the block should be done using the type's `BlockReader` methods.
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait BlockReaderIdExt: BlockReader + BlockIdReader + ReceiptProviderIdExt {
|
||||
/// Returns the block with matching tag from the database
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_by_number_or_tag(&self, id: BlockNumberOrTag) -> ProviderResult<Option<Block>> {
|
||||
self.convert_block_number(id)?.map_or_else(|| Ok(None), |num| self.block(num.into()))
|
||||
}
|
||||
|
||||
/// Returns the pending block header if available
|
||||
///
|
||||
/// Note: This returns a [SealedHeader] because it's expected that this is sealed by the
|
||||
/// provider and the caller does not know the hash.
|
||||
fn pending_header(&self) -> ProviderResult<Option<SealedHeader>> {
|
||||
self.sealed_header_by_id(BlockNumberOrTag::Pending.into())
|
||||
}
|
||||
|
||||
/// Returns the latest block header if available
|
||||
///
|
||||
/// Note: This returns a [SealedHeader] because it's expected that this is sealed by the
|
||||
/// provider and the caller does not know the hash.
|
||||
fn latest_header(&self) -> ProviderResult<Option<SealedHeader>> {
|
||||
self.sealed_header_by_id(BlockNumberOrTag::Latest.into())
|
||||
}
|
||||
|
||||
/// Returns the safe block header if available
|
||||
///
|
||||
/// Note: This returns a [SealedHeader] because it's expected that this is sealed by the
|
||||
/// provider and the caller does not know the hash.
|
||||
fn safe_header(&self) -> ProviderResult<Option<SealedHeader>> {
|
||||
self.sealed_header_by_id(BlockNumberOrTag::Safe.into())
|
||||
}
|
||||
|
||||
/// Returns the finalized block header if available
|
||||
///
|
||||
/// Note: This returns a [SealedHeader] because it's expected that this is sealed by the
|
||||
/// provider and the caller does not know the hash.
|
||||
fn finalized_header(&self) -> ProviderResult<Option<SealedHeader>> {
|
||||
self.sealed_header_by_id(BlockNumberOrTag::Finalized.into())
|
||||
}
|
||||
|
||||
/// Returns the block with the matching [BlockId] from the database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_by_id(&self, id: BlockId) -> ProviderResult<Option<Block>>;
|
||||
|
||||
/// Returns the block with senders with matching [BlockId].
|
||||
///
|
||||
/// Returns the block's transactions in the requested variant.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_with_senders_by_id(
|
||||
&self,
|
||||
id: BlockId,
|
||||
transaction_kind: TransactionVariant,
|
||||
) -> ProviderResult<Option<BlockWithSenders>> {
|
||||
match id {
|
||||
BlockId::Hash(hash) => {
|
||||
self.block_with_senders(hash.block_hash.into(), transaction_kind)
|
||||
}
|
||||
BlockId::Number(num) => self.convert_block_number(num)?.map_or_else(
|
||||
|| Ok(None),
|
||||
|num| self.block_with_senders(num.into(), transaction_kind),
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the header with matching tag from the database
|
||||
///
|
||||
/// Returns `None` if header is not found.
|
||||
fn header_by_number_or_tag(&self, id: BlockNumberOrTag) -> ProviderResult<Option<Header>> {
|
||||
self.convert_block_number(id)?
|
||||
.map_or_else(|| Ok(None), |num| self.header_by_hash_or_number(num.into()))
|
||||
}
|
||||
|
||||
/// Returns the header with matching tag from the database
|
||||
///
|
||||
/// Returns `None` if header is not found.
|
||||
fn sealed_header_by_number_or_tag(
|
||||
&self,
|
||||
id: BlockNumberOrTag,
|
||||
) -> ProviderResult<Option<SealedHeader>> {
|
||||
self.convert_block_number(id)?
|
||||
.map_or_else(|| Ok(None), |num| self.header_by_hash_or_number(num.into()))?
|
||||
.map_or_else(|| Ok(None), |h| Ok(Some(h.seal_slow())))
|
||||
}
|
||||
|
||||
/// Returns the sealed header with the matching `BlockId` from the database.
|
||||
///
|
||||
/// Returns `None` if header is not found.
|
||||
fn sealed_header_by_id(&self, id: BlockId) -> ProviderResult<Option<SealedHeader>>;
|
||||
|
||||
/// Returns the header with the matching `BlockId` from the database.
|
||||
///
|
||||
/// Returns `None` if header is not found.
|
||||
fn header_by_id(&self, id: BlockId) -> ProviderResult<Option<Header>>;
|
||||
|
||||
/// Returns the ommers with the matching tag from the database.
|
||||
fn ommers_by_number_or_tag(&self, id: BlockNumberOrTag) -> ProviderResult<Option<Vec<Header>>> {
|
||||
self.convert_block_number(id)?.map_or_else(|| Ok(None), |num| self.ommers(num.into()))
|
||||
}
|
||||
|
||||
/// Returns the ommers with the matching `BlockId` from the database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn ommers_by_id(&self, id: BlockId) -> ProviderResult<Option<Vec<Header>>>;
|
||||
}
|
||||
|
||||
/// BlockExecution Writer
|
||||
#[auto_impl(&, Arc, Box)]
|
||||
#[auto_impl::auto_impl(&, Arc, Box)]
|
||||
pub trait BlockExecutionWriter: BlockWriter + BlockReader + Send + Sync {
|
||||
/// Get range of blocks and its execution result
|
||||
fn get_block_and_execution_range(
|
||||
@ -289,7 +33,7 @@ pub trait BlockExecutionWriter: BlockWriter + BlockReader + Send + Sync {
|
||||
}
|
||||
|
||||
/// Block Writer
|
||||
#[auto_impl(&, Arc, Box)]
|
||||
#[auto_impl::auto_impl(&, Arc, Box)]
|
||||
pub trait BlockWriter: Send + Sync {
|
||||
/// Insert full block and make it canonical. Parent tx num and transition id is taken from
|
||||
/// parent block in database.
|
||||
|
||||
@ -1,53 +1,22 @@
|
||||
//! Collection of common provider traits.
|
||||
|
||||
mod account;
|
||||
pub use account::{AccountExtReader, AccountReader, ChangeSetReader};
|
||||
|
||||
mod storage;
|
||||
pub use storage::StorageReader;
|
||||
|
||||
mod block;
|
||||
pub use block::{
|
||||
BlockExecutionWriter, BlockReader, BlockReaderIdExt, BlockSource, BlockWriter,
|
||||
TransactionVariant,
|
||||
};
|
||||
|
||||
mod block_hash;
|
||||
pub use block_hash::BlockHashReader;
|
||||
|
||||
mod block_id;
|
||||
pub use block_id::{BlockIdReader, BlockNumReader};
|
||||
// Re-export all the traits
|
||||
pub use reth_storage_api::*;
|
||||
|
||||
// Re-export for convenience
|
||||
pub use reth_evm::provider::EvmEnvProvider;
|
||||
|
||||
mod block;
|
||||
pub use block::*;
|
||||
|
||||
mod chain_info;
|
||||
pub use chain_info::CanonChainTracker;
|
||||
|
||||
mod header;
|
||||
pub use header::HeaderProvider;
|
||||
|
||||
mod header_sync_gap;
|
||||
pub use header_sync_gap::{HeaderSyncGap, HeaderSyncGapProvider, HeaderSyncMode};
|
||||
|
||||
mod receipts;
|
||||
pub use receipts::{ReceiptProvider, ReceiptProviderIdExt};
|
||||
|
||||
mod state;
|
||||
pub use state::{
|
||||
BlockchainTreePendingStateProvider, BundleStateDataProvider, BundleStateForkProvider,
|
||||
FullBundleStateDataProvider, StateProvider, StateProviderBox, StateProviderFactory,
|
||||
StateWriter,
|
||||
};
|
||||
|
||||
mod trie;
|
||||
pub use trie::StateRootProvider;
|
||||
|
||||
mod transactions;
|
||||
pub use transactions::{TransactionsProvider, TransactionsProviderExt};
|
||||
|
||||
mod withdrawals;
|
||||
pub use withdrawals::WithdrawalsProvider;
|
||||
pub use state::StateWriter;
|
||||
|
||||
mod chain;
|
||||
pub use chain::{
|
||||
|
||||
@ -1,256 +1,10 @@
|
||||
use super::AccountReader;
|
||||
use crate::{
|
||||
providers::StaticFileProviderRWRefMut, BlockHashReader, BlockIdReader, BundleStateWithReceipts,
|
||||
StateRootProvider,
|
||||
};
|
||||
use auto_impl::auto_impl;
|
||||
use crate::providers::StaticFileProviderRWRefMut;
|
||||
use reth_db::transaction::{DbTx, DbTxMut};
|
||||
use reth_primitives::{
|
||||
trie::AccountProof, Address, BlockHash, BlockId, BlockNumHash, BlockNumber, BlockNumberOrTag,
|
||||
Bytecode, StorageKey, StorageValue, B256, KECCAK_EMPTY, U256,
|
||||
};
|
||||
use reth_storage_errors::provider::{ProviderError, ProviderResult};
|
||||
use reth_storage_errors::provider::ProviderResult;
|
||||
use revm::db::OriginalValuesKnown;
|
||||
|
||||
/// Type alias of boxed [StateProvider].
|
||||
pub type StateProviderBox = Box<dyn StateProvider>;
|
||||
|
||||
/// An abstraction for a type that provides state data.
|
||||
#[auto_impl(&, Arc, Box)]
|
||||
pub trait StateProvider: BlockHashReader + AccountReader + StateRootProvider + Send + Sync {
|
||||
/// Get storage of given account.
|
||||
fn storage(
|
||||
&self,
|
||||
account: Address,
|
||||
storage_key: StorageKey,
|
||||
) -> ProviderResult<Option<StorageValue>>;
|
||||
|
||||
/// Get account code by its hash
|
||||
fn bytecode_by_hash(&self, code_hash: B256) -> ProviderResult<Option<Bytecode>>;
|
||||
|
||||
/// Get account and storage proofs.
|
||||
fn proof(&self, address: Address, keys: &[B256]) -> ProviderResult<AccountProof>;
|
||||
|
||||
/// Get account code by its address.
|
||||
///
|
||||
/// Returns `None` if the account doesn't exist or account is not a contract
|
||||
fn account_code(&self, addr: Address) -> ProviderResult<Option<Bytecode>> {
|
||||
// Get basic account information
|
||||
// Returns None if acc doesn't exist
|
||||
let acc = match self.basic_account(addr)? {
|
||||
Some(acc) => acc,
|
||||
None => return Ok(None),
|
||||
};
|
||||
|
||||
if let Some(code_hash) = acc.bytecode_hash {
|
||||
if code_hash == KECCAK_EMPTY {
|
||||
return Ok(None)
|
||||
}
|
||||
// Get the code from the code hash
|
||||
return self.bytecode_by_hash(code_hash)
|
||||
}
|
||||
|
||||
// Return `None` if no code hash is set
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
/// Get account balance by its address.
|
||||
///
|
||||
/// Returns `None` if the account doesn't exist
|
||||
fn account_balance(&self, addr: Address) -> ProviderResult<Option<U256>> {
|
||||
// Get basic account information
|
||||
// Returns None if acc doesn't exist
|
||||
match self.basic_account(addr)? {
|
||||
Some(acc) => Ok(Some(acc.balance)),
|
||||
None => Ok(None),
|
||||
}
|
||||
}
|
||||
|
||||
/// Get account nonce by its address.
|
||||
///
|
||||
/// Returns `None` if the account doesn't exist
|
||||
fn account_nonce(&self, addr: Address) -> ProviderResult<Option<u64>> {
|
||||
// Get basic account information
|
||||
// Returns None if acc doesn't exist
|
||||
match self.basic_account(addr)? {
|
||||
Some(acc) => Ok(Some(acc.nonce)),
|
||||
None => Ok(None),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Light wrapper that returns `StateProvider` implementations that correspond to the given
|
||||
/// `BlockNumber`, the latest state, or the pending state.
|
||||
///
|
||||
/// This type differentiates states into `historical`, `latest` and `pending`, where the `latest`
|
||||
/// block determines what is historical or pending: `[historical..latest..pending]`.
|
||||
///
|
||||
/// The `latest` state represents the state after the most recent block has been committed to the
|
||||
/// database, `historical` states are states that have been committed to the database before the
|
||||
/// `latest` state, and `pending` states are states that have not yet been committed to the
|
||||
/// database which may or may not become the `latest` state, depending on consensus.
|
||||
///
|
||||
/// Note: the `pending` block is considered the block that extends the canonical chain but one and
|
||||
/// has the `latest` block as its parent.
|
||||
///
|
||||
/// All states are _inclusive_, meaning they include _all_ all changes made (executed transactions)
|
||||
/// in their respective blocks. For example [StateProviderFactory::history_by_block_number] for
|
||||
/// block number `n` will return the state after block `n` was executed (transactions, withdrawals).
|
||||
/// In other words, all states point to the end of the state's respective block, which is equivalent
|
||||
/// to state at the beginning of the child block.
|
||||
///
|
||||
/// This affects tracing, or replaying blocks, which will need to be executed on top of the state of
|
||||
/// the parent block. For example, in order to trace block `n`, the state after block `n - 1` needs
|
||||
/// to be used, since block `n` was executed on its parent block's state.
|
||||
#[auto_impl(&, Arc, Box)]
|
||||
pub trait StateProviderFactory: BlockIdReader + Send + Sync {
|
||||
/// Storage provider for latest block.
|
||||
fn latest(&self) -> ProviderResult<StateProviderBox>;
|
||||
|
||||
/// Returns a [StateProvider] indexed by the given [BlockId].
|
||||
///
|
||||
/// Note: if a number or hash is provided this will __only__ look at historical(canonical)
|
||||
/// state.
|
||||
fn state_by_block_id(&self, block_id: BlockId) -> ProviderResult<StateProviderBox> {
|
||||
match block_id {
|
||||
BlockId::Number(block_number) => self.state_by_block_number_or_tag(block_number),
|
||||
BlockId::Hash(block_hash) => self.history_by_block_hash(block_hash.into()),
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns a [StateProvider] indexed by the given block number or tag.
|
||||
///
|
||||
/// Note: if a number is provided this will only look at historical(canonical) state.
|
||||
fn state_by_block_number_or_tag(
|
||||
&self,
|
||||
number_or_tag: BlockNumberOrTag,
|
||||
) -> ProviderResult<StateProviderBox> {
|
||||
match number_or_tag {
|
||||
BlockNumberOrTag::Latest => self.latest(),
|
||||
BlockNumberOrTag::Finalized => {
|
||||
// we can only get the finalized state by hash, not by num
|
||||
let hash = match self.finalized_block_hash()? {
|
||||
Some(hash) => hash,
|
||||
None => return Err(ProviderError::FinalizedBlockNotFound),
|
||||
};
|
||||
// only look at historical state
|
||||
self.history_by_block_hash(hash)
|
||||
}
|
||||
BlockNumberOrTag::Safe => {
|
||||
// we can only get the safe state by hash, not by num
|
||||
let hash = match self.safe_block_hash()? {
|
||||
Some(hash) => hash,
|
||||
None => return Err(ProviderError::SafeBlockNotFound),
|
||||
};
|
||||
|
||||
self.history_by_block_hash(hash)
|
||||
}
|
||||
BlockNumberOrTag::Earliest => self.history_by_block_number(0),
|
||||
BlockNumberOrTag::Pending => self.pending(),
|
||||
BlockNumberOrTag::Number(num) => {
|
||||
// Note: The `BlockchainProvider` could also lookup the tree for the given block number, if for example the block number is `latest + 1`, however this should only support canonical state: <https://github.com/paradigmxyz/reth/issues/4515>
|
||||
self.history_by_block_number(num)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns a historical [StateProvider] indexed by the given historic block number.
|
||||
///
|
||||
///
|
||||
/// Note: this only looks at historical blocks, not pending blocks.
|
||||
fn history_by_block_number(&self, block: BlockNumber) -> ProviderResult<StateProviderBox>;
|
||||
|
||||
/// Returns a historical [StateProvider] indexed by the given block hash.
|
||||
///
|
||||
/// Note: this only looks at historical blocks, not pending blocks.
|
||||
fn history_by_block_hash(&self, block: BlockHash) -> ProviderResult<StateProviderBox>;
|
||||
|
||||
/// Returns _any_[StateProvider] with matching block hash.
|
||||
///
|
||||
/// This will return a [StateProvider] for either a historical or pending block.
|
||||
fn state_by_block_hash(&self, block: BlockHash) -> ProviderResult<StateProviderBox>;
|
||||
|
||||
/// Storage provider for pending state.
|
||||
///
|
||||
/// Represents the state at the block that extends the canonical chain by one.
|
||||
/// If there's no `pending` block, then this is equal to [StateProviderFactory::latest]
|
||||
fn pending(&self) -> ProviderResult<StateProviderBox>;
|
||||
|
||||
/// Storage provider for pending state for the given block hash.
|
||||
///
|
||||
/// Represents the state at the block that extends the canonical chain.
|
||||
///
|
||||
/// If the block couldn't be found, returns `None`.
|
||||
fn pending_state_by_hash(&self, block_hash: B256) -> ProviderResult<Option<StateProviderBox>>;
|
||||
|
||||
/// Return a [StateProvider] that contains bundle state data provider.
|
||||
/// Used to inspect or execute transaction on the pending state.
|
||||
fn pending_with_provider(
|
||||
&self,
|
||||
bundle_state_data: Box<dyn FullBundleStateDataProvider>,
|
||||
) -> ProviderResult<StateProviderBox>;
|
||||
}
|
||||
|
||||
/// Blockchain trait provider that gives access to the blockchain state that is not yet committed
|
||||
/// (pending).
|
||||
pub trait BlockchainTreePendingStateProvider: Send + Sync {
|
||||
/// Returns a state provider that includes all state changes of the given (pending) block hash.
|
||||
///
|
||||
/// In other words, the state provider will return the state after all transactions of the given
|
||||
/// hash have been executed.
|
||||
fn pending_state_provider(
|
||||
&self,
|
||||
block_hash: BlockHash,
|
||||
) -> ProviderResult<Box<dyn FullBundleStateDataProvider>> {
|
||||
self.find_pending_state_provider(block_hash)
|
||||
.ok_or(ProviderError::StateForHashNotFound(block_hash))
|
||||
}
|
||||
|
||||
/// Returns state provider if a matching block exists.
|
||||
fn find_pending_state_provider(
|
||||
&self,
|
||||
block_hash: BlockHash,
|
||||
) -> Option<Box<dyn FullBundleStateDataProvider>>;
|
||||
}
|
||||
|
||||
/// Post state data needed for execution on it.
|
||||
///
|
||||
/// State contains:
|
||||
/// * [`BundleStateWithReceipts`] contains all changed of accounts and storage of pending chain
|
||||
/// * block hashes of pending chain and canonical blocks.
|
||||
#[auto_impl(&, Box)]
|
||||
pub trait BundleStateDataProvider: Send + Sync {
|
||||
/// Return post state
|
||||
fn state(&self) -> &BundleStateWithReceipts;
|
||||
/// Return block hash by block number of pending or canonical chain.
|
||||
fn block_hash(&self, block_number: BlockNumber) -> Option<BlockHash>;
|
||||
}
|
||||
|
||||
/// Fork data needed for execution on it.
|
||||
///
|
||||
/// It contains a canonical fork, the block on what pending chain was forked from.
|
||||
#[auto_impl(&, Box)]
|
||||
pub trait BundleStateForkProvider {
|
||||
/// Return canonical fork, the block on what post state was forked from.
|
||||
///
|
||||
/// Needed to create state provider.
|
||||
fn canonical_fork(&self) -> BlockNumHash;
|
||||
}
|
||||
|
||||
/// Full post state data needed for execution on it.
|
||||
/// This trait is used to create a state provider over pending state.
|
||||
///
|
||||
/// This trait is a combination of [`BundleStateDataProvider`] and [`BundleStateForkProvider`].
|
||||
///
|
||||
/// Pending state contains:
|
||||
/// * [`BundleStateWithReceipts`] contains all changed of accounts and storage of pending chain
|
||||
/// * block hashes of pending chain and canonical blocks.
|
||||
/// * canonical fork, the block on what pending chain was forked from.
|
||||
pub trait FullBundleStateDataProvider: BundleStateDataProvider + BundleStateForkProvider {}
|
||||
|
||||
impl<T> FullBundleStateDataProvider for T where T: BundleStateDataProvider + BundleStateForkProvider {}
|
||||
|
||||
/// A helper trait for [BundleStateWithReceipts] to write state and receipts to storage.
|
||||
/// A helper trait for [BundleStateWithReceipts](reth_execution_types::BundleStateWithReceipts) to
|
||||
/// write state and receipts to storage.
|
||||
pub trait StateWriter {
|
||||
/// Write the data and receipts to the database or static files if `static_file_producer` is
|
||||
/// `Some`. It should be `None` if there is any kind of pruning/filtering over the receipts.
|
||||
|
||||
24
crates/storage/storage-api/Cargo.toml
Normal file
24
crates/storage/storage-api/Cargo.toml
Normal file
@ -0,0 +1,24 @@
|
||||
[package]
|
||||
name = "reth-storage-api"
|
||||
version.workspace = true
|
||||
edition.workspace = true
|
||||
rust-version.workspace = true
|
||||
license.workspace = true
|
||||
homepage.workspace = true
|
||||
repository.workspace = true
|
||||
description = "Reth storage provider traits and types"
|
||||
|
||||
[lints]
|
||||
workspace = true
|
||||
|
||||
[dependencies]
|
||||
# reth
|
||||
reth-execution-types.workspace = true
|
||||
reth-db.workspace = true
|
||||
reth-primitives.workspace = true
|
||||
reth-storage-errors.workspace = true
|
||||
reth-trie.workspace = true
|
||||
|
||||
revm.workspace = true
|
||||
|
||||
auto_impl.workspace = true
|
||||
261
crates/storage/storage-api/src/block.rs
Normal file
261
crates/storage/storage-api/src/block.rs
Normal file
@ -0,0 +1,261 @@
|
||||
use crate::{
|
||||
BlockIdReader, BlockNumReader, HeaderProvider, ReceiptProvider, ReceiptProviderIdExt,
|
||||
TransactionsProvider, WithdrawalsProvider,
|
||||
};
|
||||
use reth_db::models::StoredBlockBodyIndices;
|
||||
use reth_primitives::{
|
||||
Block, BlockHashOrNumber, BlockId, BlockNumber, BlockNumberOrTag, BlockWithSenders, Header,
|
||||
Receipt, SealedBlock, SealedBlockWithSenders, SealedHeader, B256,
|
||||
};
|
||||
use reth_storage_errors::provider::ProviderResult;
|
||||
use std::ops::RangeInclusive;
|
||||
|
||||
/// Enum to control transaction hash inclusion.
|
||||
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Default)]
|
||||
pub enum TransactionVariant {
|
||||
/// Indicates that transactions should be processed without including their hashes.
|
||||
NoHash,
|
||||
/// Indicates that transactions should be processed along with their hashes.
|
||||
#[default]
|
||||
WithHash,
|
||||
}
|
||||
|
||||
/// A helper enum that represents the origin of the requested block.
|
||||
///
|
||||
/// This helper type's sole purpose is to give the caller more control over from where blocks can be
|
||||
/// fetched.
|
||||
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Default)]
|
||||
pub enum BlockSource {
|
||||
/// Check all available sources.
|
||||
///
|
||||
/// Note: it's expected that looking up pending blocks is faster than looking up blocks in the
|
||||
/// database so this prioritizes Pending > Database.
|
||||
#[default]
|
||||
Any,
|
||||
/// The block was fetched from the pending block source, the blockchain tree that buffers
|
||||
/// blocks that are not yet finalized.
|
||||
Pending,
|
||||
/// The block was fetched from the database.
|
||||
Database,
|
||||
}
|
||||
|
||||
impl BlockSource {
|
||||
/// Returns `true` if the block source is `Pending` or `Any`.
|
||||
pub fn is_pending(&self) -> bool {
|
||||
matches!(self, BlockSource::Pending | BlockSource::Any)
|
||||
}
|
||||
|
||||
/// Returns `true` if the block source is `Database` or `Any`.
|
||||
pub fn is_database(&self) -> bool {
|
||||
matches!(self, BlockSource::Database | BlockSource::Any)
|
||||
}
|
||||
}
|
||||
|
||||
/// Api trait for fetching `Block` related data.
|
||||
///
|
||||
/// If not requested otherwise, implementers of this trait should prioritize fetching blocks from
|
||||
/// the database.
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait BlockReader:
|
||||
BlockNumReader
|
||||
+ HeaderProvider
|
||||
+ TransactionsProvider
|
||||
+ ReceiptProvider
|
||||
+ WithdrawalsProvider
|
||||
+ Send
|
||||
+ Sync
|
||||
{
|
||||
/// Tries to find in the given block source.
|
||||
///
|
||||
/// Note: this only operates on the hash because the number might be ambiguous.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn find_block_by_hash(&self, hash: B256, source: BlockSource) -> ProviderResult<Option<Block>>;
|
||||
|
||||
/// Returns the block with given id from the database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block(&self, id: BlockHashOrNumber) -> ProviderResult<Option<Block>>;
|
||||
|
||||
/// Returns the pending block if available
|
||||
///
|
||||
/// Note: This returns a [SealedBlock] because it's expected that this is sealed by the provider
|
||||
/// and the caller does not know the hash.
|
||||
fn pending_block(&self) -> ProviderResult<Option<SealedBlock>>;
|
||||
|
||||
/// Returns the pending block if available
|
||||
///
|
||||
/// Note: This returns a [SealedBlockWithSenders] because it's expected that this is sealed by
|
||||
/// the provider and the caller does not know the hash.
|
||||
fn pending_block_with_senders(&self) -> ProviderResult<Option<SealedBlockWithSenders>>;
|
||||
|
||||
/// Returns the pending block and receipts if available.
|
||||
fn pending_block_and_receipts(&self) -> ProviderResult<Option<(SealedBlock, Vec<Receipt>)>>;
|
||||
|
||||
/// Returns the ommers/uncle headers of the given block from the database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn ommers(&self, id: BlockHashOrNumber) -> ProviderResult<Option<Vec<Header>>>;
|
||||
|
||||
/// Returns the block with matching hash from the database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_by_hash(&self, hash: B256) -> ProviderResult<Option<Block>> {
|
||||
self.block(hash.into())
|
||||
}
|
||||
|
||||
/// Returns the block with matching number from database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_by_number(&self, num: u64) -> ProviderResult<Option<Block>> {
|
||||
self.block(num.into())
|
||||
}
|
||||
|
||||
/// Returns the block body indices with matching number from database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_body_indices(&self, num: u64) -> ProviderResult<Option<StoredBlockBodyIndices>>;
|
||||
|
||||
/// Returns the block with senders with matching number or hash from database.
|
||||
///
|
||||
/// Returns the block's transactions in the requested variant.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_with_senders(
|
||||
&self,
|
||||
id: BlockHashOrNumber,
|
||||
transaction_kind: TransactionVariant,
|
||||
) -> ProviderResult<Option<BlockWithSenders>>;
|
||||
|
||||
/// Returns all blocks in the given inclusive range.
|
||||
///
|
||||
/// Note: returns only available blocks
|
||||
fn block_range(&self, range: RangeInclusive<BlockNumber>) -> ProviderResult<Vec<Block>>;
|
||||
|
||||
/// retrieves a range of blocks from the database, along with the senders of each
|
||||
/// transaction in the blocks.
|
||||
///
|
||||
/// The `transaction_kind` parameter determines whether to return its hash
|
||||
fn block_with_senders_range(
|
||||
&self,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
) -> ProviderResult<Vec<BlockWithSenders>>;
|
||||
}
|
||||
|
||||
/// Trait extension for `BlockReader`, for types that implement `BlockId` conversion.
|
||||
///
|
||||
/// The `BlockReader` trait should be implemented on types that can retrieve a block from either
|
||||
/// a block number or hash. However, it might be desirable to fetch a block from a `BlockId` type,
|
||||
/// which can be a number, hash, or tag such as `BlockNumberOrTag::Safe`.
|
||||
///
|
||||
/// Resolving tags requires keeping track of block hashes or block numbers associated with the tag,
|
||||
/// so this trait can only be implemented for types that implement `BlockIdReader`. The
|
||||
/// `BlockIdReader` methods should be used to resolve `BlockId`s to block numbers or hashes, and
|
||||
/// retrieving the block should be done using the type's `BlockReader` methods.
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait BlockReaderIdExt: BlockReader + BlockIdReader + ReceiptProviderIdExt {
|
||||
/// Returns the block with matching tag from the database
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_by_number_or_tag(&self, id: BlockNumberOrTag) -> ProviderResult<Option<Block>> {
|
||||
self.convert_block_number(id)?.map_or_else(|| Ok(None), |num| self.block(num.into()))
|
||||
}
|
||||
|
||||
/// Returns the pending block header if available
|
||||
///
|
||||
/// Note: This returns a [SealedHeader] because it's expected that this is sealed by the
|
||||
/// provider and the caller does not know the hash.
|
||||
fn pending_header(&self) -> ProviderResult<Option<SealedHeader>> {
|
||||
self.sealed_header_by_id(BlockNumberOrTag::Pending.into())
|
||||
}
|
||||
|
||||
/// Returns the latest block header if available
|
||||
///
|
||||
/// Note: This returns a [SealedHeader] because it's expected that this is sealed by the
|
||||
/// provider and the caller does not know the hash.
|
||||
fn latest_header(&self) -> ProviderResult<Option<SealedHeader>> {
|
||||
self.sealed_header_by_id(BlockNumberOrTag::Latest.into())
|
||||
}
|
||||
|
||||
/// Returns the safe block header if available
|
||||
///
|
||||
/// Note: This returns a [SealedHeader] because it's expected that this is sealed by the
|
||||
/// provider and the caller does not know the hash.
|
||||
fn safe_header(&self) -> ProviderResult<Option<SealedHeader>> {
|
||||
self.sealed_header_by_id(BlockNumberOrTag::Safe.into())
|
||||
}
|
||||
|
||||
/// Returns the finalized block header if available
|
||||
///
|
||||
/// Note: This returns a [SealedHeader] because it's expected that this is sealed by the
|
||||
/// provider and the caller does not know the hash.
|
||||
fn finalized_header(&self) -> ProviderResult<Option<SealedHeader>> {
|
||||
self.sealed_header_by_id(BlockNumberOrTag::Finalized.into())
|
||||
}
|
||||
|
||||
/// Returns the block with the matching [BlockId] from the database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_by_id(&self, id: BlockId) -> ProviderResult<Option<Block>>;
|
||||
|
||||
/// Returns the block with senders with matching [BlockId].
|
||||
///
|
||||
/// Returns the block's transactions in the requested variant.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn block_with_senders_by_id(
|
||||
&self,
|
||||
id: BlockId,
|
||||
transaction_kind: TransactionVariant,
|
||||
) -> ProviderResult<Option<BlockWithSenders>> {
|
||||
match id {
|
||||
BlockId::Hash(hash) => {
|
||||
self.block_with_senders(hash.block_hash.into(), transaction_kind)
|
||||
}
|
||||
BlockId::Number(num) => self.convert_block_number(num)?.map_or_else(
|
||||
|| Ok(None),
|
||||
|num| self.block_with_senders(num.into(), transaction_kind),
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the header with matching tag from the database
|
||||
///
|
||||
/// Returns `None` if header is not found.
|
||||
fn header_by_number_or_tag(&self, id: BlockNumberOrTag) -> ProviderResult<Option<Header>> {
|
||||
self.convert_block_number(id)?
|
||||
.map_or_else(|| Ok(None), |num| self.header_by_hash_or_number(num.into()))
|
||||
}
|
||||
|
||||
/// Returns the header with matching tag from the database
|
||||
///
|
||||
/// Returns `None` if header is not found.
|
||||
fn sealed_header_by_number_or_tag(
|
||||
&self,
|
||||
id: BlockNumberOrTag,
|
||||
) -> ProviderResult<Option<SealedHeader>> {
|
||||
self.convert_block_number(id)?
|
||||
.map_or_else(|| Ok(None), |num| self.header_by_hash_or_number(num.into()))?
|
||||
.map_or_else(|| Ok(None), |h| Ok(Some(h.seal_slow())))
|
||||
}
|
||||
|
||||
/// Returns the sealed header with the matching `BlockId` from the database.
|
||||
///
|
||||
/// Returns `None` if header is not found.
|
||||
fn sealed_header_by_id(&self, id: BlockId) -> ProviderResult<Option<SealedHeader>>;
|
||||
|
||||
/// Returns the header with the matching `BlockId` from the database.
|
||||
///
|
||||
/// Returns `None` if header is not found.
|
||||
fn header_by_id(&self, id: BlockId) -> ProviderResult<Option<Header>>;
|
||||
|
||||
/// Returns the ommers with the matching tag from the database.
|
||||
fn ommers_by_number_or_tag(&self, id: BlockNumberOrTag) -> ProviderResult<Option<Vec<Header>>> {
|
||||
self.convert_block_number(id)?.map_or_else(|| Ok(None), |num| self.ommers(num.into()))
|
||||
}
|
||||
|
||||
/// Returns the ommers with the matching `BlockId` from the database.
|
||||
///
|
||||
/// Returns `None` if block is not found.
|
||||
fn ommers_by_id(&self, id: BlockId) -> ProviderResult<Option<Vec<Header>>>;
|
||||
}
|
||||
@ -1,9 +1,8 @@
|
||||
use auto_impl::auto_impl;
|
||||
use reth_primitives::{BlockHashOrNumber, BlockNumber, B256};
|
||||
use reth_storage_errors::provider::ProviderResult;
|
||||
|
||||
/// Client trait for fetching block hashes by number.
|
||||
#[auto_impl(&, Arc, Box)]
|
||||
#[auto_impl::auto_impl(&, Arc, Box)]
|
||||
pub trait BlockHashReader: Send + Sync {
|
||||
/// Get the hash of the block with the given number. Returns `None` if no block with this number
|
||||
/// exists.
|
||||
@ -1,4 +1,4 @@
|
||||
use super::BlockHashReader;
|
||||
use crate::BlockHashReader;
|
||||
use reth_primitives::{BlockHashOrNumber, BlockId, BlockNumber, BlockNumberOrTag, ChainInfo, B256};
|
||||
use reth_storage_errors::provider::{ProviderError, ProviderResult};
|
||||
|
||||
@ -1,10 +1,9 @@
|
||||
use auto_impl::auto_impl;
|
||||
use reth_primitives::{BlockHash, BlockHashOrNumber, BlockNumber, Header, SealedHeader, U256};
|
||||
use reth_storage_errors::provider::ProviderResult;
|
||||
use std::ops::RangeBounds;
|
||||
|
||||
/// Client trait for fetching `Header` related data.
|
||||
#[auto_impl(&, Arc)]
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait HeaderProvider: Send + Sync {
|
||||
/// Check if block is known
|
||||
fn is_known(&self, block_hash: &BlockHash) -> ProviderResult<bool> {
|
||||
42
crates/storage/storage-api/src/lib.rs
Normal file
42
crates/storage/storage-api/src/lib.rs
Normal file
@ -0,0 +1,42 @@
|
||||
//! Collection of traits and types for common storage access.
|
||||
|
||||
#![doc(
|
||||
html_logo_url = "https://raw.githubusercontent.com/paradigmxyz/reth/main/assets/reth-docs.png",
|
||||
html_favicon_url = "https://avatars0.githubusercontent.com/u/97369466?s=256",
|
||||
issue_tracker_base_url = "https://github.com/paradigmxyz/reth/issues/"
|
||||
)]
|
||||
#![cfg_attr(not(test), warn(unused_crate_dependencies))]
|
||||
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
|
||||
|
||||
mod account;
|
||||
pub use account::*;
|
||||
|
||||
mod block;
|
||||
pub use block::*;
|
||||
|
||||
mod block_id;
|
||||
pub use block_id::*;
|
||||
|
||||
mod block_hash;
|
||||
pub use block_hash::*;
|
||||
|
||||
mod header;
|
||||
pub use header::*;
|
||||
|
||||
mod receipts;
|
||||
pub use receipts::*;
|
||||
|
||||
mod state;
|
||||
pub use state::*;
|
||||
|
||||
mod storage;
|
||||
pub use storage::*;
|
||||
|
||||
mod transactions;
|
||||
pub use transactions::*;
|
||||
|
||||
mod trie;
|
||||
pub use trie::*;
|
||||
|
||||
mod withdrawals;
|
||||
pub use withdrawals::*;
|
||||
@ -1,9 +1,7 @@
|
||||
use std::ops::RangeBounds;
|
||||
|
||||
use crate::BlockIdReader;
|
||||
use reth_primitives::{BlockHashOrNumber, BlockId, BlockNumberOrTag, Receipt, TxHash, TxNumber};
|
||||
use reth_storage_errors::provider::ProviderResult;
|
||||
|
||||
use crate::BlockIdReader;
|
||||
use std::ops::RangeBounds;
|
||||
|
||||
/// Client trait for fetching [Receipt] data .
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
257
crates/storage/storage-api/src/state.rs
Normal file
257
crates/storage/storage-api/src/state.rs
Normal file
@ -0,0 +1,257 @@
|
||||
use super::{AccountReader, BlockHashReader, BlockIdReader, StateRootProvider};
|
||||
use auto_impl::auto_impl;
|
||||
use reth_execution_types::BundleStateWithReceipts;
|
||||
use reth_primitives::{
|
||||
trie::AccountProof, Address, BlockHash, BlockId, BlockNumHash, BlockNumber, BlockNumberOrTag,
|
||||
Bytecode, StorageKey, StorageValue, B256, KECCAK_EMPTY, U256,
|
||||
};
|
||||
use reth_storage_errors::provider::{ProviderError, ProviderResult};
|
||||
|
||||
/// Type alias of boxed [StateProvider].
|
||||
pub type StateProviderBox = Box<dyn StateProvider>;
|
||||
|
||||
/// An abstraction for a type that provides state data.
|
||||
#[auto_impl(&, Arc, Box)]
|
||||
pub trait StateProvider: BlockHashReader + AccountReader + StateRootProvider + Send + Sync {
|
||||
/// Get storage of given account.
|
||||
fn storage(
|
||||
&self,
|
||||
account: Address,
|
||||
storage_key: StorageKey,
|
||||
) -> ProviderResult<Option<StorageValue>>;
|
||||
|
||||
/// Get account code by its hash
|
||||
fn bytecode_by_hash(&self, code_hash: B256) -> ProviderResult<Option<Bytecode>>;
|
||||
|
||||
/// Get account and storage proofs.
|
||||
fn proof(&self, address: Address, keys: &[B256]) -> ProviderResult<AccountProof>;
|
||||
|
||||
/// Get account code by its address.
|
||||
///
|
||||
/// Returns `None` if the account doesn't exist or account is not a contract
|
||||
fn account_code(&self, addr: Address) -> ProviderResult<Option<Bytecode>> {
|
||||
// Get basic account information
|
||||
// Returns None if acc doesn't exist
|
||||
let acc = match self.basic_account(addr)? {
|
||||
Some(acc) => acc,
|
||||
None => return Ok(None),
|
||||
};
|
||||
|
||||
if let Some(code_hash) = acc.bytecode_hash {
|
||||
if code_hash == KECCAK_EMPTY {
|
||||
return Ok(None)
|
||||
}
|
||||
// Get the code from the code hash
|
||||
return self.bytecode_by_hash(code_hash)
|
||||
}
|
||||
|
||||
// Return `None` if no code hash is set
|
||||
Ok(None)
|
||||
}
|
||||
|
||||
/// Get account balance by its address.
|
||||
///
|
||||
/// Returns `None` if the account doesn't exist
|
||||
fn account_balance(&self, addr: Address) -> ProviderResult<Option<U256>> {
|
||||
// Get basic account information
|
||||
// Returns None if acc doesn't exist
|
||||
match self.basic_account(addr)? {
|
||||
Some(acc) => Ok(Some(acc.balance)),
|
||||
None => Ok(None),
|
||||
}
|
||||
}
|
||||
|
||||
/// Get account nonce by its address.
|
||||
///
|
||||
/// Returns `None` if the account doesn't exist
|
||||
fn account_nonce(&self, addr: Address) -> ProviderResult<Option<u64>> {
|
||||
// Get basic account information
|
||||
// Returns None if acc doesn't exist
|
||||
match self.basic_account(addr)? {
|
||||
Some(acc) => Ok(Some(acc.nonce)),
|
||||
None => Ok(None),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Light wrapper that returns `StateProvider` implementations that correspond to the given
|
||||
/// `BlockNumber`, the latest state, or the pending state.
|
||||
///
|
||||
/// This type differentiates states into `historical`, `latest` and `pending`, where the `latest`
|
||||
/// block determines what is historical or pending: `[historical..latest..pending]`.
|
||||
///
|
||||
/// The `latest` state represents the state after the most recent block has been committed to the
|
||||
/// database, `historical` states are states that have been committed to the database before the
|
||||
/// `latest` state, and `pending` states are states that have not yet been committed to the
|
||||
/// database which may or may not become the `latest` state, depending on consensus.
|
||||
///
|
||||
/// Note: the `pending` block is considered the block that extends the canonical chain but one and
|
||||
/// has the `latest` block as its parent.
|
||||
///
|
||||
/// All states are _inclusive_, meaning they include _all_ all changes made (executed transactions)
|
||||
/// in their respective blocks. For example [StateProviderFactory::history_by_block_number] for
|
||||
/// block number `n` will return the state after block `n` was executed (transactions, withdrawals).
|
||||
/// In other words, all states point to the end of the state's respective block, which is equivalent
|
||||
/// to state at the beginning of the child block.
|
||||
///
|
||||
/// This affects tracing, or replaying blocks, which will need to be executed on top of the state of
|
||||
/// the parent block. For example, in order to trace block `n`, the state after block `n - 1` needs
|
||||
/// to be used, since block `n` was executed on its parent block's state.
|
||||
#[auto_impl(&, Arc, Box)]
|
||||
pub trait StateProviderFactory: BlockIdReader + Send + Sync {
|
||||
/// Storage provider for latest block.
|
||||
fn latest(&self) -> ProviderResult<StateProviderBox>;
|
||||
|
||||
/// Returns a [StateProvider] indexed by the given [BlockId].
|
||||
///
|
||||
/// Note: if a number or hash is provided this will __only__ look at historical(canonical)
|
||||
/// state.
|
||||
fn state_by_block_id(&self, block_id: BlockId) -> ProviderResult<StateProviderBox> {
|
||||
match block_id {
|
||||
BlockId::Number(block_number) => self.state_by_block_number_or_tag(block_number),
|
||||
BlockId::Hash(block_hash) => self.history_by_block_hash(block_hash.into()),
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns a [StateProvider] indexed by the given block number or tag.
|
||||
///
|
||||
/// Note: if a number is provided this will only look at historical(canonical) state.
|
||||
fn state_by_block_number_or_tag(
|
||||
&self,
|
||||
number_or_tag: BlockNumberOrTag,
|
||||
) -> ProviderResult<StateProviderBox> {
|
||||
match number_or_tag {
|
||||
BlockNumberOrTag::Latest => self.latest(),
|
||||
BlockNumberOrTag::Finalized => {
|
||||
// we can only get the finalized state by hash, not by num
|
||||
let hash = match self.finalized_block_hash()? {
|
||||
Some(hash) => hash,
|
||||
None => return Err(ProviderError::FinalizedBlockNotFound),
|
||||
};
|
||||
// only look at historical state
|
||||
self.history_by_block_hash(hash)
|
||||
}
|
||||
BlockNumberOrTag::Safe => {
|
||||
// we can only get the safe state by hash, not by num
|
||||
let hash = match self.safe_block_hash()? {
|
||||
Some(hash) => hash,
|
||||
None => return Err(ProviderError::SafeBlockNotFound),
|
||||
};
|
||||
|
||||
self.history_by_block_hash(hash)
|
||||
}
|
||||
BlockNumberOrTag::Earliest => self.history_by_block_number(0),
|
||||
BlockNumberOrTag::Pending => self.pending(),
|
||||
BlockNumberOrTag::Number(num) => {
|
||||
// Note: The `BlockchainProvider` could also lookup the tree for the given block number, if for example the block number is `latest + 1`, however this should only support canonical state: <https://github.com/paradigmxyz/reth/issues/4515>
|
||||
self.history_by_block_number(num)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns a historical [StateProvider] indexed by the given historic block number.
|
||||
///
|
||||
///
|
||||
/// Note: this only looks at historical blocks, not pending blocks.
|
||||
fn history_by_block_number(&self, block: BlockNumber) -> ProviderResult<StateProviderBox>;
|
||||
|
||||
/// Returns a historical [StateProvider] indexed by the given block hash.
|
||||
///
|
||||
/// Note: this only looks at historical blocks, not pending blocks.
|
||||
fn history_by_block_hash(&self, block: BlockHash) -> ProviderResult<StateProviderBox>;
|
||||
|
||||
/// Returns _any_[StateProvider] with matching block hash.
|
||||
///
|
||||
/// This will return a [StateProvider] for either a historical or pending block.
|
||||
fn state_by_block_hash(&self, block: BlockHash) -> ProviderResult<StateProviderBox>;
|
||||
|
||||
/// Storage provider for pending state.
|
||||
///
|
||||
/// Represents the state at the block that extends the canonical chain by one.
|
||||
/// If there's no `pending` block, then this is equal to [StateProviderFactory::latest]
|
||||
fn pending(&self) -> ProviderResult<StateProviderBox>;
|
||||
|
||||
/// Storage provider for pending state for the given block hash.
|
||||
///
|
||||
/// Represents the state at the block that extends the canonical chain.
|
||||
///
|
||||
/// If the block couldn't be found, returns `None`.
|
||||
fn pending_state_by_hash(&self, block_hash: B256) -> ProviderResult<Option<StateProviderBox>>;
|
||||
|
||||
/// Return a [StateProvider] that contains bundle state data provider.
|
||||
/// Used to inspect or execute transaction on the pending state.
|
||||
fn pending_with_provider(
|
||||
&self,
|
||||
bundle_state_data: Box<dyn FullBundleStateDataProvider>,
|
||||
) -> ProviderResult<StateProviderBox>;
|
||||
}
|
||||
|
||||
/// Blockchain trait provider that gives access to the blockchain state that is not yet committed
|
||||
/// (pending).
|
||||
pub trait BlockchainTreePendingStateProvider: Send + Sync {
|
||||
/// Returns a state provider that includes all state changes of the given (pending) block hash.
|
||||
///
|
||||
/// In other words, the state provider will return the state after all transactions of the given
|
||||
/// hash have been executed.
|
||||
fn pending_state_provider(
|
||||
&self,
|
||||
block_hash: BlockHash,
|
||||
) -> ProviderResult<Box<dyn FullBundleStateDataProvider>> {
|
||||
self.find_pending_state_provider(block_hash)
|
||||
.ok_or(ProviderError::StateForHashNotFound(block_hash))
|
||||
}
|
||||
|
||||
/// Returns state provider if a matching block exists.
|
||||
fn find_pending_state_provider(
|
||||
&self,
|
||||
block_hash: BlockHash,
|
||||
) -> Option<Box<dyn FullBundleStateDataProvider>>;
|
||||
}
|
||||
|
||||
/// Post state data needed for execution on it.
|
||||
///
|
||||
/// State contains:
|
||||
/// * [`BundleStateWithReceipts`] contains all changed of accounts and storage of pending chain
|
||||
/// * block hashes of pending chain and canonical blocks.
|
||||
#[auto_impl(&, Box)]
|
||||
pub trait BundleStateDataProvider: Send + Sync {
|
||||
/// Return post state
|
||||
fn state(&self) -> &BundleStateWithReceipts;
|
||||
/// Return block hash by block number of pending or canonical chain.
|
||||
fn block_hash(&self, block_number: BlockNumber) -> Option<BlockHash>;
|
||||
}
|
||||
|
||||
impl BundleStateDataProvider for BundleStateWithReceipts {
|
||||
fn state(&self) -> &BundleStateWithReceipts {
|
||||
self
|
||||
}
|
||||
|
||||
/// Always returns [None] because we don't have any information about the block header.
|
||||
fn block_hash(&self, _block_number: BlockNumber) -> Option<BlockHash> {
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
/// Fork data needed for execution on it.
|
||||
///
|
||||
/// It contains a canonical fork, the block on what pending chain was forked from.
|
||||
#[auto_impl(&, Box)]
|
||||
pub trait BundleStateForkProvider {
|
||||
/// Return canonical fork, the block on what post state was forked from.
|
||||
///
|
||||
/// Needed to create state provider.
|
||||
fn canonical_fork(&self) -> BlockNumHash;
|
||||
}
|
||||
|
||||
/// Full post state data needed for execution on it.
|
||||
/// This trait is used to create a state provider over pending state.
|
||||
///
|
||||
/// This trait is a combination of [`BundleStateDataProvider`] and [`BundleStateForkProvider`].
|
||||
///
|
||||
/// Pending state contains:
|
||||
/// * [`BundleStateWithReceipts`] contains all changed of accounts and storage of pending chain
|
||||
/// * block hashes of pending chain and canonical blocks.
|
||||
/// * canonical fork, the block on what pending chain was forked from.
|
||||
pub trait FullBundleStateDataProvider: BundleStateDataProvider + BundleStateForkProvider {}
|
||||
|
||||
impl<T> FullBundleStateDataProvider for T where T: BundleStateDataProvider + BundleStateForkProvider {}
|
||||
@ -1,14 +1,12 @@
|
||||
use reth_primitives::{Address, BlockNumber, StorageEntry, B256};
|
||||
use reth_storage_errors::provider::ProviderResult;
|
||||
use std::{
|
||||
collections::{BTreeMap, BTreeSet},
|
||||
ops::RangeInclusive,
|
||||
};
|
||||
|
||||
use auto_impl::auto_impl;
|
||||
use reth_primitives::{Address, BlockNumber, StorageEntry, B256};
|
||||
use reth_storage_errors::provider::ProviderResult;
|
||||
|
||||
/// Storage reader
|
||||
#[auto_impl(&, Arc, Box)]
|
||||
#[auto_impl::auto_impl(&, Arc, Box)]
|
||||
pub trait StorageReader: Send + Sync {
|
||||
/// Get plainstate storages for addresses and storage keys.
|
||||
fn plain_state_storages(
|
||||
@ -1,10 +1,11 @@
|
||||
use std::ops::{Range, RangeBounds, RangeInclusive};
|
||||
|
||||
use crate::{BlockNumReader, BlockReader};
|
||||
use reth_primitives::{
|
||||
Address, BlockHashOrNumber, BlockNumber, TransactionMeta, TransactionSigned,
|
||||
TransactionSignedNoHash, TxHash, TxNumber,
|
||||
};
|
||||
use reth_storage_errors::provider::{ProviderError, ProviderResult};
|
||||
use std::ops::{Range, RangeBounds, RangeInclusive};
|
||||
|
||||
/// Client trait for fetching [TransactionSigned] related data.
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
@ -1,11 +1,10 @@
|
||||
use auto_impl::auto_impl;
|
||||
use reth_primitives::B256;
|
||||
use reth_storage_errors::provider::ProviderResult;
|
||||
use reth_trie::updates::TrieUpdates;
|
||||
use revm::db::BundleState;
|
||||
|
||||
/// A type that can compute the state root of a given post state.
|
||||
#[auto_impl(&, Box, Arc)]
|
||||
#[auto_impl::auto_impl(&, Box, Arc)]
|
||||
pub trait StateRootProvider: Send + Sync {
|
||||
/// Returns the state root of the `BundleState` on top of the current state.
|
||||
///
|
||||
Reference in New Issue
Block a user