feat: integrate builder (#6611)

This commit is contained in:
Matthias Seitz
2024-02-29 17:50:04 +01:00
committed by GitHub
parent 7d36206dfe
commit c5955f1305
73 changed files with 2201 additions and 3022 deletions

View File

@ -53,6 +53,7 @@ reth-node-optimism = { workspace = true, optional = true, features = [
"optimism",
] }
reth-node-core.workspace = true
reth-node-builder.workspace = true
# crypto
alloy-rlp.workspace = true
@ -137,8 +138,6 @@ optimism = [
"reth-blockchain-tree/optimism",
"reth-payload-builder/optimism",
"reth-optimism-payload-builder/optimism",
"reth-ethereum-payload-builder/optimism",
"reth-node-ethereum/optimism",
"dep:reth-node-optimism",
"reth-node-core/optimism",
]

View File

@ -1,849 +0,0 @@
//! Contains types and methods that can be used to launch a node based off of a [NodeConfig].
use crate::commands::debug_cmd::engine_api_store::EngineApiStore;
use eyre::Context;
use fdlimit::raise_fd_limit;
use futures::{future::Either, stream, stream_select, StreamExt};
use reth_auto_seal_consensus::AutoSealBuilder;
use reth_beacon_consensus::{
hooks::{EngineHooks, PruneHook, StaticFileHook},
BeaconConsensusEngine, MIN_BLOCKS_FOR_PIPELINE_RUN,
};
use reth_blockchain_tree::{config::BlockchainTreeConfig, ShareableBlockchainTree};
use reth_config::Config;
use reth_db::{
database::Database,
database_metrics::{DatabaseMetadata, DatabaseMetrics},
};
use reth_interfaces::p2p::either::EitherDownloader;
use reth_network::NetworkEvents;
use reth_network_api::{NetworkInfo, PeersInfo};
use reth_node_core::{
cli::{
components::{RethNodeComponentsImpl, RethRpcServerHandles},
config::RethRpcConfig,
db_type::DatabaseInstance,
ext::{DefaultRethNodeCommandConfig, RethCliExt, RethNodeCommandConfig},
},
dirs::{ChainPath, DataDirPath},
events::cl::ConsensusLayerHealthEvents,
exit::NodeExitFuture,
init::init_genesis,
version::SHORT_VERSION,
};
#[cfg(not(feature = "optimism"))]
use reth_node_ethereum::{EthEngineTypes, EthEvmConfig};
#[cfg(feature = "optimism")]
use reth_node_optimism::{OptimismEngineTypes, OptimismEvmConfig};
use reth_payload_builder::PayloadBuilderHandle;
use reth_primitives::format_ether;
use reth_provider::{providers::BlockchainProvider, ProviderFactory};
use reth_prune::PrunerBuilder;
use reth_rpc_engine_api::EngineApi;
use reth_static_file::StaticFileProducer;
use reth_tasks::{TaskExecutor, TaskManager};
use reth_transaction_pool::TransactionPool;
use std::{path::PathBuf, sync::Arc};
use tokio::sync::{mpsc::unbounded_channel, oneshot};
use tracing::*;
/// Re-export `NodeConfig` from `reth_node_core`.
pub use reth_node_core::node_config::NodeConfig;
/// Launches the node, also adding any RPC extensions passed.
///
/// # Example
/// ```rust
/// # use reth_tasks::{TaskManager, TaskSpawner};
/// # use reth_node_core::node_config::NodeConfig;
/// # use reth_node_core::cli::{
/// # ext::DefaultRethNodeCommandConfig,
/// # };
/// # use tokio::runtime::Handle;
/// # use reth::builder::launch_from_config;
///
/// async fn t() {
/// let handle = Handle::current();
/// let manager = TaskManager::new(handle);
/// let executor = manager.executor();
/// let builder = NodeConfig::default();
/// let ext = DefaultRethNodeCommandConfig::default();
/// let handle = launch_from_config::<()>(builder, ext, executor).await.unwrap();
/// }
/// ```
pub async fn launch_from_config<E: RethCliExt>(
mut config: NodeConfig,
ext: E::Node,
executor: TaskExecutor,
) -> eyre::Result<NodeHandle> {
info!(target: "reth::cli", "reth {} starting", SHORT_VERSION);
// Register the prometheus recorder before creating the database,
// because database init needs it to register metrics.
config.install_prometheus_recorder()?;
let database = std::mem::take(&mut config.database);
let db_instance = database.init_db(config.db.log_level, config.chain.chain)?;
info!(target: "reth::cli", "Database opened");
match db_instance {
DatabaseInstance::Real { db, data_dir } => {
let builder = NodeBuilderWithDatabase { config, db, data_dir };
builder.launch::<E>(ext, executor).await
}
DatabaseInstance::Test { db, data_dir } => {
let builder = NodeBuilderWithDatabase { config, db, data_dir };
builder.launch::<E>(ext, executor).await
}
}
}
/// A version of the [NodeConfig] that has an installed database. This is used to construct the
/// [NodeHandle].
///
/// This also contains a path to a data dir that cannot be changed.
#[derive(Debug)]
pub struct NodeBuilderWithDatabase<DB> {
/// The node config
pub config: NodeConfig,
/// The database
pub db: Arc<DB>,
/// The data dir
pub data_dir: ChainPath<DataDirPath>,
}
impl<DB: Database + DatabaseMetrics + DatabaseMetadata + 'static> NodeBuilderWithDatabase<DB> {
/// Launch the node with the given extensions and executor
pub async fn launch<E: RethCliExt>(
mut self,
mut ext: E::Node,
executor: TaskExecutor,
) -> eyre::Result<NodeHandle> {
// Raise the fd limit of the process.
// Does not do anything on windows.
raise_fd_limit()?;
// get config
let config = self.load_config()?;
let prometheus_handle = self.config.install_prometheus_recorder()?;
let provider_factory = ProviderFactory::new(
Arc::clone(&self.db),
Arc::clone(&self.config.chain),
self.data_dir.static_files_path(),
)?
.with_static_files_metrics();
self.config.start_metrics_endpoint(prometheus_handle, Arc::clone(&self.db)).await?;
debug!(target: "reth::cli", chain=%self.config.chain.chain, genesis=?self.config.chain.genesis_hash(), "Initializing genesis");
let genesis_hash = init_genesis(provider_factory.clone())?;
info!(target: "reth::cli", "{}", self.config.chain.display_hardforks());
let consensus = self.config.consensus();
debug!(target: "reth::cli", "Spawning stages metrics listener task");
let (sync_metrics_tx, sync_metrics_rx) = unbounded_channel();
let sync_metrics_listener = reth_stages::MetricsListener::new(sync_metrics_rx);
executor.spawn_critical("stages metrics listener task", sync_metrics_listener);
let prune_config = self
.config
.pruning
.prune_config(Arc::clone(&self.config.chain))?
.or(config.prune.clone());
// TODO: stateful node builder should be able to remove cfgs here
#[cfg(feature = "optimism")]
let evm_config = OptimismEvmConfig::default();
// The default payload builder is implemented on the unit type.
#[cfg(not(feature = "optimism"))]
let evm_config = EthEvmConfig::default();
// configure blockchain tree
let tree_config = BlockchainTreeConfig::default();
let tree = self.config.build_blockchain_tree(
provider_factory.clone(),
consensus.clone(),
prune_config.clone(),
sync_metrics_tx.clone(),
tree_config,
evm_config,
)?;
let canon_state_notification_sender = tree.canon_state_notification_sender();
let blockchain_tree = ShareableBlockchainTree::new(tree);
debug!(target: "reth::cli", "configured blockchain tree");
// fetch the head block from the database
let head = self
.config
.lookup_head(provider_factory.clone())
.wrap_err("the head block is missing")?;
// setup the blockchain provider
let blockchain_db =
BlockchainProvider::new(provider_factory.clone(), blockchain_tree.clone())?;
// build transaction pool
let transaction_pool =
self.config.build_and_spawn_txpool(&blockchain_db, head, &executor, &self.data_dir)?;
// build network
let mut network_builder = self
.config
.build_network(
&config,
provider_factory.clone(),
executor.clone(),
head,
&self.data_dir,
)
.await?;
let components = RethNodeComponentsImpl::new(
blockchain_db.clone(),
transaction_pool.clone(),
network_builder.handle(),
executor.clone(),
blockchain_db.clone(),
evm_config,
);
// allow network modifications
ext.configure_network(network_builder.network_mut(), &components)?;
// launch network
let network = self.config.start_network(
network_builder,
&executor,
transaction_pool.clone(),
provider_factory.clone(),
&self.data_dir,
);
info!(target: "reth::cli", peer_id = %network.peer_id(), local_addr = %network.local_addr(), enode = %network.local_node_record(), "Connected to P2P network");
debug!(target: "reth::cli", peer_id = ?network.peer_id(), "Full peer ID");
let network_client = network.fetch_client().await?;
ext.on_components_initialized(&components)?;
debug!(target: "reth::cli", "Spawning payload builder service");
// TODO: stateful node builder should handle this in with_payload_builder
// Optimism's payload builder is implemented on the OptimismPayloadBuilder type.
#[cfg(feature = "optimism")]
let payload_builder = reth_optimism_payload_builder::OptimismPayloadBuilder::default()
.set_compute_pending_block(self.config.builder.compute_pending_block);
#[cfg(feature = "optimism")]
let payload_builder: PayloadBuilderHandle<OptimismEngineTypes> =
ext.spawn_payload_builder_service(&self.config.builder, &components, payload_builder)?;
// The default payload builder is implemented on the unit type.
#[cfg(not(feature = "optimism"))]
let payload_builder = reth_ethereum_payload_builder::EthereumPayloadBuilder::default();
#[cfg(not(feature = "optimism"))]
let payload_builder: PayloadBuilderHandle<EthEngineTypes> =
ext.spawn_payload_builder_service(&self.config.builder, &components, payload_builder)?;
let (consensus_engine_tx, mut consensus_engine_rx) = unbounded_channel();
if let Some(store_path) = self.config.debug.engine_api_store.clone() {
let (engine_intercept_tx, engine_intercept_rx) = unbounded_channel();
let engine_api_store = EngineApiStore::new(store_path);
executor.spawn_critical(
"engine api interceptor",
engine_api_store.intercept(consensus_engine_rx, engine_intercept_tx),
);
consensus_engine_rx = engine_intercept_rx;
};
let max_block = self.config.max_block(&network_client, provider_factory.clone()).await?;
let mut hooks = EngineHooks::new();
let mut static_file_producer = StaticFileProducer::new(
provider_factory.clone(),
provider_factory.static_file_provider(),
prune_config.clone().unwrap_or_default().segments,
);
let static_file_producer_events = static_file_producer.events();
hooks.add(StaticFileHook::new(static_file_producer.clone(), Box::new(executor.clone())));
info!(target: "reth::cli", "StaticFileProducer initialized");
// Configure the pipeline
let (mut pipeline, client) = if self.config.dev.dev {
info!(target: "reth::cli", "Starting Reth in dev mode");
for (idx, (address, alloc)) in self.config.chain.genesis.alloc.iter().enumerate() {
info!(target: "reth::cli", "Allocated Genesis Account: {:02}. {} ({} ETH)", idx, address.to_string(), format_ether(alloc.balance));
}
let mining_mode =
self.config.mining_mode(transaction_pool.pending_transactions_listener());
let (_, client, mut task) = AutoSealBuilder::new(
Arc::clone(&self.config.chain),
blockchain_db.clone(),
transaction_pool.clone(),
consensus_engine_tx.clone(),
canon_state_notification_sender,
mining_mode,
evm_config,
)
.build();
let mut pipeline = self
.config
.build_networked_pipeline(
&config.stages,
client.clone(),
Arc::clone(&consensus),
provider_factory.clone(),
&executor,
sync_metrics_tx,
prune_config.clone(),
max_block,
static_file_producer,
evm_config,
)
.await?;
let pipeline_events = pipeline.events();
task.set_pipeline_events(pipeline_events);
debug!(target: "reth::cli", "Spawning auto mine task");
executor.spawn(Box::pin(task));
(pipeline, EitherDownloader::Left(client))
} else {
let pipeline = self
.config
.build_networked_pipeline(
&config.stages,
network_client.clone(),
Arc::clone(&consensus),
provider_factory.clone(),
&executor.clone(),
sync_metrics_tx,
prune_config.clone(),
max_block,
static_file_producer,
evm_config,
)
.await?;
(pipeline, EitherDownloader::Right(network_client))
};
let pipeline_events = pipeline.events();
let initial_target = self.config.initial_pipeline_target(genesis_hash);
let prune_config = prune_config.unwrap_or_default();
let mut pruner = PrunerBuilder::new(prune_config.clone())
.max_reorg_depth(tree_config.max_reorg_depth() as usize)
.prune_delete_limit(self.config.chain.prune_delete_limit)
.build(provider_factory.clone());
let pruner_events = pruner.events();
hooks.add(PruneHook::new(pruner, Box::new(executor.clone())));
info!(target: "reth::cli", ?prune_config, "Pruner initialized");
// Configure the consensus engine
let (beacon_consensus_engine, beacon_engine_handle) = BeaconConsensusEngine::with_channel(
client,
pipeline,
blockchain_db.clone(),
Box::new(executor.clone()),
Box::new(network.clone()),
max_block,
self.config.debug.continuous,
payload_builder.clone(),
initial_target,
MIN_BLOCKS_FOR_PIPELINE_RUN,
consensus_engine_tx,
consensus_engine_rx,
hooks,
)?;
info!(target: "reth::cli", "Consensus engine initialized");
let events = stream_select!(
network.event_listener().map(Into::into),
beacon_engine_handle.event_listener().map(Into::into),
pipeline_events.map(Into::into),
if self.config.debug.tip.is_none() && !self.config.dev.dev {
Either::Left(
ConsensusLayerHealthEvents::new(Box::new(blockchain_db.clone()))
.map(Into::into),
)
} else {
Either::Right(stream::empty())
},
pruner_events.map(Into::into),
static_file_producer_events.map(Into::into),
);
executor.spawn_critical(
"events task",
reth_node_core::events::node::handle_events(
Some(network.clone()),
Some(head.number),
events,
self.db.clone(),
),
);
let engine_api = EngineApi::new(
blockchain_db.clone(),
self.config.chain.clone(),
beacon_engine_handle,
payload_builder.into(),
Box::new(executor.clone()),
);
info!(target: "reth::cli", "Engine API handler initialized");
// extract the jwt secret from the args if possible
let default_jwt_path = self.data_dir.jwt_path();
let jwt_secret = self.config.rpc.auth_jwt_secret(default_jwt_path)?;
// adjust rpc port numbers based on instance number
self.config.adjust_instance_ports();
// Start RPC servers
let rpc_server_handles =
self.config.rpc.start_servers(&components, engine_api, jwt_secret, &mut ext).await?;
// Run consensus engine to completion
let (tx, rx) = oneshot::channel();
info!(target: "reth::cli", "Starting consensus engine");
executor.spawn_critical_blocking("consensus engine", async move {
let res = beacon_consensus_engine.await;
let _ = tx.send(res);
});
ext.on_node_started(&components)?;
// If `enable_genesis_walkback` is set to true, the rollup client will need to
// perform the derivation pipeline from genesis, validating the data dir.
// When set to false, set the finalized, safe, and unsafe head block hashes
// on the rollup client using a fork choice update. This prevents the rollup
// client from performing the derivation pipeline from genesis, and instead
// starts syncing from the current tip in the DB.
#[cfg(feature = "optimism")]
if self.config.chain.is_optimism() && !self.config.rollup.enable_genesis_walkback {
let client = rpc_server_handles.auth.http_client();
reth_rpc_api::EngineApiClient::<OptimismEngineTypes>::fork_choice_updated_v2(
&client,
reth_rpc_types::engine::ForkchoiceState {
head_block_hash: head.hash,
safe_block_hash: head.hash,
finalized_block_hash: head.hash,
},
None,
)
.await?;
}
// construct node handle and return
let node_handle = NodeHandle {
rpc_server_handles,
node_exit_future: NodeExitFuture::new(rx, self.config.debug.terminate),
};
Ok(node_handle)
}
/// Returns the path to the config file.
fn config_path(&self) -> PathBuf {
self.config.config.clone().unwrap_or_else(|| self.data_dir.config_path())
}
/// Loads the reth config with the given datadir root
fn load_config(&self) -> eyre::Result<Config> {
let config_path = self.config_path();
let mut config = confy::load_path::<Config>(&config_path)
.wrap_err_with(|| format!("Could not load config file {:?}", config_path))?;
info!(target: "reth::cli", path = ?config_path, "Configuration loaded");
// Update the config with the command line arguments
config.peers.connect_trusted_nodes_only = self.config.network.trusted_only;
if !self.config.network.trusted_peers.is_empty() {
info!(target: "reth::cli", "Adding trusted nodes");
self.config.network.trusted_peers.iter().for_each(|peer| {
config.peers.trusted_nodes.insert(*peer);
});
}
Ok(config)
}
}
/// The [NodeHandle] contains the [RethRpcServerHandles] returned by the reth initialization
/// process, as well as a method for waiting for the node exit.
#[derive(Debug)]
pub struct NodeHandle {
/// The handles to the RPC servers
rpc_server_handles: RethRpcServerHandles,
/// A Future which waits node exit
/// See [`NodeExitFuture`]
node_exit_future: NodeExitFuture,
}
impl NodeHandle {
/// Returns the [RethRpcServerHandles] for this node.
pub fn rpc_server_handles(&self) -> &RethRpcServerHandles {
&self.rpc_server_handles
}
/// Waits for the node to exit, if it was configured to exit.
pub async fn wait_for_node_exit(self) -> eyre::Result<()> {
self.node_exit_future.await
}
}
/// A simple function to launch a node with the specified [NodeConfig], spawning tasks on the
/// [TaskExecutor] constructed from [TaskManager::current].
///
/// # Example
/// ```
/// # use reth_node_core::{
/// # node_config::NodeConfig,
/// # args::RpcServerArgs,
/// # };
/// # use reth::builder::spawn_node;
/// async fn t() {
/// // Create a node builder with an http rpc server enabled
/// let rpc_args = RpcServerArgs::default().with_http();
///
/// let builder = NodeConfig::test().with_rpc(rpc_args);
///
/// // Spawn the builder, returning a handle to the node
/// let (_handle, _manager) = spawn_node(builder).await.unwrap();
/// }
/// ```
pub async fn spawn_node(config: NodeConfig) -> eyre::Result<(NodeHandle, TaskManager)> {
let task_manager = TaskManager::current();
let ext = DefaultRethNodeCommandConfig::default();
Ok((launch_from_config::<()>(config, ext, task_manager.executor()).await?, task_manager))
}
#[cfg(test)]
mod tests {
use super::*;
use reth_node_core::args::RpcServerArgs;
use reth_primitives::U256;
use reth_rpc_api::EthApiClient;
#[tokio::test]
async fn block_number_node_config_test() {
// this launches a test node with http
let rpc_args = RpcServerArgs::default().with_http();
let (handle, _manager) = spawn_node(NodeConfig::test().with_rpc(rpc_args)).await.unwrap();
// call a function on the node
let client = handle.rpc_server_handles().rpc.http_client().unwrap();
let block_number = client.block_number().await.unwrap();
// it should be zero, since this is an ephemeral test node
assert_eq!(block_number, U256::ZERO);
}
#[tokio::test]
async fn rpc_handles_none_without_http() {
// this launches a test node _without_ http
let (handle, _manager) = spawn_node(NodeConfig::test()).await.unwrap();
// ensure that the `http_client` is none
let maybe_client = handle.rpc_server_handles().rpc.http_client();
assert!(maybe_client.is_none());
}
#[tokio::test]
async fn launch_multiple_nodes() {
// spawn_test_node takes roughly 1 second per node, so this test takes ~4 seconds
let num_nodes = 4;
// contains handles and managers
let mut handles = Vec::new();
for _ in 0..num_nodes {
let handle = spawn_node(NodeConfig::test()).await.unwrap();
handles.push(handle);
}
}
#[cfg(feature = "optimism")]
#[tokio::test]
async fn optimism_pre_canyon_no_withdrawals_valid() {
reth_tracing::init_test_tracing();
use alloy_chains::Chain;
use jsonrpsee::http_client::HttpClient;
use reth_primitives::{ChainSpec, Genesis};
use reth_rpc_api::EngineApiClient;
use reth_rpc_types::engine::{
ForkchoiceState, OptimismPayloadAttributes, PayloadAttributes,
};
// this launches a test node with http
let rpc_args = RpcServerArgs::default().with_http();
// create optimism genesis with canyon at block 2
let spec = ChainSpec::builder()
.chain(Chain::optimism_mainnet())
.genesis(Genesis::default())
.regolith_activated()
.build();
let genesis_hash = spec.genesis_hash();
// create node config
let node_config = NodeConfig::test().with_rpc(rpc_args).with_chain(spec);
let (handle, _manager) = spawn_node(node_config).await.unwrap();
// call a function on the node
let client = handle.rpc_server_handles().auth.http_client();
let block_number = client.block_number().await.unwrap();
// it should be zero, since this is an ephemeral test node
assert_eq!(block_number, U256::ZERO);
// call the engine_forkchoiceUpdated function with payload attributes
let forkchoice_state = ForkchoiceState {
head_block_hash: genesis_hash,
safe_block_hash: genesis_hash,
finalized_block_hash: genesis_hash,
};
let payload_attributes = OptimismPayloadAttributes {
payload_attributes: PayloadAttributes {
timestamp: 1,
prev_randao: Default::default(),
suggested_fee_recipient: Default::default(),
// canyon is _not_ in the chain spec, so this should cause the engine call to fail
withdrawals: None,
parent_beacon_block_root: None,
},
no_tx_pool: None,
gas_limit: Some(1),
transactions: None,
};
// call the engine_forkchoiceUpdated function with payload attributes
let res = <HttpClient as EngineApiClient<OptimismEngineTypes>>::fork_choice_updated_v2(
&client,
forkchoice_state,
Some(payload_attributes),
)
.await;
res.expect("pre-canyon engine call without withdrawals should succeed");
}
#[cfg(feature = "optimism")]
#[tokio::test]
async fn optimism_pre_canyon_withdrawals_invalid() {
reth_tracing::init_test_tracing();
use alloy_chains::Chain;
use assert_matches::assert_matches;
use jsonrpsee::{core::Error, http_client::HttpClient, types::error::INVALID_PARAMS_CODE};
use reth_primitives::{ChainSpec, Genesis};
use reth_rpc_api::EngineApiClient;
use reth_rpc_types::engine::{
ForkchoiceState, OptimismPayloadAttributes, PayloadAttributes,
};
// this launches a test node with http
let rpc_args = RpcServerArgs::default().with_http();
// create optimism genesis with canyon at block 2
let spec = ChainSpec::builder()
.chain(Chain::optimism_mainnet())
.genesis(Genesis::default())
.regolith_activated()
.build();
let genesis_hash = spec.genesis_hash();
// create node config
let node_config = NodeConfig::test().with_rpc(rpc_args).with_chain(spec);
let (handle, _manager) = spawn_node(node_config).await.unwrap();
// call a function on the node
let client = handle.rpc_server_handles().auth.http_client();
let block_number = client.block_number().await.unwrap();
// it should be zero, since this is an ephemeral test node
assert_eq!(block_number, U256::ZERO);
// call the engine_forkchoiceUpdated function with payload attributes
let forkchoice_state = ForkchoiceState {
head_block_hash: genesis_hash,
safe_block_hash: genesis_hash,
finalized_block_hash: genesis_hash,
};
let payload_attributes = OptimismPayloadAttributes {
payload_attributes: PayloadAttributes {
timestamp: 1,
prev_randao: Default::default(),
suggested_fee_recipient: Default::default(),
// canyon is _not_ in the chain spec, so this should cause the engine call to fail
withdrawals: Some(vec![]),
parent_beacon_block_root: None,
},
no_tx_pool: None,
gas_limit: Some(1),
transactions: None,
};
// call the engine_forkchoiceUpdated function with payload attributes
let res = <HttpClient as EngineApiClient<OptimismEngineTypes>>::fork_choice_updated_v2(
&client,
forkchoice_state,
Some(payload_attributes),
)
.await;
let err = res.expect_err("pre-canyon engine call with withdrawals should fail");
assert_matches!(err, Error::Call(ref object) if object.code() == INVALID_PARAMS_CODE);
}
#[cfg(feature = "optimism")]
#[tokio::test]
async fn optimism_post_canyon_no_withdrawals_invalid() {
reth_tracing::init_test_tracing();
use alloy_chains::Chain;
use assert_matches::assert_matches;
use jsonrpsee::{core::Error, http_client::HttpClient, types::error::INVALID_PARAMS_CODE};
use reth_primitives::{ChainSpec, Genesis};
use reth_rpc_api::EngineApiClient;
use reth_rpc_types::engine::{
ForkchoiceState, OptimismPayloadAttributes, PayloadAttributes,
};
// this launches a test node with http
let rpc_args = RpcServerArgs::default().with_http();
// create optimism genesis with canyon at block 2
let spec = ChainSpec::builder()
.chain(Chain::optimism_mainnet())
.genesis(Genesis::default())
.canyon_activated()
.build();
let genesis_hash = spec.genesis_hash();
// create node config
let node_config = NodeConfig::test().with_rpc(rpc_args).with_chain(spec);
let (handle, _manager) = spawn_node(node_config).await.unwrap();
// call a function on the node
let client = handle.rpc_server_handles().auth.http_client();
let block_number = client.block_number().await.unwrap();
// it should be zero, since this is an ephemeral test node
assert_eq!(block_number, U256::ZERO);
// call the engine_forkchoiceUpdated function with payload attributes
let forkchoice_state = ForkchoiceState {
head_block_hash: genesis_hash,
safe_block_hash: genesis_hash,
finalized_block_hash: genesis_hash,
};
let payload_attributes = OptimismPayloadAttributes {
payload_attributes: PayloadAttributes {
timestamp: 1,
prev_randao: Default::default(),
suggested_fee_recipient: Default::default(),
// canyon is _not_ in the chain spec, so this should cause the engine call to fail
withdrawals: None,
parent_beacon_block_root: None,
},
no_tx_pool: None,
gas_limit: Some(1),
transactions: None,
};
// call the engine_forkchoiceUpdated function with payload attributes
let res = <HttpClient as EngineApiClient<OptimismEngineTypes>>::fork_choice_updated_v2(
&client,
forkchoice_state,
Some(payload_attributes),
)
.await;
let err = res.expect_err("post-canyon engine call with no withdrawals should fail");
assert_matches!(err, Error::Call(ref object) if object.code() == INVALID_PARAMS_CODE);
}
#[cfg(feature = "optimism")]
#[tokio::test]
async fn optimism_post_canyon_withdrawals_valid() {
reth_tracing::init_test_tracing();
use alloy_chains::Chain;
use jsonrpsee::http_client::HttpClient;
use reth_primitives::{ChainSpec, Genesis};
use reth_rpc_api::EngineApiClient;
use reth_rpc_types::engine::{
ForkchoiceState, OptimismPayloadAttributes, PayloadAttributes,
};
// this launches a test node with http
let rpc_args = RpcServerArgs::default().with_http();
// create optimism genesis with canyon at block 2
let spec = ChainSpec::builder()
.chain(Chain::optimism_mainnet())
.genesis(Genesis::default())
.canyon_activated()
.build();
let genesis_hash = spec.genesis_hash();
// create node config
let node_config = NodeConfig::test().with_rpc(rpc_args).with_chain(spec);
let (handle, _manager) = spawn_node(node_config).await.unwrap();
// call a function on the node
let client = handle.rpc_server_handles().auth.http_client();
let block_number = client.block_number().await.unwrap();
// it should be zero, since this is an ephemeral test node
assert_eq!(block_number, U256::ZERO);
// call the engine_forkchoiceUpdated function with payload attributes
let forkchoice_state = ForkchoiceState {
head_block_hash: genesis_hash,
safe_block_hash: genesis_hash,
finalized_block_hash: genesis_hash,
};
let payload_attributes = OptimismPayloadAttributes {
payload_attributes: PayloadAttributes {
timestamp: 1,
prev_randao: Default::default(),
suggested_fee_recipient: Default::default(),
// canyon is _not_ in the chain spec, so this should cause the engine call to fail
withdrawals: Some(vec![]),
parent_beacon_block_root: None,
},
no_tx_pool: None,
gas_limit: Some(1),
transactions: None,
};
// call the engine_forkchoiceUpdated function with payload attributes
let res = <HttpClient as EngineApiClient<OptimismEngineTypes>>::fork_choice_updated_v2(
&client,
forkchoice_state,
Some(payload_attributes),
)
.await;
res.expect("post-canyon engine call with withdrawals should succeed");
}
}

