feat: make RPC cache generic over primitives (#13146)

This commit is contained in:
Arsenii Kulikov
2024-12-05 17:14:51 +04:00
committed by GitHub
parent 804dc99ef4
commit b4124dd1b0
31 changed files with 354 additions and 274 deletions

1
Cargo.lock generated
View File

@ -8382,6 +8382,7 @@ dependencies = [
"reth-optimism-forks",
"reth-optimism-primitives",
"reth-primitives",
"reth-primitives-traits",
"reth-prune-types",
"reth-revm",
"revm",

View File

@ -16,6 +16,7 @@ reth-chainspec.workspace = true
reth-ethereum-forks.workspace = true
reth-evm.workspace = true
reth-primitives.workspace = true
reth-primitives-traits.workspace = true
reth-revm.workspace = true
reth-execution-errors.workspace = true
reth-execution-types.workspace = true
@ -63,6 +64,7 @@ std = [
"alloy-genesis/std",
"alloy-primitives/std",
"revm-primitives/std",
"reth-primitives-traits/std",
"revm/std",
"reth-optimism-primitives/std",
"reth-ethereum-forks/std",

View File

@ -8,7 +8,7 @@ use reth_chainspec::ChainSpec;
use reth_execution_errors::BlockExecutionError;
use reth_optimism_chainspec::OpChainSpec;
use reth_optimism_forks::OpHardfork;
use reth_primitives::BlockBody;
use reth_primitives_traits::BlockBody;
use revm::{
primitives::{Bytecode, HashMap, SpecId},
DatabaseCommit, L1BlockInfo,
@ -32,9 +32,9 @@ const L1_BLOCK_ECOTONE_SELECTOR: [u8; 4] = hex!("440a5e20");
/// transaction in the L2 block.
///
/// Returns an error if the L1 info transaction is not found, if the block is empty.
pub fn extract_l1_info(body: &BlockBody) -> Result<L1BlockInfo, OpBlockExecutionError> {
pub fn extract_l1_info<B: BlockBody>(body: &B) -> Result<L1BlockInfo, OpBlockExecutionError> {
let l1_info_tx_data = body
.transactions
.transactions()
.first()
.ok_or_else(|| OpBlockExecutionError::L1BlockInfoError {
message: "could not find l1 block info tx in the L2 block".to_string(),

View File

@ -9,10 +9,10 @@ use reth_primitives::TransactionMeta;
use reth_provider::HeaderProvider;
use reth_rpc_eth_api::{
helpers::{EthBlocks, LoadBlock, LoadPendingBlock, LoadReceipt, SpawnBlocking},
RpcNodeCore, RpcReceipt,
RpcReceipt,
};
use crate::{OpEthApi, OpEthApiError, OpReceiptBuilder};
use crate::{eth::OpNodeCore, OpEthApi, OpEthApiError, OpReceiptBuilder};
impl<N> EthBlocks for OpEthApi<N>
where
@ -20,7 +20,7 @@ where
Error = OpEthApiError,
NetworkTypes: Network<ReceiptResponse = OpTransactionReceipt>,
>,
N: RpcNodeCore<Provider: ChainSpecProvider<ChainSpec = OpChainSpec> + HeaderProvider>,
N: OpNodeCore<Provider: ChainSpecProvider<ChainSpec = OpChainSpec> + HeaderProvider>,
{
async fn block_receipts(
&self,
@ -77,6 +77,6 @@ where
impl<N> LoadBlock for OpEthApi<N>
where
Self: LoadPendingBlock + SpawnBlocking,
N: RpcNodeCore,
N: OpNodeCore,
{
}

View File

@ -1,3 +1,4 @@
use super::OpNodeCore;
use crate::{OpEthApi, OpEthApiError};
use alloy_consensus::Header;
use alloy_primitives::{Bytes, TxKind, U256};
@ -5,7 +6,7 @@ use alloy_rpc_types_eth::transaction::TransactionRequest;
use reth_evm::ConfigureEvm;
use reth_rpc_eth_api::{
helpers::{estimate::EstimateCall, Call, EthCall, LoadPendingBlock, LoadState, SpawnBlocking},
FromEthApiError, IntoEthApiError, RpcNodeCore,
FromEthApiError, IntoEthApiError,
};
use reth_rpc_eth_types::{revm_utils::CallFees, RpcInvalidTransactionError};
use revm::primitives::{BlockEnv, OptimismFields, TxEnv};
@ -13,7 +14,7 @@ use revm::primitives::{BlockEnv, OptimismFields, TxEnv};
impl<N> EthCall for OpEthApi<N>
where
Self: EstimateCall + LoadPendingBlock,
N: RpcNodeCore,
N: OpNodeCore,
{
}
@ -21,7 +22,7 @@ impl<N> EstimateCall for OpEthApi<N>
where
Self: Call,
Self::Error: From<OpEthApiError>,
N: RpcNodeCore,
N: OpNodeCore,
{
}
@ -29,7 +30,7 @@ impl<N> Call for OpEthApi<N>
where
Self: LoadState<Evm: ConfigureEvm<Header = Header>> + SpawnBlocking,
Self::Error: From<OpEthApiError>,
N: RpcNodeCore,
N: OpNodeCore,
{
#[inline]
fn call_gas_limit(&self) -> u64 {

View File

@ -8,6 +8,7 @@ mod call;
mod pending_block;
pub use receipt::{OpReceiptBuilder, OpReceiptFieldsBuilder};
use reth_node_api::NodePrimitives;
use reth_optimism_primitives::OpPrimitives;
use std::{fmt, sync::Arc};
@ -21,7 +22,8 @@ use reth_network_api::NetworkInfo;
use reth_node_builder::EthApiBuilderCtx;
use reth_provider::{
BlockNumReader, BlockReader, BlockReaderIdExt, CanonStateSubscriptions, ChainSpecProvider,
EvmEnvProvider, StageCheckpointReader, StateProviderFactory,
EvmEnvProvider, NodePrimitivesProvider, ProviderBlock, ProviderReceipt, StageCheckpointReader,
StateProviderFactory,
};
use reth_rpc::eth::{core::EthApiInner, DevSigner};
use reth_rpc_eth_api::{
@ -48,6 +50,10 @@ pub type EthApiNodeBackend<N> = EthApiInner<
<N as RpcNodeCore>::Evm,
>;
/// A helper trait with requirements for [`RpcNodeCore`] to be used in [`OpEthApi`].
pub trait OpNodeCore: RpcNodeCore<Provider: BlockReader> {}
impl<T> OpNodeCore for T where T: RpcNodeCore<Provider: BlockReader> {}
/// OP-Reth `Eth` API implementation.
///
/// This type provides the functionality for handling `eth_` related requests.
@ -59,14 +65,14 @@ pub type EthApiNodeBackend<N> = EthApiInner<
/// This type implements the [`FullEthApi`](reth_rpc_eth_api::helpers::FullEthApi) by implemented
/// all the `Eth` helper traits and prerequisite traits.
#[derive(Clone)]
pub struct OpEthApi<N: RpcNodeCore> {
pub struct OpEthApi<N: OpNodeCore> {
/// Gateway to node's core components.
inner: Arc<OpEthApiInner<N>>,
}
impl<N> OpEthApi<N>
where
N: RpcNodeCore<
N: OpNodeCore<
Provider: BlockReaderIdExt
+ ChainSpecProvider
+ CanonStateSubscriptions<Primitives = OpPrimitives>
@ -83,7 +89,7 @@ where
impl<N> EthApiTypes for OpEthApi<N>
where
Self: Send + Sync,
N: RpcNodeCore,
N: OpNodeCore,
{
type Error = OpEthApiError;
type NetworkTypes = Optimism;
@ -96,7 +102,7 @@ where
impl<N> RpcNodeCore for OpEthApi<N>
where
N: RpcNodeCore,
N: OpNodeCore,
{
type Provider = N::Provider;
type Pool = N::Pool;
@ -132,17 +138,17 @@ where
impl<N> RpcNodeCoreExt for OpEthApi<N>
where
N: RpcNodeCore,
N: OpNodeCore,
{
#[inline]
fn cache(&self) -> &EthStateCache {
fn cache(&self) -> &EthStateCache<ProviderBlock<N::Provider>, ProviderReceipt<N::Provider>> {
self.inner.eth_api.cache()
}
}
impl<N> EthApiSpec for OpEthApi<N>
where
N: RpcNodeCore<
N: OpNodeCore<
Provider: ChainSpecProvider<ChainSpec: EthereumHardforks>
+ BlockNumReader
+ StageCheckpointReader,
@ -163,7 +169,7 @@ where
impl<N> SpawnBlocking for OpEthApi<N>
where
Self: Send + Sync + Clone + 'static,
N: RpcNodeCore,
N: OpNodeCore,
{
#[inline]
fn io_task_spawner(&self) -> impl TaskSpawner {
@ -184,7 +190,7 @@ where
impl<N> LoadFee for OpEthApi<N>
where
Self: LoadBlock<Provider = N::Provider>,
N: RpcNodeCore<
N: OpNodeCore<
Provider: BlockReaderIdExt
+ EvmEnvProvider
+ ChainSpecProvider<ChainSpec: EthChainSpec + EthereumHardforks>
@ -203,7 +209,7 @@ where
}
impl<N> LoadState for OpEthApi<N> where
N: RpcNodeCore<
N: OpNodeCore<
Provider: StateProviderFactory + ChainSpecProvider<ChainSpec: EthereumHardforks>,
Pool: TransactionPool,
>
@ -213,7 +219,7 @@ impl<N> LoadState for OpEthApi<N> where
impl<N> EthState for OpEthApi<N>
where
Self: LoadState + SpawnBlocking,
N: RpcNodeCore,
N: OpNodeCore,
{
#[inline]
fn max_proof_window(&self) -> u64 {
@ -224,27 +230,27 @@ where
impl<N> EthFees for OpEthApi<N>
where
Self: LoadFee,
N: RpcNodeCore,
N: OpNodeCore,
{
}
impl<N> Trace for OpEthApi<N>
where
Self: RpcNodeCore<Provider: BlockReader> + LoadState<Evm: ConfigureEvm<Header = Header>>,
N: RpcNodeCore,
N: OpNodeCore,
{
}
impl<N> AddDevSigners for OpEthApi<N>
where
N: RpcNodeCore,
N: OpNodeCore,
{
fn with_dev_accounts(&self) {
*self.inner.eth_api.signers().write() = DevSigner::random_signers(20)
}
}
impl<N: RpcNodeCore> fmt::Debug for OpEthApi<N> {
impl<N: OpNodeCore> fmt::Debug for OpEthApi<N> {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
f.debug_struct("OpEthApi").finish_non_exhaustive()
}
@ -252,7 +258,7 @@ impl<N: RpcNodeCore> fmt::Debug for OpEthApi<N> {
/// Container type `OpEthApi`
#[allow(missing_debug_implementations)]
struct OpEthApiInner<N: RpcNodeCore> {
struct OpEthApiInner<N: OpNodeCore> {
/// Gateway to node's core components.
eth_api: EthApiNodeBackend<N>,
/// Sequencer client, configured to forward submitted transactions to sequencer of given OP
@ -285,10 +291,12 @@ impl OpEthApiBuilder {
/// Builds an instance of [`OpEthApi`]
pub fn build<N>(self, ctx: &EthApiBuilderCtx<N>) -> OpEthApi<N>
where
N: RpcNodeCore<
Provider: BlockReaderIdExt
+ ChainSpecProvider
+ CanonStateSubscriptions<Primitives = OpPrimitives>
N: OpNodeCore<
Provider: BlockReaderIdExt<
Block = <<N::Provider as NodePrimitivesProvider>::Primitives as NodePrimitives>::Block,
Receipt = <<N::Provider as NodePrimitivesProvider>::Primitives as NodePrimitives>::Receipt,
> + ChainSpecProvider
+ CanonStateSubscriptions
+ Clone
+ 'static,
>,

View File

@ -10,17 +10,17 @@ use reth_primitives::{RecoveredTx, TransactionSigned};
use reth_provider::{BlockReaderIdExt, ReceiptProvider, TransactionsProvider};
use reth_rpc_eth_api::{
helpers::{EthSigner, EthTransactions, LoadTransaction, SpawnBlocking},
FromEthApiError, FullEthApiTypes, RpcNodeCore, TransactionCompat,
FromEthApiError, FullEthApiTypes, RpcNodeCore, RpcNodeCoreExt, TransactionCompat,
};
use reth_rpc_eth_types::utils::recover_raw_transaction;
use reth_transaction_pool::{PoolTransaction, TransactionOrigin, TransactionPool};
use crate::{OpEthApi, OpEthApiError, SequencerClient};
use crate::{eth::OpNodeCore, OpEthApi, OpEthApiError, SequencerClient};
impl<N> EthTransactions for OpEthApi<N>
where
Self: LoadTransaction<Provider: BlockReaderIdExt>,
N: RpcNodeCore,
N: OpNodeCore,
{
fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn EthSigner>>> {
self.inner.eth_api.signers()
@ -56,15 +56,15 @@ where
impl<N> LoadTransaction for OpEthApi<N>
where
Self: SpawnBlocking + FullEthApiTypes,
N: RpcNodeCore<Provider: TransactionsProvider, Pool: TransactionPool>,
Self: SpawnBlocking + FullEthApiTypes + RpcNodeCoreExt,
N: OpNodeCore<Provider: TransactionsProvider, Pool: TransactionPool>,
Self::Pool: TransactionPool,
{
}
impl<N> OpEthApi<N>
where
N: RpcNodeCore,
N: OpNodeCore,
{
/// Returns the [`SequencerClient`] if one is set.
pub fn raw_tx_forwarder(&self) -> Option<SequencerClient> {

View File

@ -15,11 +15,11 @@ pub type DynEthApiBuilder<Provider, Pool, EvmConfig, Network, Tasks, Events, Eth
/// Handlers for core, filter and pubsub `eth` namespace APIs.
#[derive(Debug, Clone)]
pub struct EthHandlers<Provider, Pool, Network, Events, EthApi: EthApiTypes> {
pub struct EthHandlers<Provider: BlockReader, Pool, Network, Events, EthApi: EthApiTypes> {
/// Main `eth_` request handler
pub api: EthApi,
/// The async caching layer used by the eth handlers
pub cache: EthStateCache,
pub cache: EthStateCache<Provider::Block, Provider::Receipt>,
/// Polling based filter handler available on all transports
pub filter: EthFilter<Provider, Pool, EthApi>,
/// Handler for subscriptions only available for transports that support it (ws, ipc)

View File

@ -925,7 +925,7 @@ impl RpcModuleConfigBuilder {
/// A Helper type the holds instances of the configured modules.
#[derive(Debug, Clone)]
pub struct RpcRegistryInner<
Provider,
Provider: BlockReader,
Pool,
Network,
Tasks,
@ -1029,6 +1029,7 @@ where
impl<Provider, Pool, Network, Tasks, Events, EthApi, BlockExecutor, Consensus>
RpcRegistryInner<Provider, Pool, Network, Tasks, Events, EthApi, BlockExecutor, Consensus>
where
Provider: BlockReader,
EthApi: EthApiTypes,
{
/// Returns a reference to the installed [`EthApi`](reth_rpc::eth::EthApi).
@ -1045,7 +1046,7 @@ where
///
/// This will spawn exactly one [`EthStateCache`] service if this is the first time the cache is
/// requested.
pub const fn eth_cache(&self) -> &EthStateCache {
pub const fn eth_cache(&self) -> &EthStateCache<Provider::Block, Provider::Receipt> {
&self.eth.cache
}
@ -1089,7 +1090,7 @@ impl<Provider, Pool, Network, Tasks, Events, EthApi, BlockExecutor, Consensus>
where
Network: NetworkInfo + Clone + 'static,
EthApi: EthApiTypes,
Provider: ChainSpecProvider<ChainSpec: EthereumHardforks>,
Provider: BlockReader + ChainSpecProvider<ChainSpec: EthereumHardforks>,
BlockExecutor: BlockExecutorProvider,
{
/// Instantiates `AdminApi`

View File

@ -8,12 +8,13 @@ use alloy_network::TransactionBuilder;
use alloy_primitives::{Address, Bytes, TxHash, B256};
use alloy_rpc_types_eth::{transaction::TransactionRequest, BlockNumberOrTag, TransactionInfo};
use futures::Future;
use reth_node_api::BlockBody;
use reth_primitives::{
transaction::SignedTransactionIntoRecoveredExt, SealedBlockWithSenders, TransactionMeta,
TransactionSigned,
};
use reth_provider::{
BlockNumReader, BlockReaderIdExt, ProviderReceipt, ProviderTx, ReceiptProvider,
BlockNumReader, BlockReaderIdExt, ProviderBlock, ProviderReceipt, ProviderTx, ReceiptProvider,
TransactionsProvider,
};
use reth_rpc_eth_types::{
@ -79,15 +80,17 @@ pub trait EthTransactions: LoadTransaction<Provider: BlockReaderIdExt> {
/// Get all transactions in the block with the given hash.
///
/// Returns `None` if block does not exist.
#[expect(clippy::type_complexity)]
fn transactions_by_block(
&self,
block: B256,
) -> impl Future<Output = Result<Option<Vec<TransactionSigned>>, Self::Error>> + Send {
) -> impl Future<Output = Result<Option<Vec<ProviderTx<Self::Provider>>>, Self::Error>> + Send
{
async move {
self.cache()
.get_sealed_block_with_senders(block)
.await
.map(|b| b.map(|b| b.body.transactions.clone()))
.map(|b| b.map(|b| b.body.transactions().to_vec()))
.map_err(Self::Error::from_eth_err)
}
}
@ -568,7 +571,10 @@ pub trait LoadTransaction: SpawnBlocking + FullEthApiTypes + RpcNodeCoreExt {
hash: B256,
) -> impl Future<
Output = Result<
Option<(TransactionSource<ProviderTx<Self::Provider>>, Arc<SealedBlockWithSenders>)>,
Option<(
TransactionSource<ProviderTx<Self::Provider>>,
Arc<SealedBlockWithSenders<ProviderBlock<Self::Provider>>>,
)>,
Self::Error,
>,
> + Send {

View File

@ -1,6 +1,7 @@
//! Helper trait for interfacing with [`FullNodeComponents`].
use reth_node_api::FullNodeComponents;
use reth_provider::{BlockReader, ProviderBlock, ProviderReceipt};
use reth_rpc_eth_types::EthStateCache;
/// Helper trait to relax trait bounds on [`FullNodeComponents`].
@ -76,7 +77,9 @@ where
/// Additional components, asides the core node components, needed to run `eth_` namespace API
/// server.
pub trait RpcNodeCoreExt: RpcNodeCore {
pub trait RpcNodeCoreExt: RpcNodeCore<Provider: BlockReader> {
/// Returns handle to RPC cache service.
fn cache(&self) -> &EthStateCache;
fn cache(
&self,
) -> &EthStateCache<ProviderBlock<Self::Provider>, ProviderReceipt<Self::Provider>>;
}

View File

@ -3,7 +3,7 @@
use reth_chain_state::CanonStateSubscriptions;
use reth_chainspec::ChainSpecProvider;
use reth_primitives::NodePrimitives;
use reth_storage_api::BlockReaderIdExt;
use reth_storage_api::{BlockReader, BlockReaderIdExt};
use reth_tasks::TaskSpawner;
use crate::{
@ -13,7 +13,10 @@ use crate::{
/// Context for building the `eth` namespace API.
#[derive(Debug, Clone)]
pub struct EthApiBuilderCtx<Provider, Pool, EvmConfig, Network, Tasks, Events> {
pub struct EthApiBuilderCtx<Provider, Pool, EvmConfig, Network, Tasks, Events>
where
Provider: BlockReader,
{
/// Database handle.
pub provider: Provider,
/// Mempool handle.
@ -29,7 +32,7 @@ pub struct EthApiBuilderCtx<Provider, Pool, EvmConfig, Network, Tasks, Events> {
/// Events handle.
pub events: Events,
/// RPC cache handle.
pub cache: EthStateCache,
pub cache: EthStateCache<Provider::Block, Provider::Receipt>,
}
impl<Provider, Pool, EvmConfig, Network, Tasks, Events>
@ -38,27 +41,24 @@ where
Provider: BlockReaderIdExt + Clone,
{
/// Returns a new [`FeeHistoryCache`] for the context.
pub fn new_fee_history_cache(&self) -> FeeHistoryCache
pub fn new_fee_history_cache<N>(&self) -> FeeHistoryCache
where
Provider: ChainSpecProvider + 'static,
N: NodePrimitives,
Tasks: TaskSpawner,
Events: CanonStateSubscriptions<
Primitives: NodePrimitives<
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
>,
>,
Events: CanonStateSubscriptions<Primitives = N>,
Provider:
BlockReaderIdExt<Block = N::Block, Receipt = N::Receipt> + ChainSpecProvider + 'static,
{
let fee_history_cache =
FeeHistoryCache::new(self.cache.clone(), self.config.fee_history_cache);
let fee_history_cache = FeeHistoryCache::new(self.config.fee_history_cache);
let new_canonical_blocks = self.events.canonical_state_stream();
let fhc = fee_history_cache.clone();
let provider = self.provider.clone();
let cache = self.cache.clone();
self.executor.spawn_critical(
"cache canonical blocks for fee history task",
Box::pin(async move {
fee_history_cache_new_blocks_task(fhc, new_canonical_blocks, provider).await;
fee_history_cache_new_blocks_task(fhc, new_canonical_blocks, provider, cache).await;
}),
);

View File

@ -1,14 +1,14 @@
//! Async caching support for eth RPC
use super::{EthStateCacheConfig, MultiConsumerLruCache};
use alloy_consensus::Header;
use alloy_eips::BlockHashOrNumber;
use alloy_primitives::B256;
use futures::{future::Either, Stream, StreamExt};
use reth_chain_state::CanonStateNotification;
use reth_errors::{ProviderError, ProviderResult};
use reth_execution_types::Chain;
use reth_primitives::{Receipt, SealedBlockWithSenders, TransactionSigned};
use reth_primitives::{NodePrimitives, SealedBlockWithSenders};
use reth_primitives_traits::{Block, BlockBody};
use reth_storage_api::{BlockReader, StateProviderFactory, TransactionVariant};
use reth_tasks::{TaskSpawner, TokioTaskExecutor};
use schnellru::{ByLength, Limiter};
@ -30,41 +30,49 @@ pub mod metrics;
pub mod multi_consumer;
/// The type that can send the response to a requested [`SealedBlockWithSenders`]
type BlockTransactionsResponseSender =
oneshot::Sender<ProviderResult<Option<Vec<TransactionSigned>>>>;
type BlockTransactionsResponseSender<T> = oneshot::Sender<ProviderResult<Option<Vec<T>>>>;
/// The type that can send the response to a requested [`SealedBlockWithSenders`]
type BlockWithSendersResponseSender =
oneshot::Sender<ProviderResult<Option<Arc<SealedBlockWithSenders>>>>;
type BlockWithSendersResponseSender<B> =
oneshot::Sender<ProviderResult<Option<Arc<SealedBlockWithSenders<B>>>>>;
/// The type that can send the response to the requested receipts of a block.
type ReceiptsResponseSender = oneshot::Sender<ProviderResult<Option<Arc<Vec<Receipt>>>>>;
type ReceiptsResponseSender<R> = oneshot::Sender<ProviderResult<Option<Arc<Vec<R>>>>>;
/// The type that can send the response to a requested header
type HeaderResponseSender = oneshot::Sender<ProviderResult<Header>>;
type HeaderResponseSender<H> = oneshot::Sender<ProviderResult<H>>;
type BlockLruCache<L> = MultiConsumerLruCache<
type BlockLruCache<B, L> = MultiConsumerLruCache<
B256,
Arc<SealedBlockWithSenders>,
Arc<SealedBlockWithSenders<B>>,
L,
Either<BlockWithSendersResponseSender, BlockTransactionsResponseSender>,
Either<
BlockWithSendersResponseSender<B>,
BlockTransactionsResponseSender<<<B as Block>::Body as BlockBody>::Transaction>,
>,
>;
type ReceiptsLruCache<L> =
MultiConsumerLruCache<B256, Arc<Vec<Receipt>>, L, ReceiptsResponseSender>;
type ReceiptsLruCache<R, L> =
MultiConsumerLruCache<B256, Arc<Vec<R>>, L, ReceiptsResponseSender<R>>;
type HeaderLruCache<L> = MultiConsumerLruCache<B256, Header, L, HeaderResponseSender>;
type HeaderLruCache<H, L> = MultiConsumerLruCache<B256, H, L, HeaderResponseSender<H>>;
/// Provides async access to cached eth data
///
/// This is the frontend for the async caching service which manages cached data on a different
/// task.
#[derive(Debug, Clone)]
pub struct EthStateCache {
to_service: UnboundedSender<CacheAction>,
#[derive(Debug)]
pub struct EthStateCache<B: Block, R> {
to_service: UnboundedSender<CacheAction<B, R>>,
}
impl EthStateCache {
impl<B: Block, R> Clone for EthStateCache<B, R> {
fn clone(&self) -> Self {
Self { to_service: self.to_service.clone() }
}
}
impl<B: Block, R: Send + Sync> EthStateCache<B, R> {
/// Creates and returns both [`EthStateCache`] frontend and the memory bound service.
fn create<Provider, Tasks>(
provider: Provider,
@ -73,7 +81,10 @@ impl EthStateCache {
max_receipts: u32,
max_headers: u32,
max_concurrent_db_operations: usize,
) -> (Self, EthStateCacheService<Provider, Tasks>) {
) -> (Self, EthStateCacheService<Provider, Tasks>)
where
Provider: BlockReader<Block = B, Receipt = R>,
{
let (to_service, rx) = unbounded_channel();
let service = EthStateCacheService {
provider,
@ -95,14 +106,8 @@ impl EthStateCache {
/// See also [`Self::spawn_with`]
pub fn spawn<Provider>(provider: Provider, config: EthStateCacheConfig) -> Self
where
Provider: StateProviderFactory
+ BlockReader<
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
Header = reth_primitives::Header,
> + Clone
+ Unpin
+ 'static,
Provider:
StateProviderFactory + BlockReader<Block = B, Receipt = R> + Clone + Unpin + 'static,
{
Self::spawn_with(provider, config, TokioTaskExecutor::default())
}
@ -117,14 +122,8 @@ impl EthStateCache {
executor: Tasks,
) -> Self
where
Provider: StateProviderFactory
+ BlockReader<
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
Header = reth_primitives::Header,
> + Clone
+ Unpin
+ 'static,
Provider:
StateProviderFactory + BlockReader<Block = B, Receipt = R> + Clone + Unpin + 'static,
Tasks: TaskSpawner + Clone + 'static,
{
let EthStateCacheConfig {
@ -151,19 +150,16 @@ impl EthStateCache {
pub async fn get_sealed_block_with_senders(
&self,
block_hash: B256,
) -> ProviderResult<Option<Arc<SealedBlockWithSenders>>> {
) -> ProviderResult<Option<Arc<SealedBlockWithSenders<B>>>> {
let (response_tx, rx) = oneshot::channel();
let _ = self.to_service.send(CacheAction::GetBlockWithSenders { block_hash, response_tx });
rx.await.map_err(|_| ProviderError::CacheServiceUnavailable)?
}
/// Requests the [Receipt] for the block hash
/// Requests the receipts for the block hash
///
/// Returns `None` if the block was not found.
pub async fn get_receipts(
&self,
block_hash: B256,
) -> ProviderResult<Option<Arc<Vec<Receipt>>>> {
pub async fn get_receipts(&self, block_hash: B256) -> ProviderResult<Option<Arc<Vec<R>>>> {
let (response_tx, rx) = oneshot::channel();
let _ = self.to_service.send(CacheAction::GetReceipts { block_hash, response_tx });
rx.await.map_err(|_| ProviderError::CacheServiceUnavailable)?
@ -173,7 +169,7 @@ impl EthStateCache {
pub async fn get_block_and_receipts(
&self,
block_hash: B256,
) -> ProviderResult<Option<(Arc<SealedBlockWithSenders>, Arc<Vec<Receipt>>)>> {
) -> ProviderResult<Option<(Arc<SealedBlockWithSenders<B>>, Arc<Vec<R>>)>> {
let block = self.get_sealed_block_with_senders(block_hash);
let receipts = self.get_receipts(block_hash);
@ -185,7 +181,7 @@ impl EthStateCache {
/// Requests the header for the given hash.
///
/// Returns an error if the header is not found.
pub async fn get_header(&self, block_hash: B256) -> ProviderResult<Header> {
pub async fn get_header(&self, block_hash: B256) -> ProviderResult<B::Header> {
let (response_tx, rx) = oneshot::channel();
let _ = self.to_service.send(CacheAction::GetHeader { block_hash, response_tx });
rx.await.map_err(|_| ProviderError::CacheServiceUnavailable)?
@ -216,25 +212,26 @@ pub(crate) struct EthStateCacheService<
LimitReceipts = ByLength,
LimitHeaders = ByLength,
> where
LimitBlocks: Limiter<B256, Arc<SealedBlockWithSenders>>,
LimitReceipts: Limiter<B256, Arc<Vec<Receipt>>>,
LimitHeaders: Limiter<B256, Header>,
Provider: BlockReader,
LimitBlocks: Limiter<B256, Arc<SealedBlockWithSenders<Provider::Block>>>,
LimitReceipts: Limiter<B256, Arc<Vec<Provider::Receipt>>>,
LimitHeaders: Limiter<B256, Provider::Header>,
{
/// The type used to lookup data from disk
provider: Provider,
/// The LRU cache for full blocks grouped by their hash.
full_block_cache: BlockLruCache<LimitBlocks>,
full_block_cache: BlockLruCache<Provider::Block, LimitBlocks>,
/// The LRU cache for full blocks grouped by their hash.
receipts_cache: ReceiptsLruCache<LimitReceipts>,
receipts_cache: ReceiptsLruCache<Provider::Receipt, LimitReceipts>,
/// The LRU cache for headers.
///
/// Headers are cached because they are required to populate the environment for execution
/// (evm).
headers_cache: HeaderLruCache<LimitHeaders>,
headers_cache: HeaderLruCache<Provider::Header, LimitHeaders>,
/// Sender half of the action channel.
action_tx: UnboundedSender<CacheAction>,
action_tx: UnboundedSender<CacheAction<Provider::Block, Provider::Receipt>>,
/// Receiver half of the action channel.
action_rx: UnboundedReceiverStream<CacheAction>,
action_rx: UnboundedReceiverStream<CacheAction<Provider::Block, Provider::Receipt>>,
/// The type that's used to spawn tasks that do the actual work
action_task_spawner: Tasks,
/// Rate limiter
@ -249,7 +246,7 @@ where
fn on_new_block(
&mut self,
block_hash: B256,
res: ProviderResult<Option<Arc<SealedBlockWithSenders>>>,
res: ProviderResult<Option<Arc<SealedBlockWithSenders<Provider::Block>>>>,
) {
if let Some(queued) = self.full_block_cache.remove(&block_hash) {
// send the response to queued senders
@ -260,7 +257,7 @@ where
}
Either::Right(transaction_tx) => {
let _ = transaction_tx.send(res.clone().map(|maybe_block| {
maybe_block.map(|block| block.block.body.transactions.clone())
maybe_block.map(|block| block.block.body.transactions().to_vec())
}));
}
}
@ -276,7 +273,7 @@ where
fn on_new_receipts(
&mut self,
block_hash: B256,
res: ProviderResult<Option<Arc<Vec<Receipt>>>>,
res: ProviderResult<Option<Arc<Vec<Provider::Receipt>>>>,
) {
if let Some(queued) = self.receipts_cache.remove(&block_hash) {
// send the response to queued senders
@ -294,7 +291,7 @@ where
fn on_reorg_block(
&mut self,
block_hash: B256,
res: ProviderResult<Option<SealedBlockWithSenders>>,
res: ProviderResult<Option<SealedBlockWithSenders<Provider::Block>>>,
) {
let res = res.map(|b| b.map(Arc::new));
if let Some(queued) = self.full_block_cache.remove(&block_hash) {
@ -306,7 +303,7 @@ where
}
Either::Right(transaction_tx) => {
let _ = transaction_tx.send(res.clone().map(|maybe_block| {
maybe_block.map(|block| block.block.body.transactions.clone())
maybe_block.map(|block| block.block.body.transactions().to_vec())
}));
}
}
@ -317,7 +314,7 @@ where
fn on_reorg_receipts(
&mut self,
block_hash: B256,
res: ProviderResult<Option<Arc<Vec<Receipt>>>>,
res: ProviderResult<Option<Arc<Vec<Provider::Receipt>>>>,
) {
if let Some(queued) = self.receipts_cache.remove(&block_hash) {
// send the response to queued senders
@ -336,14 +333,7 @@ where
impl<Provider, Tasks> Future for EthStateCacheService<Provider, Tasks>
where
Provider: StateProviderFactory
+ BlockReader<
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
Header = reth_primitives::Header,
> + Clone
+ Unpin
+ 'static,
Provider: StateProviderFactory + BlockReader + Clone + Unpin + 'static,
Tasks: TaskSpawner + Clone + 'static,
{
type Output = ();
@ -504,52 +494,55 @@ where
}
/// All message variants sent through the channel
enum CacheAction {
enum CacheAction<B: Block, R> {
GetBlockWithSenders {
block_hash: B256,
response_tx: BlockWithSendersResponseSender,
response_tx: BlockWithSendersResponseSender<B>,
},
GetHeader {
block_hash: B256,
response_tx: HeaderResponseSender,
response_tx: HeaderResponseSender<B::Header>,
},
GetReceipts {
block_hash: B256,
response_tx: ReceiptsResponseSender,
response_tx: ReceiptsResponseSender<R>,
},
BlockWithSendersResult {
block_hash: B256,
res: ProviderResult<Option<Arc<SealedBlockWithSenders>>>,
res: ProviderResult<Option<Arc<SealedBlockWithSenders<B>>>>,
},
ReceiptsResult {
block_hash: B256,
res: ProviderResult<Option<Arc<Vec<Receipt>>>>,
res: ProviderResult<Option<Arc<Vec<R>>>>,
},
HeaderResult {
block_hash: B256,
res: Box<ProviderResult<Header>>,
res: Box<ProviderResult<B::Header>>,
},
CacheNewCanonicalChain {
chain_change: ChainChange,
chain_change: ChainChange<B, R>,
},
RemoveReorgedChain {
chain_change: ChainChange,
chain_change: ChainChange<B, R>,
},
}
struct BlockReceipts {
struct BlockReceipts<R> {
block_hash: B256,
receipts: Vec<Option<Receipt>>,
receipts: Vec<Option<R>>,
}
/// A change of the canonical chain
struct ChainChange {
blocks: Vec<SealedBlockWithSenders>,
receipts: Vec<BlockReceipts>,
struct ChainChange<B: Block, R> {
blocks: Vec<SealedBlockWithSenders<B>>,
receipts: Vec<BlockReceipts<R>>,
}
impl ChainChange {
fn new(chain: Arc<Chain>) -> Self {
impl<B: Block, R: Clone> ChainChange<B, R> {
fn new<N>(chain: Arc<Chain<N>>) -> Self
where
N: NodePrimitives<Block = B, Receipt = R>,
{
let (blocks, receipts): (Vec<_>, Vec<_>) = chain
.blocks_and_receipts()
.map(|(block, receipts)| {
@ -566,9 +559,11 @@ impl ChainChange {
/// immediately before they need to be fetched from disk.
///
/// Reorged blocks are removed from the cache.
pub async fn cache_new_blocks_task<St>(eth_state_cache: EthStateCache, mut events: St)
where
St: Stream<Item = CanonStateNotification> + Unpin + 'static,
pub async fn cache_new_blocks_task<St, N: NodePrimitives>(
eth_state_cache: EthStateCache<N::Block, N::Receipt>,
mut events: St,
) where
St: Stream<Item = CanonStateNotification<N>> + Unpin + 'static,
{
while let Some(event) = events.next().await {
if let Some(reverted) = event.reverted() {

View File

@ -6,6 +6,7 @@ use std::{
sync::{atomic::Ordering::SeqCst, Arc},
};
use alloy_consensus::{BlockHeader, Transaction, TxReceipt};
use alloy_eips::eip1559::calc_next_block_base_fee;
use alloy_primitives::B256;
use alloy_rpc_types_eth::TxGasAndReward;
@ -16,8 +17,8 @@ use futures::{
use metrics::atomics::AtomicU64;
use reth_chain_state::CanonStateNotification;
use reth_chainspec::{ChainSpecProvider, EthChainSpec};
use reth_primitives::{NodePrimitives, Receipt, SealedBlock, TransactionSigned};
use reth_primitives_traits::{Block, BlockBody};
use reth_primitives::{NodePrimitives, SealedBlock};
use reth_primitives_traits::BlockBody;
use reth_rpc_server_types::constants::gas_oracle::MAX_HEADER_HISTORY;
use reth_storage_api::BlockReaderIdExt;
use revm_primitives::{calc_blob_gasprice, calc_excess_blob_gas};
@ -36,13 +37,12 @@ pub struct FeeHistoryCache {
impl FeeHistoryCache {
/// Creates new `FeeHistoryCache` instance, initialize it with the more recent data, set bounds
pub fn new(eth_cache: EthStateCache, config: FeeHistoryCacheConfig) -> Self {
pub fn new(config: FeeHistoryCacheConfig) -> Self {
let inner = FeeHistoryCacheInner {
lower_bound: Default::default(),
upper_bound: Default::default(),
config,
entries: Default::default(),
eth_cache,
};
Self { inner: Arc::new(inner) }
}
@ -73,9 +73,12 @@ impl FeeHistoryCache {
}
/// Insert block data into the cache.
async fn insert_blocks<'a, I>(&self, blocks: I)
async fn insert_blocks<'a, I, H, B, R>(&self, blocks: I)
where
I: IntoIterator<Item = (&'a SealedBlock, Arc<Vec<Receipt>>)>,
H: BlockHeader + 'a,
B: BlockBody,
R: TxReceipt,
I: IntoIterator<Item = (&'a SealedBlock<H, B>, Arc<Vec<R>>)>,
{
let mut entries = self.inner.entries.write().await;
@ -87,11 +90,11 @@ impl FeeHistoryCache {
&percentiles,
fee_history_entry.gas_used,
fee_history_entry.base_fee_per_gas,
&block.body.transactions,
block.body.transactions(),
&receipts,
)
.unwrap_or_default();
entries.insert(block.number, fee_history_entry);
entries.insert(block.number(), fee_history_entry);
}
// enforce bounds by popping the oldest entries
@ -200,7 +203,6 @@ struct FeeHistoryCacheInner {
config: FeeHistoryCacheConfig,
/// Stores the entries of the cache
entries: tokio::sync::RwLock<BTreeMap<u64, FeeHistoryEntry>>,
eth_cache: EthStateCache,
}
/// Awaits for new chain events and directly inserts them into the cache so they're available
@ -209,10 +211,12 @@ pub async fn fee_history_cache_new_blocks_task<St, Provider, N>(
fee_history_cache: FeeHistoryCache,
mut events: St,
provider: Provider,
cache: EthStateCache<N::Block, N::Receipt>,
) where
St: Stream<Item = CanonStateNotification<N>> + Unpin + 'static,
Provider: BlockReaderIdExt + ChainSpecProvider + 'static,
N: NodePrimitives<Block = reth_primitives::Block, Receipt = reth_primitives::Receipt>,
Provider:
BlockReaderIdExt<Block = N::Block, Receipt = N::Receipt> + ChainSpecProvider + 'static,
N: NodePrimitives,
{
// We're listening for new blocks emitted when the node is in live sync.
// If the node transitions to stage sync, we need to fetch the missing blocks
@ -225,12 +229,7 @@ pub async fn fee_history_cache_new_blocks_task<St, Provider, N>(
trace!(target: "rpc::fee", ?block_number, "Fetching missing block for fee history cache");
if let Ok(Some(hash)) = provider.block_hash(block_number) {
// fetch missing block
fetch_missing_block = fee_history_cache
.inner
.eth_cache
.get_block_and_receipts(hash)
.boxed()
.fuse();
fetch_missing_block = cache.get_block_and_receipts(hash).boxed().fuse();
}
}
}
@ -270,13 +269,17 @@ pub async fn fee_history_cache_new_blocks_task<St, Provider, N>(
/// the corresponding rewards for the transactions at each percentile.
///
/// The results are returned as a vector of U256 values.
pub fn calculate_reward_percentiles_for_block(
pub fn calculate_reward_percentiles_for_block<T, R>(
percentiles: &[f64],
gas_used: u64,
base_fee_per_gas: u64,
transactions: &[TransactionSigned],
receipts: &[Receipt],
) -> Result<Vec<u128>, EthApiError> {
transactions: &[T],
receipts: &[R],
) -> Result<Vec<u128>, EthApiError>
where
T: Transaction,
R: TxReceipt,
{
let mut transactions = transactions
.iter()
.zip(receipts)
@ -287,12 +290,12 @@ pub fn calculate_reward_percentiles_for_block(
// While we will sum up the gas again later, it is worth
// noting that the order of the transactions will be different,
// so the sum will also be different for each receipt.
let gas_used = receipt.cumulative_gas_used - *previous_gas;
*previous_gas = receipt.cumulative_gas_used;
let gas_used = receipt.cumulative_gas_used() - *previous_gas;
*previous_gas = receipt.cumulative_gas_used();
Some(TxGasAndReward {
gas_used,
reward: tx.effective_tip_per_gas(Some(base_fee_per_gas)).unwrap_or_default(),
gas_used: gas_used as u64,
reward: tx.effective_tip_per_gas(base_fee_per_gas).unwrap_or_default(),
})
})
.collect::<Vec<_>>();
@ -361,20 +364,20 @@ impl FeeHistoryEntry {
/// Creates a new entry from a sealed block.
///
/// Note: This does not calculate the rewards for the block.
pub fn new(block: &SealedBlock) -> Self {
pub fn new<H: BlockHeader, B: BlockBody>(block: &SealedBlock<H, B>) -> Self {
Self {
base_fee_per_gas: block.base_fee_per_gas.unwrap_or_default(),
gas_used_ratio: block.gas_used as f64 / block.gas_limit as f64,
base_fee_per_blob_gas: block.blob_fee(),
blob_gas_used_ratio: block.body().blob_gas_used() as f64 /
base_fee_per_gas: block.base_fee_per_gas().unwrap_or_default(),
gas_used_ratio: block.gas_used() as f64 / block.gas_limit() as f64,
base_fee_per_blob_gas: block.excess_blob_gas().map(calc_blob_gasprice),
blob_gas_used_ratio: block.body.blob_gas_used() as f64 /
alloy_eips::eip4844::MAX_DATA_GAS_PER_BLOCK as f64,
excess_blob_gas: block.excess_blob_gas,
blob_gas_used: block.blob_gas_used,
gas_used: block.gas_used,
excess_blob_gas: block.excess_blob_gas(),
blob_gas_used: block.blob_gas_used(),
gas_used: block.gas_used(),
header_hash: block.hash(),
gas_limit: block.gas_limit,
gas_limit: block.gas_limit(),
rewards: Vec::new(),
timestamp: block.timestamp,
timestamp: block.timestamp(),
}
}

View File

@ -1,13 +1,13 @@
//! An implementation of the eth gas price oracle, used for providing gas price estimates based on
//! previous blocks.
use alloy_consensus::{constants::GWEI_TO_WEI, BlockHeader};
use alloy_consensus::{constants::GWEI_TO_WEI, BlockHeader, Transaction};
use alloy_eips::BlockNumberOrTag;
use alloy_primitives::{B256, U256};
use alloy_rpc_types_eth::BlockId;
use derive_more::{Deref, DerefMut, From, Into};
use itertools::Itertools;
use reth_primitives_traits::SignedTransaction;
use reth_primitives_traits::{BlockBody, SignedTransaction};
use reth_rpc_server_types::{
constants,
constants::gas_oracle::{
@ -15,7 +15,7 @@ use reth_rpc_server_types::{
DEFAULT_MAX_GAS_PRICE, MAX_HEADER_HISTORY, SAMPLE_NUMBER,
},
};
use reth_storage_api::BlockReaderIdExt;
use reth_storage_api::{BlockReader, BlockReaderIdExt};
use schnellru::{ByLength, LruMap};
use serde::{Deserialize, Serialize};
use std::fmt::{self, Debug, Formatter};
@ -70,11 +70,14 @@ impl Default for GasPriceOracleConfig {
/// Calculates a gas price depending on recent blocks.
#[derive(Debug)]
pub struct GasPriceOracle<Provider> {
pub struct GasPriceOracle<Provider>
where
Provider: BlockReader,
{
/// The type used to subscribe to block events and get block info
provider: Provider,
/// The cache for blocks
cache: EthStateCache,
cache: EthStateCache<Provider::Block, Provider::Receipt>,
/// The config for the oracle
oracle_config: GasPriceOracleConfig,
/// The price under which the sample will be ignored.
@ -92,7 +95,7 @@ where
pub fn new(
provider: Provider,
mut oracle_config: GasPriceOracleConfig,
cache: EthStateCache,
cache: EthStateCache<Provider::Block, Provider::Receipt>,
) -> Self {
// sanitize the percentile to be less than 100
if oracle_config.percentile > 100 {
@ -220,43 +223,44 @@ where
None => return Ok(None),
};
let base_fee_per_gas = block.base_fee_per_gas;
let parent_hash = block.parent_hash;
let base_fee_per_gas = block.base_fee_per_gas();
let parent_hash = block.parent_hash();
// sort the functions by ascending effective tip first
let sorted_transactions = block
.body
.transactions
.iter()
.sorted_by_cached_key(|tx| tx.effective_tip_per_gas(base_fee_per_gas));
let sorted_transactions = block.body.transactions().iter().sorted_by_cached_key(|tx| {
if let Some(base_fee) = base_fee_per_gas {
(*tx).effective_tip_per_gas(base_fee)
} else {
Some((*tx).priority_fee_or_price())
}
});
let mut prices = Vec::with_capacity(limit);
for tx in sorted_transactions {
let mut effective_gas_tip = None;
let effective_tip = if let Some(base_fee) = base_fee_per_gas {
tx.effective_tip_per_gas(base_fee)
} else {
Some(tx.priority_fee_or_price())
};
// ignore transactions with a tip under the configured threshold
if let Some(ignore_under) = self.ignore_price {
let tip = tx.effective_tip_per_gas(base_fee_per_gas);
effective_gas_tip = Some(tip);
if tip < Some(ignore_under) {
if effective_tip < Some(ignore_under) {
continue
}
}
// check if the sender was the coinbase, if so, ignore
if let Some(sender) = tx.recover_signer() {
if sender == block.beneficiary {
if sender == block.beneficiary() {
continue
}
}
// a `None` effective_gas_tip represents a transaction where the max_fee_per_gas is
// less than the base fee which would be invalid
let effective_gas_tip = effective_gas_tip
.unwrap_or_else(|| tx.effective_tip_per_gas(base_fee_per_gas))
.ok_or(RpcInvalidTransactionError::FeeCapTooLow)?;
prices.push(U256::from(effective_gas_tip));
prices.push(U256::from(effective_tip.ok_or(RpcInvalidTransactionError::FeeCapTooLow)?));
// we have enough entries
if prices.len() >= limit {

View File

@ -2,14 +2,15 @@
//!
//! Log parsing for building filter.
use alloy_consensus::TxReceipt;
use alloy_eips::{eip2718::Encodable2718, BlockNumHash};
use alloy_primitives::TxHash;
use alloy_rpc_types_eth::{FilteredParams, Log};
use reth_chainspec::ChainInfo;
use reth_errors::ProviderError;
use reth_primitives::{Receipt, SealedBlockWithSenders};
use reth_primitives_traits::SignedTransaction;
use reth_storage_api::BlockReader;
use reth_primitives_traits::{BlockBody, SignedTransaction};
use reth_storage_api::{BlockReader, ProviderBlock};
use std::sync::Arc;
/// Returns all matching of a block's receipts when the transaction hashes are known.
@ -54,20 +55,23 @@ pub enum ProviderOrBlock<'a, P: BlockReader> {
/// Provider
Provider(&'a P),
/// [`SealedBlockWithSenders`]
Block(Arc<SealedBlockWithSenders>),
Block(Arc<SealedBlockWithSenders<ProviderBlock<P>>>),
}
/// Appends all matching logs of a block's receipts.
/// If the log matches, look up the corresponding transaction hash.
pub fn append_matching_block_logs<P: BlockReader<Transaction: SignedTransaction>>(
pub fn append_matching_block_logs<P>(
all_logs: &mut Vec<Log>,
provider_or_block: ProviderOrBlock<'_, P>,
filter: &FilteredParams,
block_num_hash: BlockNumHash,
receipts: &[Receipt],
receipts: &[P::Receipt],
removed: bool,
block_timestamp: u64,
) -> Result<(), ProviderError> {
) -> Result<(), ProviderError>
where
P: BlockReader<Transaction: SignedTransaction>,
{
// Tracks the index of a log in the entire block.
let mut log_index: u64 = 0;
@ -81,13 +85,13 @@ pub fn append_matching_block_logs<P: BlockReader<Transaction: SignedTransaction>
// The transaction hash of the current receipt.
let mut transaction_hash = None;
for log in &receipt.logs {
for log in receipt.logs() {
if log_matches_filter(block_num_hash, log, filter) {
// if this is the first match in the receipt's logs, look up the transaction hash
if transaction_hash.is_none() {
transaction_hash = match &provider_or_block {
ProviderOrBlock::Block(block) => {
block.body.transactions.get(receipt_idx).map(|t| t.hash())
block.body.transactions().get(receipt_idx).map(|t| t.trie_hash())
}
ProviderOrBlock::Provider(provider) => {
let first_tx_num = match loaded_first_tx_num {

View File

@ -9,7 +9,10 @@ use alloy_network::Ethereum;
use alloy_primitives::U256;
use derive_more::Deref;
use reth_primitives::NodePrimitives;
use reth_provider::{BlockReaderIdExt, CanonStateSubscriptions, ChainSpecProvider};
use reth_provider::{
BlockReader, BlockReaderIdExt, CanonStateSubscriptions, ChainSpecProvider, ProviderBlock,
ProviderReceipt,
};
use reth_rpc_eth_api::{
helpers::{EthSigner, SpawnBlocking},
node::RpcNodeCoreExt,
@ -37,7 +40,7 @@ use crate::eth::EthTxBuilder;
/// This way [`EthApi`] is not limited to [`jsonrpsee`] and can be used standalone or in other
/// network handlers (for example ipc).
#[derive(Deref)]
pub struct EthApi<Provider, Pool, Network, EvmConfig> {
pub struct EthApi<Provider: BlockReader, Pool, Network, EvmConfig> {
/// All nested fields bundled together.
#[deref]
pub(super) inner: Arc<EthApiInner<Provider, Pool, Network, EvmConfig>>,
@ -45,7 +48,10 @@ pub struct EthApi<Provider, Pool, Network, EvmConfig> {
pub tx_resp_builder: EthTxBuilder,
}
impl<Provider, Pool, Network, EvmConfig> Clone for EthApi<Provider, Pool, Network, EvmConfig> {
impl<Provider, Pool, Network, EvmConfig> Clone for EthApi<Provider, Pool, Network, EvmConfig>
where
Provider: BlockReader,
{
fn clone(&self) -> Self {
Self { inner: self.inner.clone(), tx_resp_builder: EthTxBuilder }
}
@ -61,7 +67,7 @@ where
provider: Provider,
pool: Pool,
network: Network,
eth_cache: EthStateCache,
eth_cache: EthStateCache<Provider::Block, Provider::Receipt>,
gas_oracle: GasPriceOracle<Provider>,
gas_cap: impl Into<GasCap>,
max_simulate_blocks: u64,
@ -106,8 +112,8 @@ where
Tasks: TaskSpawner + Clone + 'static,
Events: CanonStateSubscriptions<
Primitives: NodePrimitives<
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
Block = ProviderBlock<Provider>,
Receipt = ProviderReceipt<Provider>,
>,
>,
{
@ -137,6 +143,7 @@ where
impl<Provider, Pool, Network, EvmConfig> EthApiTypes for EthApi<Provider, Pool, Network, EvmConfig>
where
Self: Send + Sync,
Provider: BlockReader,
{
type Error = EthApiError;
type NetworkTypes = Ethereum;
@ -149,7 +156,7 @@ where
impl<Provider, Pool, Network, EvmConfig> RpcNodeCore for EthApi<Provider, Pool, Network, EvmConfig>
where
Provider: Send + Sync + Clone + Unpin,
Provider: BlockReader + Send + Sync + Clone + Unpin,
Pool: Send + Sync + Clone + Unpin,
Network: Send + Sync + Clone,
EvmConfig: Send + Sync + Clone + Unpin,
@ -184,16 +191,21 @@ where
impl<Provider, Pool, Network, EvmConfig> RpcNodeCoreExt
for EthApi<Provider, Pool, Network, EvmConfig>
where
Self: RpcNodeCore,
Provider: BlockReader + Send + Sync + Clone + Unpin,
Pool: Send + Sync + Clone + Unpin,
Network: Send + Sync + Clone,
EvmConfig: Send + Sync + Clone + Unpin,
{
#[inline]
fn cache(&self) -> &EthStateCache {
fn cache(&self) -> &EthStateCache<ProviderBlock<Provider>, ProviderReceipt<Provider>> {
self.inner.cache()
}
}
impl<Provider, Pool, Network, EvmConfig> std::fmt::Debug
for EthApi<Provider, Pool, Network, EvmConfig>
where
Provider: BlockReader,
{
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
f.debug_struct("EthApi").finish_non_exhaustive()
@ -204,6 +216,7 @@ impl<Provider, Pool, Network, EvmConfig> SpawnBlocking
for EthApi<Provider, Pool, Network, EvmConfig>
where
Self: Clone + Send + Sync + 'static,
Provider: BlockReader,
{
#[inline]
fn io_task_spawner(&self) -> impl TaskSpawner {
@ -223,7 +236,7 @@ where
/// Container type `EthApi`
#[allow(missing_debug_implementations)]
pub struct EthApiInner<Provider, Pool, Network, EvmConfig> {
pub struct EthApiInner<Provider: BlockReader, Pool, Network, EvmConfig> {
/// The transaction pool.
pool: Pool,
/// The provider that can interact with the chain.
@ -233,7 +246,7 @@ pub struct EthApiInner<Provider, Pool, Network, EvmConfig> {
/// All configured Signers
signers: parking_lot::RwLock<Vec<Box<dyn EthSigner>>>,
/// The async cache frontend for eth related data
eth_cache: EthStateCache,
eth_cache: EthStateCache<Provider::Block, Provider::Receipt>,
/// The async gas oracle frontend for gas price suggestions
gas_oracle: GasPriceOracle<Provider>,
/// Maximum gas limit for `eth_call` and call tracing RPC methods.
@ -269,7 +282,7 @@ where
provider: Provider,
pool: Pool,
network: Network,
eth_cache: EthStateCache,
eth_cache: EthStateCache<Provider::Block, Provider::Receipt>,
gas_oracle: GasPriceOracle<Provider>,
gas_cap: impl Into<GasCap>,
max_simulate_blocks: u64,
@ -312,7 +325,10 @@ where
}
}
impl<Provider, Pool, Network, EvmConfig> EthApiInner<Provider, Pool, Network, EvmConfig> {
impl<Provider, Pool, Network, EvmConfig> EthApiInner<Provider, Pool, Network, EvmConfig>
where
Provider: BlockReader,
{
/// Returns a handle to data on disk.
#[inline]
pub const fn provider(&self) -> &Provider {
@ -321,7 +337,7 @@ impl<Provider, Pool, Network, EvmConfig> EthApiInner<Provider, Pool, Network, Ev
/// Returns a handle to data in memory.
#[inline]
pub const fn cache(&self) -> &EthStateCache {
pub const fn cache(&self) -> &EthStateCache<Provider::Block, Provider::Receipt> {
&self.eth_cache
}
@ -455,8 +471,7 @@ mod tests {
) -> EthApi<P, TestPool, NoopNetwork, EthEvmConfig> {
let evm_config = EthEvmConfig::new(provider.chain_spec());
let cache = EthStateCache::spawn(provider.clone(), Default::default());
let fee_history_cache =
FeeHistoryCache::new(cache.clone(), FeeHistoryCacheConfig::default());
let fee_history_cache = FeeHistoryCache::new(FeeHistoryCacheConfig::default());
let gas_cap = provider.chain_spec().max_gas_limit();
EthApi::new(

View File

@ -9,8 +9,8 @@ use alloy_rpc_types_eth::{
use async_trait::async_trait;
use jsonrpsee::{core::RpcResult, server::IdProvider};
use reth_chainspec::ChainInfo;
use reth_primitives::{Receipt, SealedBlockWithSenders};
use reth_provider::{BlockIdReader, BlockReader, ProviderError};
use reth_primitives::SealedBlockWithSenders;
use reth_provider::{BlockIdReader, BlockReader, ProviderBlock, ProviderError, ProviderReceipt};
use reth_rpc_eth_api::{
EthApiTypes, EthFilterApiServer, FullEthApiTypes, RpcTransaction, TransactionCompat,
};
@ -40,7 +40,7 @@ use tracing::{error, trace};
const MAX_HEADERS_RANGE: u64 = 1_000; // with ~530bytes per header this is ~500kb
/// `Eth` filter RPC implementation.
pub struct EthFilter<Provider, Pool, Eth: EthApiTypes> {
pub struct EthFilter<Provider: BlockReader, Pool, Eth: EthApiTypes> {
/// All nested fields bundled together
inner: Arc<EthFilterInner<Provider, Pool, RpcTransaction<Eth::NetworkTypes>>>,
/// Assembles response data w.r.t. network.
@ -50,6 +50,7 @@ pub struct EthFilter<Provider, Pool, Eth: EthApiTypes> {
impl<Provider, Pool, Eth> Clone for EthFilter<Provider, Pool, Eth>
where
Eth: EthApiTypes,
Provider: BlockReader,
{
fn clone(&self) -> Self {
Self { inner: self.inner.clone(), tx_resp_builder: self.tx_resp_builder.clone() }
@ -58,7 +59,7 @@ where
impl<Provider, Pool, Eth> EthFilter<Provider, Pool, Eth>
where
Provider: Send + Sync + 'static,
Provider: BlockReader + Send + Sync + 'static,
Pool: Send + Sync + 'static,
Eth: EthApiTypes + 'static,
{
@ -73,7 +74,7 @@ where
pub fn new(
provider: Provider,
pool: Pool,
eth_cache: EthStateCache,
eth_cache: EthStateCache<Provider::Block, Provider::Receipt>,
config: EthFilterConfig,
task_spawner: Box<dyn TaskSpawner>,
tx_resp_builder: Eth::TransactionCompat,
@ -334,6 +335,7 @@ where
impl<Provider, Pool, Eth> std::fmt::Debug for EthFilter<Provider, Pool, Eth>
where
Eth: EthApiTypes,
Provider: BlockReader,
{
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
f.debug_struct("EthFilter").finish_non_exhaustive()
@ -342,7 +344,7 @@ where
/// Container type `EthFilter`
#[derive(Debug)]
struct EthFilterInner<Provider, Pool, Tx> {
struct EthFilterInner<Provider: BlockReader, Pool, Tx> {
/// The transaction pool.
pool: Pool,
/// The provider that can interact with the chain.
@ -356,7 +358,7 @@ struct EthFilterInner<Provider, Pool, Tx> {
/// Maximum number of logs that can be returned in a response
max_logs_per_response: usize,
/// The async cache frontend for eth related data
eth_cache: EthStateCache,
eth_cache: EthStateCache<Provider::Block, Provider::Receipt>,
/// maximum number of headers to read at once for range filter
max_headers_range: u64,
/// The type that can spawn tasks.
@ -536,8 +538,13 @@ where
&self,
block_num_hash: &BlockNumHash,
best_number: u64,
) -> Result<Option<(Arc<Vec<Receipt>>, Option<Arc<SealedBlockWithSenders>>)>, EthFilterError>
{
) -> Result<
Option<(
Arc<Vec<ProviderReceipt<Provider>>>,
Option<Arc<SealedBlockWithSenders<ProviderBlock<Provider>>>>,
)>,
EthFilterError,
> {
// The last 4 blocks are most likely cached, so we can just fetch them
let cached_range = best_number.saturating_sub(4)..=best_number;
let receipts_block = if cached_range.contains(&block_num_hash.number) {

View File

@ -2,10 +2,10 @@
use alloy_rpc_types_eth::{BlockId, TransactionReceipt};
use reth_primitives::TransactionMeta;
use reth_provider::{BlockReaderIdExt, HeaderProvider};
use reth_provider::{BlockReader, HeaderProvider};
use reth_rpc_eth_api::{
helpers::{EthBlocks, LoadBlock, LoadPendingBlock, LoadReceipt, SpawnBlocking},
RpcReceipt,
RpcNodeCoreExt, RpcReceipt,
};
use reth_rpc_eth_types::{EthApiError, EthReceiptBuilder};
@ -18,6 +18,7 @@ where
NetworkTypes: alloy_network::Network<ReceiptResponse = TransactionReceipt>,
Provider: HeaderProvider,
>,
Provider: BlockReader,
{
async fn block_receipts(
&self,
@ -62,7 +63,7 @@ where
impl<Provider, Pool, Network, EvmConfig> LoadBlock for EthApi<Provider, Pool, Network, EvmConfig>
where
Self: LoadPendingBlock + SpawnBlocking,
Provider: BlockReaderIdExt,
Self: LoadPendingBlock + SpawnBlocking + RpcNodeCoreExt,
Provider: BlockReader,
{
}

View File

@ -3,12 +3,15 @@
use crate::EthApi;
use alloy_consensus::Header;
use reth_evm::ConfigureEvm;
use reth_provider::BlockReader;
use reth_rpc_eth_api::helpers::{
estimate::EstimateCall, Call, EthCall, LoadPendingBlock, LoadState, SpawnBlocking,
};
impl<Provider, Pool, Network, EvmConfig> EthCall for EthApi<Provider, Pool, Network, EvmConfig> where
Self: EstimateCall + LoadPendingBlock
impl<Provider, Pool, Network, EvmConfig> EthCall for EthApi<Provider, Pool, Network, EvmConfig>
where
Self: EstimateCall + LoadPendingBlock,
Provider: BlockReader,
{
}
@ -16,6 +19,7 @@ impl<Provider, Pool, Network, EvmConfig> Call for EthApi<Provider, Pool, Network
where
Self: LoadState<Evm: ConfigureEvm<Header = Header>> + SpawnBlocking,
EvmConfig: ConfigureEvm<Header = Header>,
Provider: BlockReader,
{
#[inline]
fn call_gas_limit(&self) -> u64 {
@ -28,7 +32,9 @@ where
}
}
impl<Provider, Pool, Network, EvmConfig> EstimateCall for EthApi<Provider, Pool, Network, EvmConfig> where
Self: Call
impl<Provider, Pool, Network, EvmConfig> EstimateCall for EthApi<Provider, Pool, Network, EvmConfig>
where
Self: Call,
Provider: BlockReader,
{
}

View File

@ -1,14 +1,18 @@
//! Contains RPC handler implementations for fee history.
use reth_chainspec::{EthChainSpec, EthereumHardforks};
use reth_provider::{BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, StateProviderFactory};
use reth_provider::{
BlockReader, BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, StateProviderFactory,
};
use reth_rpc_eth_api::helpers::{EthFees, LoadBlock, LoadFee};
use reth_rpc_eth_types::{FeeHistoryCache, GasPriceOracle};
use crate::EthApi;
impl<Provider, Pool, Network, EvmConfig> EthFees for EthApi<Provider, Pool, Network, EvmConfig> where
Self: LoadFee
impl<Provider, Pool, Network, EvmConfig> EthFees for EthApi<Provider, Pool, Network, EvmConfig>
where
Self: LoadFee,
Provider: BlockReader,
{
}

View File

@ -4,7 +4,8 @@ use alloy_consensus::Header;
use reth_chainspec::{EthChainSpec, EthereumHardforks};
use reth_evm::ConfigureEvm;
use reth_provider::{
BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, ProviderTx, StateProviderFactory,
BlockReader, BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, ProviderTx,
StateProviderFactory,
};
use reth_rpc_eth_api::{
helpers::{LoadPendingBlock, SpawnBlocking},
@ -33,6 +34,7 @@ where
>,
Evm: ConfigureEvm<Header = Header, Transaction = ProviderTx<Self::Provider>>,
>,
Provider: BlockReader,
{
#[inline]
fn pending_block(&self) -> &tokio::sync::Mutex<Option<PendingBlock>> {

View File

@ -1,7 +1,7 @@
//! Builds an RPC receipt response w.r.t. data layout of network.
use reth_primitives::{Receipt, TransactionMeta, TransactionSigned};
use reth_provider::{ReceiptProvider, TransactionsProvider};
use reth_provider::{BlockReader, ReceiptProvider, TransactionsProvider};
use reth_rpc_eth_api::{helpers::LoadReceipt, FromEthApiError, RpcNodeCoreExt, RpcReceipt};
use reth_rpc_eth_types::{EthApiError, EthReceiptBuilder};
@ -13,6 +13,7 @@ where
Provider: TransactionsProvider<Transaction = TransactionSigned>
+ ReceiptProvider<Receipt = reth_primitives::Receipt>,
>,
Provider: BlockReader,
{
async fn build_transaction_receipt(
&self,

View File

@ -11,11 +11,14 @@ use alloy_rpc_types_eth::TransactionRequest;
use alloy_signer::SignerSync;
use alloy_signer_local::PrivateKeySigner;
use reth_primitives::TransactionSigned;
use reth_provider::BlockReader;
use reth_rpc_eth_api::helpers::{signer::Result, AddDevSigners, EthSigner};
use reth_rpc_eth_types::SignError;
impl<Provider, Pool, Network, EvmConfig> AddDevSigners
for EthApi<Provider, Pool, Network, EvmConfig>
where
Provider: BlockReader,
{
fn with_dev_accounts(&self) {
*self.inner.signers().write() = DevSigner::random_signers(20)

View File

@ -1,7 +1,7 @@
use alloy_primitives::U256;
use reth_chainspec::EthereumHardforks;
use reth_network_api::NetworkInfo;
use reth_provider::{BlockNumReader, ChainSpecProvider, StageCheckpointReader};
use reth_provider::{BlockNumReader, BlockReader, ChainSpecProvider, StageCheckpointReader};
use reth_rpc_eth_api::{helpers::EthApiSpec, RpcNodeCore};
use crate::EthApi;
@ -14,6 +14,7 @@ where
+ StageCheckpointReader,
Network: NetworkInfo,
>,
Provider: BlockReader,
{
fn starting_block(&self) -> U256 {
self.inner.starting_block()

View File

@ -1,12 +1,12 @@
//! Contains RPC handler implementations specific to state.
use reth_chainspec::EthereumHardforks;
use reth_provider::{ChainSpecProvider, StateProviderFactory};
use reth_provider::{BlockReader, ChainSpecProvider, StateProviderFactory};
use reth_transaction_pool::TransactionPool;
use reth_rpc_eth_api::{
helpers::{EthState, LoadState, SpawnBlocking},
RpcNodeCore,
RpcNodeCoreExt,
};
use crate::EthApi;
@ -14,17 +14,22 @@ use crate::EthApi;
impl<Provider, Pool, Network, EvmConfig> EthState for EthApi<Provider, Pool, Network, EvmConfig>
where
Self: LoadState + SpawnBlocking,
Provider: BlockReader,
{
fn max_proof_window(&self) -> u64 {
self.inner.eth_proof_window()
}
}
impl<Provider, Pool, Network, EvmConfig> LoadState for EthApi<Provider, Pool, Network, EvmConfig> where
Self: RpcNodeCore<
Provider: StateProviderFactory + ChainSpecProvider<ChainSpec: EthereumHardforks>,
impl<Provider, Pool, Network, EvmConfig> LoadState for EthApi<Provider, Pool, Network, EvmConfig>
where
Self: RpcNodeCoreExt<
Provider: BlockReader
+ StateProviderFactory
+ ChainSpecProvider<ChainSpec: EthereumHardforks>,
Pool: TransactionPool,
>
>,
Provider: BlockReader,
{
}
@ -58,12 +63,12 @@ mod tests {
pool,
NoopNetwork::default(),
cache.clone(),
GasPriceOracle::new(NoopProvider::default(), Default::default(), cache.clone()),
GasPriceOracle::new(NoopProvider::default(), Default::default(), cache),
ETHEREUM_BLOCK_GAS_LIMIT,
DEFAULT_MAX_SIMULATE_BLOCKS,
DEFAULT_ETH_PROOF_WINDOW,
BlockingTaskPool::build().expect("failed to build tracing pool"),
FeeHistoryCache::new(cache, FeeHistoryCacheConfig::default()),
FeeHistoryCache::new(FeeHistoryCacheConfig::default()),
evm_config,
DEFAULT_PROOF_PERMITS,
)
@ -84,12 +89,12 @@ mod tests {
pool,
(),
cache.clone(),
GasPriceOracle::new(mock_provider, Default::default(), cache.clone()),
GasPriceOracle::new(mock_provider, Default::default(), cache),
ETHEREUM_BLOCK_GAS_LIMIT,
DEFAULT_MAX_SIMULATE_BLOCKS,
DEFAULT_ETH_PROOF_WINDOW + 1,
BlockingTaskPool::build().expect("failed to build tracing pool"),
FeeHistoryCache::new(cache, FeeHistoryCacheConfig::default()),
FeeHistoryCache::new(FeeHistoryCacheConfig::default()),
evm_config,
DEFAULT_PROOF_PERMITS,
)

View File

@ -7,7 +7,9 @@ use reth_rpc_eth_api::helpers::{LoadState, Trace};
use crate::EthApi;
impl<Provider, Pool, Network, EvmConfig> Trace for EthApi<Provider, Pool, Network, EvmConfig> where
Self: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Header>>
impl<Provider, Pool, Network, EvmConfig> Trace for EthApi<Provider, Pool, Network, EvmConfig>
where
Self: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Header>>,
Provider: BlockReader,
{
}

View File

@ -1,9 +1,9 @@
//! Contains RPC handler implementations specific to transactions
use reth_provider::{BlockReaderIdExt, TransactionsProvider};
use reth_provider::{BlockReader, BlockReaderIdExt, TransactionsProvider};
use reth_rpc_eth_api::{
helpers::{EthSigner, EthTransactions, LoadTransaction, SpawnBlocking},
FullEthApiTypes, RpcNodeCore,
FullEthApiTypes, RpcNodeCoreExt,
};
use reth_transaction_pool::TransactionPool;
@ -13,6 +13,7 @@ impl<Provider, Pool, Network, EvmConfig> EthTransactions
for EthApi<Provider, Pool, Network, EvmConfig>
where
Self: LoadTransaction<Provider: BlockReaderIdExt>,
Provider: BlockReader,
{
#[inline]
fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn EthSigner>>> {
@ -25,7 +26,8 @@ impl<Provider, Pool, Network, EvmConfig> LoadTransaction
where
Self: SpawnBlocking
+ FullEthApiTypes
+ RpcNodeCore<Provider: TransactionsProvider, Pool: TransactionPool>,
+ RpcNodeCoreExt<Provider: TransactionsProvider, Pool: TransactionPool>,
Provider: BlockReader,
{
}
@ -58,8 +60,7 @@ mod tests {
let evm_config = EthEvmConfig::new(noop_provider.chain_spec());
let cache = EthStateCache::spawn(noop_provider, Default::default());
let fee_history_cache =
FeeHistoryCache::new(cache.clone(), FeeHistoryCacheConfig::default());
let fee_history_cache = FeeHistoryCache::new(FeeHistoryCacheConfig::default());
let eth_api = EthApi::new(
noop_provider,
pool.clone(),

View File

@ -40,6 +40,9 @@ impl BlockSource {
}
}
/// A helper type alias to access [`BlockReader::Block`].
pub type ProviderBlock<P> = <P as BlockReader>::Block;
/// Api trait for fetching `Block` related data.
///
/// If not requested otherwise, implementers of this trait should prioritize fetching blocks from

View File

@ -1,14 +1,18 @@
use crate::BlockIdReader;
use alloy_eips::{BlockHashOrNumber, BlockId, BlockNumberOrTag};
use alloy_primitives::{TxHash, TxNumber};
use reth_primitives_traits::Receipt;
use reth_storage_errors::provider::ProviderResult;
use std::ops::RangeBounds;
/// A helper type alias to access [`ReceiptProvider::Receipt`].
pub type ProviderReceipt<P> = <P as ReceiptProvider>::Receipt;
/// Client trait for fetching receipt data.
#[auto_impl::auto_impl(&, Arc)]
pub trait ReceiptProvider: Send + Sync {
/// The receipt type.
type Receipt: Send + Sync;
type Receipt: Receipt;
/// Get receipt by transaction number
///

View File

@ -1,4 +1,4 @@
use crate::{BlockNumReader, BlockReader, ReceiptProvider};
use crate::{BlockNumReader, BlockReader};
use alloy_eips::BlockHashOrNumber;
use alloy_primitives::{Address, BlockNumber, TxHash, TxNumber};
use reth_primitives::TransactionMeta;
@ -84,9 +84,6 @@ pub trait TransactionsProvider: BlockNumReader + Send + Sync {
/// A helper type alias to access [`TransactionsProvider::Transaction`].
pub type ProviderTx<P> = <P as TransactionsProvider>::Transaction;
/// A helper type alias to access [`ReceiptProvider::Receipt`].
pub type ProviderReceipt<P> = <P as ReceiptProvider>::Receipt;
/// Client trait for fetching additional transactions related data.
#[auto_impl::auto_impl(&, Arc)]
pub trait TransactionsProviderExt: BlockReader + Send + Sync {