mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: integrate ExecutorProvider (#7798)
This commit is contained in:
@ -19,6 +19,7 @@ reth-primitives = { workspace = true, features = ["arbitrary", "clap"] }
|
||||
reth-db = { workspace = true, features = ["mdbx"] }
|
||||
reth-exex.workspace = true
|
||||
reth-provider = { workspace = true }
|
||||
reth-evm.workspace = true
|
||||
reth-revm.workspace = true
|
||||
reth-stages.workspace = true
|
||||
reth-interfaces = { workspace = true, features = ["clap"] }
|
||||
|
||||
@ -6,6 +6,7 @@ use crate::{
|
||||
DatabaseArgs,
|
||||
},
|
||||
dirs::{DataDirPath, MaybePlatformPath},
|
||||
macros::block_executor,
|
||||
};
|
||||
use alloy_rlp::Decodable;
|
||||
use clap::Parser;
|
||||
@ -20,10 +21,9 @@ use reth_blockchain_tree::{
|
||||
use reth_cli_runner::CliContext;
|
||||
use reth_consensus::Consensus;
|
||||
use reth_db::{init_db, DatabaseEnv};
|
||||
use reth_evm::execute::{BlockExecutionOutput, BlockExecutorProvider, Executor};
|
||||
use reth_interfaces::RethResult;
|
||||
use reth_node_api::PayloadBuilderAttributes;
|
||||
#[cfg(not(feature = "optimism"))]
|
||||
use reth_node_ethereum::EthEvmConfig;
|
||||
use reth_payload_builder::database::CachedReads;
|
||||
use reth_primitives::{
|
||||
constants::eip4844::{LoadKzgSettingsError, MAINNET_KZG_TRUSTED_SETUP},
|
||||
@ -31,13 +31,14 @@ use reth_primitives::{
|
||||
revm_primitives::KzgSettings,
|
||||
stage::StageId,
|
||||
Address, BlobTransaction, BlobTransactionSidecar, Bytes, ChainSpec, PooledTransactionsElement,
|
||||
SealedBlock, SealedBlockWithSenders, Transaction, TransactionSigned, TxEip4844, B256, U256,
|
||||
Receipts, SealedBlock, SealedBlockWithSenders, Transaction, TransactionSigned, TxEip4844, B256,
|
||||
U256,
|
||||
};
|
||||
use reth_provider::{
|
||||
providers::BlockchainProvider, BlockHashReader, BlockReader, BlockWriter, ExecutorFactory,
|
||||
ProviderFactory, StageCheckpointReader, StateProviderFactory,
|
||||
providers::BlockchainProvider, BlockHashReader, BlockReader, BlockWriter,
|
||||
BundleStateWithReceipts, ProviderFactory, StageCheckpointReader, StateProviderFactory,
|
||||
};
|
||||
use reth_revm::EvmProcessorFactory;
|
||||
use reth_revm::database::StateProviderDatabase;
|
||||
#[cfg(feature = "optimism")]
|
||||
use reth_rpc_types::engine::OptimismPayloadAttributes;
|
||||
use reth_rpc_types::engine::{BlobsBundleV1, PayloadAttributes};
|
||||
@ -161,18 +162,11 @@ impl Command {
|
||||
|
||||
let consensus: Arc<dyn Consensus> = Arc::new(BeaconConsensus::new(Arc::clone(&self.chain)));
|
||||
|
||||
#[cfg(feature = "optimism")]
|
||||
let evm_config = reth_node_optimism::OptimismEvmConfig::default();
|
||||
|
||||
#[cfg(not(feature = "optimism"))]
|
||||
let evm_config = EthEvmConfig::default();
|
||||
let executor = block_executor!(self.chain.clone());
|
||||
|
||||
// configure blockchain tree
|
||||
let tree_externals = TreeExternals::new(
|
||||
provider_factory.clone(),
|
||||
Arc::clone(&consensus),
|
||||
EvmProcessorFactory::new(self.chain.clone(), evm_config),
|
||||
);
|
||||
let tree_externals =
|
||||
TreeExternals::new(provider_factory.clone(), Arc::clone(&consensus), executor);
|
||||
let tree = BlockchainTree::new(tree_externals, BlockchainTreeConfig::default(), None)?;
|
||||
let blockchain_tree = Arc::new(ShareableBlockchainTree::new(tree));
|
||||
|
||||
@ -309,11 +303,16 @@ impl Command {
|
||||
let block_with_senders =
|
||||
SealedBlockWithSenders::new(block.clone(), senders).unwrap();
|
||||
|
||||
let executor_factory = EvmProcessorFactory::new(self.chain.clone(), evm_config);
|
||||
let mut executor = executor_factory.with_state(blockchain_db.latest()?);
|
||||
executor
|
||||
.execute_and_verify_receipt(&block_with_senders.clone().unseal(), U256::MAX)?;
|
||||
let state = executor.take_output_state();
|
||||
let db = StateProviderDatabase::new(blockchain_db.latest()?);
|
||||
let executor = block_executor!(self.chain.clone()).executor(db);
|
||||
|
||||
let BlockExecutionOutput { state, receipts, .. } =
|
||||
executor.execute((&block_with_senders.clone().unseal(), U256::MAX).into())?;
|
||||
let state = BundleStateWithReceipts::new(
|
||||
state,
|
||||
Receipts::from_block_receipt(receipts),
|
||||
block.number,
|
||||
);
|
||||
debug!(target: "reth::cli", ?state, "Executed block");
|
||||
|
||||
let hashed_state = state.hash_state_slow();
|
||||
|
||||
@ -7,6 +7,7 @@ use crate::{
|
||||
DatabaseArgs, NetworkArgs,
|
||||
},
|
||||
dirs::{DataDirPath, MaybePlatformPath},
|
||||
macros::block_executor,
|
||||
utils::get_single_header,
|
||||
};
|
||||
use clap::Parser;
|
||||
@ -25,7 +26,6 @@ use reth_interfaces::p2p::{bodies::client::BodiesClient, headers::client::Header
|
||||
use reth_network::{NetworkEvents, NetworkHandle};
|
||||
use reth_network_api::NetworkInfo;
|
||||
use reth_node_core::init::init_genesis;
|
||||
use reth_node_ethereum::EthEvmConfig;
|
||||
use reth_primitives::{
|
||||
fs, stage::StageId, BlockHashOrNumber, BlockNumber, ChainSpec, PruneModes, B256,
|
||||
};
|
||||
@ -111,8 +111,7 @@ impl Command {
|
||||
let stage_conf = &config.stages;
|
||||
|
||||
let (tip_tx, tip_rx) = watch::channel(B256::ZERO);
|
||||
let factory =
|
||||
reth_revm::EvmProcessorFactory::new(self.chain.clone(), EthEvmConfig::default());
|
||||
let executor = block_executor!(self.chain.clone());
|
||||
|
||||
let header_mode = HeaderSyncMode::Tip(tip_rx);
|
||||
let pipeline = Pipeline::builder()
|
||||
@ -124,14 +123,14 @@ impl Command {
|
||||
Arc::clone(&consensus),
|
||||
header_downloader,
|
||||
body_downloader,
|
||||
factory.clone(),
|
||||
executor.clone(),
|
||||
stage_conf.etl.clone(),
|
||||
)
|
||||
.set(SenderRecoveryStage {
|
||||
commit_threshold: stage_conf.sender_recovery.commit_threshold,
|
||||
})
|
||||
.set(ExecutionStage::new(
|
||||
factory,
|
||||
executor,
|
||||
ExecutionStageThresholds {
|
||||
max_blocks: None,
|
||||
max_changes: None,
|
||||
|
||||
@ -7,6 +7,7 @@ use crate::{
|
||||
DatabaseArgs, NetworkArgs,
|
||||
},
|
||||
dirs::{DataDirPath, MaybePlatformPath},
|
||||
macros::block_executor,
|
||||
utils::{get_single_body, get_single_header},
|
||||
};
|
||||
use backon::{ConstantBuilder, Retryable};
|
||||
@ -14,16 +15,17 @@ use clap::Parser;
|
||||
use reth_cli_runner::CliContext;
|
||||
use reth_config::Config;
|
||||
use reth_db::{init_db, DatabaseEnv};
|
||||
use reth_evm::execute::{BlockExecutionOutput, BlockExecutorProvider, Executor};
|
||||
use reth_interfaces::executor::BlockValidationError;
|
||||
use reth_network::NetworkHandle;
|
||||
use reth_network_api::NetworkInfo;
|
||||
use reth_node_ethereum::EthEvmConfig;
|
||||
use reth_primitives::{fs, stage::StageId, BlockHashOrNumber, ChainSpec};
|
||||
use reth_primitives::{fs, stage::StageId, BlockHashOrNumber, ChainSpec, Receipts};
|
||||
use reth_provider::{
|
||||
AccountExtReader, ExecutorFactory, HashingWriter, HeaderProvider, LatestStateProviderRef,
|
||||
OriginalValuesKnown, ProviderFactory, StageCheckpointReader, StaticFileProviderFactory,
|
||||
StorageReader,
|
||||
AccountExtReader, BundleStateWithReceipts, HashingWriter, HeaderProvider,
|
||||
LatestStateProviderRef, OriginalValuesKnown, ProviderFactory, StageCheckpointReader,
|
||||
StaticFileProviderFactory, StorageReader,
|
||||
};
|
||||
use reth_revm::database::StateProviderDatabase;
|
||||
use reth_tasks::TaskExecutor;
|
||||
use reth_trie::{updates::TrieKey, StateRoot};
|
||||
use std::{net::SocketAddr, path::PathBuf, sync::Arc};
|
||||
@ -162,24 +164,31 @@ impl Command {
|
||||
)
|
||||
.await?;
|
||||
|
||||
let executor_factory =
|
||||
reth_revm::EvmProcessorFactory::new(self.chain.clone(), EthEvmConfig::default());
|
||||
let mut executor = executor_factory.with_state(LatestStateProviderRef::new(
|
||||
let db = StateProviderDatabase::new(LatestStateProviderRef::new(
|
||||
provider.tx_ref(),
|
||||
factory.static_file_provider(),
|
||||
));
|
||||
|
||||
let executor = block_executor!(self.chain.clone()).executor(db);
|
||||
|
||||
let merkle_block_td =
|
||||
provider.header_td_by_number(merkle_block_number)?.unwrap_or_default();
|
||||
executor.execute_and_verify_receipt(
|
||||
&block
|
||||
.clone()
|
||||
.unseal()
|
||||
.with_recovered_senders()
|
||||
.ok_or(BlockValidationError::SenderRecoveryError)?,
|
||||
merkle_block_td + block.difficulty,
|
||||
let BlockExecutionOutput { state, receipts, .. } = executor.execute(
|
||||
(
|
||||
&block
|
||||
.clone()
|
||||
.unseal()
|
||||
.with_recovered_senders()
|
||||
.ok_or(BlockValidationError::SenderRecoveryError)?,
|
||||
merkle_block_td + block.difficulty,
|
||||
)
|
||||
.into(),
|
||||
)?;
|
||||
let block_state = executor.take_output_state();
|
||||
let block_state = BundleStateWithReceipts::new(
|
||||
state,
|
||||
Receipts::from_block_receipt(receipts),
|
||||
block.number,
|
||||
);
|
||||
|
||||
// Unpacked `BundleState::state_root_slow` function
|
||||
let (in_memory_state_root, in_memory_updates) =
|
||||
|
||||
@ -7,6 +7,7 @@ use crate::{
|
||||
DatabaseArgs, NetworkArgs,
|
||||
},
|
||||
dirs::{DataDirPath, MaybePlatformPath},
|
||||
macros::block_executor,
|
||||
utils::get_single_header,
|
||||
};
|
||||
use backon::{ConstantBuilder, Retryable};
|
||||
@ -20,7 +21,6 @@ use reth_exex::ExExManagerHandle;
|
||||
use reth_interfaces::p2p::full_block::FullBlockClient;
|
||||
use reth_network::NetworkHandle;
|
||||
use reth_network_api::NetworkInfo;
|
||||
use reth_node_ethereum::EthEvmConfig;
|
||||
use reth_primitives::{
|
||||
fs,
|
||||
stage::{StageCheckpoint, StageId},
|
||||
@ -201,10 +201,9 @@ impl Command {
|
||||
checkpoint.stage_checkpoint.is_some()
|
||||
});
|
||||
|
||||
let factory =
|
||||
reth_revm::EvmProcessorFactory::new(self.chain.clone(), EthEvmConfig::default());
|
||||
let executor = block_executor!(self.chain.clone());
|
||||
let mut execution_stage = ExecutionStage::new(
|
||||
factory,
|
||||
executor,
|
||||
ExecutionStageThresholds {
|
||||
max_blocks: Some(1),
|
||||
max_changes: None,
|
||||
|
||||
@ -5,6 +5,7 @@ use crate::{
|
||||
DatabaseArgs, NetworkArgs,
|
||||
},
|
||||
dirs::{DataDirPath, MaybePlatformPath},
|
||||
macros::block_executor,
|
||||
};
|
||||
use clap::Parser;
|
||||
use eyre::Context;
|
||||
@ -20,15 +21,12 @@ use reth_db::{init_db, DatabaseEnv};
|
||||
use reth_network::NetworkHandle;
|
||||
use reth_network_api::NetworkInfo;
|
||||
use reth_node_core::engine::engine_store::{EngineMessageStore, StoredEngineApiMessage};
|
||||
#[cfg(not(feature = "optimism"))]
|
||||
use reth_node_ethereum::{EthEngineTypes, EthEvmConfig};
|
||||
use reth_payload_builder::{PayloadBuilderHandle, PayloadBuilderService};
|
||||
use reth_primitives::{fs, ChainSpec, PruneModes};
|
||||
use reth_provider::{
|
||||
providers::BlockchainProvider, CanonStateSubscriptions, ProviderFactory,
|
||||
StaticFileProviderFactory,
|
||||
};
|
||||
use reth_revm::EvmProcessorFactory;
|
||||
use reth_stages::Pipeline;
|
||||
use reth_static_file::StaticFileProducer;
|
||||
use reth_tasks::TaskExecutor;
|
||||
@ -126,18 +124,11 @@ impl Command {
|
||||
|
||||
let consensus: Arc<dyn Consensus> = Arc::new(BeaconConsensus::new(Arc::clone(&self.chain)));
|
||||
|
||||
#[cfg(not(feature = "optimism"))]
|
||||
let evm_config = EthEvmConfig::default();
|
||||
|
||||
#[cfg(feature = "optimism")]
|
||||
let evm_config = reth_node_optimism::OptimismEvmConfig::default();
|
||||
let executor = block_executor!(self.chain.clone());
|
||||
|
||||
// Configure blockchain tree
|
||||
let tree_externals = TreeExternals::new(
|
||||
provider_factory.clone(),
|
||||
Arc::clone(&consensus),
|
||||
EvmProcessorFactory::new(self.chain.clone(), evm_config),
|
||||
);
|
||||
let tree_externals =
|
||||
TreeExternals::new(provider_factory.clone(), Arc::clone(&consensus), executor);
|
||||
let tree = BlockchainTree::new(tree_externals, BlockchainTreeConfig::default(), None)?;
|
||||
let blockchain_tree = Arc::new(ShareableBlockchainTree::new(tree));
|
||||
|
||||
@ -184,8 +175,10 @@ impl Command {
|
||||
) = PayloadBuilderService::new(payload_generator, blockchain_db.canonical_state_stream());
|
||||
|
||||
#[cfg(not(feature = "optimism"))]
|
||||
let (payload_service, payload_builder): (_, PayloadBuilderHandle<EthEngineTypes>) =
|
||||
PayloadBuilderService::new(payload_generator, blockchain_db.canonical_state_stream());
|
||||
let (payload_service, payload_builder): (
|
||||
_,
|
||||
PayloadBuilderHandle<reth_node_ethereum::EthEngineTypes>,
|
||||
) = PayloadBuilderService::new(payload_generator, blockchain_db.canonical_state_stream());
|
||||
|
||||
ctx.task_executor.spawn_critical("payload builder service", payload_service);
|
||||
|
||||
|
||||
@ -6,6 +6,7 @@ use crate::{
|
||||
DatabaseArgs,
|
||||
},
|
||||
dirs::{DataDirPath, MaybePlatformPath},
|
||||
macros::block_executor,
|
||||
version::SHORT_VERSION,
|
||||
};
|
||||
use clap::Parser;
|
||||
@ -26,7 +27,6 @@ use reth_interfaces::p2p::{
|
||||
headers::downloader::{HeaderDownloader, SyncTarget},
|
||||
};
|
||||
use reth_node_core::init::init_genesis;
|
||||
use reth_node_ethereum::EthEvmConfig;
|
||||
use reth_node_events::node::NodeEvent;
|
||||
use reth_primitives::{stage::StageId, ChainSpec, PruneModes, B256};
|
||||
use reth_provider::{
|
||||
@ -269,8 +269,7 @@ where
|
||||
.expect("failed to set download range");
|
||||
|
||||
let (tip_tx, tip_rx) = watch::channel(B256::ZERO);
|
||||
let factory =
|
||||
reth_revm::EvmProcessorFactory::new(provider_factory.chain_spec(), EthEvmConfig::default());
|
||||
let executor = block_executor!(provider_factory.chain_spec());
|
||||
|
||||
let max_block = file_client.max_block().unwrap_or(0);
|
||||
|
||||
@ -285,14 +284,14 @@ where
|
||||
consensus.clone(),
|
||||
header_downloader,
|
||||
body_downloader,
|
||||
factory.clone(),
|
||||
executor.clone(),
|
||||
config.stages.etl.clone(),
|
||||
)
|
||||
.set(SenderRecoveryStage {
|
||||
commit_threshold: config.stages.sender_recovery.commit_threshold,
|
||||
})
|
||||
.set(ExecutionStage::new(
|
||||
factory,
|
||||
executor,
|
||||
ExecutionStageThresholds {
|
||||
max_blocks: config.stages.execution.max_blocks,
|
||||
max_changes: config.stages.execution.max_changes,
|
||||
|
||||
@ -1,15 +1,12 @@
|
||||
use super::setup;
|
||||
use crate::utils::DbTool;
|
||||
use eyre::Result;
|
||||
use crate::{macros::block_executor, utils::DbTool};
|
||||
use reth_db::{
|
||||
cursor::DbCursorRO, database::Database, table::TableImporter, tables, transaction::DbTx,
|
||||
DatabaseEnv,
|
||||
};
|
||||
use reth_node_core::dirs::{ChainPath, DataDirPath};
|
||||
use reth_node_ethereum::EthEvmConfig;
|
||||
use reth_primitives::stage::StageCheckpoint;
|
||||
use reth_provider::{ChainSpecProvider, ProviderFactory};
|
||||
use reth_revm::EvmProcessorFactory;
|
||||
use reth_stages::{stages::ExecutionStage, Stage, UnwindInput};
|
||||
use tracing::info;
|
||||
|
||||
@ -19,7 +16,7 @@ pub(crate) async fn dump_execution_stage<DB: Database>(
|
||||
to: u64,
|
||||
output_datadir: ChainPath<DataDirPath>,
|
||||
should_run: bool,
|
||||
) -> Result<()> {
|
||||
) -> eyre::Result<()> {
|
||||
let (output_db, tip_block_number) = setup(from, to, &output_datadir.db(), db_tool)?;
|
||||
|
||||
import_tables_with_range(&output_db, db_tool, from, to)?;
|
||||
@ -127,10 +124,8 @@ async fn unwind_and_copy<DB: Database>(
|
||||
) -> eyre::Result<()> {
|
||||
let provider = db_tool.provider_factory.provider_rw()?;
|
||||
|
||||
let mut exec_stage = ExecutionStage::new_with_factory(EvmProcessorFactory::new(
|
||||
db_tool.chain.clone(),
|
||||
EthEvmConfig::default(),
|
||||
));
|
||||
let executor = block_executor!(db_tool.chain.clone());
|
||||
let mut exec_stage = ExecutionStage::new_with_executor(executor);
|
||||
|
||||
exec_stage.unwind(
|
||||
&provider,
|
||||
@ -159,10 +154,8 @@ async fn dry_run<DB: Database>(
|
||||
) -> eyre::Result<()> {
|
||||
info!(target: "reth::cli", "Executing stage. [dry-run]");
|
||||
|
||||
let mut exec_stage = ExecutionStage::new_with_factory(EvmProcessorFactory::new(
|
||||
output_provider_factory.chain_spec(),
|
||||
EthEvmConfig::default(),
|
||||
));
|
||||
let executor = block_executor!(output_provider_factory.chain_spec());
|
||||
let mut exec_stage = ExecutionStage::new_with_executor(executor);
|
||||
|
||||
let input =
|
||||
reth_stages::ExecInput { target: Some(to), checkpoint: Some(StageCheckpoint::new(from)) };
|
||||
|
||||
@ -1,11 +1,10 @@
|
||||
use super::setup;
|
||||
use crate::utils::DbTool;
|
||||
use crate::{macros::block_executor, utils::DbTool};
|
||||
use eyre::Result;
|
||||
use reth_config::config::EtlConfig;
|
||||
use reth_db::{database::Database, table::TableImporter, tables, DatabaseEnv};
|
||||
use reth_exex::ExExManagerHandle;
|
||||
use reth_node_core::dirs::{ChainPath, DataDirPath};
|
||||
use reth_node_ethereum::EthEvmConfig;
|
||||
use reth_primitives::{stage::StageCheckpoint, BlockNumber, PruneModes};
|
||||
use reth_provider::ProviderFactory;
|
||||
use reth_stages::{
|
||||
@ -81,9 +80,11 @@ async fn unwind_and_copy<DB: Database>(
|
||||
|
||||
MerkleStage::default_unwind().unwind(&provider, unwind)?;
|
||||
|
||||
let executor = block_executor!(db_tool.chain.clone());
|
||||
|
||||
// Bring Plainstate to TO (hashing stage execution requires it)
|
||||
let mut exec_stage = ExecutionStage::new(
|
||||
reth_revm::EvmProcessorFactory::new(db_tool.chain.clone(), EthEvmConfig::default()),
|
||||
executor,
|
||||
ExecutionStageThresholds {
|
||||
max_blocks: Some(u64::MAX),
|
||||
max_changes: None,
|
||||
|
||||
@ -9,6 +9,7 @@ use crate::{
|
||||
DatabaseArgs, NetworkArgs, StageEnum,
|
||||
},
|
||||
dirs::{DataDirPath, MaybePlatformPath},
|
||||
macros::block_executor,
|
||||
prometheus_exporter,
|
||||
version::SHORT_VERSION,
|
||||
};
|
||||
@ -19,7 +20,6 @@ use reth_config::{config::EtlConfig, Config};
|
||||
use reth_db::init_db;
|
||||
use reth_downloaders::bodies::bodies::BodiesDownloaderBuilder;
|
||||
use reth_exex::ExExManagerHandle;
|
||||
use reth_node_ethereum::EthEvmConfig;
|
||||
use reth_primitives::ChainSpec;
|
||||
use reth_provider::{
|
||||
ProviderFactory, StageCheckpointReader, StageCheckpointWriter, StaticFileProviderFactory,
|
||||
@ -224,13 +224,10 @@ impl Command {
|
||||
}
|
||||
StageEnum::Senders => (Box::new(SenderRecoveryStage::new(batch_size)), None),
|
||||
StageEnum::Execution => {
|
||||
let factory = reth_revm::EvmProcessorFactory::new(
|
||||
self.chain.clone(),
|
||||
EthEvmConfig::default(),
|
||||
);
|
||||
let executor = block_executor!(self.chain.clone());
|
||||
(
|
||||
Box::new(ExecutionStage::new(
|
||||
factory,
|
||||
executor,
|
||||
ExecutionStageThresholds {
|
||||
max_blocks: Some(batch_size),
|
||||
max_changes: None,
|
||||
|
||||
@ -1,12 +1,5 @@
|
||||
//! Unwinding a certain block range
|
||||
|
||||
use crate::{
|
||||
args::{
|
||||
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
|
||||
DatabaseArgs,
|
||||
},
|
||||
dirs::{DataDirPath, MaybePlatformPath},
|
||||
};
|
||||
use clap::{Parser, Subcommand};
|
||||
use reth_beacon_consensus::BeaconConsensus;
|
||||
use reth_config::{Config, PruneConfig};
|
||||
@ -21,7 +14,6 @@ use reth_node_core::{
|
||||
args::{get_secret_key, NetworkArgs},
|
||||
dirs::ChainPath,
|
||||
};
|
||||
use reth_node_ethereum::EthEvmConfig;
|
||||
use reth_primitives::{BlockHashOrNumber, ChainSpec, PruneModes, B256};
|
||||
use reth_provider::{
|
||||
BlockExecutionWriter, BlockNumReader, ChainSpecProvider, HeaderSyncMode, ProviderFactory,
|
||||
@ -42,6 +34,15 @@ use std::{ops::RangeInclusive, sync::Arc};
|
||||
use tokio::sync::watch;
|
||||
use tracing::info;
|
||||
|
||||
use crate::{
|
||||
args::{
|
||||
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
|
||||
DatabaseArgs,
|
||||
},
|
||||
dirs::{DataDirPath, MaybePlatformPath},
|
||||
macros::block_executor,
|
||||
};
|
||||
|
||||
/// `reth stage unwind` command
|
||||
#[derive(Debug, Parser)]
|
||||
pub struct Command {
|
||||
@ -178,10 +179,7 @@ impl Command {
|
||||
let stage_conf = &config.stages;
|
||||
|
||||
let (tip_tx, tip_rx) = watch::channel(B256::ZERO);
|
||||
let factory = reth_revm::EvmProcessorFactory::new(
|
||||
provider_factory.chain_spec(),
|
||||
EthEvmConfig::default(),
|
||||
);
|
||||
let executor = block_executor!(provider_factory.chain_spec());
|
||||
|
||||
let header_mode = HeaderSyncMode::Tip(tip_rx);
|
||||
let pipeline = Pipeline::builder()
|
||||
@ -193,14 +191,14 @@ impl Command {
|
||||
Arc::clone(&consensus),
|
||||
header_downloader,
|
||||
body_downloader,
|
||||
factory.clone(),
|
||||
executor.clone(),
|
||||
stage_conf.etl.clone(),
|
||||
)
|
||||
.set(SenderRecoveryStage {
|
||||
commit_threshold: stage_conf.sender_recovery.commit_threshold,
|
||||
})
|
||||
.set(ExecutionStage::new(
|
||||
factory,
|
||||
executor,
|
||||
ExecutionStageThresholds {
|
||||
max_blocks: None,
|
||||
max_changes: None,
|
||||
|
||||
@ -31,6 +31,7 @@
|
||||
|
||||
pub mod cli;
|
||||
pub mod commands;
|
||||
mod macros;
|
||||
pub mod utils;
|
||||
|
||||
/// Re-exported payload related types
|
||||
|
||||
20
bin/reth/src/macros.rs
Normal file
20
bin/reth/src/macros.rs
Normal file
@ -0,0 +1,20 @@
|
||||
//! Helper macros
|
||||
|
||||
/// Creates the block executor type based on the configured feature.
|
||||
///
|
||||
/// Note(mattsse): This is incredibly horrible and will be replaced
|
||||
#[cfg(not(feature = "optimism"))]
|
||||
macro_rules! block_executor {
|
||||
($chain_spec:expr) => {
|
||||
reth_node_ethereum::EthExecutorProvider::ethereum($chain_spec)
|
||||
};
|
||||
}
|
||||
|
||||
#[cfg(feature = "optimism")]
|
||||
macro_rules! block_executor {
|
||||
($chain_spec:expr) => {
|
||||
reth_node_optimism::OpExecutorProvider::optimism($chain_spec)
|
||||
};
|
||||
}
|
||||
|
||||
pub(crate) use block_executor;
|
||||
Reference in New Issue
Block a user