View File

@ -5,18 +5,19 @@ use crate::{
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
LogArgs,
},
cli::ext::RethCliExt,
commands::{
config_cmd, db, debug_cmd, dump_genesis, import, init_cmd, node, p2p, recover, stage,
test_vectors,
config_cmd, db, debug_cmd, dump_genesis, import, init_cmd, node, node::NoArgs, p2p,
recover, stage, test_vectors,
},
core::cli::runner::CliRunner,
version::{LONG_VERSION, SHORT_VERSION},
};
use clap::{value_parser, Parser, Subcommand};
use reth_db::DatabaseEnv;
use reth_node_builder::{InitState, WithLaunchContext};
use reth_primitives::ChainSpec;
use reth_tracing::FileWorkerGuard;
use std::sync::Arc;
use std::{ffi::OsString, fmt, future::Future, sync::Arc};
/// Re-export of the `reth_node_core` types specifically in the `cli` module.
///
@ -30,7 +31,7 @@ pub use crate::core::cli::*;
/// This is the entrypoint to the executable.
#[derive(Debug, Parser)]
#[command(author, version = SHORT_VERSION, long_version = LONG_VERSION, about = "Reth", long_about = None)]
pub struct Cli<Ext: RethCliExt = ()> {
pub struct Cli<Ext: clap::Args + fmt::Debug = NoArgs> {
/// The command to run
#[command(subcommand)]
command: Commands<Ext>,
@ -68,9 +69,70 @@ pub struct Cli<Ext: RethCliExt = ()> {
logs: LogArgs,
}
impl<Ext: RethCliExt> Cli<Ext> {
impl Cli {
/// Parsers only the default CLI arguments
pub fn parse_args() -> Self {
Self::parse()
}
/// Parsers only the default CLI arguments from the given iterator
pub fn try_parse_args_from<I, T>(itr: I) -> Result<Self, clap::error::Error>
where
I: IntoIterator<Item = T>,
T: Into<OsString> + Clone,
{
Cli::try_parse_from(itr)
}
}
impl<Ext: clap::Args + fmt::Debug> Cli<Ext> {
/// Execute the configured cli command.
pub fn run(mut self) -> eyre::Result<()> {
///
/// This accepts a closure that is used to launch the node via the
/// [NodeCommand](node::NodeCommand).
///
///
/// # Example
///
/// ```no_run
/// use reth::cli::Cli;
/// use reth_node_ethereum::EthereumNode;
///
/// Cli::parse_args()
/// .run(|builder, _| async move {
/// let handle = builder.launch_node(EthereumNode::default()).await?;
///
/// handle.wait_for_node_exit().await
/// })
/// .unwrap();
/// ```
///
/// # Example
///
/// Parse additional CLI arguments for the node command and use it to configure the node.
///
/// ```no_run
/// use clap::Parser;
/// use reth::cli::Cli;
///
/// #[derive(Debug, Parser)]
/// pub struct MyArgs {
/// pub enable: bool,
/// }
///
/// Cli::parse()
/// .run(|builder, my_args: MyArgs| async move {
/// // launch the node
///
/// Ok(())
/// })
/// .unwrap();
/// ````
pub fn run<L, Fut>(mut self, launcher: L) -> eyre::Result<()>
where
L: FnOnce(WithLaunchContext<Arc<DatabaseEnv>, InitState>, Ext) -> Fut,
Fut: Future<Output = eyre::Result<()>>,
{
// add network name to logs dir
self.logs.log_file_directory =
self.logs.log_file_directory.join(self.chain.chain.to_string());
@ -79,7 +141,9 @@ impl<Ext: RethCliExt> Cli<Ext> {
let runner = CliRunner::default();
match self.command {
Commands::Node(command) => runner.run_command_until_exit(|ctx| command.execute(ctx)),
Commands::Node(command) => {
runner.run_command_until_exit(|ctx| command.execute(ctx, launcher))
}
Commands::Init(command) => runner.run_blocking_until_ctrl_c(command.execute()),
Commands::Import(command) => runner.run_blocking_until_ctrl_c(command.execute()),
Commands::DumpGenesis(command) => runner.run_blocking_until_ctrl_c(command.execute()),
@ -101,26 +165,11 @@ impl<Ext: RethCliExt> Cli<Ext> {
let guard = self.logs.init_tracing()?;
Ok(guard)
}
/// Configures the given node extension.
pub fn with_node_extension<C>(mut self, conf: C) -> Self
where
C: Into<Ext::Node>,
{
self.command.set_node_extension(conf.into());
self
}
}
/// Convenience function for parsing CLI options, set up logging and run the chosen command.
#[inline]
pub fn run() -> eyre::Result<()> {
Cli::<()>::parse().run()
}
/// Commands to be executed
#[derive(Debug, Subcommand)]
pub enum Commands<Ext: RethCliExt = ()> {
pub enum Commands<Ext: clap::Args + fmt::Debug = NoArgs> {
/// Start the node
#[command(name = "node")]
Node(node::NodeCommand<Ext>),
@ -155,17 +204,6 @@ pub enum Commands<Ext: RethCliExt = ()> {
Recover(recover::Command),
}
impl<Ext: RethCliExt> Commands<Ext> {
/// Sets the node extension if it is the [NodeCommand](node::NodeCommand).
///
/// This is a noop if the command is not the [NodeCommand](node::NodeCommand).
pub fn set_node_extension(&mut self, ext: Ext::Node) {
if let Commands::Node(command) = self {
command.ext = ext
}
}
}
#[cfg(test)]
mod tests {
use super::*;
@ -174,7 +212,7 @@ mod tests {
#[test]
fn parse_color_mode() {
let reth = Cli::<()>::try_parse_from(["reth", "node", "--color", "always"]).unwrap();
let reth = Cli::try_parse_args_from(["reth", "node", "--color", "always"]).unwrap();
assert_eq!(reth.logs.color, ColorMode::Always);
}
@ -183,9 +221,9 @@ mod tests {
/// runtime
#[test]
fn test_parse_help_all_subcommands() {
let reth = Cli::<()>::command();
let reth = Cli::<NoArgs>::command();
for sub_command in reth.get_subcommands() {
let err = Cli::<()>::try_parse_from(["reth", sub_command.get_name(), "--help"])
let err = Cli::try_parse_args_from(["reth", sub_command.get_name(), "--help"])
.err()
.unwrap_or_else(|| {
panic!("Failed to parse help message {}", sub_command.get_name())
@ -201,7 +239,7 @@ mod tests {
/// name
#[test]
fn parse_logs_path() {
let mut reth = Cli::<()>::try_parse_from(["reth", "node"]).unwrap();
let mut reth = Cli::try_parse_args_from(["reth", "node"]).unwrap();
reth.logs.log_file_directory =
reth.logs.log_file_directory.join(reth.chain.chain.to_string());
let log_dir = reth.logs.log_file_directory;
@ -211,7 +249,7 @@ mod tests {
let mut iter = SUPPORTED_CHAINS.iter();
iter.next();
for chain in iter {
let mut reth = Cli::<()>::try_parse_from(["reth", "node", "--chain", chain]).unwrap();
let mut reth = Cli::try_parse_args_from(["reth", "node", "--chain", chain]).unwrap();
reth.logs.log_file_directory =
reth.logs.log_file_directory.join(reth.chain.chain.to_string());
let log_dir = reth.logs.log_file_directory;
@ -220,21 +258,12 @@ mod tests {
}
}
#[test]
fn override_trusted_setup_file() {
// We already have a test that asserts that this has been initialized,
// so we cheat a little bit and check that loading a random file errors.
let reth = Cli::<()>::try_parse_from(["reth", "node", "--trusted-setup-file", "README.md"])
.unwrap();
assert!(reth.run().is_err());
}
#[test]
fn parse_env_filter_directives() {
let temp_dir = tempfile::tempdir().unwrap();
std::env::set_var("RUST_LOG", "info,evm=debug");
let reth = Cli::<()>::try_parse_from([
let reth = Cli::try_parse_args_from([
"reth",
"init",
"--datadir",
@ -243,6 +272,6 @@ mod tests {
"debug,net=trace",
])
.unwrap();
assert!(reth.run().is_ok());
assert!(reth.run(|_, _| async move { Ok(()) }).is_ok());
}
}

View File

@ -1,130 +0,0 @@
//! Stores engine API messages to disk for later inspection and replay.
use reth_beacon_consensus::BeaconEngineMessage;
use reth_node_api::EngineTypes;
use reth_primitives::fs::{self};
use reth_rpc_types::{
engine::{CancunPayloadFields, ForkchoiceState},
ExecutionPayload,
};
use serde::{Deserialize, Serialize};
use std::{collections::BTreeMap, path::PathBuf, time::SystemTime};
use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender};
use tracing::*;
/// A message from the engine API that has been stored to disk.
#[derive(Debug, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub enum StoredEngineApiMessage<Attributes> {
/// The on-disk representation of an `engine_forkchoiceUpdated` method call.
ForkchoiceUpdated {
/// The [ForkchoiceState] sent in the persisted call.
state: ForkchoiceState,
/// The payload attributes sent in the persisted call, if any.
payload_attrs: Option<Attributes>,
},
/// The on-disk representation of an `engine_newPayload` method call.
NewPayload {
/// The [ExecutionPayload] sent in the persisted call.
payload: ExecutionPayload,
/// The Cancun-specific fields sent in the persisted call, if any.
cancun_fields: Option<CancunPayloadFields>,
},
}
/// This can read and write engine API messages in a specific directory.
#[derive(Debug)]
pub struct EngineApiStore {
/// The path to the directory that stores the engine API messages.
path: PathBuf,
}
impl EngineApiStore {
/// Creates a new [EngineApiStore] at the given path.
///
/// The path is expected to be a directory, where individual message JSON files will be stored.
pub fn new(path: PathBuf) -> Self {
Self { path }
}
/// Stores the received [BeaconEngineMessage] to disk, appending the `received_at` time to the
/// path.
pub fn on_message<Engine>(
&self,
msg: &BeaconEngineMessage<Engine>,
received_at: SystemTime,
) -> eyre::Result<()>
where
Engine: EngineTypes,
{
fs::create_dir_all(&self.path)?; // ensure that store path had been created
let timestamp = received_at.duration_since(SystemTime::UNIX_EPOCH).unwrap().as_millis();
match msg {
BeaconEngineMessage::ForkchoiceUpdated { state, payload_attrs, tx: _tx } => {
let filename = format!("{}-fcu-{}.json", timestamp, state.head_block_hash);
fs::write(
self.path.join(filename),
serde_json::to_vec(&StoredEngineApiMessage::ForkchoiceUpdated {
state: *state,
payload_attrs: payload_attrs.clone(),
})?,
)?;
}
BeaconEngineMessage::NewPayload { payload, cancun_fields, tx: _tx } => {
let filename = format!("{}-new_payload-{}.json", timestamp, payload.block_hash());
fs::write(
self.path.join(filename),
serde_json::to_vec(
&StoredEngineApiMessage::<Engine::PayloadAttributes>::NewPayload {
payload: payload.clone(),
cancun_fields: cancun_fields.clone(),
},
)?,
)?;
}
// noop
BeaconEngineMessage::TransitionConfigurationExchanged |
BeaconEngineMessage::EventListener(_) => (),
};
Ok(())
}
/// Finds and iterates through any stored engine API message files, ordered by timestamp.
pub fn engine_messages_iter(&self) -> eyre::Result<impl Iterator<Item = PathBuf>> {
let mut filenames_by_ts = BTreeMap::<u64, Vec<PathBuf>>::default();
for entry in fs::read_dir(&self.path)? {
let entry = entry?;
let filename = entry.file_name();
if let Some(filename) = filename.to_str().filter(|n| n.ends_with(".json")) {
if let Some(Ok(timestamp)) = filename.split('-').next().map(|n| n.parse::<u64>()) {
filenames_by_ts.entry(timestamp).or_default().push(entry.path());
tracing::debug!(target: "engine::store", timestamp, filename, "Queued engine API message");
} else {
tracing::warn!(target: "engine::store", %filename, "Could not parse timestamp from filename")
}
} else {
tracing::warn!(target: "engine::store", ?filename, "Skipping non json file");
}
}
Ok(filenames_by_ts.into_iter().flat_map(|(_, paths)| paths))
}
/// Intercepts an incoming engine API message, storing it to disk and forwarding it to the
/// engine channel.
pub async fn intercept<Engine>(
self,
mut rx: UnboundedReceiver<BeaconEngineMessage<Engine>>,
to_engine: UnboundedSender<BeaconEngineMessage<Engine>>,
) where
Engine: EngineTypes,
BeaconEngineMessage<Engine>: std::fmt::Debug,
{
loop {
let Some(msg) = rx.recv().await else { break };
if let Err(error) = self.on_message(&msg, SystemTime::now()) {
error!(target: "engine::intercept", ?msg, %error, "Error handling Engine API message");
}
let _ = to_engine.send(msg);
}
}
}

View File

@ -2,9 +2,7 @@
use crate::core::cli::runner::CliContext;
use clap::{Parser, Subcommand};
mod build_block;
pub mod engine_api_store;
mod execution;
mod in_memory_merkle;
mod merkle;

View File

@ -4,7 +4,6 @@ use crate::{
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs, NetworkArgs,
},
commands::debug_cmd::engine_api_store::{EngineApiStore, StoredEngineApiMessage},
core::cli::runner::CliContext,
dirs::{DataDirPath, MaybePlatformPath},
};
@ -20,6 +19,7 @@ use reth_db::{init_db, mdbx::DatabaseArguments, DatabaseEnv};
use reth_interfaces::consensus::Consensus;
use reth_network::NetworkHandle;
use reth_network_api::NetworkInfo;
use reth_node_core::engine_api_store::{EngineApiStore, StoredEngineApiMessage};
#[cfg(not(feature = "optimism"))]
use reth_node_ethereum::{EthEngineTypes, EthEvmConfig};
#[cfg(feature = "optimism")]

View File

@ -1,6 +1,4 @@
//! Main node command
//!
//! Starts the client
//! Main node command for launching a node
use crate::{
args::{
@ -8,21 +6,19 @@ use crate::{
DatabaseArgs, DebugArgs, DevArgs, NetworkArgs, PayloadBuilderArgs, PruningArgs,
RpcServerArgs, TxPoolArgs,
},
builder::{launch_from_config, NodeConfig},
cli::{db_type::DatabaseBuilder, ext::RethCliExt},
core::cli::runner::CliContext,
dirs::{DataDirPath, MaybePlatformPath},
};
use clap::{value_parser, Parser};
use reth_auto_seal_consensus::AutoSealConsensus;
use reth_beacon_consensus::BeaconConsensus;
use reth_interfaces::consensus::Consensus;
use clap::{value_parser, Args, Parser};
use reth_db::{init_db, mdbx::DatabaseArguments, DatabaseEnv};
use reth_node_builder::{InitState, NodeBuilder, WithLaunchContext};
use reth_node_core::node_config::NodeConfig;
use reth_primitives::ChainSpec;
use std::{net::SocketAddr, path::PathBuf, sync::Arc};
use std::{ffi::OsString, fmt, future::Future, net::SocketAddr, path::PathBuf, sync::Arc};
/// Start the node
#[derive(Debug, Parser)]
pub struct NodeCommand<Ext: RethCliExt = ()> {
pub struct NodeCommand<Ext: clap::Args + fmt::Debug = NoArgs> {
/// The path to the data dir for all reth files and subdirectories.
///
/// Defaults to the OS-specific data directory:
@ -116,63 +112,37 @@ pub struct NodeCommand<Ext: RethCliExt = ()> {
#[command(flatten)]
pub pruning: PruningArgs,
/// Rollup related arguments
#[cfg(feature = "optimism")]
#[command(flatten)]
pub rollup: crate::args::RollupArgs,
/// Additional cli arguments
#[command(flatten, next_help_heading = "Extension")]
pub ext: Ext::Node,
pub ext: Ext,
}
impl<Ext: RethCliExt> NodeCommand<Ext> {
/// Replaces the extension of the node command
pub fn with_ext<E: RethCliExt>(self, ext: E::Node) -> NodeCommand<E> {
let Self {
datadir,
config,
chain,
metrics,
trusted_setup_file,
instance,
with_unused_ports,
network,
rpc,
txpool,
builder,
debug,
db,
dev,
pruning,
#[cfg(feature = "optimism")]
rollup,
..
} = self;
NodeCommand {
datadir,
config,
chain,
metrics,
instance,
with_unused_ports,
trusted_setup_file,
network,
rpc,
txpool,
builder,
debug,
db,
dev,
pruning,
#[cfg(feature = "optimism")]
rollup,
ext,
}
impl NodeCommand {
/// Parsers only the default CLI arguments
pub fn parse_args() -> Self {
Self::parse()
}
/// Execute `node` command
pub async fn execute(self, ctx: CliContext) -> eyre::Result<()> {
/// Parsers only the default [NodeCommand] arguments from the given iterator
pub fn try_parse_args_from<I, T>(itr: I) -> Result<Self, clap::error::Error>
where
I: IntoIterator<Item = T>,
T: Into<OsString> + Clone,
{
Self::try_parse_from(itr)
}
}
impl<Ext: clap::Args + fmt::Debug> NodeCommand<Ext> {
/// Launches the node
///
/// This transforms the node command into a node config and launches the node using the given
/// closure.
pub async fn execute<L, Fut>(self, ctx: CliContext, launcher: L) -> eyre::Result<()>
where
L: FnOnce(WithLaunchContext<Arc<DatabaseEnv>, InitState>, Ext) -> Fut,
Fut: Future<Output = eyre::Result<()>>,
{
let Self {
datadir,
config,
@ -189,17 +159,11 @@ impl<Ext: RethCliExt> NodeCommand<Ext> {
db,
dev,
pruning,
#[cfg(feature = "optimism")]
rollup,
ext,
} = self;
// set up real database
let database = DatabaseBuilder::Real(datadir);
// set up node config
let mut node_config = NodeConfig {
database,
config,
chain,
metrics,
@ -213,35 +177,38 @@ impl<Ext: RethCliExt> NodeCommand<Ext> {
db,
dev,
pruning,
#[cfg(feature = "optimism")]
rollup,
};
// Register the prometheus recorder before creating the database,
// because database init needs it to register metrics.
let _ = node_config.install_prometheus_recorder()?;
let data_dir = datadir.unwrap_or_chain_default(node_config.chain.chain);
let db_path = data_dir.db_path();
tracing::info!(target: "reth::cli", path = ?db_path, "Opening database");
let database = Arc::new(
init_db(db_path.clone(), DatabaseArguments::default().log_level(db.log_level))?
.with_metrics(),
);
if with_unused_ports {
node_config = node_config.with_unused_ports();
}
let executor = ctx.task_executor;
let builder = NodeBuilder::new(node_config)
.with_database(database)
.with_launch_context(ctx.task_executor, data_dir);
// launch the node
let handle = launch_from_config::<Ext>(node_config, ext, executor).await?;
handle.wait_for_node_exit().await
}
/// Returns the [Consensus] instance to use.
///
/// By default this will be a [BeaconConsensus] instance, but if the `--dev` flag is set, it
/// will be an [AutoSealConsensus] instance.
pub fn consensus(&self) -> Arc<dyn Consensus> {
if self.dev.dev {
Arc::new(AutoSealConsensus::new(Arc::clone(&self.chain)))
} else {
Arc::new(BeaconConsensus::new(Arc::clone(&self.chain)))
}
launcher(builder, ext).await
}
}
/// No Additional arguments
#[derive(Debug, Clone, Copy, Default, Args)]
#[non_exhaustive]
pub struct NoArgs;
#[cfg(test)]
mod tests {
use super::*;
@ -253,14 +220,14 @@ mod tests {
#[test]
fn parse_help_node_command() {
let err = NodeCommand::<()>::try_parse_from(["reth", "--help"]).unwrap_err();
let err = NodeCommand::try_parse_args_from(["reth", "--help"]).unwrap_err();
assert_eq!(err.kind(), clap::error::ErrorKind::DisplayHelp);
}
#[test]
fn parse_common_node_command_chain_args() {
for chain in SUPPORTED_CHAINS {
let args: NodeCommand = NodeCommand::<()>::parse_from(["reth", "--chain", chain]);
let args: NodeCommand = NodeCommand::<NoArgs>::parse_from(["reth", "--chain", chain]);
assert_eq!(args.chain.chain, chain.parse::<reth_primitives::Chain>().unwrap());
}
}
@ -268,13 +235,13 @@ mod tests {
#[test]
fn parse_discovery_addr() {
let cmd =
NodeCommand::<()>::try_parse_from(["reth", "--discovery.addr", "127.0.0.1"]).unwrap();
NodeCommand::try_parse_args_from(["reth", "--discovery.addr", "127.0.0.1"]).unwrap();
assert_eq!(cmd.network.discovery.addr, Ipv4Addr::LOCALHOST);
}
#[test]
fn parse_addr() {
let cmd = NodeCommand::<()>::try_parse_from([
let cmd = NodeCommand::try_parse_args_from([
"reth",
"--discovery.addr",
"127.0.0.1",
@ -288,14 +255,14 @@ mod tests {
#[test]
fn parse_discovery_port() {
let cmd = NodeCommand::<()>::try_parse_from(["reth", "--discovery.port", "300"]).unwrap();
let cmd = NodeCommand::try_parse_args_from(["reth", "--discovery.port", "300"]).unwrap();
assert_eq!(cmd.network.discovery.port, 300);
}
#[test]
fn parse_port() {
let cmd =
NodeCommand::<()>::try_parse_from(["reth", "--discovery.port", "300", "--port", "99"])
NodeCommand::try_parse_args_from(["reth", "--discovery.port", "300", "--port", "99"])
.unwrap();
assert_eq!(cmd.network.discovery.port, 300);
assert_eq!(cmd.network.port, 99);
@ -303,27 +270,27 @@ mod tests {
#[test]
fn parse_metrics_port() {
let cmd = NodeCommand::<()>::try_parse_from(["reth", "--metrics", "9001"]).unwrap();
let cmd = NodeCommand::try_parse_args_from(["reth", "--metrics", "9001"]).unwrap();
assert_eq!(cmd.metrics, Some(SocketAddr::new(IpAddr::V4(Ipv4Addr::LOCALHOST), 9001)));
let cmd = NodeCommand::<()>::try_parse_from(["reth", "--metrics", ":9001"]).unwrap();
let cmd = NodeCommand::try_parse_args_from(["reth", "--metrics", ":9001"]).unwrap();
assert_eq!(cmd.metrics, Some(SocketAddr::new(IpAddr::V4(Ipv4Addr::LOCALHOST), 9001)));
let cmd =
NodeCommand::<()>::try_parse_from(["reth", "--metrics", "localhost:9001"]).unwrap();
NodeCommand::try_parse_args_from(["reth", "--metrics", "localhost:9001"]).unwrap();
assert_eq!(cmd.metrics, Some(SocketAddr::new(IpAddr::V4(Ipv4Addr::LOCALHOST), 9001)));
}
#[test]
fn parse_config_path() {
let cmd = NodeCommand::<()>::try_parse_from(["reth", "--config", "my/path/to/reth.toml"])
.unwrap();
let cmd =
NodeCommand::try_parse_args_from(["reth", "--config", "my/path/to/reth.toml"]).unwrap();
// always store reth.toml in the data dir, not the chain specific data dir
let data_dir = cmd.datadir.unwrap_or_chain_default(cmd.chain.chain);
let config_path = cmd.config.unwrap_or(data_dir.config_path());
assert_eq!(config_path, Path::new("my/path/to/reth.toml"));
let cmd = NodeCommand::<()>::try_parse_from(["reth"]).unwrap();
let cmd = NodeCommand::try_parse_args_from(["reth"]).unwrap();
// always store reth.toml in the data dir, not the chain specific data dir
let data_dir = cmd.datadir.unwrap_or_chain_default(cmd.chain.chain);
@ -334,14 +301,14 @@ mod tests {
#[test]
fn parse_db_path() {
let cmd = NodeCommand::<()>::try_parse_from(["reth"]).unwrap();
let cmd = NodeCommand::try_parse_args_from(["reth"]).unwrap();
let data_dir = cmd.datadir.unwrap_or_chain_default(cmd.chain.chain);
let db_path = data_dir.db_path();
let end = format!("reth/{}/db", SUPPORTED_CHAINS[0]);
assert!(db_path.ends_with(end), "{:?}", cmd.config);
let cmd =
NodeCommand::<()>::try_parse_from(["reth", "--datadir", "my/custom/path"]).unwrap();
NodeCommand::try_parse_args_from(["reth", "--datadir", "my/custom/path"]).unwrap();
let data_dir = cmd.datadir.unwrap_or_chain_default(cmd.chain.chain);
let db_path = data_dir.db_path();
assert_eq!(db_path, Path::new("my/custom/path/db"));
@ -350,7 +317,7 @@ mod tests {
#[test]
#[cfg(not(feature = "optimism"))] // dev mode not yet supported in op-reth
fn parse_dev() {
let cmd = NodeCommand::<()>::parse_from(["reth", "--dev"]);
let cmd = NodeCommand::<NoArgs>::parse_from(["reth", "--dev"]);
let chain = reth_primitives::DEV.clone();
assert_eq!(cmd.chain.chain, chain.chain);
assert_eq!(cmd.chain.genesis_hash, chain.genesis_hash);
@ -368,7 +335,7 @@ mod tests {
#[test]
fn parse_instance() {
let mut cmd = NodeCommand::<()>::parse_from(["reth"]);
let mut cmd = NodeCommand::<NoArgs>::parse_from(["reth"]);
cmd.rpc.adjust_instance_ports(cmd.instance);
cmd.network.port = DEFAULT_DISCOVERY_PORT + cmd.instance - 1;
// check rpc port numbers
@ -378,7 +345,7 @@ mod tests {
// check network listening port number
assert_eq!(cmd.network.port, 30303);
let mut cmd = NodeCommand::<()>::parse_from(["reth", "--instance", "2"]);
let mut cmd = NodeCommand::<NoArgs>::parse_from(["reth", "--instance", "2"]);
cmd.rpc.adjust_instance_ports(cmd.instance);
cmd.network.port = DEFAULT_DISCOVERY_PORT + cmd.instance - 1;
// check rpc port numbers
@ -388,7 +355,7 @@ mod tests {
// check network listening port number
assert_eq!(cmd.network.port, 30304);
let mut cmd = NodeCommand::<()>::parse_from(["reth", "--instance", "3"]);
let mut cmd = NodeCommand::<NoArgs>::parse_from(["reth", "--instance", "3"]);
cmd.rpc.adjust_instance_ports(cmd.instance);
cmd.network.port = DEFAULT_DISCOVERY_PORT + cmd.instance - 1;
// check rpc port numbers
@ -401,21 +368,21 @@ mod tests {
#[test]
fn parse_with_unused_ports() {
let cmd = NodeCommand::<()>::parse_from(["reth", "--with-unused-ports"]);
let cmd = NodeCommand::<NoArgs>::parse_from(["reth", "--with-unused-ports"]);
assert!(cmd.with_unused_ports);
}
#[test]
fn with_unused_ports_conflicts_with_instance() {
let err =
NodeCommand::<()>::try_parse_from(["reth", "--with-unused-ports", "--instance", "2"])
NodeCommand::try_parse_args_from(["reth", "--with-unused-ports", "--instance", "2"])
.unwrap_err();
assert_eq!(err.kind(), clap::error::ErrorKind::ArgumentConflict);
}
#[test]
fn with_unused_ports_check_zero() {
let mut cmd = NodeCommand::<()>::parse_from(["reth"]);
let mut cmd = NodeCommand::<NoArgs>::parse_from(["reth"]);
cmd.rpc = cmd.rpc.with_unused_ports();
cmd.network = cmd.network.with_unused_ports();

View File

@ -26,7 +26,6 @@
)]
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
pub mod builder;
pub mod cli;
pub mod commands;
pub mod utils;
@ -62,6 +61,11 @@ pub mod version {
pub use reth_node_core::version::*;
}
/// Re-exported from `reth_node_builder`
pub mod builder {
pub use reth_node_builder::*;
}
/// Re-exported from `reth_node_core`, also to prevent a breaking change. See the comment on
/// the `reth_node_core::args` re-export for more details.
pub mod dirs {

View File

@ -10,12 +10,18 @@ compile_error!("Cannot build the `reth` binary with the `optimism` feature flag
#[cfg(not(feature = "optimism"))]
fn main() {
use reth::cli::Cli;
use reth_node_ethereum::EthereumNode;
// Enable backtraces unless a RUST_BACKTRACE value has already been explicitly provided.
if std::env::var("RUST_BACKTRACE").is_err() {
std::env::set_var("RUST_BACKTRACE", "1");
}
if let Err(err) = reth::cli::run() {
if let Err(err) = Cli::parse_args().run(|builder, _| async {
let handle = builder.launch_node(EthereumNode::default()).await?;
handle.node_exit_future.await
}) {
eprintln!("Error: {err:?}");
std::process::exit(1);
}

View File

@ -1,4 +1,11 @@
#![allow(missing_docs, rustdoc::missing_crate_level_docs)]
use clap::Parser;
use reth::cli::Cli;
use reth_node_builder::NodeHandle;
use reth_node_optimism::{args::RollupArgs, OptimismEngineTypes, OptimismNode};
use reth_provider::BlockReaderIdExt;
// We use jemalloc for performance reasons
#[cfg(all(feature = "jemalloc", unix))]
#[global_allocator]
@ -14,7 +21,34 @@ fn main() {
std::env::set_var("RUST_BACKTRACE", "1");
}
if let Err(err) = reth::cli::run() {
if let Err(err) = Cli::<RollupArgs>::parse().run(|builder, rollup_args| async move {
let NodeHandle { node, node_exit_future } =
builder.launch_node(OptimismNode::new(rollup_args.clone())).await?;
// If `enable_genesis_walkback` is set to true, the rollup client will need to
// perform the derivation pipeline from genesis, validating the data dir.
// When set to false, set the finalized, safe, and unsafe head block hashes
// on the rollup client using a fork choice update. This prevents the rollup
// client from performing the derivation pipeline from genesis, and instead
// starts syncing from the current tip in the DB.
if node.chain_spec().is_optimism() && !rollup_args.enable_genesis_walkback {
let client = node.rpc_server_handles.auth.http_client();
if let Ok(Some(head)) = node.provider.latest_header() {
reth_rpc_api::EngineApiClient::<OptimismEngineTypes>::fork_choice_updated_v2(
&client,
reth_rpc_types::engine::ForkchoiceState {
head_block_hash: head.hash(),
safe_block_hash: head.hash(),
finalized_block_hash: head.hash(),
},
None,
)
.await?;
}
}
node_exit_future.await
}) {
eprintln!("Error: {err:?}");
std::process::exit(1);
}