diff --git a/crates/rpc/rpc/Cargo.toml b/crates/rpc/rpc/Cargo.toml index dc62f8b9d93f..9cfa3653de94 100644 --- a/crates/rpc/rpc/Cargo.toml +++ b/crates/rpc/rpc/Cargo.toml @@ -75,7 +75,7 @@ tracing.workspace = true tracing-futures = "0.2" schnellru.workspace = true futures.workspace = true -derive_more.workspace = true +derive_more = { workspace = true, default-features = false, features = ["deref", "deref_mut", "constructor"] } dyn-clone.workspace = true auto_impl.workspace = true diff --git a/crates/rpc/rpc/src/eth/api/block.rs b/crates/rpc/rpc/src/eth/api/block.rs index e7aea4e370a8..0fded533d74f 100644 --- a/crates/rpc/rpc/src/eth/api/block.rs +++ b/crates/rpc/rpc/src/eth/api/block.rs @@ -1,10 +1,5 @@ //! Contains RPC handler implementations specific to blocks. -use crate::{ - eth::error::{EthApiError, EthResult}, - EthApi, -}; - use reth_evm::ConfigureEvm; use reth_network_api::NetworkInfo; use reth_primitives::BlockId; @@ -16,7 +11,13 @@ use reth_rpc_types::{Header, Index, RichBlock}; use reth_rpc_types_compat::block::{from_block, uncle_block_from_header}; use reth_transaction_pool::TransactionPool; -use crate::eth::api::EthBlocks; +use crate::{ + eth::{ + api::{EthBlocks, LoadPendingBlock}, + error::{EthApiError, EthResult}, + }, + EthApi, +}; impl EthBlocks for EthApi where @@ -94,7 +95,10 @@ where pub(crate) async fn block( &self, block_id: impl Into, - ) -> EthResult> { + ) -> EthResult> + where + Self: LoadPendingBlock, + { self.block_with_senders(block_id) .await .map(|maybe_block| maybe_block.map(|block| block.block)) @@ -104,7 +108,10 @@ where pub(crate) async fn block_with_senders( &self, block_id: impl Into, - ) -> EthResult> { + ) -> EthResult> + where + Self: LoadPendingBlock, + { let block_id = block_id.into(); if block_id.is_pending() { @@ -133,7 +140,10 @@ where &self, block_id: impl Into, full: bool, - ) -> EthResult> { + ) -> EthResult> + where + Self: LoadPendingBlock, + { let block = match self.block_with_senders(block_id).await? { Some(block) => block, None => return Ok(None), @@ -151,7 +161,10 @@ where pub(crate) async fn rpc_block_header( &self, block_id: impl Into, - ) -> EthResult> { + ) -> EthResult> + where + Self: LoadPendingBlock, + { let header = self.rpc_block(block_id, false).await?.map(|block| block.inner.header); Ok(header) } diff --git a/crates/rpc/rpc/src/eth/api/call.rs b/crates/rpc/rpc/src/eth/api/call.rs index 4d20487f233e..25b49c278024 100644 --- a/crates/rpc/rpc/src/eth/api/call.rs +++ b/crates/rpc/rpc/src/eth/api/call.rs @@ -1,17 +1,5 @@ //! Contains RPC handler implementations specific to endpoints that call/execute within evm. -use crate::{ - eth::{ - api::{EthTransactions, SpawnBlocking}, - error::{ensure_success, EthApiError, EthResult, RevertError, RpcInvalidTransactionError}, - revm_utils::{ - apply_state_overrides, build_call_evm_env, caller_gas_allowance, - cap_tx_gas_limit_with_caller_allowance, get_precompiles, prepare_call_env, - EvmOverrides, - }, - }, - EthApi, -}; use reth_evm::ConfigureEvm; use reth_network_api::NetworkInfo; use reth_primitives::{revm::env::tx_env_with_recovered, BlockId, Bytes, TxKind, U256}; @@ -34,7 +22,18 @@ use revm::{ use revm_inspectors::access_list::AccessListInspector; use tracing::trace; -use super::LoadState; +use crate::{ + eth::{ + api::{EthTransactions, LoadPendingBlock, LoadState, SpawnBlocking}, + error::{ensure_success, EthApiError, EthResult, RevertError, RpcInvalidTransactionError}, + revm_utils::{ + apply_state_overrides, build_call_evm_env, caller_gas_allowance, + cap_tx_gas_limit_with_caller_allowance, get_precompiles, prepare_call_env, + EvmOverrides, + }, + }, + EthApi, +}; // Gas per transaction not creating a contract. const MIN_TRANSACTION_GAS: u64 = 21_000u64; @@ -88,7 +87,10 @@ where bundle: Bundle, state_context: Option, mut state_override: Option, - ) -> EthResult> { + ) -> EthResult> + where + Self: LoadPendingBlock, + { let Bundle { transactions, block_override } = bundle; if transactions.is_empty() { return Err(EthApiError::InvalidParams(String::from("transactions are empty."))) diff --git a/crates/rpc/rpc/src/eth/api/fees.rs b/crates/rpc/rpc/src/eth/api/fees.rs index 68789674ce5d..e67d5caaee1d 100644 --- a/crates/rpc/rpc/src/eth/api/fees.rs +++ b/crates/rpc/rpc/src/eth/api/fees.rs @@ -1,12 +1,5 @@ //! Contains RPC handler implementations for fee history. -use crate::{ - eth::{ - api::fee_history::{calculate_reward_percentiles_for_block, FeeHistoryEntry}, - error::{EthApiError, EthResult}, - }, - EthApi, -}; use reth_evm::ConfigureEvm; use reth_network_api::NetworkInfo; use reth_primitives::{BlockNumberOrTag, U256}; @@ -15,8 +8,20 @@ use reth_rpc_types::FeeHistory; use reth_transaction_pool::TransactionPool; use tracing::debug; +use crate::{ + eth::{ + api::{ + fee_history::{calculate_reward_percentiles_for_block, FeeHistoryEntry}, + LoadPendingBlock, + }, + error::{EthApiError, EthResult}, + }, + EthApi, +}; + impl EthApi where + Self: LoadPendingBlock, Pool: TransactionPool + Clone + 'static, Provider: BlockReaderIdExt + ChainSpecProvider + StateProviderFactory + EvmEnvProvider + 'static, diff --git a/crates/rpc/rpc/src/eth/api/mod.rs b/crates/rpc/rpc/src/eth/api/mod.rs index 76204fde33b1..807bb6355f19 100644 --- a/crates/rpc/rpc/src/eth/api/mod.rs +++ b/crates/rpc/rpc/src/eth/api/mod.rs @@ -1,43 +1,29 @@ //! The entire implementation of the namespace is quite large, hence it is divided across several //! files. +use std::{fmt::Debug, sync::Arc}; + use async_trait::async_trait; use reth_errors::{RethError, RethResult}; use reth_evm::ConfigureEvm; use reth_network_api::NetworkInfo; -use reth_primitives::{ - revm_primitives::{BlockEnv, CfgEnvWithHandlerCfg}, - Address, BlockNumberOrTag, ChainInfo, SealedBlockWithSenders, SealedHeader, U256, U64, -}; +use reth_primitives::{Address, BlockNumberOrTag, ChainInfo, U256, U64}; use reth_provider::{BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, StateProviderFactory}; use reth_rpc_types::{SyncInfo, SyncStatus}; use reth_tasks::{pool::BlockingTaskPool, TaskSpawner, TokioTaskExecutor}; use reth_transaction_pool::TransactionPool; -use revm_primitives::{CfgEnv, SpecId}; -use std::{ - fmt::Debug, - sync::Arc, - time::{Duration, Instant}, -}; use tokio::sync::Mutex; use crate::eth::{ - api::{ - fee_history::FeeHistoryCache, - pending_block::{PendingBlock, PendingBlockEnv, PendingBlockEnvOrigin}, - }, - cache::EthStateCache, - error::{EthApiError, EthResult}, - gas_oracle::GasPriceOracle, + api::fee_history::FeeHistoryCache, cache::EthStateCache, gas_oracle::GasPriceOracle, signer::EthSigner, }; pub mod block; mod call; pub(crate) mod fee_history; - mod fees; -mod pending_block; +pub mod pending_block; pub mod receipt; mod server; mod sign; @@ -45,10 +31,11 @@ mod state; pub mod traits; pub mod transactions; +pub use pending_block::PendingBlock; pub use receipt::ReceiptBuilder; pub use traits::{ - BuildReceipt, EthBlocks, EthState, EthTransactions, LoadState, RawTransactionForwarder, - SpawnBlocking, StateCacheDB, + BuildReceipt, EthBlocks, EthState, EthTransactions, LoadPendingBlock, LoadState, + RawTransactionForwarder, SpawnBlocking, StateCacheDB, }; pub use transactions::TransactionSource; @@ -211,120 +198,15 @@ where } } -impl EthApi -where - Provider: - BlockReaderIdExt + ChainSpecProvider + StateProviderFactory + EvmEnvProvider + 'static, - Pool: TransactionPool + 'static, - Network: NetworkInfo + 'static, - EvmConfig: ConfigureEvm, -{ - /// Configures the [`CfgEnvWithHandlerCfg`] and [`BlockEnv`] for the pending block - /// - /// If no pending block is available, this will derive it from the `latest` block - pub(crate) fn pending_block_env_and_cfg(&self) -> EthResult { - let origin: PendingBlockEnvOrigin = if let Some(pending) = - self.provider().pending_block_with_senders()? - { - PendingBlockEnvOrigin::ActualPending(pending) - } else { - // no pending block from the CL yet, so we use the latest block and modify the env - // values that we can - let latest = - self.provider().latest_header()?.ok_or_else(|| EthApiError::UnknownBlockNumber)?; - - let (mut latest_header, block_hash) = latest.split(); - // child block - latest_header.number += 1; - // assumed child block is in the next slot: 12s - latest_header.timestamp += 12; - // base fee of the child block - let chain_spec = self.provider().chain_spec(); - - latest_header.base_fee_per_gas = latest_header.next_block_base_fee( - chain_spec.base_fee_params_at_timestamp(latest_header.timestamp), - ); - - // update excess blob gas consumed above target - latest_header.excess_blob_gas = latest_header.next_block_excess_blob_gas(); - - // we're reusing the same block hash because we need this to lookup the block's state - let latest = SealedHeader::new(latest_header, block_hash); - - PendingBlockEnvOrigin::DerivedFromLatest(latest) - }; - - let mut cfg = CfgEnvWithHandlerCfg::new_with_spec_id(CfgEnv::default(), SpecId::LATEST); - - let mut block_env = BlockEnv::default(); - // Note: for the PENDING block we assume it is past the known merge block and thus this will - // not fail when looking up the total difficulty value for the blockenv. - self.provider().fill_env_with_header( - &mut cfg, - &mut block_env, - origin.header(), - self.inner.evm_config.clone(), - )?; - - Ok(PendingBlockEnv { cfg, block_env, origin }) - } - - /// Returns the locally built pending block - pub(crate) async fn local_pending_block(&self) -> EthResult> { - let pending = self.pending_block_env_and_cfg()?; - if pending.origin.is_actual_pending() { - return Ok(pending.origin.into_actual_pending()) - } - - let mut lock = self.inner.pending_block.lock().await; - - let now = Instant::now(); - - // check if the block is still good - if let Some(pending_block) = lock.as_ref() { - // this is guaranteed to be the `latest` header - if pending.block_env.number.to::() == pending_block.block.number && - pending.origin.header().hash() == pending_block.block.parent_hash && - now <= pending_block.expires_at - { - return Ok(Some(pending_block.block.clone())) - } - } - - // no pending block from the CL yet, so we need to build it ourselves via txpool - let pending_block = match self - .spawn_blocking_io(move |this| { - // we rebuild the block - pending.build_block(this.provider(), this.pool()) - }) - .await - { - Ok(block) => block, - Err(err) => { - tracing::debug!(target: "rpc", "Failed to build pending block: {:?}", err); - return Ok(None) - } - }; - - let now = Instant::now(); - *lock = Some(PendingBlock { - block: pending_block.clone(), - expires_at: now + Duration::from_secs(1), - }); - - Ok(Some(pending_block)) - } -} - -impl std::fmt::Debug - for EthApi +impl std::fmt::Debug + for EthApi { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("EthApi").finish_non_exhaustive() } } -impl Clone for EthApi { +impl Clone for EthApi { fn clone(&self) -> Self { Self { inner: Arc::clone(&self.inner) } } @@ -474,6 +356,12 @@ impl EthApiInner &Mutex> { + &self.pending_block + } + /// Returns a handle to the task spawner. #[inline] pub const fn task_spawner(&self) -> &dyn TaskSpawner { diff --git a/crates/rpc/rpc/src/eth/api/pending_block.rs b/crates/rpc/rpc/src/eth/api/pending_block.rs index 276796b137ea..eeec611a0d2c 100644 --- a/crates/rpc/rpc/src/eth/api/pending_block.rs +++ b/crates/rpc/rpc/src/eth/api/pending_block.rs @@ -1,7 +1,10 @@ //! Support for building a pending block via local txpool. -use crate::eth::error::{EthApiError, EthResult}; +use std::time::Instant; + +use derive_more::Constructor; use reth_errors::ProviderError; +use reth_evm::ConfigureEvm; use reth_primitives::{ constants::{eip4844::MAX_DATA_GAS_PER_BLOCK, BEACON_NONCE}, proofs, @@ -13,7 +16,10 @@ use reth_primitives::{ Block, BlockId, BlockNumberOrTag, ChainSpec, Header, IntoRecoveredTransaction, Receipt, Receipts, Requests, SealedBlockWithSenders, SealedHeader, B256, EMPTY_OMMER_ROOT_HASH, U256, }; -use reth_provider::{BundleStateWithReceipts, ChainSpecProvider, StateProviderFactory}; +use reth_provider::{ + BlockReaderIdExt, BundleStateWithReceipts, ChainSpecProvider, EvmEnvProvider, + StateProviderFactory, +}; use reth_revm::{ database::StateProviderDatabase, state_change::{ @@ -24,17 +30,56 @@ use reth_revm::{ use reth_transaction_pool::{BestTransactionsAttributes, TransactionPool}; use revm::{db::states::bundle_state::BundleRetention, Database, DatabaseCommit, State}; use revm_primitives::EnvWithHandlerCfg; -use std::time::Instant; +use tokio::sync::Mutex; + +use crate::{ + eth::{ + api::{LoadPendingBlock, SpawnBlocking}, + error::{EthApiError, EthResult}, + }, + EthApi, +}; + +impl LoadPendingBlock + for EthApi +where + Self: SpawnBlocking, + Provider: BlockReaderIdExt + EvmEnvProvider + ChainSpecProvider + StateProviderFactory, + Pool: TransactionPool, + EvmConfig: ConfigureEvm, +{ + #[inline] + fn provider( + &self, + ) -> &(impl BlockReaderIdExt + EvmEnvProvider + ChainSpecProvider + StateProviderFactory) { + self.inner.provider() + } + + #[inline] + fn pool(&self) -> &impl TransactionPool { + self.inner.pool() + } + + #[inline] + fn pending_block(&self) -> &Mutex> { + self.inner.pending_block() + } + + #[inline] + fn evm_config(&self) -> &impl ConfigureEvm { + self.inner.evm_config() + } +} /// Configured [`BlockEnv`] and [`CfgEnvWithHandlerCfg`] for a pending block -#[derive(Debug, Clone)] -pub(crate) struct PendingBlockEnv { +#[derive(Debug, Clone, Constructor)] +pub struct PendingBlockEnv { /// Configured [`CfgEnvWithHandlerCfg`] for the pending block. - pub(crate) cfg: CfgEnvWithHandlerCfg, + pub cfg: CfgEnvWithHandlerCfg, /// Configured [`BlockEnv`] for the pending block. - pub(crate) block_env: BlockEnv, + pub block_env: BlockEnv, /// Origin block for the config - pub(crate) origin: PendingBlockEnvOrigin, + pub origin: PendingBlockEnvOrigin, } impl PendingBlockEnv { @@ -44,7 +89,7 @@ impl PendingBlockEnv { /// /// After Cancun, if the origin is the actual pending block, the block includes the EIP-4788 pre /// block contract call using the parent beacon block root received from the CL. - pub(crate) fn build_block( + pub fn build_block( self, client: &Client, pool: &Pool, @@ -358,7 +403,7 @@ where /// The origin for a configured [`PendingBlockEnv`] #[derive(Clone, Debug)] -pub(crate) enum PendingBlockEnvOrigin { +pub enum PendingBlockEnvOrigin { /// The pending block as received from the CL. ActualPending(SealedBlockWithSenders), /// The _modified_ header of the latest block. @@ -372,12 +417,12 @@ pub(crate) enum PendingBlockEnvOrigin { impl PendingBlockEnvOrigin { /// Returns true if the origin is the actual pending block as received from the CL. - pub(crate) const fn is_actual_pending(&self) -> bool { + pub const fn is_actual_pending(&self) -> bool { matches!(self, Self::ActualPending(_)) } /// Consumes the type and returns the actual pending block. - pub(crate) fn into_actual_pending(self) -> Option { + pub fn into_actual_pending(self) -> Option { match self { Self::ActualPending(block) => Some(block), _ => None, @@ -388,7 +433,7 @@ impl PendingBlockEnvOrigin { /// /// If this is the actual pending block, the state is the "Pending" tag, otherwise we can safely /// identify the block by its hash (latest block). - pub(crate) fn state_block_id(&self) -> BlockId { + pub fn state_block_id(&self) -> BlockId { match self { Self::ActualPending(_) => BlockNumberOrTag::Pending.into(), Self::DerivedFromLatest(header) => BlockId::Hash(header.hash().into()), @@ -408,7 +453,7 @@ impl PendingBlockEnvOrigin { } /// Returns the header this pending block is based on. - pub(crate) fn header(&self) -> &SealedHeader { + pub fn header(&self) -> &SealedHeader { match self { Self::ActualPending(block) => &block.header, Self::DerivedFromLatest(header) => header, @@ -417,10 +462,10 @@ impl PendingBlockEnvOrigin { } /// In memory pending block for `pending` tag -#[derive(Debug)] -pub(crate) struct PendingBlock { +#[derive(Debug, Constructor)] +pub struct PendingBlock { /// The cached pending block - pub(crate) block: SealedBlockWithSenders, + pub block: SealedBlockWithSenders, /// Timestamp when the pending block is considered outdated - pub(crate) expires_at: Instant, + pub expires_at: Instant, } diff --git a/crates/rpc/rpc/src/eth/api/traits/mod.rs b/crates/rpc/rpc/src/eth/api/traits/mod.rs index caa4b4853340..b81a8940a000 100644 --- a/crates/rpc/rpc/src/eth/api/traits/mod.rs +++ b/crates/rpc/rpc/src/eth/api/traits/mod.rs @@ -2,12 +2,14 @@ pub mod block; pub mod blocking_task; +pub mod pending_block; pub mod receipt; pub mod state; pub mod transaction; pub use block::EthBlocks; pub use blocking_task::SpawnBlocking; +pub use pending_block::LoadPendingBlock; pub use receipt::BuildReceipt; pub use state::{EthState, LoadState}; pub use transaction::{EthTransactions, RawTransactionForwarder, StateCacheDB}; diff --git a/crates/rpc/rpc/src/eth/api/traits/pending_block.rs b/crates/rpc/rpc/src/eth/api/traits/pending_block.rs new file mode 100644 index 000000000000..6790731c8edc --- /dev/null +++ b/crates/rpc/rpc/src/eth/api/traits/pending_block.rs @@ -0,0 +1,144 @@ +//! Loads a pending block from database. Helper trait for `eth_` block and transaction RPC methods. + +use std::time::{Duration, Instant}; + +use futures::Future; +use reth_evm::ConfigureEvm; +use reth_primitives::{SealedBlockWithSenders, SealedHeader}; +use reth_provider::{ + BlockReader, BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, StateProviderFactory, +}; +use reth_transaction_pool::TransactionPool; +use revm_primitives::{BlockEnv, CfgEnv, CfgEnvWithHandlerCfg, SpecId}; +use tokio::sync::Mutex; +use tracing::debug; + +use crate::eth::{ + api::{ + pending_block::{PendingBlock, PendingBlockEnv, PendingBlockEnvOrigin}, + SpawnBlocking, + }, + error::{EthApiError, EthResult}, +}; + +/// Loads a pending block from database. +pub trait LoadPendingBlock: SpawnBlocking { + /// Returns a handle for reading data from disk. + /// + /// Data access in default (L1) trait method implementations. + fn provider( + &self, + ) -> &(impl BlockReaderIdExt + EvmEnvProvider + ChainSpecProvider + StateProviderFactory); + + /// Returns a handle for reading data from transaction pool. + /// + /// Data access in default (L1) trait method implementations. + fn pool(&self) -> &impl TransactionPool; + + /// Returns a handle to the pending block. + /// + /// Data access in default (L1) trait method implementations. + fn pending_block(&self) -> &Mutex>; + + /// Returns a handle for reading evm config. + /// + /// Data access in default (L1) trait method implementations. + fn evm_config(&self) -> &impl ConfigureEvm; + + /// Configures the [`CfgEnvWithHandlerCfg`] and [`BlockEnv`] for the pending block + /// + /// If no pending block is available, this will derive it from the `latest` block + fn pending_block_env_and_cfg(&self) -> EthResult { + let origin: PendingBlockEnvOrigin = if let Some(pending) = + self.provider().pending_block_with_senders()? + { + PendingBlockEnvOrigin::ActualPending(pending) + } else { + // no pending block from the CL yet, so we use the latest block and modify the env + // values that we can + let latest = + self.provider().latest_header()?.ok_or_else(|| EthApiError::UnknownBlockNumber)?; + + let (mut latest_header, block_hash) = latest.split(); + // child block + latest_header.number += 1; + // assumed child block is in the next slot: 12s + latest_header.timestamp += 12; + // base fee of the child block + let chain_spec = self.provider().chain_spec(); + + latest_header.base_fee_per_gas = latest_header.next_block_base_fee( + chain_spec.base_fee_params_at_timestamp(latest_header.timestamp), + ); + + // update excess blob gas consumed above target + latest_header.excess_blob_gas = latest_header.next_block_excess_blob_gas(); + + // we're reusing the same block hash because we need this to lookup the block's state + let latest = SealedHeader::new(latest_header, block_hash); + + PendingBlockEnvOrigin::DerivedFromLatest(latest) + }; + + let mut cfg = CfgEnvWithHandlerCfg::new_with_spec_id(CfgEnv::default(), SpecId::LATEST); + + let mut block_env = BlockEnv::default(); + // Note: for the PENDING block we assume it is past the known merge block and thus this will + // not fail when looking up the total difficulty value for the blockenv. + self.provider().fill_env_with_header( + &mut cfg, + &mut block_env, + origin.header(), + self.evm_config().clone(), + )?; + + Ok(PendingBlockEnv::new(cfg, block_env, origin)) + } + + /// Returns the locally built pending block + fn local_pending_block( + &self, + ) -> impl Future>> { + async move { + let pending = self.pending_block_env_and_cfg()?; + if pending.origin.is_actual_pending() { + return Ok(pending.origin.into_actual_pending()) + } + + let mut lock = self.pending_block().lock().await; + + let now = Instant::now(); + + // check if the block is still good + if let Some(pending_block) = lock.as_ref() { + // this is guaranteed to be the `latest` header + if pending.block_env.number.to::() == pending_block.block.number && + pending.origin.header().hash() == pending_block.block.parent_hash && + now <= pending_block.expires_at + { + return Ok(Some(pending_block.block.clone())) + } + } + + // no pending block from the CL yet, so we need to build it ourselves via txpool + let pending_block = match self + .spawn_blocking_io(move |this| { + // we rebuild the block + pending.build_block(this.provider(), this.pool()) + }) + .await + { + Ok(block) => block, + Err(err) => { + debug!(target: "rpc", "Failed to build pending block: {:?}", err); + return Ok(None) + } + }; + + let now = Instant::now(); + *lock = Some(PendingBlock::new(pending_block.clone(), now + Duration::from_secs(1))); + + Ok(Some(pending_block)) + } + } +} diff --git a/crates/rpc/rpc/src/eth/api/transactions.rs b/crates/rpc/rpc/src/eth/api/transactions.rs index abb2a60d8511..771bec08c820 100644 --- a/crates/rpc/rpc/src/eth/api/transactions.rs +++ b/crates/rpc/rpc/src/eth/api/transactions.rs @@ -37,8 +37,8 @@ use revm::{ use crate::{ eth::{ api::{ - pending_block::PendingBlockEnv, BuildReceipt, EthState, EthTransactions, LoadState, - RawTransactionForwarder, SpawnBlocking, StateCacheDB, + pending_block::PendingBlockEnv, BuildReceipt, EthState, EthTransactions, + LoadPendingBlock, LoadState, RawTransactionForwarder, SpawnBlocking, StateCacheDB, }, cache::EthStateCache, error::{EthApiError, EthResult, RpcInvalidTransactionError, SignError}, @@ -97,7 +97,7 @@ where async fn evm_env_at(&self, at: BlockId) -> EthResult<(CfgEnvWithHandlerCfg, BlockEnv, BlockId)> where - Self: LoadState, + Self: LoadState + LoadPendingBlock, { if at.is_pending() { let PendingBlockEnv { cfg, block_env, origin } = self.pending_block_env_and_cfg()?; diff --git a/crates/rpc/rpc/src/eth/bundle.rs b/crates/rpc/rpc/src/eth/bundle.rs index 494ba12629a3..6d07ba536ee2 100644 --- a/crates/rpc/rpc/src/eth/bundle.rs +++ b/crates/rpc/rpc/src/eth/bundle.rs @@ -1,11 +1,7 @@ //! `Eth` bundle implementation and helpers. -use crate::eth::{ - api::{EthTransactions, SpawnBlocking}, - error::{EthApiError, EthResult, RpcInvalidTransactionError}, - revm_utils::FillableTransaction, - utils::recover_raw_transaction, -}; +use std::sync::Arc; + use jsonrpsee::core::RpcResult; use reth_primitives::{ constants::eip4844::MAINNET_KZG_TRUSTED_SETUP, @@ -22,9 +18,13 @@ use revm::{ primitives::{ResultAndState, TxEnv}, }; use revm_primitives::{EnvWithHandlerCfg, MAX_BLOB_GAS_PER_BLOCK}; -use std::sync::Arc; -use super::api::LoadState; +use crate::eth::{ + api::{EthTransactions, LoadState, SpawnBlocking}, + error::{EthApiError, EthResult, RpcInvalidTransactionError}, + revm_utils::FillableTransaction, + utils::recover_raw_transaction, +}; /// `Eth` bundle implementation. pub struct EthBundle {