mirror of
https://github.com/paradigmxyz/reth.git
synced 2026-01-09 23:38:10 -05:00
chore: remove BlockExecutorProvider trait (#15989)
Co-authored-by: Arsenii Kulikov <klkvrr@gmail.com>
This commit is contained in:
@@ -29,7 +29,7 @@ fn main() {
|
||||
install_ress_subprotocol(
|
||||
ress_args,
|
||||
node.provider,
|
||||
node.block_executor,
|
||||
node.evm_config,
|
||||
node.network,
|
||||
node.task_executor,
|
||||
node.add_ons_handle.engine_events.new_listener(),
|
||||
|
||||
@@ -1,5 +1,5 @@
|
||||
use reth_ethereum_primitives::EthPrimitives;
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_network::{protocol::IntoRlpxSubProtocol, NetworkProtocols};
|
||||
use reth_network_api::FullNetwork;
|
||||
use reth_node_api::BeaconConsensusEngineEvent;
|
||||
@@ -16,14 +16,14 @@ use tracing::*;
|
||||
pub fn install_ress_subprotocol<P, E, N>(
|
||||
args: RessArgs,
|
||||
provider: BlockchainProvider<P>,
|
||||
block_executor: E,
|
||||
evm_config: E,
|
||||
network: N,
|
||||
task_executor: TaskExecutor,
|
||||
engine_events: EventStream<BeaconConsensusEngineEvent<EthPrimitives>>,
|
||||
) -> eyre::Result<()>
|
||||
where
|
||||
P: ProviderNodeTypes<Primitives = EthPrimitives>,
|
||||
E: BlockExecutorProvider<Primitives = EthPrimitives> + Clone,
|
||||
E: ConfigureEvm<Primitives = EthPrimitives> + Clone + 'static,
|
||||
N: FullNetwork + NetworkProtocols,
|
||||
{
|
||||
info!(target: "reth::cli", "Installing ress subprotocol");
|
||||
@@ -39,7 +39,7 @@ where
|
||||
let (tx, mut rx) = mpsc::unbounded_channel();
|
||||
let provider = RethRessProtocolProvider::new(
|
||||
provider,
|
||||
block_executor,
|
||||
evm_config,
|
||||
Box::new(task_executor.clone()),
|
||||
args.max_witness_window,
|
||||
args.witness_max_parallel,
|
||||
|
||||
@@ -9,11 +9,7 @@ use reth_consensus::{noop::NoopConsensus, ConsensusError, FullConsensus};
|
||||
use reth_db::{init_db, open_db_read_only, DatabaseEnv};
|
||||
use reth_db_common::init::init_genesis;
|
||||
use reth_downloaders::{bodies::noop::NoopBodiesDownloader, headers::noop::NoopHeaderDownloader};
|
||||
use reth_evm::{
|
||||
execute::{BasicBlockExecutorProvider, BlockExecutorProvider},
|
||||
noop::NoopEvmConfig,
|
||||
ConfigureEvm,
|
||||
};
|
||||
use reth_evm::{noop::NoopEvmConfig, ConfigureEvm};
|
||||
use reth_node_api::FullNodeTypesAdapter;
|
||||
use reth_node_builder::{
|
||||
Node, NodeComponents, NodeComponentsBuilder, NodeTypes, NodeTypesWithDBAdapter,
|
||||
@@ -165,7 +161,7 @@ impl<C: ChainSpecParser> EnvironmentArgs<C> {
|
||||
Arc::new(NoopConsensus::default()),
|
||||
NoopHeaderDownloader::default(),
|
||||
NoopBodiesDownloader::default(),
|
||||
BasicBlockExecutorProvider::new(NoopEvmConfig::<N::Evm>::default()),
|
||||
NoopEvmConfig::<N::Evm>::default(),
|
||||
config.stages.clone(),
|
||||
prune_modes.clone(),
|
||||
))
|
||||
@@ -229,26 +225,26 @@ where
|
||||
|
||||
/// Helper trait aggregating components required for the CLI.
|
||||
pub trait CliNodeComponents<N: CliNodeTypes> {
|
||||
/// Block executor.
|
||||
type Executor: BlockExecutorProvider<Primitives = N::Primitives>;
|
||||
/// Evm to use.
|
||||
type Evm: ConfigureEvm<Primitives = N::Primitives> + 'static;
|
||||
/// Consensus implementation.
|
||||
type Consensus: FullConsensus<N::Primitives, Error = ConsensusError> + Clone + 'static;
|
||||
|
||||
/// Returns the block executor.
|
||||
fn executor(&self) -> &Self::Executor;
|
||||
/// Returns the configured EVM.
|
||||
fn evm_config(&self) -> &Self::Evm;
|
||||
/// Returns the consensus implementation.
|
||||
fn consensus(&self) -> &Self::Consensus;
|
||||
}
|
||||
|
||||
impl<N: CliNodeTypes, E, C> CliNodeComponents<N> for (E, C)
|
||||
where
|
||||
E: BlockExecutorProvider<Primitives = N::Primitives>,
|
||||
E: ConfigureEvm<Primitives = N::Primitives> + 'static,
|
||||
C: FullConsensus<N::Primitives, Error = ConsensusError> + Clone + 'static,
|
||||
{
|
||||
type Executor = E;
|
||||
type Evm = E;
|
||||
type Consensus = C;
|
||||
|
||||
fn executor(&self) -> &Self::Executor {
|
||||
fn evm_config(&self) -> &Self::Evm {
|
||||
&self.0
|
||||
}
|
||||
|
||||
|
||||
@@ -13,7 +13,7 @@ use reth_downloaders::{
|
||||
file_client::{ChunkedFileReader, FileClient, DEFAULT_BYTE_LEN_CHUNK_CHAIN_FILE},
|
||||
headers::reverse_headers::ReverseHeadersDownloaderBuilder,
|
||||
};
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_network_p2p::{
|
||||
bodies::downloader::BodyDownloader,
|
||||
headers::downloader::{HeaderDownloader, SyncTarget},
|
||||
@@ -76,7 +76,7 @@ impl<C: ChainSpecParser<ChainSpec: EthChainSpec + EthereumHardforks>> ImportComm
|
||||
let Environment { provider_factory, config, .. } = self.env.init::<N>(AccessRights::RW)?;
|
||||
|
||||
let components = components(provider_factory.chain_spec());
|
||||
let executor = components.executor().clone();
|
||||
let executor = components.evm_config().clone();
|
||||
let consensus = Arc::new(components.consensus().clone());
|
||||
info!(target: "reth::cli", "Consensus engine initialized");
|
||||
|
||||
@@ -181,12 +181,12 @@ pub fn build_import_pipeline<N, C, E>(
|
||||
file_client: Arc<FileClient<BlockTy<N>>>,
|
||||
static_file_producer: StaticFileProducer<ProviderFactory<N>>,
|
||||
disable_exec: bool,
|
||||
executor: E,
|
||||
evm_config: E,
|
||||
) -> eyre::Result<(Pipeline<N>, impl Stream<Item = NodeEvent<N::Primitives>>)>
|
||||
where
|
||||
N: ProviderNodeTypes,
|
||||
C: FullConsensus<N::Primitives, Error = ConsensusError> + 'static,
|
||||
E: BlockExecutorProvider<Primitives = N::Primitives>,
|
||||
E: ConfigureEvm<Primitives = N::Primitives> + 'static,
|
||||
{
|
||||
if !file_client.has_canonical_blocks() {
|
||||
eyre::bail!("unable to import non canonical blocks");
|
||||
@@ -231,7 +231,7 @@ where
|
||||
consensus.clone(),
|
||||
header_downloader,
|
||||
body_downloader,
|
||||
executor,
|
||||
evm_config,
|
||||
config.stages.clone(),
|
||||
PruneModes::default(),
|
||||
)
|
||||
|
||||
@@ -5,7 +5,7 @@ use reth_db_api::{
|
||||
cursor::DbCursorRO, database::Database, table::TableImporter, tables, transaction::DbTx,
|
||||
};
|
||||
use reth_db_common::DbTool;
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_node_builder::NodeTypesWithDB;
|
||||
use reth_node_core::dirs::{ChainPath, DataDirPath};
|
||||
use reth_provider::{
|
||||
@@ -22,19 +22,19 @@ pub(crate) async fn dump_execution_stage<N, E, C>(
|
||||
to: u64,
|
||||
output_datadir: ChainPath<DataDirPath>,
|
||||
should_run: bool,
|
||||
executor: E,
|
||||
evm_config: E,
|
||||
consensus: C,
|
||||
) -> eyre::Result<()>
|
||||
where
|
||||
N: ProviderNodeTypes<DB = Arc<DatabaseEnv>>,
|
||||
E: BlockExecutorProvider<Primitives = N::Primitives>,
|
||||
E: ConfigureEvm<Primitives = N::Primitives> + 'static,
|
||||
C: FullConsensus<E::Primitives, Error = ConsensusError> + 'static,
|
||||
{
|
||||
let (output_db, tip_block_number) = setup(from, to, &output_datadir.db(), db_tool)?;
|
||||
|
||||
import_tables_with_range(&output_db, db_tool, from, to)?;
|
||||
|
||||
unwind_and_copy(db_tool, from, tip_block_number, &output_db, executor.clone())?;
|
||||
unwind_and_copy(db_tool, from, tip_block_number, &output_db, evm_config.clone())?;
|
||||
|
||||
if should_run {
|
||||
dry_run(
|
||||
@@ -45,7 +45,7 @@ where
|
||||
),
|
||||
to,
|
||||
from,
|
||||
executor,
|
||||
evm_config,
|
||||
consensus,
|
||||
)?;
|
||||
}
|
||||
@@ -139,11 +139,11 @@ fn unwind_and_copy<N: ProviderNodeTypes>(
|
||||
from: u64,
|
||||
tip_block_number: u64,
|
||||
output_db: &DatabaseEnv,
|
||||
executor: impl BlockExecutorProvider<Primitives = N::Primitives>,
|
||||
evm_config: impl ConfigureEvm<Primitives = N::Primitives>,
|
||||
) -> eyre::Result<()> {
|
||||
let provider = db_tool.provider_factory.database_provider_rw()?;
|
||||
|
||||
let mut exec_stage = ExecutionStage::new_with_executor(executor, NoopConsensus::arc());
|
||||
let mut exec_stage = ExecutionStage::new_with_executor(evm_config, NoopConsensus::arc());
|
||||
|
||||
exec_stage.unwind(
|
||||
&provider,
|
||||
@@ -169,17 +169,17 @@ fn dry_run<N, E, C>(
|
||||
output_provider_factory: ProviderFactory<N>,
|
||||
to: u64,
|
||||
from: u64,
|
||||
executor: E,
|
||||
evm_config: E,
|
||||
consensus: C,
|
||||
) -> eyre::Result<()>
|
||||
where
|
||||
N: ProviderNodeTypes,
|
||||
E: BlockExecutorProvider<Primitives = N::Primitives>,
|
||||
E: ConfigureEvm<Primitives = N::Primitives> + 'static,
|
||||
C: FullConsensus<E::Primitives, Error = ConsensusError> + 'static,
|
||||
{
|
||||
info!(target: "reth::cli", "Executing stage. [dry-run]");
|
||||
|
||||
let mut exec_stage = ExecutionStage::new_with_executor(executor, Arc::new(consensus));
|
||||
let mut exec_stage = ExecutionStage::new_with_executor(evm_config, Arc::new(consensus));
|
||||
|
||||
let input =
|
||||
reth_stages::ExecInput { target: Some(to), checkpoint: Some(StageCheckpoint::new(from)) };
|
||||
|
||||
@@ -8,7 +8,7 @@ use reth_consensus::{ConsensusError, FullConsensus};
|
||||
use reth_db::DatabaseEnv;
|
||||
use reth_db_api::{database::Database, table::TableImporter, tables};
|
||||
use reth_db_common::DbTool;
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_exex::ExExManagerHandle;
|
||||
use reth_node_core::dirs::{ChainPath, DataDirPath};
|
||||
use reth_provider::{
|
||||
@@ -30,7 +30,7 @@ pub(crate) async fn dump_merkle_stage<N>(
|
||||
to: BlockNumber,
|
||||
output_datadir: ChainPath<DataDirPath>,
|
||||
should_run: bool,
|
||||
executor: impl BlockExecutorProvider<Primitives = N::Primitives>,
|
||||
evm_config: impl ConfigureEvm<Primitives = N::Primitives>,
|
||||
consensus: impl FullConsensus<N::Primitives, Error = ConsensusError> + 'static,
|
||||
) -> Result<()>
|
||||
where
|
||||
@@ -54,7 +54,7 @@ where
|
||||
)
|
||||
})??;
|
||||
|
||||
unwind_and_copy(db_tool, (from, to), tip_block_number, &output_db, executor, consensus)?;
|
||||
unwind_and_copy(db_tool, (from, to), tip_block_number, &output_db, evm_config, consensus)?;
|
||||
|
||||
if should_run {
|
||||
dry_run(
|
||||
@@ -77,7 +77,7 @@ fn unwind_and_copy<N: ProviderNodeTypes>(
|
||||
range: (u64, u64),
|
||||
tip_block_number: u64,
|
||||
output_db: &DatabaseEnv,
|
||||
executor: impl BlockExecutorProvider<Primitives = N::Primitives>,
|
||||
evm_config: impl ConfigureEvm<Primitives = N::Primitives>,
|
||||
consensus: impl FullConsensus<N::Primitives, Error = ConsensusError> + 'static,
|
||||
) -> eyre::Result<()> {
|
||||
let (from, to) = range;
|
||||
@@ -100,7 +100,7 @@ fn unwind_and_copy<N: ProviderNodeTypes>(
|
||||
|
||||
// Bring Plainstate to TO (hashing stage execution requires it)
|
||||
let mut exec_stage = ExecutionStage::new(
|
||||
executor, // Not necessary for unwinding.
|
||||
evm_config, // Not necessary for unwinding.
|
||||
Arc::new(consensus),
|
||||
ExecutionStageThresholds {
|
||||
max_blocks: Some(u64::MAX),
|
||||
|
||||
@@ -98,17 +98,17 @@ impl<C: ChainSpecParser<ChainSpec: EthChainSpec + EthereumHardforks>> Command<C>
|
||||
let Environment { provider_factory, .. } = self.env.init::<N>(AccessRights::RO)?;
|
||||
let tool = DbTool::new(provider_factory)?;
|
||||
let components = components(tool.chain());
|
||||
let executor = components.executor().clone();
|
||||
let evm_config = components.evm_config().clone();
|
||||
let consensus = components.consensus().clone();
|
||||
|
||||
match &self.command {
|
||||
Stages::Execution(cmd) => {
|
||||
handle_stage!(dump_execution_stage, &tool, cmd, executor, consensus)
|
||||
handle_stage!(dump_execution_stage, &tool, cmd, evm_config, consensus)
|
||||
}
|
||||
Stages::StorageHashing(cmd) => handle_stage!(dump_hashing_storage_stage, &tool, cmd),
|
||||
Stages::AccountHashing(cmd) => handle_stage!(dump_hashing_account_stage, &tool, cmd),
|
||||
Stages::Merkle(cmd) => {
|
||||
handle_stage!(dump_merkle_stage, &tool, cmd, executor, consensus)
|
||||
handle_stage!(dump_merkle_stage, &tool, cmd, evm_config, consensus)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -263,7 +263,7 @@ impl<C: ChainSpecParser<ChainSpec: EthChainSpec + Hardforks + EthereumHardforks>
|
||||
),
|
||||
StageEnum::Execution => (
|
||||
Box::new(ExecutionStage::new(
|
||||
components.executor().clone(),
|
||||
components.evm_config().clone(),
|
||||
Arc::new(components.consensus().clone()),
|
||||
ExecutionStageThresholds {
|
||||
max_blocks: Some(batch_size),
|
||||
|
||||
@@ -13,7 +13,7 @@ use reth_config::Config;
|
||||
use reth_consensus::noop::NoopConsensus;
|
||||
use reth_db::DatabaseEnv;
|
||||
use reth_downloaders::{bodies::noop::NoopBodiesDownloader, headers::noop::NoopHeaderDownloader};
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_exex::ExExManagerHandle;
|
||||
use reth_provider::{
|
||||
providers::ProviderNodeTypes, BlockExecutionWriter, BlockNumReader, ChainStateBlockReader,
|
||||
@@ -85,7 +85,7 @@ impl<C: ChainSpecParser<ChainSpec: EthChainSpec + EthereumHardforks>> Command<C>
|
||||
|
||||
// This will build an offline-only pipeline if the `offline` flag is enabled
|
||||
let mut pipeline =
|
||||
self.build_pipeline(config, provider_factory, components.executor().clone())?;
|
||||
self.build_pipeline(config, provider_factory, components.evm_config().clone())?;
|
||||
|
||||
// Move all applicable data from database to static files.
|
||||
pipeline.move_to_static_files()?;
|
||||
@@ -117,7 +117,7 @@ impl<C: ChainSpecParser<ChainSpec: EthChainSpec + EthereumHardforks>> Command<C>
|
||||
self,
|
||||
config: Config,
|
||||
provider_factory: ProviderFactory<N>,
|
||||
executor: impl BlockExecutorProvider<Primitives = N::Primitives>,
|
||||
evm_config: impl ConfigureEvm<Primitives = N::Primitives> + 'static,
|
||||
) -> Result<Pipeline<N>, eyre::Error> {
|
||||
let stage_conf = &config.stages;
|
||||
let prune_modes = config.prune.clone().map(|prune| prune.segments).unwrap_or_default();
|
||||
@@ -127,7 +127,7 @@ impl<C: ChainSpecParser<ChainSpec: EthChainSpec + EthereumHardforks>> Command<C>
|
||||
let builder = if self.offline {
|
||||
Pipeline::<N>::builder().add_stages(
|
||||
OfflineStages::new(
|
||||
executor,
|
||||
evm_config,
|
||||
NoopConsensus::arc(),
|
||||
config.stages,
|
||||
prune_modes.clone(),
|
||||
@@ -143,12 +143,12 @@ impl<C: ChainSpecParser<ChainSpec: EthChainSpec + EthereumHardforks>> Command<C>
|
||||
Arc::new(NoopConsensus::default()),
|
||||
NoopHeaderDownloader::default(),
|
||||
NoopBodiesDownloader::default(),
|
||||
executor.clone(),
|
||||
evm_config.clone(),
|
||||
stage_conf.clone(),
|
||||
prune_modes.clone(),
|
||||
)
|
||||
.set(ExecutionStage::new(
|
||||
executor,
|
||||
evm_config,
|
||||
Arc::new(NoopConsensus::default()),
|
||||
ExecutionStageThresholds {
|
||||
max_blocks: None,
|
||||
|
||||
@@ -4,7 +4,7 @@ use alloy_rpc_types_debug::ExecutionWitness;
|
||||
use pretty_assertions::Comparison;
|
||||
use reth_chainspec::{EthChainSpec, EthereumHardforks};
|
||||
use reth_engine_primitives::InvalidBlockHook;
|
||||
use reth_evm::execute::{BlockExecutorProvider, Executor};
|
||||
use reth_evm::{execute::Executor, ConfigureEvm};
|
||||
use reth_primitives_traits::{NodePrimitives, RecoveredBlock, SealedHeader};
|
||||
use reth_provider::{BlockExecutionOutput, ChainSpecProvider, StateProviderFactory};
|
||||
use reth_revm::{database::StateProviderDatabase, db::BundleState, state::AccountInfo};
|
||||
@@ -116,7 +116,7 @@ pub struct InvalidBlockWitnessHook<P, E> {
|
||||
/// The provider to read the historical state and do the EVM execution.
|
||||
provider: P,
|
||||
/// The EVM configuration to use for the execution.
|
||||
executor: E,
|
||||
evm_config: E,
|
||||
/// The directory to write the witness to. Additionally, diff files will be written to this
|
||||
/// directory in case of failed sanity checks.
|
||||
output_directory: PathBuf,
|
||||
@@ -128,11 +128,11 @@ impl<P, E> InvalidBlockWitnessHook<P, E> {
|
||||
/// Creates a new witness hook.
|
||||
pub const fn new(
|
||||
provider: P,
|
||||
executor: E,
|
||||
evm_config: E,
|
||||
output_directory: PathBuf,
|
||||
healthy_node_client: Option<jsonrpsee::http_client::HttpClient>,
|
||||
) -> Self {
|
||||
Self { provider, executor, output_directory, healthy_node_client }
|
||||
Self { provider, evm_config, output_directory, healthy_node_client }
|
||||
}
|
||||
}
|
||||
|
||||
@@ -143,7 +143,7 @@ where
|
||||
+ Send
|
||||
+ Sync
|
||||
+ 'static,
|
||||
E: BlockExecutorProvider<Primitives = N>,
|
||||
E: ConfigureEvm<Primitives = N> + 'static,
|
||||
N: NodePrimitives,
|
||||
{
|
||||
fn on_invalid_block(
|
||||
@@ -158,7 +158,7 @@ where
|
||||
{
|
||||
// TODO(alexey): unify with `DebugApi::debug_execution_witness`
|
||||
|
||||
let mut executor = self.executor.executor(StateProviderDatabase::new(
|
||||
let mut executor = self.evm_config.batch_executor(StateProviderDatabase::new(
|
||||
self.provider.state_by_block_hash(parent_header.hash())?,
|
||||
));
|
||||
|
||||
@@ -366,7 +366,7 @@ where
|
||||
+ Send
|
||||
+ Sync
|
||||
+ 'static,
|
||||
E: BlockExecutorProvider<Primitives = N>,
|
||||
E: ConfigureEvm<Primitives = N> + 'static,
|
||||
{
|
||||
fn on_invalid_block(
|
||||
&self,
|
||||
|
||||
@@ -3090,7 +3090,6 @@ impl PersistingKind {
|
||||
matches!(self, Self::PersistingDescendant)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
|
||||
@@ -18,11 +18,11 @@ use reth_consensus::{Consensus, FullConsensus};
|
||||
use reth_errors::{ConsensusError, RethResult};
|
||||
use reth_ethereum_payload_builder::EthereumBuilderConfig;
|
||||
use reth_ethereum_primitives::{EthPrimitives, TransactionSigned};
|
||||
use reth_evm::execute::{BlockExecutorProvider, Executor};
|
||||
use reth_evm::{execute::Executor, ConfigureEvm};
|
||||
use reth_execution_types::ExecutionOutcome;
|
||||
use reth_fs_util as fs;
|
||||
use reth_node_api::{BlockTy, EngineApiMessageVersion, PayloadBuilderAttributes};
|
||||
use reth_node_ethereum::{consensus::EthBeaconConsensus, EthEvmConfig, EthExecutorProvider};
|
||||
use reth_node_ethereum::{consensus::EthBeaconConsensus, EthEvmConfig};
|
||||
use reth_primitives_traits::{Block as _, SealedBlock, SealedHeader, SignedTransaction};
|
||||
use reth_provider::{
|
||||
providers::{BlockchainProvider, ProviderNodeTypes},
|
||||
@@ -225,8 +225,8 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
|
||||
let state_provider = blockchain_db.latest()?;
|
||||
let db = StateProviderDatabase::new(&state_provider);
|
||||
let executor =
|
||||
EthExecutorProvider::ethereum(provider_factory.chain_spec()).executor(db);
|
||||
let evm_config = EthEvmConfig::ethereum(provider_factory.chain_spec());
|
||||
let executor = evm_config.batch_executor(db);
|
||||
|
||||
let block_execution_output = executor.execute(&block_with_senders)?;
|
||||
let execution_outcome =
|
||||
|
||||
@@ -11,7 +11,7 @@ use reth_cli_runner::CliContext;
|
||||
use reth_cli_util::get_secret_key;
|
||||
use reth_config::Config;
|
||||
use reth_ethereum_primitives::EthPrimitives;
|
||||
use reth_evm::execute::{BlockExecutorProvider, Executor};
|
||||
use reth_evm::{execute::Executor, ConfigureEvm};
|
||||
use reth_execution_types::ExecutionOutcome;
|
||||
use reth_network::{BlockDownloaderProvider, NetworkHandle};
|
||||
use reth_network_api::NetworkInfo;
|
||||
@@ -20,7 +20,7 @@ use reth_node_core::{
|
||||
args::NetworkArgs,
|
||||
utils::{get_single_body, get_single_header},
|
||||
};
|
||||
use reth_node_ethereum::{consensus::EthBeaconConsensus, EthExecutorProvider};
|
||||
use reth_node_ethereum::{consensus::EthBeaconConsensus, EthEvmConfig};
|
||||
use reth_primitives_traits::SealedBlock;
|
||||
use reth_provider::{
|
||||
providers::ProviderNodeTypes, AccountExtReader, ChainSpecProvider, DatabaseProviderFactory,
|
||||
@@ -146,7 +146,8 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
let state_provider = LatestStateProviderRef::new(&provider);
|
||||
let db = StateProviderDatabase::new(&state_provider);
|
||||
|
||||
let executor = EthExecutorProvider::ethereum(provider_factory.chain_spec()).executor(db);
|
||||
let evm_config = EthEvmConfig::ethereum(provider_factory.chain_spec());
|
||||
let executor = evm_config.batch_executor(db);
|
||||
let block_execution_output = executor.execute(&block.clone().try_recover()?)?;
|
||||
let execution_outcome = ExecutionOutcome::from((block_execution_output, block.number()));
|
||||
|
||||
|
||||
@@ -11,7 +11,7 @@ use reth_config::Config;
|
||||
use reth_consensus::{Consensus, ConsensusError};
|
||||
use reth_db_api::{cursor::DbCursorRO, tables, transaction::DbTx};
|
||||
use reth_ethereum_primitives::EthPrimitives;
|
||||
use reth_evm::execute::{BlockExecutorProvider, Executor};
|
||||
use reth_evm::{execute::Executor, ConfigureEvm};
|
||||
use reth_execution_types::ExecutionOutcome;
|
||||
use reth_network::{BlockDownloaderProvider, NetworkHandle};
|
||||
use reth_network_api::NetworkInfo;
|
||||
@@ -155,8 +155,9 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
|
||||
provider_rw.insert_block(sealed_block.clone(), StorageLocation::Database)?;
|
||||
|
||||
let executor = executor_provider
|
||||
.executor(StateProviderDatabase::new(LatestStateProviderRef::new(&provider_rw)));
|
||||
let executor = executor_provider.batch_executor(StateProviderDatabase::new(
|
||||
LatestStateProviderRef::new(&provider_rw),
|
||||
));
|
||||
let output = executor.execute(&sealed_block)?;
|
||||
|
||||
provider_rw.write_state(
|
||||
|
||||
@@ -1,30 +1,12 @@
|
||||
//! Ethereum block execution strategy.
|
||||
|
||||
use crate::EthEvmConfig;
|
||||
use alloc::sync::Arc;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_evm::execute::BasicBlockExecutorProvider;
|
||||
|
||||
/// Helper type with backwards compatible methods to obtain Ethereum executor
|
||||
/// providers.
|
||||
#[derive(Debug)]
|
||||
pub struct EthExecutorProvider;
|
||||
|
||||
impl EthExecutorProvider {
|
||||
/// Creates a new default ethereum executor provider.
|
||||
pub fn ethereum(chain_spec: Arc<ChainSpec>) -> BasicBlockExecutorProvider<EthEvmConfig> {
|
||||
BasicBlockExecutorProvider::new(EthEvmConfig::new(chain_spec))
|
||||
}
|
||||
|
||||
/// Returns a new provider for the mainnet.
|
||||
pub fn mainnet() -> BasicBlockExecutorProvider<EthEvmConfig> {
|
||||
BasicBlockExecutorProvider::new(EthEvmConfig::mainnet())
|
||||
}
|
||||
}
|
||||
pub type EthExecutorProvider = crate::EthEvmConfig;
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::EthEvmConfig;
|
||||
use alloy_consensus::{constants::ETH_TO_WEI, Header, TxLegacy};
|
||||
use alloy_eips::{
|
||||
eip2935::{HISTORY_SERVE_WINDOW, HISTORY_STORAGE_ADDRESS, HISTORY_STORAGE_CODE},
|
||||
@@ -35,9 +17,9 @@ mod tests {
|
||||
};
|
||||
use alloy_evm::block::BlockValidationError;
|
||||
use alloy_primitives::{b256, fixed_bytes, keccak256, Bytes, TxKind, B256, U256};
|
||||
use reth_chainspec::{ChainSpecBuilder, EthereumHardfork, ForkCondition, MAINNET};
|
||||
use reth_chainspec::{ChainSpec, ChainSpecBuilder, EthereumHardfork, ForkCondition, MAINNET};
|
||||
use reth_ethereum_primitives::{Block, BlockBody, Transaction};
|
||||
use reth_evm::execute::{BasicBlockExecutorProvider, BlockExecutorProvider, Executor};
|
||||
use reth_evm::{execute::Executor, ConfigureEvm};
|
||||
use reth_execution_types::BlockExecutionResult;
|
||||
use reth_primitives_traits::{
|
||||
crypto::secp256k1::public_key_to_address, Block as _, RecoveredBlock,
|
||||
@@ -49,7 +31,7 @@ mod tests {
|
||||
state::{AccountInfo, Bytecode, EvmState},
|
||||
Database,
|
||||
};
|
||||
use std::sync::mpsc;
|
||||
use std::sync::{mpsc, Arc};
|
||||
|
||||
fn create_database_with_beacon_root_contract() -> CacheDB<EmptyDB> {
|
||||
let mut db = CacheDB::new(Default::default());
|
||||
@@ -84,8 +66,8 @@ mod tests {
|
||||
db
|
||||
}
|
||||
|
||||
fn executor_provider(chain_spec: Arc<ChainSpec>) -> BasicBlockExecutorProvider<EthEvmConfig> {
|
||||
BasicBlockExecutorProvider::new(EthEvmConfig::new(chain_spec))
|
||||
fn evm_config(chain_spec: Arc<ChainSpec>) -> EthEvmConfig {
|
||||
EthEvmConfig::new(chain_spec)
|
||||
}
|
||||
|
||||
#[test]
|
||||
@@ -102,9 +84,9 @@ mod tests {
|
||||
.build(),
|
||||
);
|
||||
|
||||
let provider = executor_provider(chain_spec);
|
||||
let provider = evm_config(chain_spec);
|
||||
|
||||
let mut executor = provider.executor(db);
|
||||
let mut executor = provider.batch_executor(db);
|
||||
|
||||
// attempt to execute a block without parent beacon block root, expect err
|
||||
let err = executor
|
||||
@@ -184,11 +166,11 @@ mod tests {
|
||||
.build(),
|
||||
);
|
||||
|
||||
let provider = executor_provider(chain_spec);
|
||||
let provider = evm_config(chain_spec);
|
||||
|
||||
// attempt to execute an empty block with parent beacon block root, this should not fail
|
||||
provider
|
||||
.executor(db)
|
||||
.batch_executor(db)
|
||||
.execute_one(&RecoveredBlock::new_unhashed(
|
||||
Block {
|
||||
header,
|
||||
@@ -218,7 +200,7 @@ mod tests {
|
||||
.build(),
|
||||
);
|
||||
|
||||
let provider = executor_provider(chain_spec);
|
||||
let provider = evm_config(chain_spec);
|
||||
|
||||
// construct the header for block one
|
||||
let header = Header {
|
||||
@@ -229,7 +211,7 @@ mod tests {
|
||||
..Header::default()
|
||||
};
|
||||
|
||||
let mut executor = provider.executor(db);
|
||||
let mut executor = provider.batch_executor(db);
|
||||
|
||||
// attempt to execute an empty block with parent beacon block root, this should not fail
|
||||
executor
|
||||
@@ -263,8 +245,8 @@ mod tests {
|
||||
);
|
||||
|
||||
let mut header = chain_spec.genesis_header().clone();
|
||||
let provider = executor_provider(chain_spec);
|
||||
let mut executor = provider.executor(db);
|
||||
let provider = evm_config(chain_spec);
|
||||
let mut executor = provider.batch_executor(db);
|
||||
|
||||
// attempt to execute the genesis block with non-zero parent beacon block root, expect err
|
||||
header.parent_beacon_block_root = Some(B256::with_last_byte(0x69));
|
||||
@@ -325,10 +307,10 @@ mod tests {
|
||||
.build(),
|
||||
);
|
||||
|
||||
let provider = executor_provider(chain_spec);
|
||||
let provider = evm_config(chain_spec);
|
||||
|
||||
// execute header
|
||||
let mut executor = provider.executor(db);
|
||||
let mut executor = provider.batch_executor(db);
|
||||
|
||||
// Now execute a block with the fixed header, ensure that it does not fail
|
||||
executor
|
||||
@@ -392,8 +374,8 @@ mod tests {
|
||||
.build(),
|
||||
);
|
||||
|
||||
let provider = executor_provider(chain_spec);
|
||||
let mut executor = provider.executor(db);
|
||||
let provider = evm_config(chain_spec);
|
||||
let mut executor = provider.batch_executor(db);
|
||||
|
||||
// construct the header for block one
|
||||
let header = Header { timestamp: 1, number: 1, ..Header::default() };
|
||||
@@ -432,8 +414,8 @@ mod tests {
|
||||
);
|
||||
|
||||
let header = chain_spec.genesis_header().clone();
|
||||
let provider = executor_provider(chain_spec);
|
||||
let mut executor = provider.executor(db);
|
||||
let provider = evm_config(chain_spec);
|
||||
let mut executor = provider.batch_executor(db);
|
||||
|
||||
// attempt to execute genesis block, this should not fail
|
||||
executor
|
||||
@@ -478,8 +460,8 @@ mod tests {
|
||||
parent_beacon_block_root: Some(B256::random()),
|
||||
..Header::default()
|
||||
};
|
||||
let provider = executor_provider(chain_spec);
|
||||
let mut executor = provider.executor(db);
|
||||
let provider = evm_config(chain_spec);
|
||||
let mut executor = provider.batch_executor(db);
|
||||
|
||||
// attempt to execute the fork activation block, this should not fail
|
||||
executor
|
||||
@@ -524,8 +506,8 @@ mod tests {
|
||||
.build(),
|
||||
);
|
||||
|
||||
let provider = executor_provider(chain_spec);
|
||||
let mut executor = provider.executor(db);
|
||||
let provider = evm_config(chain_spec);
|
||||
let mut executor = provider.batch_executor(db);
|
||||
|
||||
let header = Header {
|
||||
parent_hash: B256::random(),
|
||||
@@ -567,8 +549,8 @@ mod tests {
|
||||
let header = chain_spec.genesis_header().clone();
|
||||
let header_hash = header.hash_slow();
|
||||
|
||||
let provider = executor_provider(chain_spec);
|
||||
let mut executor = provider.executor(db);
|
||||
let provider = evm_config(chain_spec);
|
||||
let mut executor = provider.batch_executor(db);
|
||||
|
||||
// attempt to execute the genesis block, this should not fail
|
||||
executor
|
||||
@@ -712,9 +694,9 @@ mod tests {
|
||||
}),
|
||||
);
|
||||
|
||||
let provider = executor_provider(chain_spec);
|
||||
let provider = evm_config(chain_spec);
|
||||
|
||||
let mut executor = provider.executor(db);
|
||||
let mut executor = provider.batch_executor(db);
|
||||
|
||||
let BlockExecutionResult { receipts, requests, .. } = executor
|
||||
.execute_one(
|
||||
@@ -788,7 +770,8 @@ mod tests {
|
||||
);
|
||||
|
||||
// Create an executor from the state provider
|
||||
let mut executor = executor_provider(chain_spec).executor(db);
|
||||
let evm_config = evm_config(chain_spec);
|
||||
let mut executor = evm_config.batch_executor(db);
|
||||
|
||||
// Execute the block and capture the result
|
||||
let exec_result = executor.execute_one(
|
||||
@@ -852,8 +835,8 @@ mod tests {
|
||||
vec![],
|
||||
);
|
||||
|
||||
let provider = executor_provider(chain_spec);
|
||||
let executor = provider.executor(db);
|
||||
let provider = evm_config(chain_spec);
|
||||
let executor = provider.batch_executor(db);
|
||||
|
||||
let (tx, rx) = mpsc::channel();
|
||||
let tx_clone = tx.clone();
|
||||
|
||||
@@ -9,7 +9,6 @@ use reth_evm::{
|
||||
BlockExecutionError, BlockExecutor, BlockExecutorFactory, BlockExecutorFor, ExecutableTx,
|
||||
},
|
||||
eth::{EthBlockExecutionCtx, EthEvmContext},
|
||||
execute::BasicBlockExecutorProvider,
|
||||
ConfigureEvm, Database, EthEvm, EthEvmFactory, Evm, EvmEnvFor, EvmFactory,
|
||||
};
|
||||
use reth_execution_types::{BlockExecutionResult, ExecutionOutcome};
|
||||
@@ -21,7 +20,7 @@ use revm::{
|
||||
};
|
||||
|
||||
/// A helper type alias for mocked block executor provider.
|
||||
pub type MockExecutorProvider = BasicBlockExecutorProvider<MockEvmConfig>;
|
||||
pub type MockExecutorProvider = MockEvmConfig;
|
||||
|
||||
/// A block executor provider that returns mocked execution results.
|
||||
#[derive(Clone, Debug)]
|
||||
|
||||
@@ -1,7 +1,5 @@
|
||||
//! Ethereum EVM support
|
||||
|
||||
#[doc(inline)]
|
||||
pub use reth_evm::execute::BasicBlockExecutorProvider;
|
||||
#[doc(inline)]
|
||||
pub use reth_evm_ethereum::execute::EthExecutorProvider;
|
||||
#[doc(inline)]
|
||||
|
||||
@@ -17,7 +17,7 @@ use revm as _;
|
||||
pub use reth_ethereum_engine_primitives::EthEngineTypes;
|
||||
|
||||
pub mod evm;
|
||||
pub use evm::{BasicBlockExecutorProvider, EthEvmConfig, EthExecutorProvider};
|
||||
pub use evm::{EthEvmConfig, EthExecutorProvider};
|
||||
|
||||
pub use reth_ethereum_consensus as consensus;
|
||||
pub mod node;
|
||||
|
||||
@@ -10,10 +10,7 @@ use reth_ethereum_engine_primitives::{
|
||||
EthBuiltPayload, EthPayloadAttributes, EthPayloadBuilderAttributes,
|
||||
};
|
||||
use reth_ethereum_primitives::{EthPrimitives, PooledTransaction, TransactionSigned};
|
||||
use reth_evm::{
|
||||
execute::BasicBlockExecutorProvider, ConfigureEvm, EvmFactory, EvmFactoryFor,
|
||||
NextBlockEnvAttributes,
|
||||
};
|
||||
use reth_evm::{ConfigureEvm, EvmFactory, EvmFactoryFor, NextBlockEnvAttributes};
|
||||
use reth_network::{EthNetworkPrimitives, NetworkHandle, PeersInfo};
|
||||
use reth_node_api::{AddOnsContext, FullNodeComponents, NodeAddOns, NodePrimitives, TxTy};
|
||||
use reth_node_builder::{
|
||||
@@ -191,7 +188,7 @@ where
|
||||
let validation_api = ValidationApi::new(
|
||||
ctx.node.provider().clone(),
|
||||
Arc::new(ctx.node.consensus().clone()),
|
||||
ctx.node.block_executor().clone(),
|
||||
ctx.node.evm_config().clone(),
|
||||
ctx.config.rpc.flashbots_config(),
|
||||
Box::new(ctx.node.task_executor().clone()),
|
||||
Arc::new(EthereumEngineValidator::new(ctx.config.chain.clone())),
|
||||
@@ -304,17 +301,11 @@ where
|
||||
Node: FullNodeTypes<Types = Types>,
|
||||
{
|
||||
type EVM = EthEvmConfig;
|
||||
type Executor = BasicBlockExecutorProvider<EthEvmConfig>;
|
||||
|
||||
async fn build_evm(
|
||||
self,
|
||||
ctx: &BuilderContext<Node>,
|
||||
) -> eyre::Result<(Self::EVM, Self::Executor)> {
|
||||
async fn build_evm(self, ctx: &BuilderContext<Node>) -> eyre::Result<Self::EVM> {
|
||||
let evm_config = EthEvmConfig::new(ctx.chain_spec())
|
||||
.with_extra_data(ctx.payload_builder_config().extra_data_bytes());
|
||||
let executor = BasicBlockExecutorProvider::new(evm_config.clone());
|
||||
|
||||
Ok((evm_config, executor))
|
||||
Ok(evm_config)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -1,35 +1,12 @@
|
||||
//! Helper type that represents one of two possible executor types
|
||||
|
||||
use crate::{
|
||||
execute::{BlockExecutorProvider, Executor},
|
||||
Database, OnStateHook,
|
||||
};
|
||||
use crate::{execute::Executor, Database, OnStateHook};
|
||||
|
||||
// re-export Either
|
||||
pub use futures_util::future::Either;
|
||||
use reth_execution_types::{BlockExecutionOutput, BlockExecutionResult};
|
||||
use reth_primitives_traits::{NodePrimitives, RecoveredBlock};
|
||||
|
||||
impl<A, B> BlockExecutorProvider for Either<A, B>
|
||||
where
|
||||
A: BlockExecutorProvider,
|
||||
B: BlockExecutorProvider<Primitives = A::Primitives>,
|
||||
{
|
||||
type Primitives = A::Primitives;
|
||||
|
||||
type Executor<DB: Database> = Either<A::Executor<DB>, B::Executor<DB>>;
|
||||
|
||||
fn executor<DB>(&self, db: DB) -> Self::Executor<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
match self {
|
||||
Self::Left(a) => Either::Left(a.executor(db)),
|
||||
Self::Right(b) => Either::Right(b.executor(db)),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<A, B, DB> Executor<DB> for Either<A, B>
|
||||
where
|
||||
A: Executor<DB>,
|
||||
|
||||
@@ -129,36 +129,6 @@ pub trait Executor<DB: Database>: Sized {
|
||||
fn size_hint(&self) -> usize;
|
||||
}
|
||||
|
||||
/// A type that can create a new executor for block execution.
|
||||
pub trait BlockExecutorProvider: Clone + Debug + Send + Sync + Unpin + 'static {
|
||||
/// Receipt type.
|
||||
type Primitives: NodePrimitives;
|
||||
|
||||
/// An executor that can execute a single block given a database.
|
||||
///
|
||||
/// # Verification
|
||||
///
|
||||
/// The on [`Executor::execute`] the executor is expected to validate the execution output of
|
||||
/// the input, this includes:
|
||||
/// - Cumulative gas used must match the input's gas used.
|
||||
/// - Receipts must match the input's receipts root.
|
||||
///
|
||||
/// It is not expected to validate the state trie root, this must be done by the caller using
|
||||
/// the returned state.
|
||||
type Executor<DB: Database>: Executor<
|
||||
DB,
|
||||
Primitives = Self::Primitives,
|
||||
Error = BlockExecutionError,
|
||||
>;
|
||||
|
||||
/// Creates a new executor for single block execution.
|
||||
///
|
||||
/// This is used to execute a single block and get the changed state.
|
||||
fn executor<DB>(&self, db: DB) -> Self::Executor<DB>
|
||||
where
|
||||
DB: Database;
|
||||
}
|
||||
|
||||
/// Helper type for the output of executing a block.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct ExecuteOutput<R> {
|
||||
@@ -405,44 +375,6 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<F> Clone for BasicBlockExecutorProvider<F>
|
||||
where
|
||||
F: Clone,
|
||||
{
|
||||
fn clone(&self) -> Self {
|
||||
Self { strategy_factory: self.strategy_factory.clone() }
|
||||
}
|
||||
}
|
||||
|
||||
/// A generic block executor provider that can create executors using a strategy factory.
|
||||
#[derive(Debug, Default)]
|
||||
pub struct BasicBlockExecutorProvider<F> {
|
||||
strategy_factory: F,
|
||||
}
|
||||
|
||||
impl<F> BasicBlockExecutorProvider<F> {
|
||||
/// Creates a new `BasicBlockExecutorProvider` with the given strategy factory.
|
||||
pub const fn new(strategy_factory: F) -> Self {
|
||||
Self { strategy_factory }
|
||||
}
|
||||
}
|
||||
|
||||
impl<F> BlockExecutorProvider for BasicBlockExecutorProvider<F>
|
||||
where
|
||||
F: ConfigureEvm + 'static,
|
||||
{
|
||||
type Primitives = F::Primitives;
|
||||
|
||||
type Executor<DB: Database> = BasicBlockExecutor<F, DB>;
|
||||
|
||||
fn executor<DB>(&self, db: DB) -> Self::Executor<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
BasicBlockExecutor::new(self.strategy_factory.clone(), db)
|
||||
}
|
||||
}
|
||||
|
||||
/// A generic block executor that uses a [`BlockExecutor`] to
|
||||
/// execute blocks.
|
||||
#[expect(missing_debug_implementations)]
|
||||
@@ -530,11 +462,8 @@ mod tests {
|
||||
#[derive(Clone, Debug, Default)]
|
||||
struct TestExecutorProvider;
|
||||
|
||||
impl BlockExecutorProvider for TestExecutorProvider {
|
||||
type Primitives = EthPrimitives;
|
||||
type Executor<DB: Database> = TestExecutor<DB>;
|
||||
|
||||
fn executor<DB>(&self, _db: DB) -> Self::Executor<DB>
|
||||
impl TestExecutorProvider {
|
||||
fn executor<DB>(&self, _db: DB) -> TestExecutor<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
|
||||
@@ -23,7 +23,7 @@ use alloy_eips::{eip2930::AccessList, eip4895::Withdrawals};
|
||||
use alloy_evm::block::{BlockExecutorFactory, BlockExecutorFor};
|
||||
use alloy_primitives::{Address, B256};
|
||||
use core::{error::Error, fmt::Debug};
|
||||
use execute::{BlockAssembler, BlockBuilder};
|
||||
use execute::{BasicBlockExecutor, BlockAssembler, BlockBuilder};
|
||||
use reth_primitives_traits::{
|
||||
BlockTy, HeaderTy, NodePrimitives, ReceiptTy, SealedBlock, SealedHeader, TxTy,
|
||||
};
|
||||
@@ -270,6 +270,12 @@ pub trait ConfigureEvm: Clone + Debug + Send + Sync + Unpin {
|
||||
let ctx = self.context_for_next_block(parent, attributes);
|
||||
Ok(self.create_block_builder(evm, parent, ctx))
|
||||
}
|
||||
|
||||
/// Returns a new [`BasicBlockExecutor`].
|
||||
#[auto_impl(keep_default_for(&, Arc))]
|
||||
fn batch_executor<DB: Database>(&self, db: DB) -> BasicBlockExecutor<&Self, DB> {
|
||||
BasicBlockExecutor::new(self, db)
|
||||
}
|
||||
}
|
||||
|
||||
/// Represents additional attributes required to configure the next block.
|
||||
|
||||
@@ -11,7 +11,7 @@ use super::stream::DEFAULT_PARALLELISM;
|
||||
/// Factory for creating new backfill jobs.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct BackfillJobFactory<E, P> {
|
||||
executor: E,
|
||||
evm_config: E,
|
||||
provider: P,
|
||||
prune_modes: PruneModes,
|
||||
thresholds: ExecutionStageThresholds,
|
||||
@@ -20,9 +20,9 @@ pub struct BackfillJobFactory<E, P> {
|
||||
|
||||
impl<E, P> BackfillJobFactory<E, P> {
|
||||
/// Creates a new [`BackfillJobFactory`].
|
||||
pub fn new(executor: E, provider: P) -> Self {
|
||||
pub fn new(evm_config: E, provider: P) -> Self {
|
||||
Self {
|
||||
executor,
|
||||
evm_config,
|
||||
provider,
|
||||
prune_modes: PruneModes::none(),
|
||||
thresholds: ExecutionStageThresholds {
|
||||
@@ -64,7 +64,7 @@ impl<E: Clone, P: Clone> BackfillJobFactory<E, P> {
|
||||
/// Creates a new backfill job for the given range.
|
||||
pub fn backfill(&self, range: RangeInclusive<BlockNumber>) -> BackfillJob<E, P> {
|
||||
BackfillJob {
|
||||
executor: self.executor.clone(),
|
||||
evm_config: self.evm_config.clone(),
|
||||
provider: self.provider.clone(),
|
||||
prune_modes: self.prune_modes.clone(),
|
||||
range,
|
||||
@@ -78,9 +78,9 @@ impl BackfillJobFactory<(), ()> {
|
||||
/// Creates a new [`BackfillJobFactory`] from [`FullNodeComponents`].
|
||||
pub fn new_from_components<Node: FullNodeComponents>(
|
||||
components: Node,
|
||||
) -> BackfillJobFactory<Node::Executor, Node::Provider> {
|
||||
) -> BackfillJobFactory<Node::Evm, Node::Provider> {
|
||||
BackfillJobFactory::<_, _>::new(
|
||||
components.block_executor().clone(),
|
||||
components.evm_config().clone(),
|
||||
components.provider().clone(),
|
||||
)
|
||||
}
|
||||
|
||||
@@ -1,4 +1,5 @@
|
||||
use crate::StreamBackfillJob;
|
||||
use reth_evm::ConfigureEvm;
|
||||
use std::{
|
||||
ops::RangeInclusive,
|
||||
time::{Duration, Instant},
|
||||
@@ -7,9 +8,7 @@ use std::{
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_primitives::BlockNumber;
|
||||
use reth_ethereum_primitives::Receipt;
|
||||
use reth_evm::execute::{
|
||||
BlockExecutionError, BlockExecutionOutput, BlockExecutorProvider, Executor,
|
||||
};
|
||||
use reth_evm::execute::{BlockExecutionError, BlockExecutionOutput, Executor};
|
||||
use reth_node_api::{Block as _, BlockBody as _, NodePrimitives};
|
||||
use reth_primitives_traits::{format_gas_throughput, RecoveredBlock, SignedTransaction};
|
||||
use reth_provider::{
|
||||
@@ -30,7 +29,7 @@ pub(super) type BackfillJobResult<T> = Result<T, BlockExecutionError>;
|
||||
/// depending on the configured thresholds.
|
||||
#[derive(Debug)]
|
||||
pub struct BackfillJob<E, P> {
|
||||
pub(crate) executor: E,
|
||||
pub(crate) evm_config: E,
|
||||
pub(crate) provider: P,
|
||||
pub(crate) prune_modes: PruneModes,
|
||||
pub(crate) thresholds: ExecutionStageThresholds,
|
||||
@@ -40,7 +39,7 @@ pub struct BackfillJob<E, P> {
|
||||
|
||||
impl<E, P> Iterator for BackfillJob<E, P>
|
||||
where
|
||||
E: BlockExecutorProvider<Primitives: NodePrimitives<Block = P::Block>>,
|
||||
E: ConfigureEvm<Primitives: NodePrimitives<Block = P::Block>> + 'static,
|
||||
P: HeaderProvider + BlockReader<Transaction: SignedTransaction> + StateProviderFactory,
|
||||
{
|
||||
type Item = BackfillJobResult<Chain<E::Primitives>>;
|
||||
@@ -56,7 +55,7 @@ where
|
||||
|
||||
impl<E, P> BackfillJob<E, P>
|
||||
where
|
||||
E: BlockExecutorProvider<Primitives: NodePrimitives<Block = P::Block>>,
|
||||
E: ConfigureEvm<Primitives: NodePrimitives<Block = P::Block>> + 'static,
|
||||
P: BlockReader<Transaction: SignedTransaction> + HeaderProvider + StateProviderFactory,
|
||||
{
|
||||
/// Converts the backfill job into a single block backfill job.
|
||||
@@ -76,7 +75,7 @@ where
|
||||
"Executing block range"
|
||||
);
|
||||
|
||||
let mut executor = self.executor.executor(StateProviderDatabase::new(
|
||||
let mut executor = self.evm_config.batch_executor(StateProviderDatabase::new(
|
||||
self.provider
|
||||
.history_by_block_number(self.range.start().saturating_sub(1))
|
||||
.map_err(BlockExecutionError::other)?,
|
||||
@@ -162,7 +161,7 @@ where
|
||||
/// iterator is advanced and yields ([`RecoveredBlock`], [`BlockExecutionOutput`])
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct SingleBlockBackfillJob<E, P> {
|
||||
pub(crate) executor: E,
|
||||
pub(crate) evm_config: E,
|
||||
pub(crate) provider: P,
|
||||
pub(crate) range: RangeInclusive<BlockNumber>,
|
||||
pub(crate) stream_parallelism: usize,
|
||||
@@ -170,7 +169,7 @@ pub struct SingleBlockBackfillJob<E, P> {
|
||||
|
||||
impl<E, P> Iterator for SingleBlockBackfillJob<E, P>
|
||||
where
|
||||
E: BlockExecutorProvider<Primitives: NodePrimitives<Block = P::Block>>,
|
||||
E: ConfigureEvm<Primitives: NodePrimitives<Block = P::Block>> + 'static,
|
||||
P: HeaderProvider + BlockReader + StateProviderFactory,
|
||||
{
|
||||
type Item = BackfillJobResult<(
|
||||
@@ -185,7 +184,7 @@ where
|
||||
|
||||
impl<E, P> SingleBlockBackfillJob<E, P>
|
||||
where
|
||||
E: BlockExecutorProvider<Primitives: NodePrimitives<Block = P::Block>>,
|
||||
E: ConfigureEvm<Primitives: NodePrimitives<Block = P::Block>> + 'static,
|
||||
P: HeaderProvider + BlockReader + StateProviderFactory,
|
||||
{
|
||||
/// Converts the single block backfill job into a stream.
|
||||
@@ -216,7 +215,7 @@ where
|
||||
.map_err(BlockExecutionError::other)?;
|
||||
|
||||
// Configure the executor to use the previous block's state.
|
||||
let executor = self.executor.executor(StateProviderDatabase::new(
|
||||
let executor = self.evm_config.batch_executor(StateProviderDatabase::new(
|
||||
self.provider
|
||||
.history_by_block_number(block_number.saturating_sub(1))
|
||||
.map_err(BlockExecutionError::other)?,
|
||||
@@ -233,7 +232,7 @@ where
|
||||
impl<E, P> From<BackfillJob<E, P>> for SingleBlockBackfillJob<E, P> {
|
||||
fn from(job: BackfillJob<E, P>) -> Self {
|
||||
Self {
|
||||
executor: job.executor,
|
||||
evm_config: job.evm_config,
|
||||
provider: job.provider,
|
||||
range: job.range,
|
||||
stream_parallelism: job.stream_parallelism,
|
||||
|
||||
@@ -6,7 +6,10 @@ use futures::{
|
||||
StreamExt,
|
||||
};
|
||||
use reth_ethereum_primitives::EthPrimitives;
|
||||
use reth_evm::execute::{BlockExecutionError, BlockExecutionOutput, BlockExecutorProvider};
|
||||
use reth_evm::{
|
||||
execute::{BlockExecutionError, BlockExecutionOutput},
|
||||
ConfigureEvm,
|
||||
};
|
||||
use reth_node_api::NodePrimitives;
|
||||
use reth_primitives_traits::RecoveredBlock;
|
||||
use reth_provider::{BlockReader, Chain, StateProviderFactory};
|
||||
@@ -50,7 +53,7 @@ type BatchBlockStreamItem<N = EthPrimitives> = Chain<N>;
|
||||
/// processed asynchronously but in order within a specified range.
|
||||
#[derive(Debug)]
|
||||
pub struct StreamBackfillJob<E, P, T> {
|
||||
executor: E,
|
||||
evm_config: E,
|
||||
provider: P,
|
||||
prune_modes: PruneModes,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
@@ -115,7 +118,7 @@ where
|
||||
|
||||
impl<E, P> Stream for StreamBackfillJob<E, P, SingleBlockStreamItem<E::Primitives>>
|
||||
where
|
||||
E: BlockExecutorProvider<Primitives: NodePrimitives<Block = P::Block>> + Clone + 'static,
|
||||
E: ConfigureEvm<Primitives: NodePrimitives<Block = P::Block>> + 'static,
|
||||
P: BlockReader + StateProviderFactory + Clone + Unpin + 'static,
|
||||
{
|
||||
type Item = BackfillJobResult<SingleBlockStreamItem<E::Primitives>>;
|
||||
@@ -134,7 +137,7 @@ where
|
||||
// Spawn a new task for that block
|
||||
debug!(target: "exex::backfill", tasks = %this.tasks.len(), ?block_number, "Spawning new single block backfill task");
|
||||
let job = Box::new(SingleBlockBackfillJob {
|
||||
executor: this.executor.clone(),
|
||||
evm_config: this.evm_config.clone(),
|
||||
provider: this.provider.clone(),
|
||||
range: block_number..=block_number,
|
||||
stream_parallelism: this.parallelism,
|
||||
@@ -148,7 +151,7 @@ where
|
||||
|
||||
impl<E, P> Stream for StreamBackfillJob<E, P, BatchBlockStreamItem<E::Primitives>>
|
||||
where
|
||||
E: BlockExecutorProvider<Primitives: NodePrimitives<Block = P::Block>> + Clone + 'static,
|
||||
E: ConfigureEvm<Primitives: NodePrimitives<Block = P::Block>> + 'static,
|
||||
P: BlockReader + StateProviderFactory + Clone + Unpin + 'static,
|
||||
{
|
||||
type Item = BackfillJobResult<BatchBlockStreamItem<E::Primitives>>;
|
||||
@@ -173,7 +176,7 @@ where
|
||||
// Spawn a new task for that range
|
||||
debug!(target: "exex::backfill", tasks = %this.tasks.len(), ?range, "Spawning new block batch backfill task");
|
||||
let job = Box::new(BackfillJob {
|
||||
executor: this.executor.clone(),
|
||||
evm_config: this.evm_config.clone(),
|
||||
provider: this.provider.clone(),
|
||||
prune_modes: this.prune_modes.clone(),
|
||||
thresholds: this.thresholds.clone(),
|
||||
@@ -200,7 +203,7 @@ where
|
||||
impl<E, P> From<SingleBlockBackfillJob<E, P>> for StreamBackfillJob<E, P, SingleBlockStreamItem> {
|
||||
fn from(job: SingleBlockBackfillJob<E, P>) -> Self {
|
||||
Self {
|
||||
executor: job.executor,
|
||||
evm_config: job.evm_config,
|
||||
provider: job.provider,
|
||||
prune_modes: PruneModes::default(),
|
||||
range: job.range,
|
||||
@@ -214,12 +217,12 @@ impl<E, P> From<SingleBlockBackfillJob<E, P>> for StreamBackfillJob<E, P, Single
|
||||
|
||||
impl<E, P> From<BackfillJob<E, P>> for StreamBackfillJob<E, P, BatchBlockStreamItem<E::Primitives>>
|
||||
where
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
{
|
||||
fn from(job: BackfillJob<E, P>) -> Self {
|
||||
let batch_size = job.thresholds.max_blocks.map_or(DEFAULT_BATCH_SIZE, |max| max as usize);
|
||||
Self {
|
||||
executor: job.executor,
|
||||
evm_config: job.evm_config,
|
||||
provider: job.provider,
|
||||
prune_modes: job.prune_modes,
|
||||
range: job.range,
|
||||
|
||||
@@ -5,8 +5,11 @@ use alloy_genesis::{Genesis, GenesisAccount};
|
||||
use alloy_primitives::{b256, Address, TxKind, U256};
|
||||
use reth_chainspec::{ChainSpec, ChainSpecBuilder, EthereumHardfork, MAINNET, MIN_TRANSACTION_GAS};
|
||||
use reth_ethereum_primitives::{Block, BlockBody, Receipt, Transaction};
|
||||
use reth_evm::execute::{BlockExecutionOutput, BlockExecutorProvider, Executor};
|
||||
use reth_evm_ethereum::execute::EthExecutorProvider;
|
||||
use reth_evm::{
|
||||
execute::{BlockExecutionOutput, Executor},
|
||||
ConfigureEvm,
|
||||
};
|
||||
use reth_evm_ethereum::{execute::EthExecutorProvider, EthEvmConfig};
|
||||
use reth_node_api::FullNodePrimitives;
|
||||
use reth_primitives_traits::{Block as _, RecoveredBlock};
|
||||
use reth_provider::{
|
||||
@@ -66,7 +69,7 @@ where
|
||||
|
||||
// Execute the block to produce a block execution output
|
||||
let mut block_execution_output = EthExecutorProvider::ethereum(chain_spec)
|
||||
.executor(StateProviderDatabase::new(LatestStateProviderRef::new(&provider)))
|
||||
.batch_executor(StateProviderDatabase::new(LatestStateProviderRef::new(&provider)))
|
||||
.execute(block)?;
|
||||
block_execution_output.state.reverts.sort();
|
||||
|
||||
@@ -200,8 +203,9 @@ where
|
||||
|
||||
let provider = provider_factory.provider()?;
|
||||
|
||||
let executor = EthExecutorProvider::ethereum(chain_spec)
|
||||
.executor(StateProviderDatabase::new(LatestStateProviderRef::new(&provider)));
|
||||
let evm_config = EthEvmConfig::new(chain_spec);
|
||||
let executor = evm_config
|
||||
.batch_executor(StateProviderDatabase::new(LatestStateProviderRef::new(&provider)));
|
||||
|
||||
let mut execution_outcome = executor.execute_batch(vec![&block1, &block2])?;
|
||||
execution_outcome.state_mut().reverts.sort();
|
||||
|
||||
@@ -33,7 +33,7 @@ pub struct ExExContext<Node: FullNodeComponents> {
|
||||
///
|
||||
/// Once an [`ExExNotification`](crate::ExExNotification) is sent over the channel, it is
|
||||
/// considered delivered by the node.
|
||||
pub notifications: ExExNotifications<Node::Provider, Node::Executor>,
|
||||
pub notifications: ExExNotifications<Node::Provider, Node::Evm>,
|
||||
|
||||
/// Node components
|
||||
pub components: Node,
|
||||
@@ -43,7 +43,6 @@ impl<Node> Debug for ExExContext<Node>
|
||||
where
|
||||
Node: FullNodeComponents,
|
||||
Node::Provider: Debug,
|
||||
Node::Executor: Debug,
|
||||
{
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("ExExContext")
|
||||
@@ -61,7 +60,6 @@ impl<Node> ExExContext<Node>
|
||||
where
|
||||
Node: FullNodeComponents,
|
||||
Node::Provider: Debug + BlockReader,
|
||||
Node::Executor: Debug,
|
||||
Node::Types: NodeTypes<Primitives: NodePrimitives>,
|
||||
{
|
||||
/// Returns dynamic version of the context
|
||||
@@ -85,11 +83,6 @@ where
|
||||
self.components.evm_config()
|
||||
}
|
||||
|
||||
/// Returns the node's executor type.
|
||||
pub fn block_executor(&self) -> &Node::Executor {
|
||||
self.components.block_executor()
|
||||
}
|
||||
|
||||
/// Returns the provider of the node.
|
||||
pub fn provider(&self) -> &Node::Provider {
|
||||
self.components.provider()
|
||||
@@ -159,7 +152,7 @@ mod tests {
|
||||
{
|
||||
async fn _test_bounds(mut self) -> eyre::Result<()> {
|
||||
self.ctx.pool();
|
||||
self.ctx.block_executor();
|
||||
self.ctx.evm_config();
|
||||
self.ctx.provider();
|
||||
self.ctx.network();
|
||||
self.ctx.payload_builder_handle();
|
||||
|
||||
@@ -54,7 +54,6 @@ impl<Node> From<ExExContext<Node>> for ExExContextDyn<PrimitivesTy<Node::Types>>
|
||||
where
|
||||
Node: FullNodeComponents<Types: NodeTypes<Primitives: NodePrimitives>>,
|
||||
Node::Provider: Debug + BlockReader,
|
||||
Node::Executor: Debug,
|
||||
{
|
||||
fn from(ctx: ExExContext<Node>) -> Self {
|
||||
let config = ctx.config.map_chainspec(|chainspec| {
|
||||
|
||||
@@ -8,7 +8,7 @@ use itertools::Itertools;
|
||||
use metrics::Gauge;
|
||||
use reth_chain_state::ForkChoiceStream;
|
||||
use reth_ethereum_primitives::EthPrimitives;
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_metrics::{metrics::Counter, Metrics};
|
||||
use reth_node_api::NodePrimitives;
|
||||
use reth_primitives_traits::SealedHeader;
|
||||
@@ -94,17 +94,17 @@ impl<N: NodePrimitives> ExExHandle<N> {
|
||||
///
|
||||
/// Returns the handle, as well as a [`UnboundedSender`] for [`ExExEvent`]s and a
|
||||
/// [`mpsc::Receiver`] for [`ExExNotification`]s that should be given to the `ExEx`.
|
||||
pub fn new<P, E: BlockExecutorProvider<Primitives = N>>(
|
||||
pub fn new<P, E: ConfigureEvm<Primitives = N>>(
|
||||
id: String,
|
||||
node_head: BlockNumHash,
|
||||
provider: P,
|
||||
executor: E,
|
||||
evm_config: E,
|
||||
wal_handle: WalHandle<N>,
|
||||
) -> (Self, UnboundedSender<ExExEvent>, ExExNotifications<P, E>) {
|
||||
let (notification_tx, notification_rx) = mpsc::channel(1);
|
||||
let (event_tx, event_rx) = mpsc::unbounded_channel();
|
||||
let notifications =
|
||||
ExExNotifications::new(node_head, provider, executor, notification_rx, wal_handle);
|
||||
ExExNotifications::new(node_head, provider, evm_config, notification_rx, wal_handle);
|
||||
|
||||
(
|
||||
Self {
|
||||
@@ -663,7 +663,7 @@ mod tests {
|
||||
use futures::{StreamExt, TryStreamExt};
|
||||
use rand::Rng;
|
||||
use reth_db_common::init::init_genesis;
|
||||
use reth_evm_ethereum::{execute::EthExecutorProvider, MockExecutorProvider};
|
||||
use reth_evm_ethereum::{execute::EthExecutorProvider, EthEvmConfig};
|
||||
use reth_primitives_traits::RecoveredBlock;
|
||||
use reth_provider::{
|
||||
providers::BlockchainProvider, test_utils::create_test_provider_factory, BlockReader,
|
||||
@@ -687,7 +687,7 @@ mod tests {
|
||||
"test_exex".to_string(),
|
||||
Default::default(),
|
||||
(),
|
||||
MockExecutorProvider::default(),
|
||||
EthEvmConfig::mainnet(),
|
||||
wal.handle(),
|
||||
);
|
||||
|
||||
@@ -707,7 +707,7 @@ mod tests {
|
||||
"test_exex_1".to_string(),
|
||||
Default::default(),
|
||||
(),
|
||||
MockExecutorProvider::default(),
|
||||
EthEvmConfig::mainnet(),
|
||||
wal.handle(),
|
||||
);
|
||||
|
||||
@@ -729,7 +729,7 @@ mod tests {
|
||||
"test_exex_1".to_string(),
|
||||
Default::default(),
|
||||
(),
|
||||
MockExecutorProvider::default(),
|
||||
EthEvmConfig::mainnet(),
|
||||
wal.handle(),
|
||||
);
|
||||
|
||||
@@ -757,7 +757,7 @@ mod tests {
|
||||
"test_exex".to_string(),
|
||||
Default::default(),
|
||||
(),
|
||||
MockExecutorProvider::default(),
|
||||
EthEvmConfig::mainnet(),
|
||||
wal.handle(),
|
||||
);
|
||||
|
||||
@@ -812,7 +812,7 @@ mod tests {
|
||||
"test_exex".to_string(),
|
||||
Default::default(),
|
||||
(),
|
||||
MockExecutorProvider::default(),
|
||||
EthEvmConfig::mainnet(),
|
||||
wal.handle(),
|
||||
);
|
||||
|
||||
@@ -863,7 +863,7 @@ mod tests {
|
||||
"test_exex".to_string(),
|
||||
Default::default(),
|
||||
(),
|
||||
MockExecutorProvider::default(),
|
||||
EthEvmConfig::mainnet(),
|
||||
wal.handle(),
|
||||
);
|
||||
|
||||
@@ -918,14 +918,14 @@ mod tests {
|
||||
"test_exex1".to_string(),
|
||||
Default::default(),
|
||||
(),
|
||||
MockExecutorProvider::default(),
|
||||
EthEvmConfig::mainnet(),
|
||||
wal.handle(),
|
||||
);
|
||||
let (exex_handle2, event_tx2, _) = ExExHandle::new(
|
||||
"test_exex2".to_string(),
|
||||
Default::default(),
|
||||
(),
|
||||
MockExecutorProvider::default(),
|
||||
EthEvmConfig::mainnet(),
|
||||
wal.handle(),
|
||||
);
|
||||
|
||||
@@ -975,14 +975,14 @@ mod tests {
|
||||
"test_exex1".to_string(),
|
||||
Default::default(),
|
||||
(),
|
||||
MockExecutorProvider::default(),
|
||||
EthEvmConfig::mainnet(),
|
||||
wal.handle(),
|
||||
);
|
||||
let (exex_handle2, event_tx2, _) = ExExHandle::new(
|
||||
"test_exex2".to_string(),
|
||||
Default::default(),
|
||||
(),
|
||||
MockExecutorProvider::default(),
|
||||
EthEvmConfig::mainnet(),
|
||||
wal.handle(),
|
||||
);
|
||||
|
||||
@@ -1038,7 +1038,7 @@ mod tests {
|
||||
"test_exex_1".to_string(),
|
||||
Default::default(),
|
||||
(),
|
||||
MockExecutorProvider::default(),
|
||||
EthEvmConfig::mainnet(),
|
||||
wal.handle(),
|
||||
);
|
||||
|
||||
|
||||
@@ -3,7 +3,7 @@ use alloy_consensus::BlockHeader;
|
||||
use alloy_eips::BlockNumHash;
|
||||
use futures::{Stream, StreamExt};
|
||||
use reth_ethereum_primitives::EthPrimitives;
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_exex_types::ExExHead;
|
||||
use reth_node_api::NodePrimitives;
|
||||
use reth_provider::{BlockReader, Chain, HeaderProvider, StateProviderFactory};
|
||||
@@ -22,7 +22,7 @@ use tokio::sync::mpsc::Receiver;
|
||||
#[derive(Debug)]
|
||||
pub struct ExExNotifications<P, E>
|
||||
where
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
{
|
||||
inner: ExExNotificationsInner<P, E>,
|
||||
}
|
||||
@@ -66,7 +66,7 @@ pub trait ExExNotificationsStream<N: NodePrimitives = EthPrimitives>:
|
||||
#[derive(Debug)]
|
||||
enum ExExNotificationsInner<P, E>
|
||||
where
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
{
|
||||
/// A stream of [`ExExNotification`]s. The stream will emit notifications for all blocks.
|
||||
WithoutHead(ExExNotificationsWithoutHead<P, E>),
|
||||
@@ -80,13 +80,13 @@ where
|
||||
|
||||
impl<P, E> ExExNotifications<P, E>
|
||||
where
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
{
|
||||
/// Creates a new stream of [`ExExNotifications`] without a head.
|
||||
pub const fn new(
|
||||
node_head: BlockNumHash,
|
||||
provider: P,
|
||||
executor: E,
|
||||
evm_config: E,
|
||||
notifications: Receiver<ExExNotification<E::Primitives>>,
|
||||
wal_handle: WalHandle<E::Primitives>,
|
||||
) -> Self {
|
||||
@@ -94,7 +94,7 @@ where
|
||||
inner: ExExNotificationsInner::WithoutHead(ExExNotificationsWithoutHead::new(
|
||||
node_head,
|
||||
provider,
|
||||
executor,
|
||||
evm_config,
|
||||
notifications,
|
||||
wal_handle,
|
||||
)),
|
||||
@@ -105,10 +105,7 @@ where
|
||||
impl<P, E> ExExNotificationsStream<E::Primitives> for ExExNotifications<P, E>
|
||||
where
|
||||
P: BlockReader + HeaderProvider + StateProviderFactory + Clone + Unpin + 'static,
|
||||
E: BlockExecutorProvider<Primitives: NodePrimitives<Block = P::Block>>
|
||||
+ Clone
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
E: ConfigureEvm<Primitives: NodePrimitives<Block = P::Block>> + Clone + Unpin + 'static,
|
||||
{
|
||||
fn set_without_head(&mut self) {
|
||||
let current = std::mem::replace(&mut self.inner, ExExNotificationsInner::Invalid);
|
||||
@@ -117,7 +114,7 @@ where
|
||||
ExExNotificationsInner::WithHead(notifications) => ExExNotificationsWithoutHead::new(
|
||||
notifications.initial_local_head,
|
||||
notifications.provider,
|
||||
notifications.executor,
|
||||
notifications.evm_config,
|
||||
notifications.notifications,
|
||||
notifications.wal_handle,
|
||||
),
|
||||
@@ -135,7 +132,7 @@ where
|
||||
Box::new(ExExNotificationsWithHead::new(
|
||||
notifications.initial_local_head,
|
||||
notifications.provider,
|
||||
notifications.executor,
|
||||
notifications.evm_config,
|
||||
notifications.notifications,
|
||||
notifications.wal_handle,
|
||||
exex_head,
|
||||
@@ -159,10 +156,7 @@ where
|
||||
impl<P, E> Stream for ExExNotifications<P, E>
|
||||
where
|
||||
P: BlockReader + HeaderProvider + StateProviderFactory + Clone + Unpin + 'static,
|
||||
E: BlockExecutorProvider<Primitives: NodePrimitives<Block = P::Block>>
|
||||
+ Clone
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
E: ConfigureEvm<Primitives: NodePrimitives<Block = P::Block>> + 'static,
|
||||
{
|
||||
type Item = eyre::Result<ExExNotification<E::Primitives>>;
|
||||
|
||||
@@ -183,23 +177,23 @@ where
|
||||
/// A stream of [`ExExNotification`]s. The stream will emit notifications for all blocks.
|
||||
pub struct ExExNotificationsWithoutHead<P, E>
|
||||
where
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
{
|
||||
node_head: BlockNumHash,
|
||||
provider: P,
|
||||
executor: E,
|
||||
evm_config: E,
|
||||
notifications: Receiver<ExExNotification<E::Primitives>>,
|
||||
wal_handle: WalHandle<E::Primitives>,
|
||||
}
|
||||
|
||||
impl<P: Debug, E> Debug for ExExNotificationsWithoutHead<P, E>
|
||||
where
|
||||
E: Debug + BlockExecutorProvider,
|
||||
E: ConfigureEvm + Debug,
|
||||
{
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("ExExNotifications")
|
||||
.field("provider", &self.provider)
|
||||
.field("executor", &self.executor)
|
||||
.field("evm_config", &self.evm_config)
|
||||
.field("notifications", &self.notifications)
|
||||
.finish()
|
||||
}
|
||||
@@ -207,17 +201,17 @@ where
|
||||
|
||||
impl<P, E> ExExNotificationsWithoutHead<P, E>
|
||||
where
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
{
|
||||
/// Creates a new instance of [`ExExNotificationsWithoutHead`].
|
||||
const fn new(
|
||||
node_head: BlockNumHash,
|
||||
provider: P,
|
||||
executor: E,
|
||||
evm_config: E,
|
||||
notifications: Receiver<ExExNotification<E::Primitives>>,
|
||||
wal_handle: WalHandle<E::Primitives>,
|
||||
) -> Self {
|
||||
Self { node_head, provider, executor, notifications, wal_handle }
|
||||
Self { node_head, provider, evm_config, notifications, wal_handle }
|
||||
}
|
||||
|
||||
/// Subscribe to notifications with the given head.
|
||||
@@ -225,7 +219,7 @@ where
|
||||
ExExNotificationsWithHead::new(
|
||||
self.node_head,
|
||||
self.provider,
|
||||
self.executor,
|
||||
self.evm_config,
|
||||
self.notifications,
|
||||
self.wal_handle,
|
||||
head,
|
||||
@@ -235,7 +229,7 @@ where
|
||||
|
||||
impl<P: Unpin, E> Stream for ExExNotificationsWithoutHead<P, E>
|
||||
where
|
||||
E: Unpin + BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
{
|
||||
type Item = ExExNotification<E::Primitives>;
|
||||
|
||||
@@ -255,12 +249,12 @@ where
|
||||
#[derive(Debug)]
|
||||
pub struct ExExNotificationsWithHead<P, E>
|
||||
where
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
{
|
||||
/// The node's local head at launch.
|
||||
initial_local_head: BlockNumHash,
|
||||
provider: P,
|
||||
executor: E,
|
||||
evm_config: E,
|
||||
notifications: Receiver<ExExNotification<E::Primitives>>,
|
||||
wal_handle: WalHandle<E::Primitives>,
|
||||
/// The exex head at launch
|
||||
@@ -278,13 +272,13 @@ where
|
||||
|
||||
impl<P, E> ExExNotificationsWithHead<P, E>
|
||||
where
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
{
|
||||
/// Creates a new [`ExExNotificationsWithHead`].
|
||||
const fn new(
|
||||
node_head: BlockNumHash,
|
||||
provider: P,
|
||||
executor: E,
|
||||
evm_config: E,
|
||||
notifications: Receiver<ExExNotification<E::Primitives>>,
|
||||
wal_handle: WalHandle<E::Primitives>,
|
||||
exex_head: ExExHead,
|
||||
@@ -292,7 +286,7 @@ where
|
||||
Self {
|
||||
initial_local_head: node_head,
|
||||
provider,
|
||||
executor,
|
||||
evm_config,
|
||||
notifications,
|
||||
wal_handle,
|
||||
initial_exex_head: exex_head,
|
||||
@@ -306,10 +300,7 @@ where
|
||||
impl<P, E> ExExNotificationsWithHead<P, E>
|
||||
where
|
||||
P: BlockReader + HeaderProvider + StateProviderFactory + Clone + Unpin + 'static,
|
||||
E: BlockExecutorProvider<Primitives: NodePrimitives<Block = P::Block>>
|
||||
+ Clone
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
E: ConfigureEvm<Primitives: NodePrimitives<Block = P::Block>> + Clone + Unpin + 'static,
|
||||
{
|
||||
/// Checks if the ExEx head is on the canonical chain.
|
||||
///
|
||||
@@ -369,7 +360,7 @@ where
|
||||
/// exex_head.number`). Nothing to do.
|
||||
fn check_backfill(&mut self) -> eyre::Result<()> {
|
||||
let backfill_job_factory =
|
||||
BackfillJobFactory::new(self.executor.clone(), self.provider.clone());
|
||||
BackfillJobFactory::new(self.evm_config.clone(), self.provider.clone());
|
||||
match self.initial_exex_head.block.number.cmp(&self.initial_local_head.number) {
|
||||
std::cmp::Ordering::Less => {
|
||||
// ExEx is behind the node head, start backfill
|
||||
@@ -396,10 +387,7 @@ where
|
||||
impl<P, E> Stream for ExExNotificationsWithHead<P, E>
|
||||
where
|
||||
P: BlockReader + HeaderProvider + StateProviderFactory + Clone + Unpin + 'static,
|
||||
E: BlockExecutorProvider<Primitives: NodePrimitives<Block = P::Block>>
|
||||
+ Clone
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
E: ConfigureEvm<Primitives: NodePrimitives<Block = P::Block>> + Clone + Unpin + 'static,
|
||||
{
|
||||
type Item = eyre::Result<ExExNotification<E::Primitives>>;
|
||||
|
||||
@@ -535,7 +523,7 @@ mod tests {
|
||||
Some(ExExNotification::ChainCommitted {
|
||||
new: Arc::new(
|
||||
BackfillJobFactory::new(
|
||||
notifications.executor.clone(),
|
||||
notifications.evm_config.clone(),
|
||||
notifications.provider.clone()
|
||||
)
|
||||
.backfill(1..=1)
|
||||
|
||||
@@ -25,7 +25,7 @@ use reth_db::{
|
||||
};
|
||||
use reth_db_common::init::init_genesis;
|
||||
use reth_ethereum_primitives::{EthPrimitives, TransactionSigned};
|
||||
use reth_evm_ethereum::{MockEvmConfig, MockExecutorProvider};
|
||||
use reth_evm_ethereum::MockEvmConfig;
|
||||
use reth_execution_types::Chain;
|
||||
use reth_exex::{ExExContext, ExExEvent, ExExNotification, ExExNotifications, Wal};
|
||||
use reth_network::{config::rng_secret_key, NetworkConfigBuilder, NetworkManager};
|
||||
@@ -72,7 +72,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
/// A test [`ExecutorBuilder`] that builds a [`MockExecutorProvider`].
|
||||
/// A test [`ExecutorBuilder`] that builds a [`MockEvmConfig`] for testing.
|
||||
#[derive(Debug, Default, Clone, Copy)]
|
||||
#[non_exhaustive]
|
||||
pub struct TestExecutorBuilder;
|
||||
@@ -82,16 +82,10 @@ where
|
||||
Node: FullNodeTypes<Types: NodeTypes<ChainSpec = ChainSpec, Primitives = EthPrimitives>>,
|
||||
{
|
||||
type EVM = MockEvmConfig;
|
||||
type Executor = MockExecutorProvider;
|
||||
|
||||
async fn build_evm(
|
||||
self,
|
||||
_ctx: &BuilderContext<Node>,
|
||||
) -> eyre::Result<(Self::EVM, Self::Executor)> {
|
||||
async fn build_evm(self, _ctx: &BuilderContext<Node>) -> eyre::Result<Self::EVM> {
|
||||
let evm_config = MockEvmConfig::default();
|
||||
let executor = MockExecutorProvider::default();
|
||||
|
||||
Ok((evm_config, executor))
|
||||
Ok(evm_config)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -257,7 +251,6 @@ pub async fn test_exex_context_with_chain_spec(
|
||||
) -> eyre::Result<(ExExContext<Adapter>, TestExExHandle)> {
|
||||
let transaction_pool = testing_pool();
|
||||
let evm_config = MockEvmConfig::default();
|
||||
let executor = MockExecutorProvider::default();
|
||||
let consensus = Arc::new(TestConsensus::default());
|
||||
|
||||
let (static_dir, _) = create_test_static_files_dir();
|
||||
@@ -289,7 +282,6 @@ pub async fn test_exex_context_with_chain_spec(
|
||||
components: Components {
|
||||
transaction_pool,
|
||||
evm_config,
|
||||
executor,
|
||||
consensus,
|
||||
network,
|
||||
payload_builder_handle,
|
||||
@@ -314,7 +306,7 @@ pub async fn test_exex_context_with_chain_spec(
|
||||
let notifications = ExExNotifications::new(
|
||||
head,
|
||||
components.provider.clone(),
|
||||
components.components.executor.clone(),
|
||||
components.components.evm_config.clone(),
|
||||
notifications_rx,
|
||||
wal.handle(),
|
||||
);
|
||||
|
||||
@@ -6,7 +6,7 @@ use reth_basic_payload_builder::PayloadBuilder;
|
||||
use reth_consensus::{ConsensusError, FullConsensus};
|
||||
use reth_db_api::{database_metrics::DatabaseMetrics, Database};
|
||||
use reth_engine_primitives::{BeaconConsensusEngineEvent, BeaconConsensusEngineHandle};
|
||||
use reth_evm::{execute::BlockExecutorProvider, ConfigureEvm};
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_network_api::FullNetwork;
|
||||
use reth_node_core::node_config::NodeConfig;
|
||||
use reth_node_types::{NodeTypes, NodeTypesWithDBAdapter, TxTy};
|
||||
@@ -70,9 +70,6 @@ pub trait FullNodeComponents: FullNodeTypes + Clone + 'static {
|
||||
/// The node's EVM configuration, defining settings for the Ethereum Virtual Machine.
|
||||
type Evm: ConfigureEvm<Primitives = <Self::Types as NodeTypes>::Primitives>;
|
||||
|
||||
/// The type that knows how to execute blocks.
|
||||
type Executor: BlockExecutorProvider<Primitives = <Self::Types as NodeTypes>::Primitives>;
|
||||
|
||||
/// The consensus type of the node.
|
||||
type Consensus: FullConsensus<<Self::Types as NodeTypes>::Primitives, Error = ConsensusError>
|
||||
+ Clone
|
||||
@@ -88,9 +85,6 @@ pub trait FullNodeComponents: FullNodeTypes + Clone + 'static {
|
||||
/// Returns the node's evm config.
|
||||
fn evm_config(&self) -> &Self::Evm;
|
||||
|
||||
/// Returns the node's executor type.
|
||||
fn block_executor(&self) -> &Self::Executor;
|
||||
|
||||
/// Returns the node's consensus type.
|
||||
fn consensus(&self) -> &Self::Consensus;
|
||||
|
||||
|
||||
@@ -12,6 +12,7 @@ use crate::{
|
||||
rpc::{RethRpcAddOns, RethRpcServerHandles, RpcContext},
|
||||
AddOns, FullNode,
|
||||
};
|
||||
|
||||
use reth_exex::ExExContext;
|
||||
use reth_node_api::{FullNodeComponents, FullNodeTypes, NodeAddOns, NodeTypes};
|
||||
use reth_node_core::node_config::NodeConfig;
|
||||
@@ -91,7 +92,6 @@ impl<T: FullNodeTypes, C: NodeComponents<T>> FullNodeTypes for NodeAdapter<T, C>
|
||||
impl<T: FullNodeTypes, C: NodeComponents<T>> FullNodeComponents for NodeAdapter<T, C> {
|
||||
type Pool = C::Pool;
|
||||
type Evm = C::Evm;
|
||||
type Executor = C::Executor;
|
||||
type Consensus = C::Consensus;
|
||||
type Network = C::Network;
|
||||
|
||||
@@ -103,10 +103,6 @@ impl<T: FullNodeTypes, C: NodeComponents<T>> FullNodeComponents for NodeAdapter<
|
||||
self.components.evm_config()
|
||||
}
|
||||
|
||||
fn block_executor(&self) -> &Self::Executor {
|
||||
self.components.block_executor()
|
||||
}
|
||||
|
||||
fn consensus(&self) -> &Self::Consensus {
|
||||
self.components.consensus()
|
||||
}
|
||||
|
||||
@@ -8,7 +8,6 @@ use crate::{
|
||||
BuilderContext, ConfigureEvm, FullNodeTypes,
|
||||
};
|
||||
use reth_consensus::{ConsensusError, FullConsensus};
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_network::NetworkPrimitives;
|
||||
use reth_node_api::{BlockTy, BodyTy, HeaderTy, PrimitivesTy, TxTy};
|
||||
use reth_transaction_pool::{PoolTransaction, TransactionPool};
|
||||
@@ -317,14 +316,8 @@ where
|
||||
ExecB: ExecutorBuilder<Node>,
|
||||
ConsB: ConsensusBuilder<Node>,
|
||||
{
|
||||
type Components = Components<
|
||||
Node,
|
||||
NetworkB::Primitives,
|
||||
PoolB::Pool,
|
||||
ExecB::EVM,
|
||||
ExecB::Executor,
|
||||
ConsB::Consensus,
|
||||
>;
|
||||
type Components =
|
||||
Components<Node, NetworkB::Primitives, PoolB::Pool, ExecB::EVM, ConsB::Consensus>;
|
||||
|
||||
async fn build_components(
|
||||
self,
|
||||
@@ -339,7 +332,7 @@ where
|
||||
_marker,
|
||||
} = self;
|
||||
|
||||
let (evm_config, executor) = evm_builder.build_evm(context).await?;
|
||||
let evm_config = evm_builder.build_evm(context).await?;
|
||||
let pool = pool_builder.build_pool(context).await?;
|
||||
let network = network_builder.build_network(context, pool.clone()).await?;
|
||||
let payload_builder_handle = payload_builder
|
||||
@@ -352,7 +345,6 @@ where
|
||||
evm_config,
|
||||
network,
|
||||
payload_builder_handle,
|
||||
executor,
|
||||
consensus,
|
||||
})
|
||||
}
|
||||
@@ -391,7 +383,7 @@ pub trait NodeComponentsBuilder<Node: FullNodeTypes>: Send {
|
||||
) -> impl Future<Output = eyre::Result<Self::Components>> + Send;
|
||||
}
|
||||
|
||||
impl<Node, N, F, Fut, Pool, EVM, Executor, Cons> NodeComponentsBuilder<Node> for F
|
||||
impl<Node, N, F, Fut, Pool, EVM, Cons> NodeComponentsBuilder<Node> for F
|
||||
where
|
||||
N: NetworkPrimitives<
|
||||
BlockHeader = HeaderTy<Node::Types>,
|
||||
@@ -400,16 +392,15 @@ where
|
||||
>,
|
||||
Node: FullNodeTypes,
|
||||
F: FnOnce(&BuilderContext<Node>) -> Fut + Send,
|
||||
Fut: Future<Output = eyre::Result<Components<Node, N, Pool, EVM, Executor, Cons>>> + Send,
|
||||
Fut: Future<Output = eyre::Result<Components<Node, N, Pool, EVM, Cons>>> + Send,
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
EVM: ConfigureEvm<Primitives = PrimitivesTy<Node::Types>> + 'static,
|
||||
Executor: BlockExecutorProvider<Primitives = PrimitivesTy<Node::Types>>,
|
||||
Cons:
|
||||
FullConsensus<PrimitivesTy<Node::Types>, Error = ConsensusError> + Clone + Unpin + 'static,
|
||||
{
|
||||
type Components = Components<Node, N, Pool, EVM, Executor, Cons>;
|
||||
type Components = Components<Node, N, Pool, EVM, Cons>;
|
||||
|
||||
fn build_components(
|
||||
self,
|
||||
|
||||
@@ -1,6 +1,5 @@
|
||||
//! EVM component for the node builder.
|
||||
use crate::{BuilderContext, ConfigureEvm, FullNodeTypes};
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_node_api::PrimitivesTy;
|
||||
use std::future::Future;
|
||||
|
||||
@@ -11,31 +10,26 @@ pub trait ExecutorBuilder<Node: FullNodeTypes>: Send {
|
||||
/// This provides the node with the necessary configuration to configure an EVM.
|
||||
type EVM: ConfigureEvm<Primitives = PrimitivesTy<Node::Types>> + 'static;
|
||||
|
||||
/// The type that knows how to execute blocks.
|
||||
type Executor: BlockExecutorProvider<Primitives = PrimitivesTy<Node::Types>>;
|
||||
|
||||
/// Creates the EVM config.
|
||||
fn build_evm(
|
||||
self,
|
||||
ctx: &BuilderContext<Node>,
|
||||
) -> impl Future<Output = eyre::Result<(Self::EVM, Self::Executor)>> + Send;
|
||||
) -> impl Future<Output = eyre::Result<Self::EVM>> + Send;
|
||||
}
|
||||
|
||||
impl<Node, F, Fut, EVM, Executor> ExecutorBuilder<Node> for F
|
||||
impl<Node, F, Fut, EVM> ExecutorBuilder<Node> for F
|
||||
where
|
||||
Node: FullNodeTypes,
|
||||
EVM: ConfigureEvm<Primitives = PrimitivesTy<Node::Types>> + 'static,
|
||||
Executor: BlockExecutorProvider<Primitives = PrimitivesTy<Node::Types>>,
|
||||
F: FnOnce(&BuilderContext<Node>) -> Fut + Send,
|
||||
Fut: Future<Output = eyre::Result<(EVM, Executor)>> + Send,
|
||||
Fut: Future<Output = eyre::Result<EVM>> + Send,
|
||||
{
|
||||
type EVM = EVM;
|
||||
type Executor = Executor;
|
||||
|
||||
fn build_evm(
|
||||
self,
|
||||
ctx: &BuilderContext<Node>,
|
||||
) -> impl Future<Output = eyre::Result<(Self::EVM, Self::Executor)>> {
|
||||
) -> impl Future<Output = eyre::Result<Self::EVM>> {
|
||||
self(ctx)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,13 +20,13 @@ pub use execute::*;
|
||||
pub use network::*;
|
||||
pub use payload::*;
|
||||
pub use pool::*;
|
||||
|
||||
use reth_network_p2p::BlockClient;
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use std::fmt::Debug;
|
||||
|
||||
use crate::{ConfigureEvm, FullNodeTypes};
|
||||
use reth_consensus::{ConsensusError, FullConsensus};
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_network::{NetworkHandle, NetworkPrimitives};
|
||||
use reth_network_api::FullNetwork;
|
||||
use reth_node_api::{BlockTy, BodyTy, HeaderTy, NodeTypes, PrimitivesTy, TxTy};
|
||||
@@ -44,9 +44,6 @@ pub trait NodeComponents<T: FullNodeTypes>: Clone + Debug + Unpin + Send + Sync
|
||||
/// The node's EVM configuration, defining settings for the Ethereum Virtual Machine.
|
||||
type Evm: ConfigureEvm<Primitives = <T::Types as NodeTypes>::Primitives>;
|
||||
|
||||
/// The type that knows how to execute blocks.
|
||||
type Executor: BlockExecutorProvider<Primitives = <T::Types as NodeTypes>::Primitives>;
|
||||
|
||||
/// The consensus type of the node.
|
||||
type Consensus: FullConsensus<<T::Types as NodeTypes>::Primitives, Error = ConsensusError>
|
||||
+ Clone
|
||||
@@ -62,9 +59,6 @@ pub trait NodeComponents<T: FullNodeTypes>: Clone + Debug + Unpin + Send + Sync
|
||||
/// Returns the node's evm config.
|
||||
fn evm_config(&self) -> &Self::Evm;
|
||||
|
||||
/// Returns the node's executor type.
|
||||
fn block_executor(&self) -> &Self::Executor;
|
||||
|
||||
/// Returns the node's consensus type.
|
||||
fn consensus(&self) -> &Self::Consensus;
|
||||
|
||||
@@ -80,13 +74,11 @@ pub trait NodeComponents<T: FullNodeTypes>: Clone + Debug + Unpin + Send + Sync
|
||||
///
|
||||
/// This provides access to all the components of the node.
|
||||
#[derive(Debug)]
|
||||
pub struct Components<Node: FullNodeTypes, N: NetworkPrimitives, Pool, EVM, Executor, Consensus> {
|
||||
pub struct Components<Node: FullNodeTypes, N: NetworkPrimitives, Pool, EVM, Consensus> {
|
||||
/// The transaction pool of the node.
|
||||
pub transaction_pool: Pool,
|
||||
/// The node's EVM configuration, defining settings for the Ethereum Virtual Machine.
|
||||
pub evm_config: EVM,
|
||||
/// The node's executor type used to execute individual blocks and batches of blocks.
|
||||
pub executor: Executor,
|
||||
/// The consensus implementation of the node.
|
||||
pub consensus: Consensus,
|
||||
/// The network implementation of the node.
|
||||
@@ -95,8 +87,7 @@ pub struct Components<Node: FullNodeTypes, N: NetworkPrimitives, Pool, EVM, Exec
|
||||
pub payload_builder_handle: PayloadBuilderHandle<<Node::Types as NodeTypes>::Payload>,
|
||||
}
|
||||
|
||||
impl<Node, Pool, EVM, Executor, Cons, N> NodeComponents<Node>
|
||||
for Components<Node, N, Pool, EVM, Executor, Cons>
|
||||
impl<Node, Pool, EVM, Cons, N> NodeComponents<Node> for Components<Node, N, Pool, EVM, Cons>
|
||||
where
|
||||
Node: FullNodeTypes,
|
||||
N: NetworkPrimitives<
|
||||
@@ -108,13 +99,11 @@ where
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
EVM: ConfigureEvm<Primitives = PrimitivesTy<Node::Types>> + 'static,
|
||||
Executor: BlockExecutorProvider<Primitives = PrimitivesTy<Node::Types>>,
|
||||
Cons:
|
||||
FullConsensus<PrimitivesTy<Node::Types>, Error = ConsensusError> + Clone + Unpin + 'static,
|
||||
{
|
||||
type Pool = Pool;
|
||||
type Evm = EVM;
|
||||
type Executor = Executor;
|
||||
type Consensus = Cons;
|
||||
type Network = NetworkHandle<N>;
|
||||
|
||||
@@ -126,10 +115,6 @@ where
|
||||
&self.evm_config
|
||||
}
|
||||
|
||||
fn block_executor(&self) -> &Self::Executor {
|
||||
&self.executor
|
||||
}
|
||||
|
||||
fn consensus(&self) -> &Self::Consensus {
|
||||
&self.consensus
|
||||
}
|
||||
@@ -143,20 +128,18 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<Node, N, Pool, EVM, Executor, Cons> Clone for Components<Node, N, Pool, EVM, Executor, Cons>
|
||||
impl<Node, N, Pool, EVM, Cons> Clone for Components<Node, N, Pool, EVM, Cons>
|
||||
where
|
||||
N: NetworkPrimitives,
|
||||
Node: FullNodeTypes,
|
||||
Pool: TransactionPool,
|
||||
EVM: ConfigureEvm,
|
||||
Executor: BlockExecutorProvider,
|
||||
Cons: Clone,
|
||||
{
|
||||
fn clone(&self) -> Self {
|
||||
Self {
|
||||
transaction_pool: self.transaction_pool.clone(),
|
||||
evm_config: self.evm_config.clone(),
|
||||
executor: self.executor.clone(),
|
||||
consensus: self.consensus.clone(),
|
||||
network: self.network.clone(),
|
||||
payload_builder_handle: self.payload_builder_handle.clone(),
|
||||
|
||||
@@ -17,7 +17,7 @@ use reth_db_common::init::{init_genesis, InitStorageError};
|
||||
use reth_downloaders::{bodies::noop::NoopBodiesDownloader, headers::noop::NoopHeaderDownloader};
|
||||
use reth_engine_local::MiningMode;
|
||||
use reth_engine_tree::tree::{InvalidBlockHook, InvalidBlockHooks, NoopInvalidBlockHook};
|
||||
use reth_evm::{execute::BasicBlockExecutorProvider, noop::NoopEvmConfig, ConfigureEvm};
|
||||
use reth_evm::{noop::NoopEvmConfig, ConfigureEvm};
|
||||
use reth_fs_util as fs;
|
||||
use reth_invalid_block_hooks::InvalidBlockWitnessHook;
|
||||
use reth_network_p2p::headers::client::HeadersClient;
|
||||
@@ -423,7 +423,7 @@ where
|
||||
Arc::new(NoopConsensus::default()),
|
||||
NoopHeaderDownloader::default(),
|
||||
NoopBodiesDownloader::default(),
|
||||
BasicBlockExecutorProvider::new(NoopEvmConfig::<Evm>::default()),
|
||||
NoopEvmConfig::<Evm>::default(),
|
||||
self.toml_config().stages.clone(),
|
||||
self.prune_modes(),
|
||||
))
|
||||
@@ -905,7 +905,7 @@ where
|
||||
Ok(match hook {
|
||||
InvalidBlockHookType::Witness => Box::new(InvalidBlockWitnessHook::new(
|
||||
self.blockchain_db().clone(),
|
||||
self.components().block_executor().clone(),
|
||||
self.components().evm_config().clone(),
|
||||
output_directory,
|
||||
healthy_node_rpc_client.clone(),
|
||||
)),
|
||||
|
||||
@@ -163,7 +163,7 @@ where
|
||||
ctx.prune_config(),
|
||||
max_block,
|
||||
static_file_producer,
|
||||
ctx.components().block_executor().clone(),
|
||||
ctx.components().evm_config().clone(),
|
||||
pipeline_exex_handle,
|
||||
)?;
|
||||
|
||||
@@ -362,7 +362,6 @@ where
|
||||
|
||||
let full_node = FullNode {
|
||||
evm_config: ctx.components().evm_config().clone(),
|
||||
block_executor: ctx.components().block_executor().clone(),
|
||||
pool: ctx.components().pool().clone(),
|
||||
network: ctx.components().network().clone(),
|
||||
provider: ctx.node_adapter().provider.clone(),
|
||||
|
||||
@@ -69,7 +69,7 @@ impl<Node: FullNodeComponents + Clone> ExExLauncher<Node> {
|
||||
id.clone(),
|
||||
head,
|
||||
components.provider().clone(),
|
||||
components.block_executor().clone(),
|
||||
components.evm_config().clone(),
|
||||
exex_wal.handle(),
|
||||
);
|
||||
exex_handles.push(handle);
|
||||
|
||||
@@ -1,12 +1,13 @@
|
||||
// re-export the node api types
|
||||
pub use reth_node_api::{FullNodeTypes, NodeTypes};
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
|
||||
use crate::{components::NodeComponentsBuilder, rpc::RethRpcAddOns, NodeAdapter, NodeAddOns};
|
||||
use reth_node_api::{EngineTypes, FullNodeComponents, PayloadTypes};
|
||||
use reth_node_core::{
|
||||
dirs::{ChainPath, DataDirPath},
|
||||
node_config::NodeConfig,
|
||||
};
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_provider::ChainSpecProvider;
|
||||
use reth_rpc_api::EngineApiClient;
|
||||
use reth_rpc_builder::{auth::AuthServerHandle, RpcServerHandle};
|
||||
@@ -18,8 +19,6 @@ use std::{
|
||||
sync::Arc,
|
||||
};
|
||||
|
||||
use crate::{components::NodeComponentsBuilder, rpc::RethRpcAddOns, NodeAdapter, NodeAddOns};
|
||||
|
||||
/// A [`crate::Node`] is a [`NodeTypes`] that comes with preconfigured components.
|
||||
///
|
||||
/// This can be used to configure the builder with a preset of components.
|
||||
@@ -102,8 +101,6 @@ where
|
||||
pub struct FullNode<Node: FullNodeComponents, AddOns: NodeAddOns<Node>> {
|
||||
/// The evm configuration.
|
||||
pub evm_config: Node::Evm,
|
||||
/// The executor of the node.
|
||||
pub block_executor: Node::Executor,
|
||||
/// The node's transaction pool.
|
||||
pub pool: Node::Pool,
|
||||
/// Handle to the node's network.
|
||||
@@ -126,7 +123,6 @@ impl<Node: FullNodeComponents, AddOns: NodeAddOns<Node>> Clone for FullNode<Node
|
||||
fn clone(&self) -> Self {
|
||||
Self {
|
||||
evm_config: self.evm_config.clone(),
|
||||
block_executor: self.block_executor.clone(),
|
||||
pool: self.pool.clone(),
|
||||
network: self.network.clone(),
|
||||
provider: self.provider.clone(),
|
||||
|
||||
@@ -195,7 +195,7 @@ pub struct RpcRegistry<Node: FullNodeComponents, EthApi: EthApiTypes> {
|
||||
Node::Network,
|
||||
TaskExecutor,
|
||||
EthApi,
|
||||
Node::Executor,
|
||||
Node::Evm,
|
||||
Node::Consensus,
|
||||
>,
|
||||
}
|
||||
@@ -211,7 +211,7 @@ where
|
||||
Node::Network,
|
||||
TaskExecutor,
|
||||
EthApi,
|
||||
Node::Executor,
|
||||
Node::Evm,
|
||||
Node::Consensus,
|
||||
>;
|
||||
|
||||
@@ -497,7 +497,6 @@ where
|
||||
.with_network(node.network().clone())
|
||||
.with_executor(node.task_executor().clone())
|
||||
.with_evm_config(node.evm_config().clone())
|
||||
.with_block_executor(node.block_executor().clone())
|
||||
.with_consensus(node.consensus().clone())
|
||||
.build_with_auth_server(module_config, engine_api, eth_api);
|
||||
|
||||
|
||||
@@ -10,7 +10,7 @@ use reth_downloaders::{
|
||||
bodies::bodies::BodiesDownloaderBuilder,
|
||||
headers::reverse_headers::ReverseHeadersDownloaderBuilder,
|
||||
};
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_exex::ExExManagerHandle;
|
||||
use reth_network_p2p::{
|
||||
bodies::downloader::BodyDownloader, headers::downloader::HeaderDownloader, BlockClient,
|
||||
@@ -25,7 +25,7 @@ use tokio::sync::watch;
|
||||
|
||||
/// Constructs a [Pipeline] that's wired to the network
|
||||
#[expect(clippy::too_many_arguments)]
|
||||
pub fn build_networked_pipeline<N, Client, Executor>(
|
||||
pub fn build_networked_pipeline<N, Client, Evm>(
|
||||
config: &StageConfig,
|
||||
client: Client,
|
||||
consensus: Arc<dyn FullConsensus<N::Primitives, Error = ConsensusError>>,
|
||||
@@ -35,13 +35,13 @@ pub fn build_networked_pipeline<N, Client, Executor>(
|
||||
prune_config: Option<PruneConfig>,
|
||||
max_block: Option<BlockNumber>,
|
||||
static_file_producer: StaticFileProducer<ProviderFactory<N>>,
|
||||
executor: Executor,
|
||||
evm_config: Evm,
|
||||
exex_manager_handle: ExExManagerHandle<N::Primitives>,
|
||||
) -> eyre::Result<Pipeline<N>>
|
||||
where
|
||||
N: ProviderNodeTypes,
|
||||
Client: BlockClient<Block = BlockTy<N>> + 'static,
|
||||
Executor: BlockExecutorProvider<Primitives = N::Primitives>,
|
||||
Evm: ConfigureEvm<Primitives = N::Primitives> + 'static,
|
||||
{
|
||||
// building network downloaders using the fetch client
|
||||
let header_downloader = ReverseHeadersDownloaderBuilder::new(config.headers)
|
||||
@@ -62,7 +62,7 @@ where
|
||||
metrics_tx,
|
||||
prune_config,
|
||||
static_file_producer,
|
||||
executor,
|
||||
evm_config,
|
||||
exex_manager_handle,
|
||||
)?;
|
||||
|
||||
@@ -71,7 +71,7 @@ where
|
||||
|
||||
/// Builds the [Pipeline] with the given [`ProviderFactory`] and downloaders.
|
||||
#[expect(clippy::too_many_arguments)]
|
||||
pub fn build_pipeline<N, H, B, Executor>(
|
||||
pub fn build_pipeline<N, H, B, Evm>(
|
||||
provider_factory: ProviderFactory<N>,
|
||||
stage_config: &StageConfig,
|
||||
header_downloader: H,
|
||||
@@ -81,14 +81,14 @@ pub fn build_pipeline<N, H, B, Executor>(
|
||||
metrics_tx: reth_stages::MetricEventsSender,
|
||||
prune_config: Option<PruneConfig>,
|
||||
static_file_producer: StaticFileProducer<ProviderFactory<N>>,
|
||||
executor: Executor,
|
||||
evm_config: Evm,
|
||||
exex_manager_handle: ExExManagerHandle<N::Primitives>,
|
||||
) -> eyre::Result<Pipeline<N>>
|
||||
where
|
||||
N: ProviderNodeTypes,
|
||||
H: HeaderDownloader<Header = HeaderTy<N>> + 'static,
|
||||
B: BodyDownloader<Block = BlockTy<N>> + 'static,
|
||||
Executor: BlockExecutorProvider<Primitives = N::Primitives>,
|
||||
Evm: ConfigureEvm<Primitives = N::Primitives> + 'static,
|
||||
{
|
||||
let mut builder = Pipeline::<N>::builder();
|
||||
|
||||
@@ -111,12 +111,12 @@ where
|
||||
Arc::clone(&consensus),
|
||||
header_downloader,
|
||||
body_downloader,
|
||||
executor.clone(),
|
||||
evm_config.clone(),
|
||||
stage_config.clone(),
|
||||
prune_modes,
|
||||
)
|
||||
.set(ExecutionStage::new(
|
||||
executor,
|
||||
evm_config,
|
||||
consensus,
|
||||
stage_config.execution.into(),
|
||||
stage_config.execution_external_clean_threshold(),
|
||||
|
||||
@@ -1,34 +1,18 @@
|
||||
//! Optimism block execution strategy.
|
||||
|
||||
use crate::{OpEvmConfig, OpRethReceiptBuilder};
|
||||
use alloc::sync::Arc;
|
||||
use reth_evm::execute::BasicBlockExecutorProvider;
|
||||
use reth_optimism_chainspec::OpChainSpec;
|
||||
|
||||
/// Helper type with backwards compatible methods to obtain executor providers.
|
||||
#[derive(Debug)]
|
||||
pub struct OpExecutorProvider;
|
||||
|
||||
impl OpExecutorProvider {
|
||||
/// Creates a new default optimism executor strategy factory.
|
||||
pub fn optimism(chain_spec: Arc<OpChainSpec>) -> BasicBlockExecutorProvider<OpEvmConfig> {
|
||||
BasicBlockExecutorProvider::new(OpEvmConfig::new(
|
||||
chain_spec,
|
||||
OpRethReceiptBuilder::default(),
|
||||
))
|
||||
}
|
||||
}
|
||||
pub type OpExecutorProvider = crate::OpEvmConfig;
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::OpChainSpec;
|
||||
use crate::{OpChainSpec, OpEvmConfig, OpRethReceiptBuilder};
|
||||
use alloc::sync::Arc;
|
||||
use alloy_consensus::{Block, BlockBody, Header, SignableTransaction, TxEip1559};
|
||||
use alloy_primitives::{b256, Address, Signature, StorageKey, StorageValue, U256};
|
||||
use op_alloy_consensus::TxDeposit;
|
||||
use op_revm::constants::L1_BLOCK_CONTRACT;
|
||||
use reth_chainspec::MIN_TRANSACTION_GAS;
|
||||
use reth_evm::execute::{BasicBlockExecutorProvider, BlockExecutorProvider, Executor};
|
||||
use reth_evm::{execute::Executor, ConfigureEvm};
|
||||
use reth_optimism_chainspec::OpChainSpecBuilder;
|
||||
use reth_optimism_primitives::{OpReceipt, OpTransactionSigned};
|
||||
use reth_primitives_traits::{Account, RecoveredBlock};
|
||||
@@ -62,11 +46,8 @@ mod tests {
|
||||
db
|
||||
}
|
||||
|
||||
fn executor_provider(chain_spec: Arc<OpChainSpec>) -> BasicBlockExecutorProvider<OpEvmConfig> {
|
||||
BasicBlockExecutorProvider::new(OpEvmConfig::new(
|
||||
chain_spec,
|
||||
OpRethReceiptBuilder::default(),
|
||||
))
|
||||
fn evm_config(chain_spec: Arc<OpChainSpec>) -> OpEvmConfig {
|
||||
OpEvmConfig::new(chain_spec, OpRethReceiptBuilder::default())
|
||||
}
|
||||
|
||||
#[test]
|
||||
@@ -108,8 +89,8 @@ mod tests {
|
||||
}
|
||||
.into();
|
||||
|
||||
let provider = executor_provider(chain_spec);
|
||||
let mut executor = provider.executor(StateProviderDatabase::new(&db));
|
||||
let provider = evm_config(chain_spec);
|
||||
let mut executor = provider.batch_executor(StateProviderDatabase::new(&db));
|
||||
|
||||
// make sure the L1 block contract state is preloaded.
|
||||
executor.with_state_mut(|state| {
|
||||
@@ -181,8 +162,8 @@ mod tests {
|
||||
}
|
||||
.into();
|
||||
|
||||
let provider = executor_provider(chain_spec);
|
||||
let mut executor = provider.executor(StateProviderDatabase::new(&db));
|
||||
let provider = evm_config(chain_spec);
|
||||
let mut executor = provider.batch_executor(StateProviderDatabase::new(&db));
|
||||
|
||||
// make sure the L1 block contract state is preloaded.
|
||||
executor.with_state_mut(|state| {
|
||||
|
||||
@@ -8,7 +8,7 @@ use crate::{
|
||||
};
|
||||
use op_alloy_consensus::{interop::SafetyLevel, OpPooledTransaction};
|
||||
use reth_chainspec::{EthChainSpec, Hardforks};
|
||||
use reth_evm::{execute::BasicBlockExecutorProvider, ConfigureEvm, EvmFactory, EvmFactoryFor};
|
||||
use reth_evm::{ConfigureEvm, EvmFactory, EvmFactoryFor};
|
||||
use reth_network::{NetworkConfig, NetworkHandle, NetworkManager, NetworkPrimitives, PeersInfo};
|
||||
use reth_node_api::{
|
||||
AddOnsContext, FullNodeComponents, KeyHasherTy, NodeAddOns, NodePrimitives, PrimitivesTy, TxTy,
|
||||
@@ -469,16 +469,11 @@ where
|
||||
Node: FullNodeTypes<Types: NodeTypes<ChainSpec = OpChainSpec, Primitives = OpPrimitives>>,
|
||||
{
|
||||
type EVM = OpEvmConfig;
|
||||
type Executor = BasicBlockExecutorProvider<Self::EVM>;
|
||||
|
||||
async fn build_evm(
|
||||
self,
|
||||
ctx: &BuilderContext<Node>,
|
||||
) -> eyre::Result<(Self::EVM, Self::Executor)> {
|
||||
async fn build_evm(self, ctx: &BuilderContext<Node>) -> eyre::Result<Self::EVM> {
|
||||
let evm_config = OpEvmConfig::optimism(ctx.chain_spec());
|
||||
let executor = BasicBlockExecutorProvider::new(evm_config.clone());
|
||||
|
||||
Ok((evm_config, executor))
|
||||
Ok(evm_config)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -7,7 +7,7 @@ use alloy_primitives::{Bytes, B256};
|
||||
use parking_lot::Mutex;
|
||||
use reth_chain_state::{ExecutedBlock, ExecutedBlockWithTrieUpdates, MemoryOverlayStateProvider};
|
||||
use reth_ethereum_primitives::{Block, BlockBody, EthPrimitives};
|
||||
use reth_evm::execute::{BlockExecutorProvider, Executor};
|
||||
use reth_evm::{execute::Executor, ConfigureEvm};
|
||||
use reth_primitives_traits::{Block as _, Header, RecoveredBlock};
|
||||
use reth_provider::{
|
||||
BlockReader, BlockSource, ProviderError, ProviderResult, StateProvider, StateProviderFactory,
|
||||
@@ -29,9 +29,10 @@ pub use pending_state::*;
|
||||
|
||||
/// Reth provider implementing [`RessProtocolProvider`].
|
||||
#[expect(missing_debug_implementations)]
|
||||
#[derive(Clone)]
|
||||
pub struct RethRessProtocolProvider<P, E> {
|
||||
provider: P,
|
||||
block_executor: E,
|
||||
evm_config: E,
|
||||
task_spawner: Box<dyn TaskSpawner>,
|
||||
max_witness_window: u64,
|
||||
witness_semaphore: Arc<Semaphore>,
|
||||
@@ -39,29 +40,15 @@ pub struct RethRessProtocolProvider<P, E> {
|
||||
pending_state: PendingState<EthPrimitives>,
|
||||
}
|
||||
|
||||
impl<P: Clone, E: Clone> Clone for RethRessProtocolProvider<P, E> {
|
||||
fn clone(&self) -> Self {
|
||||
Self {
|
||||
provider: self.provider.clone(),
|
||||
block_executor: self.block_executor.clone(),
|
||||
task_spawner: self.task_spawner.clone(),
|
||||
max_witness_window: self.max_witness_window,
|
||||
witness_semaphore: self.witness_semaphore.clone(),
|
||||
witness_cache: self.witness_cache.clone(),
|
||||
pending_state: self.pending_state.clone(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<P, E> RethRessProtocolProvider<P, E>
|
||||
where
|
||||
P: BlockReader<Block = Block> + StateProviderFactory,
|
||||
E: BlockExecutorProvider<Primitives = EthPrimitives> + Clone,
|
||||
E: ConfigureEvm<Primitives = EthPrimitives> + 'static,
|
||||
{
|
||||
/// Create new ress protocol provider.
|
||||
pub fn new(
|
||||
provider: P,
|
||||
block_executor: E,
|
||||
evm_config: E,
|
||||
task_spawner: Box<dyn TaskSpawner>,
|
||||
max_witness_window: u64,
|
||||
witness_max_parallel: usize,
|
||||
@@ -70,7 +57,7 @@ where
|
||||
) -> eyre::Result<Self> {
|
||||
Ok(Self {
|
||||
provider,
|
||||
block_executor,
|
||||
evm_config,
|
||||
task_spawner,
|
||||
max_witness_window,
|
||||
witness_semaphore: Arc::new(Semaphore::new(witness_max_parallel)),
|
||||
@@ -158,7 +145,7 @@ where
|
||||
|
||||
// We allow block execution to fail, since we still want to record all accessed state by
|
||||
// invalid blocks.
|
||||
if let Err(error) = self.block_executor.executor(&mut db).execute_with_state_closure(
|
||||
if let Err(error) = self.evm_config.batch_executor(&mut db).execute_with_state_closure(
|
||||
&block,
|
||||
|state: &State<_>| {
|
||||
record.record_executed_state(state);
|
||||
@@ -205,7 +192,7 @@ where
|
||||
impl<P, E> RessProtocolProvider for RethRessProtocolProvider<P, E>
|
||||
where
|
||||
P: BlockReader<Block = Block> + StateProviderFactory + Clone + 'static,
|
||||
E: BlockExecutorProvider<Primitives = EthPrimitives> + Clone,
|
||||
E: ConfigureEvm<Primitives = EthPrimitives> + 'static,
|
||||
{
|
||||
fn header(&self, block_hash: B256) -> ProviderResult<Option<Header>> {
|
||||
trace!(target: "reth::ress_provider", %block_hash, "Serving header");
|
||||
|
||||
@@ -34,7 +34,7 @@ use jsonrpsee::{
|
||||
};
|
||||
use reth_chainspec::EthereumHardforks;
|
||||
use reth_consensus::{ConsensusError, FullConsensus};
|
||||
use reth_evm::{execute::BlockExecutorProvider, ConfigureEvm};
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_network_api::{noop::NoopNetwork, NetworkInfo, Peers};
|
||||
use reth_primitives_traits::NodePrimitives;
|
||||
use reth_provider::{
|
||||
@@ -101,7 +101,7 @@ pub mod rate_limiter;
|
||||
|
||||
/// Convenience function for starting a server in one step.
|
||||
#[expect(clippy::too_many_arguments)]
|
||||
pub async fn launch<N, Provider, Pool, Network, Tasks, EvmConfig, EthApi, BlockExecutor>(
|
||||
pub async fn launch<N, Provider, Pool, Network, Tasks, EvmConfig, EthApi>(
|
||||
provider: Provider,
|
||||
pool: Pool,
|
||||
network: Network,
|
||||
@@ -110,7 +110,6 @@ pub async fn launch<N, Provider, Pool, Network, Tasks, EvmConfig, EthApi, BlockE
|
||||
executor: Tasks,
|
||||
evm_config: EvmConfig,
|
||||
eth: EthApi,
|
||||
block_executor: BlockExecutor,
|
||||
consensus: Arc<dyn FullConsensus<N, Error = ConsensusError>>,
|
||||
) -> Result<RpcServerHandle, RpcError>
|
||||
where
|
||||
@@ -122,24 +121,15 @@ where
|
||||
Pool: TransactionPool + 'static,
|
||||
Network: NetworkInfo + Peers + Clone + 'static,
|
||||
Tasks: TaskSpawner + Clone + 'static,
|
||||
EvmConfig: ConfigureEvm<Primitives = N>,
|
||||
EvmConfig: ConfigureEvm<Primitives = N> + 'static,
|
||||
EthApi: FullEthApiServer<Provider = Provider, Pool = Pool>,
|
||||
BlockExecutor: BlockExecutorProvider<Primitives = N>,
|
||||
{
|
||||
let module_config = module_config.into();
|
||||
server_config
|
||||
.into()
|
||||
.start(
|
||||
&RpcModuleBuilder::new(
|
||||
provider,
|
||||
pool,
|
||||
network,
|
||||
executor,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
)
|
||||
.build(module_config, eth),
|
||||
&RpcModuleBuilder::new(provider, pool, network, executor, evm_config, consensus)
|
||||
.build(module_config, eth),
|
||||
)
|
||||
.await
|
||||
}
|
||||
@@ -148,8 +138,7 @@ where
|
||||
///
|
||||
/// This is the main entrypoint and the easiest way to configure an RPC server.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct RpcModuleBuilder<N, Provider, Pool, Network, Tasks, EvmConfig, BlockExecutor, Consensus>
|
||||
{
|
||||
pub struct RpcModuleBuilder<N, Provider, Pool, Network, Tasks, EvmConfig, Consensus> {
|
||||
/// The Provider type to when creating all rpc handlers
|
||||
provider: Provider,
|
||||
/// The Pool type to when creating all rpc handlers
|
||||
@@ -160,8 +149,6 @@ pub struct RpcModuleBuilder<N, Provider, Pool, Network, Tasks, EvmConfig, BlockE
|
||||
executor: Tasks,
|
||||
/// Defines how the EVM should be configured before execution.
|
||||
evm_config: EvmConfig,
|
||||
/// The provider for getting a block executor that executes blocks
|
||||
block_executor: BlockExecutor,
|
||||
/// The consensus implementation.
|
||||
consensus: Consensus,
|
||||
/// Node data primitives.
|
||||
@@ -170,10 +157,11 @@ pub struct RpcModuleBuilder<N, Provider, Pool, Network, Tasks, EvmConfig, BlockE
|
||||
|
||||
// === impl RpcBuilder ===
|
||||
|
||||
impl<N, Provider, Pool, Network, Tasks, EvmConfig, BlockExecutor, Consensus>
|
||||
RpcModuleBuilder<N, Provider, Pool, Network, Tasks, EvmConfig, BlockExecutor, Consensus>
|
||||
impl<N, Provider, Pool, Network, Tasks, EvmConfig, Consensus>
|
||||
RpcModuleBuilder<N, Provider, Pool, Network, Tasks, EvmConfig, Consensus>
|
||||
where
|
||||
N: NodePrimitives,
|
||||
EvmConfig: Clone,
|
||||
{
|
||||
/// Create a new instance of the builder
|
||||
pub const fn new(
|
||||
@@ -182,81 +170,35 @@ where
|
||||
network: Network,
|
||||
executor: Tasks,
|
||||
evm_config: EvmConfig,
|
||||
block_executor: BlockExecutor,
|
||||
consensus: Consensus,
|
||||
) -> Self {
|
||||
Self {
|
||||
provider,
|
||||
pool,
|
||||
network,
|
||||
executor,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives: PhantomData,
|
||||
}
|
||||
Self { provider, pool, network, executor, evm_config, consensus, _primitives: PhantomData }
|
||||
}
|
||||
|
||||
/// Configure the provider instance.
|
||||
pub fn with_provider<P>(
|
||||
self,
|
||||
provider: P,
|
||||
) -> RpcModuleBuilder<N, P, Pool, Network, Tasks, EvmConfig, BlockExecutor, Consensus>
|
||||
) -> RpcModuleBuilder<N, P, Pool, Network, Tasks, EvmConfig, Consensus>
|
||||
where
|
||||
P: BlockReader<Block = N::Block, Header = N::BlockHeader, Receipt = N::Receipt>
|
||||
+ StateProviderFactory
|
||||
+ 'static,
|
||||
{
|
||||
let Self {
|
||||
pool,
|
||||
network,
|
||||
executor,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
..
|
||||
} = self;
|
||||
RpcModuleBuilder {
|
||||
provider,
|
||||
network,
|
||||
pool,
|
||||
executor,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
}
|
||||
let Self { pool, network, executor, evm_config, consensus, _primitives, .. } = self;
|
||||
RpcModuleBuilder { provider, network, pool, executor, evm_config, consensus, _primitives }
|
||||
}
|
||||
|
||||
/// Configure the transaction pool instance.
|
||||
pub fn with_pool<P>(
|
||||
self,
|
||||
pool: P,
|
||||
) -> RpcModuleBuilder<N, Provider, P, Network, Tasks, EvmConfig, BlockExecutor, Consensus>
|
||||
) -> RpcModuleBuilder<N, Provider, P, Network, Tasks, EvmConfig, Consensus>
|
||||
where
|
||||
P: TransactionPool<Transaction: PoolTransaction<Consensus = N::SignedTx>> + 'static,
|
||||
{
|
||||
let Self {
|
||||
provider,
|
||||
network,
|
||||
executor,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
..
|
||||
} = self;
|
||||
RpcModuleBuilder {
|
||||
provider,
|
||||
network,
|
||||
pool,
|
||||
executor,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
}
|
||||
let Self { provider, network, executor, evm_config, consensus, _primitives, .. } = self;
|
||||
RpcModuleBuilder { provider, network, pool, executor, evm_config, consensus, _primitives }
|
||||
}
|
||||
|
||||
/// Configure a [`NoopTransactionPool`] instance.
|
||||
@@ -266,32 +208,14 @@ where
|
||||
/// [`EthApi`] which requires a [`TransactionPool`] implementation.
|
||||
pub fn with_noop_pool(
|
||||
self,
|
||||
) -> RpcModuleBuilder<
|
||||
N,
|
||||
Provider,
|
||||
NoopTransactionPool,
|
||||
Network,
|
||||
Tasks,
|
||||
EvmConfig,
|
||||
BlockExecutor,
|
||||
Consensus,
|
||||
> {
|
||||
let Self {
|
||||
provider,
|
||||
executor,
|
||||
network,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
..
|
||||
} = self;
|
||||
) -> RpcModuleBuilder<N, Provider, NoopTransactionPool, Network, Tasks, EvmConfig, Consensus>
|
||||
{
|
||||
let Self { provider, executor, network, evm_config, consensus, _primitives, .. } = self;
|
||||
RpcModuleBuilder {
|
||||
provider,
|
||||
executor,
|
||||
network,
|
||||
evm_config,
|
||||
block_executor,
|
||||
pool: NoopTransactionPool::default(),
|
||||
consensus,
|
||||
_primitives,
|
||||
@@ -302,30 +226,12 @@ where
|
||||
pub fn with_network<Net>(
|
||||
self,
|
||||
network: Net,
|
||||
) -> RpcModuleBuilder<N, Provider, Pool, Net, Tasks, EvmConfig, BlockExecutor, Consensus>
|
||||
) -> RpcModuleBuilder<N, Provider, Pool, Net, Tasks, EvmConfig, Consensus>
|
||||
where
|
||||
Net: NetworkInfo + Peers + 'static,
|
||||
{
|
||||
let Self {
|
||||
provider,
|
||||
pool,
|
||||
executor,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
..
|
||||
} = self;
|
||||
RpcModuleBuilder {
|
||||
provider,
|
||||
network,
|
||||
pool,
|
||||
executor,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
}
|
||||
let Self { provider, pool, executor, evm_config, consensus, _primitives, .. } = self;
|
||||
RpcModuleBuilder { provider, network, pool, executor, evm_config, consensus, _primitives }
|
||||
}
|
||||
|
||||
/// Configure a [`NoopNetwork`] instance.
|
||||
@@ -335,25 +241,14 @@ where
|
||||
/// [`EthApi`] which requires a [`NetworkInfo`] implementation.
|
||||
pub fn with_noop_network(
|
||||
self,
|
||||
) -> RpcModuleBuilder<N, Provider, Pool, NoopNetwork, Tasks, EvmConfig, BlockExecutor, Consensus>
|
||||
{
|
||||
let Self {
|
||||
provider,
|
||||
pool,
|
||||
executor,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
..
|
||||
} = self;
|
||||
) -> RpcModuleBuilder<N, Provider, Pool, NoopNetwork, Tasks, EvmConfig, Consensus> {
|
||||
let Self { provider, pool, executor, evm_config, consensus, _primitives, .. } = self;
|
||||
RpcModuleBuilder {
|
||||
provider,
|
||||
pool,
|
||||
executor,
|
||||
network: NoopNetwork::default(),
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
}
|
||||
@@ -363,30 +258,12 @@ where
|
||||
pub fn with_executor<T>(
|
||||
self,
|
||||
executor: T,
|
||||
) -> RpcModuleBuilder<N, Provider, Pool, Network, T, EvmConfig, BlockExecutor, Consensus>
|
||||
) -> RpcModuleBuilder<N, Provider, Pool, Network, T, EvmConfig, Consensus>
|
||||
where
|
||||
T: TaskSpawner + 'static,
|
||||
{
|
||||
let Self {
|
||||
pool,
|
||||
network,
|
||||
provider,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
..
|
||||
} = self;
|
||||
RpcModuleBuilder {
|
||||
provider,
|
||||
network,
|
||||
pool,
|
||||
executor,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
}
|
||||
let Self { pool, network, provider, evm_config, consensus, _primitives, .. } = self;
|
||||
RpcModuleBuilder { provider, network, pool, executor, evm_config, consensus, _primitives }
|
||||
}
|
||||
|
||||
/// Configure [`TokioTaskExecutor`] as the task executor to use for additional tasks.
|
||||
@@ -395,33 +272,14 @@ where
|
||||
/// [`TokioTaskExecutor`].
|
||||
pub fn with_tokio_executor(
|
||||
self,
|
||||
) -> RpcModuleBuilder<
|
||||
N,
|
||||
Provider,
|
||||
Pool,
|
||||
Network,
|
||||
TokioTaskExecutor,
|
||||
EvmConfig,
|
||||
BlockExecutor,
|
||||
Consensus,
|
||||
> {
|
||||
let Self {
|
||||
pool,
|
||||
network,
|
||||
provider,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
..
|
||||
} = self;
|
||||
) -> RpcModuleBuilder<N, Provider, Pool, Network, TokioTaskExecutor, EvmConfig, Consensus> {
|
||||
let Self { pool, network, provider, evm_config, consensus, _primitives, .. } = self;
|
||||
RpcModuleBuilder {
|
||||
provider,
|
||||
network,
|
||||
pool,
|
||||
executor: TokioTaskExecutor::default(),
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
}
|
||||
@@ -431,65 +289,22 @@ where
|
||||
pub fn with_evm_config<E>(
|
||||
self,
|
||||
evm_config: E,
|
||||
) -> RpcModuleBuilder<N, Provider, Pool, Network, Tasks, E, BlockExecutor, Consensus>
|
||||
) -> RpcModuleBuilder<N, Provider, Pool, Network, Tasks, E, Consensus>
|
||||
where
|
||||
EvmConfig: 'static,
|
||||
E: ConfigureEvm + Clone,
|
||||
{
|
||||
let Self {
|
||||
provider, pool, executor, network, block_executor, consensus, _primitives, ..
|
||||
} = self;
|
||||
RpcModuleBuilder {
|
||||
provider,
|
||||
network,
|
||||
pool,
|
||||
executor,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
}
|
||||
}
|
||||
|
||||
/// Configure the block executor provider
|
||||
pub fn with_block_executor<BE>(
|
||||
self,
|
||||
block_executor: BE,
|
||||
) -> RpcModuleBuilder<N, Provider, Pool, Network, Tasks, EvmConfig, BE, Consensus>
|
||||
where
|
||||
BE: BlockExecutorProvider<Primitives = N> + 'static,
|
||||
{
|
||||
let Self { provider, network, pool, executor, evm_config, consensus, _primitives, .. } =
|
||||
self;
|
||||
RpcModuleBuilder {
|
||||
provider,
|
||||
network,
|
||||
pool,
|
||||
executor,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
}
|
||||
let Self { provider, pool, executor, network, consensus, _primitives, .. } = self;
|
||||
RpcModuleBuilder { provider, network, pool, executor, evm_config, consensus, _primitives }
|
||||
}
|
||||
|
||||
/// Configure the consensus implementation.
|
||||
pub fn with_consensus<C>(
|
||||
self,
|
||||
consensus: C,
|
||||
) -> RpcModuleBuilder<N, Provider, Pool, Network, Tasks, EvmConfig, BlockExecutor, C> {
|
||||
let Self {
|
||||
provider, network, pool, executor, evm_config, block_executor, _primitives, ..
|
||||
} = self;
|
||||
RpcModuleBuilder {
|
||||
provider,
|
||||
network,
|
||||
pool,
|
||||
executor,
|
||||
evm_config,
|
||||
block_executor,
|
||||
consensus,
|
||||
_primitives,
|
||||
}
|
||||
) -> RpcModuleBuilder<N, Provider, Pool, Network, Tasks, EvmConfig, C> {
|
||||
let Self { provider, network, pool, executor, evm_config, _primitives, .. } = self;
|
||||
RpcModuleBuilder { provider, network, pool, executor, evm_config, consensus, _primitives }
|
||||
}
|
||||
|
||||
/// Instantiates a new [`EthApiBuilder`] from the configured components.
|
||||
@@ -530,8 +345,8 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<N, Provider, Pool, Network, Tasks, EvmConfig, BlockExecutor, Consensus>
|
||||
RpcModuleBuilder<N, Provider, Pool, Network, Tasks, EvmConfig, BlockExecutor, Consensus>
|
||||
impl<N, Provider, Pool, Network, Tasks, EvmConfig, Consensus>
|
||||
RpcModuleBuilder<N, Provider, Pool, Network, Tasks, EvmConfig, Consensus>
|
||||
where
|
||||
N: NodePrimitives,
|
||||
Provider: FullRpcProvider<Block = N::Block, Receipt = N::Receipt, Header = N::BlockHeader>
|
||||
@@ -541,8 +356,7 @@ where
|
||||
Pool: TransactionPool + 'static,
|
||||
Network: NetworkInfo + Peers + Clone + 'static,
|
||||
Tasks: TaskSpawner + Clone + 'static,
|
||||
EvmConfig: ConfigureEvm<Primitives = N>,
|
||||
BlockExecutor: BlockExecutorProvider<Primitives = N>,
|
||||
EvmConfig: ConfigureEvm<Primitives = N> + 'static,
|
||||
Consensus: FullConsensus<N, Error = ConsensusError> + Clone + 'static,
|
||||
{
|
||||
/// Configures all [`RpcModule`]s specific to the given [`TransportRpcModuleConfig`] which can
|
||||
@@ -560,27 +374,17 @@ where
|
||||
) -> (
|
||||
TransportRpcModules,
|
||||
AuthRpcModule,
|
||||
RpcRegistryInner<Provider, Pool, Network, Tasks, EthApi, BlockExecutor, Consensus>,
|
||||
RpcRegistryInner<Provider, Pool, Network, Tasks, EthApi, EvmConfig, Consensus>,
|
||||
)
|
||||
where
|
||||
EthApi: FullEthApiServer<Provider = Provider, Pool = Pool>,
|
||||
{
|
||||
let Self {
|
||||
provider, pool, network, executor, evm_config, block_executor, consensus, ..
|
||||
} = self;
|
||||
let Self { provider, pool, network, executor, consensus, evm_config, .. } = self;
|
||||
|
||||
let config = module_config.config.clone().unwrap_or_default();
|
||||
|
||||
let mut registry = RpcRegistryInner::new(
|
||||
provider,
|
||||
pool,
|
||||
network,
|
||||
executor,
|
||||
consensus,
|
||||
config,
|
||||
evm_config,
|
||||
eth,
|
||||
block_executor,
|
||||
provider, pool, network, executor, consensus, config, evm_config, eth,
|
||||
);
|
||||
|
||||
let modules = registry.create_transport_rpc_modules(module_config);
|
||||
@@ -598,24 +402,12 @@ where
|
||||
self,
|
||||
config: RpcModuleConfig,
|
||||
eth: EthApi,
|
||||
) -> RpcRegistryInner<Provider, Pool, Network, Tasks, EthApi, BlockExecutor, Consensus>
|
||||
) -> RpcRegistryInner<Provider, Pool, Network, Tasks, EthApi, EvmConfig, Consensus>
|
||||
where
|
||||
EthApi: EthApiTypes + 'static,
|
||||
{
|
||||
let Self {
|
||||
provider, pool, network, executor, evm_config, block_executor, consensus, ..
|
||||
} = self;
|
||||
RpcRegistryInner::new(
|
||||
provider,
|
||||
pool,
|
||||
network,
|
||||
executor,
|
||||
consensus,
|
||||
config,
|
||||
evm_config,
|
||||
eth,
|
||||
block_executor,
|
||||
)
|
||||
let Self { provider, pool, network, executor, consensus, evm_config, .. } = self;
|
||||
RpcRegistryInner::new(provider, pool, network, executor, consensus, config, evm_config, eth)
|
||||
}
|
||||
|
||||
/// Configures all [`RpcModule`]s specific to the given [`TransportRpcModuleConfig`] which can
|
||||
@@ -630,9 +422,7 @@ where
|
||||
{
|
||||
let mut modules = TransportRpcModules::default();
|
||||
|
||||
let Self {
|
||||
provider, pool, network, executor, evm_config, block_executor, consensus, ..
|
||||
} = self;
|
||||
let Self { provider, pool, network, executor, consensus, evm_config, .. } = self;
|
||||
|
||||
if !module_config.is_empty() {
|
||||
let TransportRpcModuleConfig { http, ws, ipc, config } = module_config.clone();
|
||||
@@ -646,7 +436,6 @@ where
|
||||
config.unwrap_or_default(),
|
||||
evm_config,
|
||||
eth,
|
||||
block_executor,
|
||||
);
|
||||
|
||||
modules.config = module_config;
|
||||
@@ -659,9 +448,9 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<N: NodePrimitives> Default for RpcModuleBuilder<N, (), (), (), (), (), (), ()> {
|
||||
impl<N: NodePrimitives> Default for RpcModuleBuilder<N, (), (), (), (), (), ()> {
|
||||
fn default() -> Self {
|
||||
Self::new((), (), (), (), (), (), ())
|
||||
Self::new((), (), (), (), (), ())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -751,14 +540,14 @@ pub struct RpcRegistryInner<
|
||||
Network,
|
||||
Tasks,
|
||||
EthApi: EthApiTypes,
|
||||
BlockExecutor,
|
||||
EvmConfig,
|
||||
Consensus,
|
||||
> {
|
||||
provider: Provider,
|
||||
pool: Pool,
|
||||
network: Network,
|
||||
executor: Tasks,
|
||||
block_executor: BlockExecutor,
|
||||
evm_config: EvmConfig,
|
||||
consensus: Consensus,
|
||||
/// Holds a all `eth_` namespace handlers
|
||||
eth: EthHandlers<EthApi>,
|
||||
@@ -772,8 +561,8 @@ pub struct RpcRegistryInner<
|
||||
|
||||
// === impl RpcRegistryInner ===
|
||||
|
||||
impl<N, Provider, Pool, Network, Tasks, EthApi, BlockExecutor, Consensus>
|
||||
RpcRegistryInner<Provider, Pool, Network, Tasks, EthApi, BlockExecutor, Consensus>
|
||||
impl<N, Provider, Pool, Network, Tasks, EthApi, EvmConfig, Consensus>
|
||||
RpcRegistryInner<Provider, Pool, Network, Tasks, EthApi, EvmConfig, Consensus>
|
||||
where
|
||||
N: NodePrimitives,
|
||||
Provider: StateProviderFactory
|
||||
@@ -786,20 +575,19 @@ where
|
||||
Network: Clone + 'static,
|
||||
Tasks: TaskSpawner + Clone + 'static,
|
||||
EthApi: EthApiTypes + 'static,
|
||||
BlockExecutor: BlockExecutorProvider<Primitives = N>,
|
||||
EvmConfig: ConfigureEvm<Primitives = N>,
|
||||
{
|
||||
/// Creates a new, empty instance.
|
||||
#[expect(clippy::too_many_arguments)]
|
||||
pub fn new<EvmConfig>(
|
||||
pub fn new(
|
||||
provider: Provider,
|
||||
pool: Pool,
|
||||
network: Network,
|
||||
executor: Tasks,
|
||||
consensus: Consensus,
|
||||
config: RpcModuleConfig,
|
||||
_evm_config: EvmConfig,
|
||||
evm_config: EvmConfig,
|
||||
eth_api: EthApi,
|
||||
block_executor: BlockExecutor,
|
||||
) -> Self
|
||||
where
|
||||
EvmConfig: ConfigureEvm<Primitives = N>,
|
||||
@@ -817,8 +605,8 @@ where
|
||||
consensus,
|
||||
modules: Default::default(),
|
||||
blocking_pool_guard,
|
||||
block_executor,
|
||||
eth_config: config.eth,
|
||||
evm_config,
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -869,13 +657,13 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<Provider, Pool, Network, Tasks, EthApi, BlockExecutor, Consensus>
|
||||
RpcRegistryInner<Provider, Pool, Network, Tasks, EthApi, BlockExecutor, Consensus>
|
||||
impl<Provider, Pool, Network, Tasks, EthApi, EvmConfig, Consensus>
|
||||
RpcRegistryInner<Provider, Pool, Network, Tasks, EthApi, EvmConfig, Consensus>
|
||||
where
|
||||
Network: NetworkInfo + Clone + 'static,
|
||||
EthApi: EthApiTypes,
|
||||
Provider: BlockReader + ChainSpecProvider<ChainSpec: EthereumHardforks>,
|
||||
BlockExecutor: BlockExecutorProvider,
|
||||
EvmConfig: ConfigureEvm,
|
||||
{
|
||||
/// Instantiates `AdminApi`
|
||||
pub fn admin_api(&self) -> AdminApi<Network, Provider::ChainSpec>
|
||||
@@ -908,8 +696,8 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<N, Provider, Pool, Network, Tasks, EthApi, BlockExecutor, Consensus>
|
||||
RpcRegistryInner<Provider, Pool, Network, Tasks, EthApi, BlockExecutor, Consensus>
|
||||
impl<N, Provider, Pool, Network, Tasks, EthApi, EvmConfig, Consensus>
|
||||
RpcRegistryInner<Provider, Pool, Network, Tasks, EthApi, EvmConfig, Consensus>
|
||||
where
|
||||
N: NodePrimitives,
|
||||
Provider: FullRpcProvider<
|
||||
@@ -927,7 +715,7 @@ where
|
||||
RpcReceipt<EthApi::NetworkTypes>,
|
||||
RpcHeader<EthApi::NetworkTypes>,
|
||||
> + EthApiTypes,
|
||||
BlockExecutor: BlockExecutorProvider<Primitives = N>,
|
||||
EvmConfig: ConfigureEvm<Primitives = N> + 'static,
|
||||
{
|
||||
/// Register Eth Namespace
|
||||
///
|
||||
@@ -962,7 +750,7 @@ where
|
||||
pub fn register_debug(&mut self) -> &mut Self
|
||||
where
|
||||
EthApi: EthApiSpec + EthTransactions + TraceExt,
|
||||
BlockExecutor::Primitives: NodePrimitives<Block = ProviderBlock<EthApi::Provider>>,
|
||||
EvmConfig::Primitives: NodePrimitives<Block = ProviderBlock<EthApi::Provider>>,
|
||||
{
|
||||
let debug_api = self.debug_api();
|
||||
self.modules.insert(RethRpcModule::Debug, debug_api.into_rpc().into());
|
||||
@@ -1023,8 +811,8 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<N, Provider, Pool, Network, Tasks, EthApi, BlockExecutor, Consensus>
|
||||
RpcRegistryInner<Provider, Pool, Network, Tasks, EthApi, BlockExecutor, Consensus>
|
||||
impl<N, Provider, Pool, Network, Tasks, EthApi, EvmConfig, Consensus>
|
||||
RpcRegistryInner<Provider, Pool, Network, Tasks, EthApi, EvmConfig, Consensus>
|
||||
where
|
||||
N: NodePrimitives,
|
||||
Provider: FullRpcProvider<
|
||||
@@ -1037,7 +825,7 @@ where
|
||||
Network: NetworkInfo + Peers + Clone + 'static,
|
||||
Tasks: TaskSpawner + Clone + 'static,
|
||||
EthApi: EthApiTypes,
|
||||
BlockExecutor: BlockExecutorProvider<Primitives = N>,
|
||||
EvmConfig: ConfigureEvm<Primitives = N>,
|
||||
{
|
||||
/// Instantiates `TraceApi`
|
||||
///
|
||||
@@ -1069,15 +857,15 @@ where
|
||||
/// # Panics
|
||||
///
|
||||
/// If called outside of the tokio runtime. See also [`Self::eth_api`]
|
||||
pub fn debug_api(&self) -> DebugApi<EthApi, BlockExecutor>
|
||||
pub fn debug_api(&self) -> DebugApi<EthApi, EvmConfig>
|
||||
where
|
||||
EthApi: EthApiSpec + EthTransactions + TraceExt,
|
||||
BlockExecutor::Primitives: NodePrimitives<Block = ProviderBlock<EthApi::Provider>>,
|
||||
EvmConfig::Primitives: NodePrimitives<Block = ProviderBlock<EthApi::Provider>>,
|
||||
{
|
||||
DebugApi::new(
|
||||
self.eth_api().clone(),
|
||||
self.blocking_pool_guard.clone(),
|
||||
self.block_executor.clone(),
|
||||
self.evm_config.clone(),
|
||||
)
|
||||
}
|
||||
|
||||
@@ -1100,8 +888,8 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<N, Provider, Pool, Network, Tasks, EthApi, BlockExecutor, Consensus>
|
||||
RpcRegistryInner<Provider, Pool, Network, Tasks, EthApi, BlockExecutor, Consensus>
|
||||
impl<N, Provider, Pool, Network, Tasks, EthApi, EvmConfig, Consensus>
|
||||
RpcRegistryInner<Provider, Pool, Network, Tasks, EthApi, EvmConfig, Consensus>
|
||||
where
|
||||
N: NodePrimitives,
|
||||
Provider: FullRpcProvider<Block = N::Block>
|
||||
@@ -1112,7 +900,7 @@ where
|
||||
Network: NetworkInfo + Peers + Clone + 'static,
|
||||
Tasks: TaskSpawner + Clone + 'static,
|
||||
EthApi: FullEthApiServer<Provider = Provider, Pool = Pool>,
|
||||
BlockExecutor: BlockExecutorProvider<Primitives = N>,
|
||||
EvmConfig: ConfigureEvm<Primitives = N> + 'static,
|
||||
Consensus: FullConsensus<N, Error = ConsensusError> + Clone + 'static,
|
||||
{
|
||||
/// Configures the auth module that includes the
|
||||
@@ -1199,7 +987,7 @@ where
|
||||
RethRpcModule::Debug => DebugApi::new(
|
||||
eth_api.clone(),
|
||||
self.blocking_pool_guard.clone(),
|
||||
self.block_executor.clone(),
|
||||
self.evm_config.clone(),
|
||||
)
|
||||
.into_rpc()
|
||||
.into(),
|
||||
|
||||
@@ -6,7 +6,7 @@ use reth_consensus::noop::NoopConsensus;
|
||||
use reth_engine_primitives::BeaconConsensusEngineHandle;
|
||||
use reth_ethereum_engine_primitives::EthEngineTypes;
|
||||
use reth_ethereum_primitives::EthPrimitives;
|
||||
use reth_evm::execute::BasicBlockExecutorProvider;
|
||||
|
||||
use reth_evm_ethereum::EthEvmConfig;
|
||||
use reth_network_api::noop::NoopNetwork;
|
||||
use reth_node_ethereum::EthereumEngineValidator;
|
||||
@@ -125,7 +125,6 @@ pub fn test_rpc_builder() -> RpcModuleBuilder<
|
||||
NoopNetwork,
|
||||
TokioTaskExecutor,
|
||||
EthEvmConfig,
|
||||
BasicBlockExecutorProvider<EthEvmConfig>,
|
||||
NoopConsensus,
|
||||
> {
|
||||
RpcModuleBuilder::default()
|
||||
@@ -134,6 +133,5 @@ pub fn test_rpc_builder() -> RpcModuleBuilder<
|
||||
.with_network(NoopNetwork::default())
|
||||
.with_executor(TokioTaskExecutor::default())
|
||||
.with_evm_config(EthEvmConfig::mainnet())
|
||||
.with_block_executor(BasicBlockExecutorProvider::new(EthEvmConfig::mainnet()))
|
||||
.with_consensus(NoopConsensus::default())
|
||||
}
|
||||
|
||||
@@ -16,10 +16,7 @@ use alloy_rpc_types_trace::geth::{
|
||||
use async_trait::async_trait;
|
||||
use jsonrpsee::core::RpcResult;
|
||||
use reth_chainspec::{ChainSpecProvider, EthChainSpec, EthereumHardforks};
|
||||
use reth_evm::{
|
||||
execute::{BlockExecutorProvider, Executor},
|
||||
ConfigureEvm, EvmEnvFor, TxEnvFor,
|
||||
};
|
||||
use reth_evm::{execute::Executor, ConfigureEvm, EvmEnvFor, TxEnvFor};
|
||||
use reth_primitives_traits::{
|
||||
Block as _, BlockBody, NodePrimitives, ReceiptWithBloom, RecoveredBlock, SignedTransaction,
|
||||
};
|
||||
@@ -56,14 +53,10 @@ pub struct DebugApi<Eth, BlockExecutor> {
|
||||
|
||||
// === impl DebugApi ===
|
||||
|
||||
impl<Eth, BlockExecutor> DebugApi<Eth, BlockExecutor> {
|
||||
impl<Eth, Evm> DebugApi<Eth, Evm> {
|
||||
/// Create a new instance of the [`DebugApi`]
|
||||
pub fn new(
|
||||
eth: Eth,
|
||||
blocking_task_guard: BlockingTaskGuard,
|
||||
block_executor: BlockExecutor,
|
||||
) -> Self {
|
||||
let inner = Arc::new(DebugApiInner { eth_api: eth, blocking_task_guard, block_executor });
|
||||
pub fn new(eth: Eth, blocking_task_guard: BlockingTaskGuard, evm_config: Evm) -> Self {
|
||||
let inner = Arc::new(DebugApiInner { eth_api: eth, blocking_task_guard, evm_config });
|
||||
Self { inner }
|
||||
}
|
||||
|
||||
@@ -82,11 +75,10 @@ impl<Eth: RpcNodeCore, BlockExecutor> DebugApi<Eth, BlockExecutor> {
|
||||
|
||||
// === impl DebugApi ===
|
||||
|
||||
impl<Eth, BlockExecutor> DebugApi<Eth, BlockExecutor>
|
||||
impl<Eth, Evm> DebugApi<Eth, Evm>
|
||||
where
|
||||
Eth: EthApiTypes + TraceExt + 'static,
|
||||
BlockExecutor:
|
||||
BlockExecutorProvider<Primitives: NodePrimitives<Block = ProviderBlock<Eth::Provider>>>,
|
||||
Evm: ConfigureEvm<Primitives: NodePrimitives<Block = ProviderBlock<Eth::Provider>>> + 'static,
|
||||
{
|
||||
/// Acquires a permit to execute a tracing call.
|
||||
async fn acquire_trace_permit(&self) -> Result<OwnedSemaphorePermit, AcquireError> {
|
||||
@@ -637,7 +629,7 @@ where
|
||||
.eth_api()
|
||||
.spawn_with_state_at_block(block.parent_hash().into(), move |state_provider| {
|
||||
let db = StateProviderDatabase::new(&state_provider);
|
||||
let block_executor = this.inner.block_executor.executor(db);
|
||||
let block_executor = this.inner.evm_config.batch_executor(db);
|
||||
|
||||
let mut witness_record = ExecutionWitnessRecord::default();
|
||||
|
||||
@@ -873,11 +865,10 @@ where
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl<Eth, BlockExecutor> DebugApiServer for DebugApi<Eth, BlockExecutor>
|
||||
impl<Eth, Evm> DebugApiServer for DebugApi<Eth, Evm>
|
||||
where
|
||||
Eth: EthApiTypes + EthTransactions + TraceExt + 'static,
|
||||
BlockExecutor:
|
||||
BlockExecutorProvider<Primitives: NodePrimitives<Block = ProviderBlock<Eth::Provider>>>,
|
||||
Evm: ConfigureEvm<Primitives: NodePrimitives<Block = ProviderBlock<Eth::Provider>>> + 'static,
|
||||
{
|
||||
/// Handler for `debug_getRawHeader`
|
||||
async fn raw_header(&self, block_id: BlockId) -> RpcResult<Bytes> {
|
||||
@@ -1274,23 +1265,23 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<Eth, BlockExecutor> std::fmt::Debug for DebugApi<Eth, BlockExecutor> {
|
||||
impl<Eth, Evm> std::fmt::Debug for DebugApi<Eth, Evm> {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("DebugApi").finish_non_exhaustive()
|
||||
}
|
||||
}
|
||||
|
||||
impl<Eth, BlockExecutor> Clone for DebugApi<Eth, BlockExecutor> {
|
||||
impl<Eth, Evm> Clone for DebugApi<Eth, Evm> {
|
||||
fn clone(&self) -> Self {
|
||||
Self { inner: Arc::clone(&self.inner) }
|
||||
}
|
||||
}
|
||||
|
||||
struct DebugApiInner<Eth, BlockExecutor> {
|
||||
struct DebugApiInner<Eth, Evm> {
|
||||
/// The implementation of `eth` API
|
||||
eth_api: Eth,
|
||||
// restrict the number of concurrent calls to blocking calls
|
||||
blocking_task_guard: BlockingTaskGuard,
|
||||
/// block executor for debug & trace apis
|
||||
block_executor: BlockExecutor,
|
||||
evm_config: Evm,
|
||||
}
|
||||
|
||||
@@ -18,7 +18,7 @@ use reth_chainspec::{ChainSpecProvider, EthereumHardforks};
|
||||
use reth_consensus::{Consensus, FullConsensus};
|
||||
use reth_engine_primitives::PayloadValidator;
|
||||
use reth_errors::{BlockExecutionError, ConsensusError, ProviderError};
|
||||
use reth_evm::execute::{BlockExecutorProvider, Executor};
|
||||
use reth_evm::{execute::Executor, ConfigureEvm};
|
||||
use reth_execution_types::BlockExecutionOutput;
|
||||
use reth_metrics::{metrics, metrics::Gauge, Metrics};
|
||||
use reth_node_api::NewPayloadError;
|
||||
@@ -39,20 +39,20 @@ use tracing::warn;
|
||||
|
||||
/// The type that implements the `validation` rpc namespace trait
|
||||
#[derive(Clone, Debug, derive_more::Deref)]
|
||||
pub struct ValidationApi<Provider, E: BlockExecutorProvider> {
|
||||
pub struct ValidationApi<Provider, E: ConfigureEvm> {
|
||||
#[deref]
|
||||
inner: Arc<ValidationApiInner<Provider, E>>,
|
||||
}
|
||||
|
||||
impl<Provider, E> ValidationApi<Provider, E>
|
||||
where
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
{
|
||||
/// Create a new instance of the [`ValidationApi`]
|
||||
pub fn new(
|
||||
provider: Provider,
|
||||
consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
|
||||
executor_provider: E,
|
||||
evm_config: E,
|
||||
config: ValidationApiConfig,
|
||||
task_spawner: Box<dyn TaskSpawner>,
|
||||
payload_validator: Arc<
|
||||
@@ -68,7 +68,7 @@ where
|
||||
provider,
|
||||
consensus,
|
||||
payload_validator,
|
||||
executor_provider,
|
||||
evm_config,
|
||||
disallow,
|
||||
validation_window,
|
||||
cached_state: Default::default(),
|
||||
@@ -107,7 +107,7 @@ where
|
||||
+ ChainSpecProvider<ChainSpec: EthereumHardforks>
|
||||
+ StateProviderFactory
|
||||
+ 'static,
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm + 'static,
|
||||
{
|
||||
/// Validates the given block and a [`BidTrace`] against it.
|
||||
pub async fn validate_message_against_block(
|
||||
@@ -168,7 +168,7 @@ where
|
||||
let mut request_cache = self.cached_reads(parent_header_hash).await;
|
||||
|
||||
let cached_db = request_cache.as_db_mut(StateProviderDatabase::new(&state_provider));
|
||||
let executor = self.executor_provider.executor(cached_db);
|
||||
let executor = self.evm_config.batch_executor(cached_db);
|
||||
|
||||
let mut accessed_blacklisted = None;
|
||||
let output = executor.execute_with_state_closure(&block, |state| {
|
||||
@@ -414,7 +414,7 @@ where
|
||||
+ StateProviderFactory
|
||||
+ Clone
|
||||
+ 'static,
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm + 'static,
|
||||
{
|
||||
async fn validate_builder_submission_v1(
|
||||
&self,
|
||||
@@ -469,7 +469,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
pub struct ValidationApiInner<Provider, E: BlockExecutorProvider> {
|
||||
pub struct ValidationApiInner<Provider, E: ConfigureEvm> {
|
||||
/// The provider that can interact with the chain.
|
||||
provider: Provider,
|
||||
/// Consensus implementation.
|
||||
@@ -482,7 +482,7 @@ pub struct ValidationApiInner<Provider, E: BlockExecutorProvider> {
|
||||
>,
|
||||
>,
|
||||
/// Block executor factory.
|
||||
executor_provider: E,
|
||||
evm_config: E,
|
||||
/// Set of disallowed addresses
|
||||
disallow: HashSet<Address>,
|
||||
/// The maximum block distance - parent to latest - allowed for validation
|
||||
@@ -498,7 +498,7 @@ pub struct ValidationApiInner<Provider, E: BlockExecutorProvider> {
|
||||
metrics: ValidationMetrics,
|
||||
}
|
||||
|
||||
impl<Provider, E: BlockExecutorProvider> fmt::Debug for ValidationApiInner<Provider, E> {
|
||||
impl<Provider, E: ConfigureEvm> fmt::Debug for ValidationApiInner<Provider, E> {
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
f.debug_struct("ValidationApiInner").finish_non_exhaustive()
|
||||
}
|
||||
|
||||
@@ -15,16 +15,16 @@
|
||||
//! # use reth_chainspec::MAINNET;
|
||||
//! # use reth_prune_types::PruneModes;
|
||||
//! # use reth_evm_ethereum::EthEvmConfig;
|
||||
//! # use reth_evm::ConfigureEvm;
|
||||
//! # use reth_provider::StaticFileProviderFactory;
|
||||
//! # use reth_provider::test_utils::{create_test_provider_factory, MockNodeTypesWithDB};
|
||||
//! # use reth_static_file::StaticFileProducer;
|
||||
//! # use reth_config::config::StageConfig;
|
||||
//! # use reth_evm::execute::BlockExecutorProvider;
|
||||
//! # use reth_ethereum_primitives::EthPrimitives;
|
||||
//! # use std::sync::Arc;
|
||||
//! # use reth_consensus::{FullConsensus, ConsensusError};
|
||||
//!
|
||||
//! # fn create(exec: impl BlockExecutorProvider<Primitives = EthPrimitives>, consensus: impl FullConsensus<EthPrimitives, Error = ConsensusError> + 'static) {
|
||||
//! # fn create(exec: impl ConfigureEvm<Primitives = EthPrimitives> + 'static, consensus: impl FullConsensus<EthPrimitives, Error = ConsensusError> + 'static) {
|
||||
//!
|
||||
//! let provider_factory = create_test_provider_factory();
|
||||
//! let static_file_producer =
|
||||
@@ -47,7 +47,7 @@ use crate::{
|
||||
use alloy_primitives::B256;
|
||||
use reth_config::config::StageConfig;
|
||||
use reth_consensus::{ConsensusError, FullConsensus};
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_network_p2p::{bodies::downloader::BodyDownloader, headers::downloader::HeaderDownloader};
|
||||
use reth_primitives_traits::{Block, NodePrimitives};
|
||||
use reth_provider::HeaderSyncGapProvider;
|
||||
@@ -84,12 +84,12 @@ pub struct DefaultStages<Provider, H, B, E>
|
||||
where
|
||||
H: HeaderDownloader,
|
||||
B: BodyDownloader,
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
{
|
||||
/// Configuration for the online stages
|
||||
online: OnlineStages<Provider, H, B>,
|
||||
/// Executor factory needs for execution stage
|
||||
executor_provider: E,
|
||||
evm_config: E,
|
||||
/// Consensus instance
|
||||
consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
|
||||
/// Configuration for each stage in the pipeline
|
||||
@@ -102,7 +102,7 @@ impl<Provider, H, B, E> DefaultStages<Provider, H, B, E>
|
||||
where
|
||||
H: HeaderDownloader,
|
||||
B: BodyDownloader,
|
||||
E: BlockExecutorProvider<Primitives: NodePrimitives<BlockHeader = H::Header, Block = B::Block>>,
|
||||
E: ConfigureEvm<Primitives: NodePrimitives<BlockHeader = H::Header, Block = B::Block>>,
|
||||
{
|
||||
/// Create a new set of default stages with default values.
|
||||
#[expect(clippy::too_many_arguments)]
|
||||
@@ -112,7 +112,7 @@ where
|
||||
consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
|
||||
header_downloader: H,
|
||||
body_downloader: B,
|
||||
executor_provider: E,
|
||||
evm_config: E,
|
||||
stages_config: StageConfig,
|
||||
prune_modes: PruneModes,
|
||||
) -> Self {
|
||||
@@ -124,7 +124,7 @@ where
|
||||
body_downloader,
|
||||
stages_config.clone(),
|
||||
),
|
||||
executor_provider,
|
||||
evm_config,
|
||||
consensus,
|
||||
stages_config,
|
||||
prune_modes,
|
||||
@@ -134,14 +134,14 @@ where
|
||||
|
||||
impl<P, H, B, E> DefaultStages<P, H, B, E>
|
||||
where
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
H: HeaderDownloader,
|
||||
B: BodyDownloader,
|
||||
{
|
||||
/// Appends the default offline stages and default finish stage to the given builder.
|
||||
pub fn add_offline_stages<Provider>(
|
||||
default_offline: StageSetBuilder<Provider>,
|
||||
executor_provider: E,
|
||||
evm_config: E,
|
||||
consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
|
||||
stages_config: StageConfig,
|
||||
prune_modes: PruneModes,
|
||||
@@ -151,7 +151,7 @@ where
|
||||
{
|
||||
StageSetBuilder::default()
|
||||
.add_set(default_offline)
|
||||
.add_set(OfflineStages::new(executor_provider, consensus, stages_config, prune_modes))
|
||||
.add_set(OfflineStages::new(evm_config, consensus, stages_config, prune_modes))
|
||||
.add_stage(FinishStage)
|
||||
}
|
||||
}
|
||||
@@ -161,14 +161,14 @@ where
|
||||
P: HeaderSyncGapProvider + 'static,
|
||||
H: HeaderDownloader + 'static,
|
||||
B: BodyDownloader + 'static,
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
OnlineStages<P, H, B>: StageSet<Provider>,
|
||||
OfflineStages<E>: StageSet<Provider>,
|
||||
{
|
||||
fn builder(self) -> StageSetBuilder<Provider> {
|
||||
Self::add_offline_stages(
|
||||
self.online.builder(),
|
||||
self.executor_provider,
|
||||
self.evm_config,
|
||||
self.consensus,
|
||||
self.stages_config.clone(),
|
||||
self.prune_modes,
|
||||
@@ -283,9 +283,9 @@ where
|
||||
/// - [`PruneStage`]
|
||||
#[derive(Debug)]
|
||||
#[non_exhaustive]
|
||||
pub struct OfflineStages<E: BlockExecutorProvider> {
|
||||
pub struct OfflineStages<E: ConfigureEvm> {
|
||||
/// Executor factory needs for execution stage
|
||||
executor_provider: E,
|
||||
evm_config: E,
|
||||
/// Consensus instance for validating blocks.
|
||||
consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
|
||||
/// Configuration for each stage in the pipeline
|
||||
@@ -294,21 +294,21 @@ pub struct OfflineStages<E: BlockExecutorProvider> {
|
||||
prune_modes: PruneModes,
|
||||
}
|
||||
|
||||
impl<E: BlockExecutorProvider> OfflineStages<E> {
|
||||
impl<E: ConfigureEvm> OfflineStages<E> {
|
||||
/// Create a new set of offline stages with default values.
|
||||
pub const fn new(
|
||||
executor_provider: E,
|
||||
evm_config: E,
|
||||
consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
|
||||
stages_config: StageConfig,
|
||||
prune_modes: PruneModes,
|
||||
) -> Self {
|
||||
Self { executor_provider, consensus, stages_config, prune_modes }
|
||||
Self { evm_config, consensus, stages_config, prune_modes }
|
||||
}
|
||||
}
|
||||
|
||||
impl<E, Provider> StageSet<Provider> for OfflineStages<E>
|
||||
where
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
ExecutionStages<E>: StageSet<Provider>,
|
||||
PruneSenderRecoveryStage: Stage<Provider>,
|
||||
HashingStages: StageSet<Provider>,
|
||||
@@ -316,7 +316,7 @@ where
|
||||
PruneStage: Stage<Provider>,
|
||||
{
|
||||
fn builder(self) -> StageSetBuilder<Provider> {
|
||||
ExecutionStages::new(self.executor_provider, self.consensus, self.stages_config.clone())
|
||||
ExecutionStages::new(self.evm_config, self.consensus, self.stages_config.clone())
|
||||
.builder()
|
||||
// If sender recovery prune mode is set, add the prune sender recovery stage.
|
||||
.add_stage_opt(self.prune_modes.sender_recovery.map(|prune_mode| {
|
||||
@@ -339,29 +339,29 @@ where
|
||||
/// A set containing all stages that are required to execute pre-existing block data.
|
||||
#[derive(Debug)]
|
||||
#[non_exhaustive]
|
||||
pub struct ExecutionStages<E: BlockExecutorProvider> {
|
||||
pub struct ExecutionStages<E: ConfigureEvm> {
|
||||
/// Executor factory that will create executors.
|
||||
executor_provider: E,
|
||||
evm_config: E,
|
||||
/// Consensus instance for validating blocks.
|
||||
consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
|
||||
/// Configuration for each stage in the pipeline
|
||||
stages_config: StageConfig,
|
||||
}
|
||||
|
||||
impl<E: BlockExecutorProvider> ExecutionStages<E> {
|
||||
impl<E: ConfigureEvm> ExecutionStages<E> {
|
||||
/// Create a new set of execution stages with default values.
|
||||
pub const fn new(
|
||||
executor_provider: E,
|
||||
consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
|
||||
stages_config: StageConfig,
|
||||
) -> Self {
|
||||
Self { executor_provider, consensus, stages_config }
|
||||
Self { evm_config: executor_provider, consensus, stages_config }
|
||||
}
|
||||
}
|
||||
|
||||
impl<E, Provider> StageSet<Provider> for ExecutionStages<E>
|
||||
where
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm + 'static,
|
||||
SenderRecoveryStage: Stage<Provider>,
|
||||
ExecutionStage<E>: Stage<Provider>,
|
||||
{
|
||||
@@ -369,7 +369,7 @@ where
|
||||
StageSetBuilder::default()
|
||||
.add_stage(SenderRecoveryStage::new(self.stages_config.sender_recovery))
|
||||
.add_stage(ExecutionStage::from_config(
|
||||
self.executor_provider,
|
||||
self.evm_config,
|
||||
self.consensus,
|
||||
self.stages_config.execution,
|
||||
self.stages_config.execution_external_clean_threshold(),
|
||||
|
||||
@@ -5,10 +5,7 @@ use num_traits::Zero;
|
||||
use reth_config::config::ExecutionConfig;
|
||||
use reth_consensus::{ConsensusError, FullConsensus};
|
||||
use reth_db::{static_file::HeaderMask, tables};
|
||||
use reth_evm::{
|
||||
execute::{BlockExecutorProvider, Executor},
|
||||
metrics::ExecutorMetrics,
|
||||
};
|
||||
use reth_evm::{execute::Executor, metrics::ExecutorMetrics, ConfigureEvm};
|
||||
use reth_execution_types::Chain;
|
||||
use reth_exex::{ExExManagerHandle, ExExNotification, ExExNotificationSource};
|
||||
use reth_primitives_traits::{format_gas_throughput, Block, BlockBody, NodePrimitives};
|
||||
@@ -68,10 +65,10 @@ use super::missing_static_data_error;
|
||||
#[derive(Debug)]
|
||||
pub struct ExecutionStage<E>
|
||||
where
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
{
|
||||
/// The stage's internal block executor
|
||||
executor_provider: E,
|
||||
evm_config: E,
|
||||
/// The consensus instance for validating blocks.
|
||||
consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
|
||||
/// The consensu
|
||||
@@ -98,11 +95,11 @@ where
|
||||
|
||||
impl<E> ExecutionStage<E>
|
||||
where
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
{
|
||||
/// Create new execution stage with specified config.
|
||||
pub fn new(
|
||||
executor_provider: E,
|
||||
evm_config: E,
|
||||
consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
|
||||
thresholds: ExecutionStageThresholds,
|
||||
external_clean_threshold: u64,
|
||||
@@ -110,7 +107,7 @@ where
|
||||
) -> Self {
|
||||
Self {
|
||||
external_clean_threshold,
|
||||
executor_provider,
|
||||
evm_config,
|
||||
consensus,
|
||||
thresholds,
|
||||
post_execute_commit_input: None,
|
||||
@@ -124,11 +121,11 @@ where
|
||||
///
|
||||
/// The commit threshold will be set to [`MERKLE_STAGE_DEFAULT_CLEAN_THRESHOLD`].
|
||||
pub fn new_with_executor(
|
||||
executor_provider: E,
|
||||
evm_config: E,
|
||||
consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
|
||||
) -> Self {
|
||||
Self::new(
|
||||
executor_provider,
|
||||
evm_config,
|
||||
consensus,
|
||||
ExecutionStageThresholds::default(),
|
||||
MERKLE_STAGE_DEFAULT_CLEAN_THRESHOLD,
|
||||
@@ -138,13 +135,13 @@ where
|
||||
|
||||
/// Create new instance of [`ExecutionStage`] from configuration.
|
||||
pub fn from_config(
|
||||
executor_provider: E,
|
||||
evm_config: E,
|
||||
consensus: Arc<dyn FullConsensus<E::Primitives, Error = ConsensusError>>,
|
||||
config: ExecutionConfig,
|
||||
external_clean_threshold: u64,
|
||||
) -> Self {
|
||||
Self::new(
|
||||
executor_provider,
|
||||
evm_config,
|
||||
consensus,
|
||||
config.into(),
|
||||
external_clean_threshold,
|
||||
@@ -255,7 +252,7 @@ where
|
||||
|
||||
impl<E, Provider> Stage<Provider> for ExecutionStage<E>
|
||||
where
|
||||
E: BlockExecutorProvider,
|
||||
E: ConfigureEvm,
|
||||
Provider: DBProvider
|
||||
+ BlockReader<
|
||||
Block = <E::Primitives as NodePrimitives>::Block,
|
||||
@@ -294,7 +291,7 @@ where
|
||||
self.ensure_consistency(provider, input.checkpoint().block_number, None)?;
|
||||
|
||||
let db = StateProviderDatabase(LatestStateProviderRef::new(provider));
|
||||
let mut executor = self.executor_provider.executor(db);
|
||||
let mut executor = self.evm_config.batch_executor(db);
|
||||
|
||||
// Progress tracking
|
||||
let mut stage_progress = start_block;
|
||||
@@ -670,7 +667,6 @@ mod tests {
|
||||
};
|
||||
use reth_ethereum_consensus::EthBeaconConsensus;
|
||||
use reth_ethereum_primitives::Block;
|
||||
use reth_evm::execute::BasicBlockExecutorProvider;
|
||||
use reth_evm_ethereum::EthEvmConfig;
|
||||
use reth_primitives_traits::{Account, Bytecode, SealedBlock, StorageEntry};
|
||||
use reth_provider::{
|
||||
@@ -682,15 +678,14 @@ mod tests {
|
||||
use reth_stages_api::StageUnitCheckpoint;
|
||||
use std::collections::BTreeMap;
|
||||
|
||||
fn stage() -> ExecutionStage<BasicBlockExecutorProvider<EthEvmConfig>> {
|
||||
let strategy_factory =
|
||||
fn stage() -> ExecutionStage<EthEvmConfig> {
|
||||
let evm_config =
|
||||
EthEvmConfig::new(Arc::new(ChainSpecBuilder::mainnet().berlin_activated().build()));
|
||||
let executor_provider = BasicBlockExecutorProvider::new(strategy_factory);
|
||||
let consensus = Arc::new(EthBeaconConsensus::new(Arc::new(
|
||||
ChainSpecBuilder::mainnet().berlin_activated().build(),
|
||||
)));
|
||||
ExecutionStage::new(
|
||||
executor_provider,
|
||||
evm_config,
|
||||
consensus,
|
||||
ExecutionStageThresholds {
|
||||
max_blocks: Some(100),
|
||||
|
||||
@@ -34,7 +34,7 @@ use reth_ethereum::{
|
||||
node::{
|
||||
api::{ConfigureEvm, FullNodeTypes, NodeTypes},
|
||||
node::EthereumAddOns,
|
||||
BasicBlockExecutorProvider, EthereumNode,
|
||||
EthereumNode,
|
||||
},
|
||||
primitives::{Header, SealedHeader},
|
||||
provider::BlockExecutionResult,
|
||||
@@ -76,16 +76,11 @@ where
|
||||
Node: FullNodeTypes<Types = Types>,
|
||||
{
|
||||
type EVM = CustomEvmConfig;
|
||||
type Executor = BasicBlockExecutorProvider<Self::EVM>;
|
||||
|
||||
async fn build_evm(
|
||||
self,
|
||||
ctx: &BuilderContext<Node>,
|
||||
) -> eyre::Result<(Self::EVM, Self::Executor)> {
|
||||
async fn build_evm(self, ctx: &BuilderContext<Node>) -> eyre::Result<Self::EVM> {
|
||||
let evm_config = CustomEvmConfig { inner: EthEvmConfig::new(ctx.chain_spec()) };
|
||||
let executor = BasicBlockExecutorProvider::new(evm_config.clone());
|
||||
|
||||
Ok((evm_config, executor))
|
||||
Ok(evm_config)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -32,7 +32,7 @@ use reth_ethereum::{
|
||||
api::{FullNodeTypes, NodeTypes},
|
||||
core::{args::RpcServerArgs, node_config::NodeConfig},
|
||||
node::EthereumAddOns,
|
||||
BasicBlockExecutorProvider, EthereumNode,
|
||||
EthereumNode,
|
||||
},
|
||||
EthPrimitives,
|
||||
};
|
||||
@@ -84,15 +84,11 @@ where
|
||||
Node: FullNodeTypes<Types: NodeTypes<ChainSpec = ChainSpec, Primitives = EthPrimitives>>,
|
||||
{
|
||||
type EVM = EthEvmConfig<MyEvmFactory>;
|
||||
type Executor = BasicBlockExecutorProvider<Self::EVM>;
|
||||
|
||||
async fn build_evm(
|
||||
self,
|
||||
ctx: &BuilderContext<Node>,
|
||||
) -> eyre::Result<(Self::EVM, Self::Executor)> {
|
||||
async fn build_evm(self, ctx: &BuilderContext<Node>) -> eyre::Result<Self::EVM> {
|
||||
let evm_config =
|
||||
EthEvmConfig::new_with_evm_factory(ctx.chain_spec(), MyEvmFactory::default());
|
||||
Ok((evm_config.clone(), BasicBlockExecutorProvider::new(evm_config)))
|
||||
Ok(evm_config)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -32,7 +32,7 @@ use reth_ethereum::{
|
||||
core::{args::RpcServerArgs, node_config::NodeConfig},
|
||||
evm::EthEvm,
|
||||
node::EthereumAddOns,
|
||||
BasicBlockExecutorProvider, EthEvmConfig, EthereumNode,
|
||||
EthEvmConfig, EthereumNode,
|
||||
},
|
||||
EthPrimitives,
|
||||
};
|
||||
@@ -181,17 +181,13 @@ where
|
||||
Node: FullNodeTypes<Types: NodeTypes<ChainSpec = ChainSpec, Primitives = EthPrimitives>>,
|
||||
{
|
||||
type EVM = EthEvmConfig<MyEvmFactory>;
|
||||
type Executor = BasicBlockExecutorProvider<Self::EVM>;
|
||||
|
||||
async fn build_evm(
|
||||
self,
|
||||
ctx: &BuilderContext<Node>,
|
||||
) -> eyre::Result<(Self::EVM, Self::Executor)> {
|
||||
async fn build_evm(self, ctx: &BuilderContext<Node>) -> eyre::Result<Self::EVM> {
|
||||
let evm_config = EthEvmConfig::new_with_evm_factory(
|
||||
ctx.chain_spec(),
|
||||
MyEvmFactory { precompile_cache: self.precompile_cache.clone() },
|
||||
);
|
||||
Ok((evm_config.clone(), BasicBlockExecutorProvider::new(evm_config)))
|
||||
Ok(evm_config)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -20,12 +20,12 @@ use reth::beacon_consensus::EthBeaconConsensus;
|
||||
use reth_ethereum::{
|
||||
chainspec::ChainSpecBuilder,
|
||||
network::api::noop::NoopNetwork,
|
||||
node::{api::NodeTypesWithDBAdapter, EthEvmConfig, EthExecutorProvider, EthereumNode},
|
||||
node::{api::NodeTypesWithDBAdapter, EthEvmConfig, EthereumNode},
|
||||
pool::noop::NoopTransactionPool,
|
||||
provider::{
|
||||
db::{mdbx::DatabaseArguments, open_db_read_only, ClientVersion, DatabaseEnv},
|
||||
providers::{BlockchainProvider, StaticFileProvider},
|
||||
ChainSpecProvider, ProviderFactory,
|
||||
ProviderFactory,
|
||||
},
|
||||
rpc::{
|
||||
builder::{RethRpcModule, RpcModuleBuilder, RpcServerConfig, TransportRpcModuleConfig},
|
||||
@@ -67,7 +67,6 @@ async fn main() -> eyre::Result<()> {
|
||||
.with_noop_network()
|
||||
.with_executor(TokioTaskExecutor::default())
|
||||
.with_evm_config(EthEvmConfig::new(spec.clone()))
|
||||
.with_block_executor(EthExecutorProvider::ethereum(provider.chain_spec()))
|
||||
.with_consensus(EthBeaconConsensus::new(spec.clone()));
|
||||
|
||||
let eth_api = EthApiBuilder::new(
|
||||
|
||||
@@ -11,7 +11,7 @@ use reth_consensus::{Consensus, HeaderValidator};
|
||||
use reth_db_common::init::{insert_genesis_hashes, insert_genesis_history, insert_genesis_state};
|
||||
use reth_ethereum_consensus::{validate_block_post_execution, EthBeaconConsensus};
|
||||
use reth_ethereum_primitives::Block;
|
||||
use reth_evm::execute::{BlockExecutorProvider, Executor};
|
||||
use reth_evm::{execute::Executor, ConfigureEvm};
|
||||
use reth_evm_ethereum::execute::EthExecutorProvider;
|
||||
use reth_primitives_traits::{RecoveredBlock, SealedBlock};
|
||||
use reth_provider::{
|
||||
@@ -126,10 +126,10 @@ impl BlockchainTestCase {
|
||||
// Non‑processing error – forward as‑is.
|
||||
//
|
||||
// This should only happen if we get an unexpected error from processing the block.
|
||||
// Since it is unexpected, we treat it as a test failure.
|
||||
// Since it is unexpected, we treat it as a test failure.
|
||||
//
|
||||
// One reason for this happening is when one forgets to wrap the error from `run_case`
|
||||
// so that it produces a `Error::BlockProcessingFailed`
|
||||
// so that it produces a `Error::BlockProcessingFailed`
|
||||
Err(other) => Err(other),
|
||||
}
|
||||
}
|
||||
@@ -228,7 +228,7 @@ fn run_case(case: &BlockchainTest) -> Result<(), Error> {
|
||||
|
||||
// Execute the block
|
||||
let state_db = StateProviderDatabase(provider.latest());
|
||||
let executor = executor_provider.executor(state_db);
|
||||
let executor = executor_provider.batch_executor(state_db);
|
||||
let output =
|
||||
executor.execute(block).map_err(|_| Error::BlockProcessingFailed { block_number })?;
|
||||
|
||||
|
||||
Reference in New Issue
Block a user