mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: add Block AT to BlockReader (#12837)
This commit is contained in:
@ -19,6 +19,7 @@ reth-consensus.workspace = true
|
||||
reth-network-api.workspace = true
|
||||
reth-node-core.workspace = true
|
||||
reth-provider.workspace = true
|
||||
reth-primitives.workspace = true
|
||||
reth-rpc.workspace = true
|
||||
reth-rpc-api.workspace = true
|
||||
reth-rpc-eth-api.workspace = true
|
||||
|
||||
@ -27,7 +27,12 @@ pub struct EthHandlers<Provider, Pool, Network, Events, EthApi: EthApiTypes> {
|
||||
|
||||
impl<Provider, Pool, Network, Events, EthApi> EthHandlers<Provider, Pool, Network, Events, EthApi>
|
||||
where
|
||||
Provider: StateProviderFactory + BlockReader + EvmEnvProvider + Clone + Unpin + 'static,
|
||||
Provider: StateProviderFactory
|
||||
+ BlockReader<Block = reth_primitives::Block>
|
||||
+ EvmEnvProvider
|
||||
+ Clone
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
Pool: Send + Sync + Clone + 'static,
|
||||
Network: Clone + 'static,
|
||||
Events: CanonStateSubscriptions + Clone + 'static,
|
||||
|
||||
@ -37,8 +37,9 @@
|
||||
//! block_executor: BlockExecutor,
|
||||
//! consensus: Consensus,
|
||||
//! ) where
|
||||
//! Provider:
|
||||
//! FullRpcProvider<Transaction = TransactionSigned> + AccountReader + ChangeSetReader,
|
||||
//! Provider: FullRpcProvider<Transaction = TransactionSigned, Block = reth_primitives::Block>
|
||||
//! + AccountReader
|
||||
//! + ChangeSetReader,
|
||||
//! Pool: TransactionPool + Unpin + 'static,
|
||||
//! Network: NetworkInfo + Peers + Clone + 'static,
|
||||
//! Events: CanonStateSubscriptions + Clone + 'static,
|
||||
@ -112,8 +113,9 @@
|
||||
//! block_executor: BlockExecutor,
|
||||
//! consensus: Consensus,
|
||||
//! ) where
|
||||
//! Provider:
|
||||
//! FullRpcProvider<Transaction = TransactionSigned> + AccountReader + ChangeSetReader,
|
||||
//! Provider: FullRpcProvider<Transaction = TransactionSigned, Block = reth_primitives::Block>
|
||||
//! + AccountReader
|
||||
//! + ChangeSetReader,
|
||||
//! Pool: TransactionPool + Unpin + 'static,
|
||||
//! Network: NetworkInfo + Peers + Clone + 'static,
|
||||
//! Events: CanonStateSubscriptions + Clone + 'static,
|
||||
@ -258,7 +260,7 @@ pub async fn launch<Provider, Pool, Network, Tasks, Events, EvmConfig, EthApi, B
|
||||
consensus: Arc<dyn Consensus>,
|
||||
) -> Result<RpcServerHandle, RpcError>
|
||||
where
|
||||
Provider: FullRpcProvider + AccountReader + ChangeSetReader,
|
||||
Provider: FullRpcProvider<Block = reth_primitives::Block> + AccountReader + ChangeSetReader,
|
||||
Pool: TransactionPool + 'static,
|
||||
Network: NetworkInfo + Peers + Clone + 'static,
|
||||
Tasks: TaskSpawner + Clone + 'static,
|
||||
@ -641,6 +643,7 @@ where
|
||||
EngineT: EngineTypes,
|
||||
EngineApi: EngineApiServer<EngineT>,
|
||||
EthApi: FullEthApiServer,
|
||||
Provider: BlockReader<Block = <EthApi::Provider as BlockReader>::Block>,
|
||||
{
|
||||
let Self {
|
||||
provider,
|
||||
@ -716,6 +719,7 @@ where
|
||||
) -> RpcRegistryInner<Provider, Pool, Network, Tasks, Events, EthApi, BlockExecutor, Consensus>
|
||||
where
|
||||
EthApi: EthApiTypes + 'static,
|
||||
Provider: BlockReader<Block = reth_primitives::Block>,
|
||||
{
|
||||
let Self {
|
||||
provider,
|
||||
@ -750,6 +754,7 @@ where
|
||||
) -> TransportRpcModules<()>
|
||||
where
|
||||
EthApi: FullEthApiServer,
|
||||
Provider: BlockReader<Block = <EthApi::Provider as BlockReader>::Block>,
|
||||
{
|
||||
let mut modules = TransportRpcModules::default();
|
||||
|
||||
@ -907,7 +912,12 @@ pub struct RpcRegistryInner<
|
||||
impl<Provider, Pool, Network, Tasks, Events, EthApi, BlockExecutor, Consensus>
|
||||
RpcRegistryInner<Provider, Pool, Network, Tasks, Events, EthApi, BlockExecutor, Consensus>
|
||||
where
|
||||
Provider: StateProviderFactory + BlockReader + EvmEnvProvider + Clone + Unpin + 'static,
|
||||
Provider: StateProviderFactory
|
||||
+ BlockReader<Block = reth_primitives::Block>
|
||||
+ EvmEnvProvider
|
||||
+ Clone
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
Pool: Send + Sync + Clone + 'static,
|
||||
Network: Clone + 'static,
|
||||
Events: CanonStateSubscriptions + Clone + 'static,
|
||||
@ -1112,6 +1122,7 @@ where
|
||||
pub fn register_debug(&mut self) -> &mut Self
|
||||
where
|
||||
EthApi: EthApiSpec + EthTransactions + TraceExt,
|
||||
Provider: BlockReader<Block = <EthApi::Provider as BlockReader>::Block>,
|
||||
{
|
||||
let debug_api = self.debug_api();
|
||||
self.modules.insert(RethRpcModule::Debug, debug_api.into_rpc().into());
|
||||
@ -1126,6 +1137,7 @@ where
|
||||
pub fn register_trace(&mut self) -> &mut Self
|
||||
where
|
||||
EthApi: TraceExt,
|
||||
Provider: BlockReader<Block = <EthApi::Provider as BlockReader>::Block>,
|
||||
{
|
||||
let trace_api = self.trace_api();
|
||||
self.modules.insert(RethRpcModule::Trace, trace_api.into_rpc().into());
|
||||
@ -1264,7 +1276,9 @@ where
|
||||
impl<Provider, Pool, Network, Tasks, Events, EthApi, BlockExecutor, Consensus>
|
||||
RpcRegistryInner<Provider, Pool, Network, Tasks, Events, EthApi, BlockExecutor, Consensus>
|
||||
where
|
||||
Provider: FullRpcProvider + AccountReader + ChangeSetReader,
|
||||
Provider: FullRpcProvider<Block = <EthApi::Provider as BlockReader>::Block>
|
||||
+ AccountReader
|
||||
+ ChangeSetReader,
|
||||
Pool: TransactionPool + 'static,
|
||||
Network: NetworkInfo + Peers + Clone + 'static,
|
||||
Tasks: TaskSpawner + Clone + 'static,
|
||||
|
||||
@ -75,7 +75,11 @@ struct EngineApiInner<Provider, EngineT: EngineTypes, Pool, Validator, ChainSpec
|
||||
impl<Provider, EngineT, Pool, Validator, ChainSpec>
|
||||
EngineApi<Provider, EngineT, Pool, Validator, ChainSpec>
|
||||
where
|
||||
Provider: HeaderProvider + BlockReader + StateProviderFactory + EvmEnvProvider + 'static,
|
||||
Provider: HeaderProvider
|
||||
+ BlockReader<Block = reth_primitives::Block>
|
||||
+ StateProviderFactory
|
||||
+ EvmEnvProvider
|
||||
+ 'static,
|
||||
EngineT: EngineTypes,
|
||||
Pool: TransactionPool + 'static,
|
||||
Validator: EngineValidator<EngineT>,
|
||||
@ -487,7 +491,7 @@ where
|
||||
f: F,
|
||||
) -> EngineApiResult<Vec<Option<R>>>
|
||||
where
|
||||
F: Fn(Block) -> R + Send + 'static,
|
||||
F: Fn(Provider::Block) -> R + Send + 'static,
|
||||
R: Send + 'static,
|
||||
{
|
||||
let (tx, rx) = oneshot::channel();
|
||||
@ -735,7 +739,11 @@ where
|
||||
impl<Provider, EngineT, Pool, Validator, ChainSpec> EngineApiServer<EngineT>
|
||||
for EngineApi<Provider, EngineT, Pool, Validator, ChainSpec>
|
||||
where
|
||||
Provider: HeaderProvider + BlockReader + StateProviderFactory + EvmEnvProvider + 'static,
|
||||
Provider: HeaderProvider
|
||||
+ BlockReader<Block = reth_primitives::Block>
|
||||
+ StateProviderFactory
|
||||
+ EvmEnvProvider
|
||||
+ 'static,
|
||||
EngineT: EngineTypes,
|
||||
Pool: TransactionPool + 'static,
|
||||
Validator: EngineValidator<EngineT>,
|
||||
|
||||
@ -2,21 +2,32 @@
|
||||
|
||||
use std::sync::Arc;
|
||||
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_eips::BlockId;
|
||||
use alloy_rpc_types_eth::{Block, Header, Index};
|
||||
use futures::Future;
|
||||
use reth_primitives::{Receipt, SealedBlock, SealedBlockWithSenders};
|
||||
use reth_node_api::BlockBody;
|
||||
use reth_primitives::{Receipt, SealedBlockFor, SealedBlockWithSenders};
|
||||
use reth_provider::{BlockIdReader, BlockReader, BlockReaderIdExt, HeaderProvider};
|
||||
use reth_rpc_types_compat::block::from_block;
|
||||
|
||||
use crate::{node::RpcNodeCoreExt, FromEthApiError, FullEthApiTypes, RpcBlock, RpcReceipt};
|
||||
use crate::{
|
||||
node::RpcNodeCoreExt, EthApiTypes, FromEthApiError, FullEthApiTypes, RpcBlock, RpcNodeCore,
|
||||
RpcReceipt,
|
||||
};
|
||||
|
||||
use super::{LoadPendingBlock, LoadReceipt, SpawnBlocking};
|
||||
|
||||
/// Result type of the fetched block receipts.
|
||||
pub type BlockReceiptsResult<N, E> = Result<Option<Vec<RpcReceipt<N>>>, E>;
|
||||
/// Result type of the fetched block and its receipts.
|
||||
pub type BlockAndReceiptsResult<E> = Result<Option<(SealedBlock, Arc<Vec<Receipt>>)>, E>;
|
||||
pub type BlockAndReceiptsResult<Eth> = Result<
|
||||
Option<(
|
||||
SealedBlockFor<<<Eth as RpcNodeCore>::Provider as BlockReader>::Block>,
|
||||
Arc<Vec<Receipt>>,
|
||||
)>,
|
||||
<Eth as EthApiTypes>::Error,
|
||||
>;
|
||||
|
||||
/// Block related functions for the [`EthApiServer`](crate::EthApiServer) trait in the
|
||||
/// `eth_` namespace.
|
||||
@ -49,7 +60,7 @@ pub trait EthBlocks: LoadBlock {
|
||||
let block_hash = block.hash();
|
||||
let mut total_difficulty = self
|
||||
.provider()
|
||||
.header_td_by_number(block.number)
|
||||
.header_td_by_number(block.number())
|
||||
.map_err(Self::Error::from_eth_err)?;
|
||||
if total_difficulty.is_none() {
|
||||
// if we failed to find td after we successfully loaded the block, try again using
|
||||
@ -83,7 +94,7 @@ pub trait EthBlocks: LoadBlock {
|
||||
.provider()
|
||||
.pending_block()
|
||||
.map_err(Self::Error::from_eth_err)?
|
||||
.map(|block| block.body.transactions.len()))
|
||||
.map(|block| block.body.transactions().len()))
|
||||
}
|
||||
|
||||
let block_hash = match self
|
||||
@ -120,7 +131,7 @@ pub trait EthBlocks: LoadBlock {
|
||||
fn load_block_and_receipts(
|
||||
&self,
|
||||
block_id: BlockId,
|
||||
) -> impl Future<Output = BlockAndReceiptsResult<Self::Error>> + Send
|
||||
) -> impl Future<Output = BlockAndReceiptsResult<Self>> + Send
|
||||
where
|
||||
Self: LoadReceipt,
|
||||
{
|
||||
@ -198,10 +209,16 @@ pub trait EthBlocks: LoadBlock {
|
||||
/// Behaviour shared by several `eth_` RPC methods, not exclusive to `eth_` blocks RPC methods.
|
||||
pub trait LoadBlock: LoadPendingBlock + SpawnBlocking + RpcNodeCoreExt {
|
||||
/// Returns the block object for the given block id.
|
||||
#[expect(clippy::type_complexity)]
|
||||
fn block_with_senders(
|
||||
&self,
|
||||
block_id: BlockId,
|
||||
) -> impl Future<Output = Result<Option<Arc<SealedBlockWithSenders>>, Self::Error>> + Send {
|
||||
) -> impl Future<
|
||||
Output = Result<
|
||||
Option<Arc<SealedBlockWithSenders<<Self::Provider as BlockReader>::Block>>>,
|
||||
Self::Error,
|
||||
>,
|
||||
> + Send {
|
||||
async move {
|
||||
if block_id.is_pending() {
|
||||
// Pending block can be fetched directly without need for caching
|
||||
|
||||
@ -18,6 +18,7 @@ use alloy_rpc_types_eth::{
|
||||
use futures::Future;
|
||||
use reth_chainspec::EthChainSpec;
|
||||
use reth_evm::{ConfigureEvm, ConfigureEvmEnv};
|
||||
use reth_node_api::BlockBody;
|
||||
use reth_primitives::TransactionSigned;
|
||||
use reth_provider::{BlockIdReader, ChainSpecProvider, HeaderProvider};
|
||||
use reth_revm::{
|
||||
@ -278,14 +279,15 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock {
|
||||
// we're essentially replaying the transactions in the block here, hence we need the
|
||||
// state that points to the beginning of the block, which is the state at
|
||||
// the parent block
|
||||
let mut at = block.parent_hash;
|
||||
let mut at = block.parent_hash();
|
||||
let mut replay_block_txs = true;
|
||||
|
||||
let num_txs = transaction_index.index().unwrap_or(block.body.transactions.len());
|
||||
let num_txs =
|
||||
transaction_index.index().unwrap_or_else(|| block.body.transactions().len());
|
||||
// but if all transactions are to be replayed, we can use the state at the block itself,
|
||||
// however only if we're not targeting the pending block, because for pending we can't
|
||||
// rely on the block's state being available
|
||||
if !is_block_target_pending && num_txs == block.body.transactions.len() {
|
||||
if !is_block_target_pending && num_txs == block.body.transactions().len() {
|
||||
at = block.hash();
|
||||
replay_block_txs = false;
|
||||
}
|
||||
|
||||
@ -1,5 +1,6 @@
|
||||
//! Loads fee history from database. Helper trait for `eth_` fee and transaction RPC methods.
|
||||
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_primitives::U256;
|
||||
use alloy_rpc_types_eth::{BlockNumberOrTag, FeeHistory};
|
||||
use futures::Future;
|
||||
@ -287,7 +288,7 @@ pub trait LoadFee: LoadBlock {
|
||||
.block_with_senders(BlockNumberOrTag::Pending.into())
|
||||
.await?
|
||||
.ok_or(EthApiError::HeaderNotFound(BlockNumberOrTag::Pending.into()))?
|
||||
.base_fee_per_gas
|
||||
.base_fee_per_gas()
|
||||
.ok_or(EthApiError::InvalidTransaction(
|
||||
RpcInvalidTransactionError::TxTypeNotSupported,
|
||||
))?;
|
||||
@ -324,7 +325,7 @@ pub trait LoadFee: LoadBlock {
|
||||
let suggested_tip = self.suggested_priority_fee();
|
||||
async move {
|
||||
let (header, suggested_tip) = futures::try_join!(header, suggested_tip)?;
|
||||
let base_fee = header.and_then(|h| h.base_fee_per_gas).unwrap_or_default();
|
||||
let base_fee = header.and_then(|h| h.base_fee_per_gas()).unwrap_or_default();
|
||||
Ok(suggested_tip + U256::from(base_fee))
|
||||
}
|
||||
}
|
||||
|
||||
@ -45,7 +45,7 @@ use tracing::debug;
|
||||
pub trait LoadPendingBlock:
|
||||
EthApiTypes
|
||||
+ RpcNodeCore<
|
||||
Provider: BlockReaderIdExt
|
||||
Provider: BlockReaderIdExt<Block = reth_primitives::Block>
|
||||
+ EvmEnvProvider
|
||||
+ ChainSpecProvider<ChainSpec: EthChainSpec + EthereumHardforks>
|
||||
+ StateProviderFactory,
|
||||
@ -114,9 +114,15 @@ pub trait LoadPendingBlock:
|
||||
}
|
||||
|
||||
/// Returns the locally built pending block
|
||||
#[expect(clippy::type_complexity)]
|
||||
fn local_pending_block(
|
||||
&self,
|
||||
) -> impl Future<Output = Result<Option<(SealedBlockWithSenders, Vec<Receipt>)>, Self::Error>> + Send
|
||||
) -> impl Future<
|
||||
Output = Result<
|
||||
Option<(SealedBlockWithSenders<<Self::Provider as BlockReader>::Block>, Vec<Receipt>)>,
|
||||
Self::Error,
|
||||
>,
|
||||
> + Send
|
||||
where
|
||||
Self: SpawnBlocking,
|
||||
{
|
||||
|
||||
@ -10,6 +10,7 @@ use futures::Future;
|
||||
use reth_chainspec::ChainSpecProvider;
|
||||
use reth_evm::{system_calls::SystemCaller, ConfigureEvm, ConfigureEvmEnv};
|
||||
use reth_primitives::SealedBlockWithSenders;
|
||||
use reth_provider::BlockReader;
|
||||
use reth_revm::database::StateProviderDatabase;
|
||||
use reth_rpc_eth_types::{
|
||||
cache::db::{StateCacheDb, StateCacheDbRefMutWrapper, StateProviderTraitObjWrapper},
|
||||
@ -24,7 +25,7 @@ use revm_primitives::{
|
||||
use super::{Call, LoadBlock, LoadPendingBlock, LoadState, LoadTransaction};
|
||||
|
||||
/// Executes CPU heavy tasks.
|
||||
pub trait Trace: LoadState<Evm: ConfigureEvm<Header = Header>> {
|
||||
pub trait Trace: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Header>> {
|
||||
/// Executes the [`EnvWithHandlerCfg`] against the given [Database] without committing state
|
||||
/// changes.
|
||||
fn inspect<DB, I>(
|
||||
@ -230,7 +231,7 @@ pub trait Trace: LoadState<Evm: ConfigureEvm<Header = Header>> {
|
||||
fn trace_block_until<F, R>(
|
||||
&self,
|
||||
block_id: BlockId,
|
||||
block: Option<Arc<SealedBlockWithSenders>>,
|
||||
block: Option<Arc<SealedBlockWithSenders<<Self::Provider as BlockReader>::Block>>>,
|
||||
highest_index: Option<u64>,
|
||||
config: TracingInspectorConfig,
|
||||
f: F,
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
//! Database access for `eth_` transaction RPC methods. Loads transaction and receipt data w.r.t.
|
||||
//! network.
|
||||
|
||||
use alloy_consensus::Transaction;
|
||||
use alloy_consensus::{BlockHeader, Transaction};
|
||||
use alloy_dyn_abi::TypedData;
|
||||
use alloy_eips::{eip2718::Encodable2718, BlockId};
|
||||
use alloy_network::TransactionBuilder;
|
||||
@ -199,8 +199,8 @@ pub trait EthTransactions: LoadTransaction<Provider: BlockReaderIdExt> {
|
||||
async move {
|
||||
if let Some(block) = self.block_with_senders(block_id).await? {
|
||||
let block_hash = block.hash();
|
||||
let block_number = block.number;
|
||||
let base_fee_per_gas = block.base_fee_per_gas;
|
||||
let block_number = block.number();
|
||||
let base_fee_per_gas = block.base_fee_per_gas();
|
||||
if let Some((signer, tx)) = block.transactions_with_sender().nth(index) {
|
||||
let tx_info = TransactionInfo {
|
||||
hash: Some(tx.hash()),
|
||||
@ -275,8 +275,8 @@ pub trait EthTransactions: LoadTransaction<Provider: BlockReaderIdExt> {
|
||||
.await?
|
||||
.and_then(|block| {
|
||||
let block_hash = block.hash();
|
||||
let block_number = block.number;
|
||||
let base_fee_per_gas = block.base_fee_per_gas;
|
||||
let block_number = block.number();
|
||||
let base_fee_per_gas = block.base_fee_per_gas();
|
||||
|
||||
block
|
||||
.transactions_with_sender()
|
||||
|
||||
@ -29,6 +29,7 @@ reth-transaction-pool.workspace = true
|
||||
reth-trie.workspace = true
|
||||
|
||||
# ethereum
|
||||
alloy-eips.workspace = true
|
||||
alloy-primitives.workspace = true
|
||||
alloy-consensus.workspace = true
|
||||
alloy-sol-types.workspace = true
|
||||
@ -36,7 +37,6 @@ alloy-rpc-types-eth.workspace = true
|
||||
revm.workspace = true
|
||||
revm-inspectors.workspace = true
|
||||
revm-primitives = { workspace = true, features = ["dev"] }
|
||||
alloy-eips.workspace = true
|
||||
|
||||
# rpc
|
||||
jsonrpsee-core.workspace = true
|
||||
|
||||
21
crates/rpc/rpc-eth-types/src/cache/mod.rs
vendored
21
crates/rpc/rpc-eth-types/src/cache/mod.rs
vendored
@ -105,7 +105,12 @@ impl EthStateCache {
|
||||
evm_config: EvmConfig,
|
||||
) -> Self
|
||||
where
|
||||
Provider: StateProviderFactory + BlockReader + EvmEnvProvider + Clone + Unpin + 'static,
|
||||
Provider: StateProviderFactory
|
||||
+ BlockReader<Block = reth_primitives::Block>
|
||||
+ EvmEnvProvider
|
||||
+ Clone
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
EvmConfig: ConfigureEvm<Header = Header>,
|
||||
{
|
||||
Self::spawn_with(provider, config, TokioTaskExecutor::default(), evm_config)
|
||||
@ -122,7 +127,12 @@ impl EthStateCache {
|
||||
evm_config: EvmConfig,
|
||||
) -> Self
|
||||
where
|
||||
Provider: StateProviderFactory + BlockReader + EvmEnvProvider + Clone + Unpin + 'static,
|
||||
Provider: StateProviderFactory
|
||||
+ BlockReader<Block = reth_primitives::Block>
|
||||
+ EvmEnvProvider
|
||||
+ Clone
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
Tasks: TaskSpawner + Clone + 'static,
|
||||
EvmConfig: ConfigureEvm<Header = Header>,
|
||||
{
|
||||
@ -337,7 +347,12 @@ where
|
||||
|
||||
impl<Provider, Tasks, EvmConfig> Future for EthStateCacheService<Provider, Tasks, EvmConfig>
|
||||
where
|
||||
Provider: StateProviderFactory + BlockReader + EvmEnvProvider + Clone + Unpin + 'static,
|
||||
Provider: StateProviderFactory
|
||||
+ BlockReader<Block = reth_primitives::Block>
|
||||
+ EvmEnvProvider
|
||||
+ Clone
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
Tasks: TaskSpawner + Clone + 'static,
|
||||
EvmConfig: ConfigureEvm<Header = Header>,
|
||||
{
|
||||
|
||||
@ -8,6 +8,7 @@ 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 std::sync::Arc;
|
||||
|
||||
@ -58,7 +59,7 @@ pub enum ProviderOrBlock<'a, P: BlockReader> {
|
||||
|
||||
/// 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>(
|
||||
pub fn append_matching_block_logs<P: BlockReader<Transaction: SignedTransaction>>(
|
||||
all_logs: &mut Vec<Log>,
|
||||
provider_or_block: ProviderOrBlock<'_, P>,
|
||||
filter: &FilteredParams,
|
||||
|
||||
@ -4,10 +4,12 @@
|
||||
|
||||
use std::time::Instant;
|
||||
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_eips::{BlockId, BlockNumberOrTag};
|
||||
use alloy_primitives::B256;
|
||||
use derive_more::Constructor;
|
||||
use reth_primitives::{Receipt, SealedBlockWithSenders, SealedHeader};
|
||||
use reth_primitives_traits::Block;
|
||||
use revm_primitives::{BlockEnv, CfgEnvWithHandlerCfg};
|
||||
|
||||
/// Configured [`BlockEnv`] and [`CfgEnvWithHandlerCfg`] for a pending block.
|
||||
@ -23,26 +25,26 @@ pub struct PendingBlockEnv {
|
||||
|
||||
/// The origin for a configured [`PendingBlockEnv`]
|
||||
#[derive(Clone, Debug)]
|
||||
pub enum PendingBlockEnvOrigin {
|
||||
pub enum PendingBlockEnvOrigin<B: Block = reth_primitives::Block> {
|
||||
/// The pending block as received from the CL.
|
||||
ActualPending(SealedBlockWithSenders),
|
||||
ActualPending(SealedBlockWithSenders<B>),
|
||||
/// The _modified_ header of the latest block.
|
||||
///
|
||||
/// This derives the pending state based on the latest header by modifying:
|
||||
/// - the timestamp
|
||||
/// - the block number
|
||||
/// - fees
|
||||
DerivedFromLatest(SealedHeader),
|
||||
DerivedFromLatest(SealedHeader<B::Header>),
|
||||
}
|
||||
|
||||
impl PendingBlockEnvOrigin {
|
||||
impl<B: Block> PendingBlockEnvOrigin<B> {
|
||||
/// Returns true if the origin is the actual pending block as received from the CL.
|
||||
pub const fn is_actual_pending(&self) -> bool {
|
||||
matches!(self, Self::ActualPending(_))
|
||||
}
|
||||
|
||||
/// Consumes the type and returns the actual pending block.
|
||||
pub fn into_actual_pending(self) -> Option<SealedBlockWithSenders> {
|
||||
pub fn into_actual_pending(self) -> Option<SealedBlockWithSenders<B>> {
|
||||
match self {
|
||||
Self::ActualPending(block) => Some(block),
|
||||
_ => None,
|
||||
@ -67,13 +69,13 @@ impl PendingBlockEnvOrigin {
|
||||
/// header.
|
||||
pub fn build_target_hash(&self) -> B256 {
|
||||
match self {
|
||||
Self::ActualPending(block) => block.parent_hash,
|
||||
Self::ActualPending(block) => block.header().parent_hash(),
|
||||
Self::DerivedFromLatest(header) => header.hash(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the header this pending block is based on.
|
||||
pub fn header(&self) -> &SealedHeader {
|
||||
pub fn header(&self) -> &SealedHeader<B::Header> {
|
||||
match self {
|
||||
Self::ActualPending(block) => &block.header,
|
||||
Self::DerivedFromLatest(header) => header,
|
||||
|
||||
@ -793,7 +793,7 @@ where
|
||||
#[async_trait]
|
||||
impl<Provider, Eth, BlockExecutor> DebugApiServer for DebugApi<Provider, Eth, BlockExecutor>
|
||||
where
|
||||
Provider: BlockReaderIdExt
|
||||
Provider: BlockReaderIdExt<Block: Encodable>
|
||||
+ HeaderProvider
|
||||
+ ChainSpecProvider<ChainSpec: EthereumHardforks>
|
||||
+ StateProviderFactory
|
||||
|
||||
@ -432,7 +432,7 @@ mod tests {
|
||||
use crate::EthApi;
|
||||
|
||||
fn build_test_eth_api<
|
||||
P: BlockReaderIdExt
|
||||
P: BlockReaderIdExt<Block = reth_primitives::Block>
|
||||
+ BlockReader
|
||||
+ ChainSpecProvider<ChainSpec = ChainSpec>
|
||||
+ EvmEnvProvider
|
||||
|
||||
@ -18,7 +18,7 @@ impl<Provider, Pool, Network, EvmConfig> LoadPendingBlock
|
||||
where
|
||||
Self: SpawnBlocking
|
||||
+ RpcNodeCore<
|
||||
Provider: BlockReaderIdExt
|
||||
Provider: BlockReaderIdExt<Block = reth_primitives::Block>
|
||||
+ EvmEnvProvider
|
||||
+ ChainSpecProvider<ChainSpec: EthChainSpec + EthereumHardforks>
|
||||
+ StateProviderFactory,
|
||||
|
||||
@ -2,11 +2,12 @@
|
||||
|
||||
use alloy_consensus::Header;
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_provider::BlockReader;
|
||||
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<Evm: ConfigureEvm<Header = Header>>
|
||||
Self: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Header>>
|
||||
{
|
||||
}
|
||||
|
||||
@ -74,7 +74,7 @@ impl<Provider, Eth> TraceApi<Provider, Eth> {
|
||||
|
||||
impl<Provider, Eth> TraceApi<Provider, Eth>
|
||||
where
|
||||
Provider: BlockReader
|
||||
Provider: BlockReader<Block = <Eth::Provider as BlockReader>::Block>
|
||||
+ StateProviderFactory
|
||||
+ EvmEnvProvider
|
||||
+ ChainSpecProvider<ChainSpec: EthereumHardforks>
|
||||
@ -565,7 +565,7 @@ where
|
||||
#[async_trait]
|
||||
impl<Provider, Eth> TraceApiServer for TraceApi<Provider, Eth>
|
||||
where
|
||||
Provider: BlockReader
|
||||
Provider: BlockReader<Block = <Eth::Provider as BlockReader>::Block>
|
||||
+ StateProviderFactory
|
||||
+ EvmEnvProvider
|
||||
+ ChainSpecProvider<ChainSpec: EthereumHardforks>
|
||||
|
||||
Reference in New Issue
Block a user