chore: add EnvironmentArgs and use across all CLI commands (#8590)

This commit is contained in:
joshieDo
2024-06-04 21:30:05 +02:00
committed by GitHub
parent e1f39bdc2f
commit a8095740fc
46 changed files with 521 additions and 1124 deletions

View File

@ -0,0 +1,115 @@
//! Contains common `reth` arguments
use clap::Parser;
use reth_config::{config::EtlConfig, Config};
use reth_db::{init_db, open_db_read_only, DatabaseEnv};
use reth_db_common::init::init_genesis;
use reth_node_core::{
args::{
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs, DatadirArgs,
},
dirs::{ChainPath, DataDirPath},
};
use reth_primitives::ChainSpec;
use reth_provider::{providers::StaticFileProvider, ProviderFactory};
use std::{path::PathBuf, sync::Arc};
use tracing::{debug, info};
/// Struct to hold config and datadir paths
#[derive(Debug, Parser)]
pub struct EnvironmentArgs {
/// Parameters for datadir configuration
#[command(flatten)]
pub datadir: DatadirArgs,
/// The path to the configuration file to use.
#[arg(long, value_name = "FILE")]
pub config: Option<PathBuf>,
/// The chain this node is running.
///
/// Possible values are either a built-in chain or the path to a chain specification file.
#[arg(
long,
value_name = "CHAIN_OR_PATH",
long_help = chain_help(),
default_value = SUPPORTED_CHAINS[0],
value_parser = genesis_value_parser
)]
pub chain: Arc<ChainSpec>,
/// All database related arguments
#[command(flatten)]
pub db: DatabaseArgs,
}
impl EnvironmentArgs {
/// Initializes environment according to [`AccessRights`] and returns an instance of
/// [`Environment`].
pub fn init(&self, access: AccessRights) -> eyre::Result<Environment> {
let data_dir = self.datadir.clone().resolve_datadir(self.chain.chain);
let db_path = data_dir.db();
let sf_path = data_dir.static_files();
if access.is_read_write() {
reth_fs_util::create_dir_all(&db_path)?;
reth_fs_util::create_dir_all(&sf_path)?;
}
let config_path = self.config.clone().unwrap_or_else(|| data_dir.config());
let mut config: Config = confy::load_path(config_path).unwrap_or_default();
// Make sure ETL doesn't default to /tmp/, but to whatever datadir is set to
if config.stages.etl.dir.is_none() {
config.stages.etl.dir = Some(EtlConfig::from_datadir(data_dir.data_dir()));
}
info!(target: "reth::cli", ?db_path, ?sf_path, "Opening storage");
let (db, sfp) = match access {
AccessRights::RW => (
Arc::new(init_db(db_path, self.db.database_args())?),
StaticFileProvider::read_write(sf_path)?,
),
AccessRights::RO => (
Arc::new(open_db_read_only(&db_path, self.db.database_args())?),
StaticFileProvider::read_only(sf_path)?,
),
};
let provider_factory = ProviderFactory::new(db, self.chain.clone(), sfp);
if access.is_read_write() {
debug!(target: "reth::cli", chain=%self.chain.chain, genesis=?self.chain.genesis_hash(), "Initializing genesis");
init_genesis(provider_factory.clone())?;
}
Ok(Environment { config, provider_factory, data_dir })
}
}
/// Environment built from [`EnvironmentArgs`].
#[derive(Debug)]
pub struct Environment {
/// Configuration for reth node
pub config: Config,
/// Provider factory.
pub provider_factory: ProviderFactory<Arc<DatabaseEnv>>,
/// Datadir path.
pub data_dir: ChainPath<DataDirPath>,
}
/// Environment access rights.
#[derive(Debug, Copy, Clone)]
pub enum AccessRights {
/// Read-write access
RW,
/// Read-only access
RO,
}
impl AccessRights {
/// Returns `true` if it requires read-write access to the environment.
pub const fn is_read_write(&self) -> bool {
matches!(self, Self::RW)
}
}

View File

@ -10,6 +10,7 @@ use reth_db::{
};
use std::{
hash::{BuildHasher, Hasher},
sync::Arc,
time::{Duration, Instant},
};
use tracing::{info, warn};
@ -36,7 +37,7 @@ pub struct Command {
impl Command {
/// Execute `db checksum` command
pub fn execute(self, tool: &DbTool<DatabaseEnv>) -> eyre::Result<()> {
pub fn execute(self, tool: &DbTool<Arc<DatabaseEnv>>) -> eyre::Result<()> {
warn!("This command should be run without the node running!");
self.table.view(&ChecksumViewer {
tool,

View File

@ -15,6 +15,7 @@ use std::{
hash::Hash,
io::Write,
path::{Path, PathBuf},
sync::Arc,
};
use tracing::{info, warn};
@ -52,7 +53,7 @@ impl Command {
///
/// The discrepancies and extra elements, along with a brief summary of the diff results are
/// then written to a file in the output directory.
pub fn execute(self, tool: &DbTool<DatabaseEnv>) -> eyre::Result<()> {
pub fn execute(self, tool: &DbTool<Arc<DatabaseEnv>>) -> eyre::Result<()> {
warn!("Make sure the node is not running when running `reth db diff`!");
// open second db
let second_db_path: PathBuf = self.secondary_datadir.join("db").into();

View File

@ -4,7 +4,7 @@ use clap::Parser;
use eyre::WrapErr;
use reth_db::{database::Database, table::Table, DatabaseEnv, RawValue, TableViewer, Tables};
use reth_primitives::hex;
use std::cell::RefCell;
use std::{cell::RefCell, sync::Arc};
use tracing::error;
#[derive(Parser, Debug)]
@ -50,7 +50,7 @@ pub struct Command {
impl Command {
/// Execute `db list` command
pub fn execute(self, tool: &DbTool<DatabaseEnv>) -> eyre::Result<()> {
pub fn execute(self, tool: &DbTool<Arc<DatabaseEnv>>) -> eyre::Result<()> {
self.table.view(&ListTableViewer { tool, args: &self })
}
@ -81,7 +81,7 @@ impl Command {
}
struct ListTableViewer<'a> {
tool: &'a DbTool<DatabaseEnv>,
tool: &'a DbTool<Arc<DatabaseEnv>>,
args: &'a Command,
}

View File

@ -1,24 +1,12 @@
//! Database debugging tool
use crate::{
args::{
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs,
},
commands::common::{AccessRights, Environment, EnvironmentArgs},
utils::DbTool,
};
use clap::{Parser, Subcommand};
use reth_db::{
open_db, open_db_read_only,
version::{get_db_version, DatabaseVersionError, DB_VERSION},
};
use reth_node_core::args::DatadirArgs;
use reth_primitives::ChainSpec;
use reth_provider::{providers::StaticFileProvider, ProviderFactory};
use std::{
io::{self, Write},
sync::Arc,
};
use reth_db::version::{get_db_version, DatabaseVersionError, DB_VERSION};
use std::io::{self, Write};
mod checksum;
mod clear;
@ -32,24 +20,8 @@ mod tui;
/// `reth db` command
#[derive(Debug, Parser)]
pub struct Command {
/// The chain this node is running.
///
/// Possible values are either a built-in chain or the path to a chain specification file.
#[arg(
long,
value_name = "CHAIN_OR_PATH",
long_help = chain_help(),
default_value = SUPPORTED_CHAINS[0],
value_parser = genesis_value_parser,
global = true,
)]
chain: Arc<ChainSpec>,
#[command(flatten)]
datadir: DatadirArgs,
#[command(flatten)]
db: DatabaseArgs,
env: EnvironmentArgs,
#[command(subcommand)]
command: Subcommands,
@ -84,12 +56,10 @@ pub enum Subcommands {
/// `db_ro_exec` opens a database in read-only mode, and then execute with the provided command
macro_rules! db_ro_exec {
($chain:expr, $db_path:expr, $db_args:ident, $sfp:ident, $tool:ident, $command:block) => {
let db = open_db_read_only($db_path, $db_args)?;
let provider_factory =
ProviderFactory::new(db, $chain.clone(), StaticFileProvider::read_only($sfp)?);
($env:expr, $tool:ident, $command:block) => {
let Environment { provider_factory, .. } = $env.init(AccessRights::RO)?;
let $tool = DbTool::new(provider_factory)?;
let $tool = DbTool::new(provider_factory.clone())?;
$command;
};
}
@ -97,36 +67,34 @@ macro_rules! db_ro_exec {
impl Command {
/// Execute `db` command
pub async fn execute(self) -> eyre::Result<()> {
// add network name to data dir
let data_dir = self.datadir.resolve_datadir(self.chain.chain);
let data_dir = self.env.datadir.clone().resolve_datadir(self.env.chain.chain);
let db_path = data_dir.db();
let db_args = self.db.database_args();
let static_files_path = data_dir.static_files();
match self.command {
// TODO: We'll need to add this on the DB trait.
Subcommands::Stats(command) => {
db_ro_exec!(self.chain, &db_path, db_args, static_files_path, tool, {
db_ro_exec!(self.env, tool, {
command.execute(data_dir, &tool)?;
});
}
Subcommands::List(command) => {
db_ro_exec!(self.chain, &db_path, db_args, static_files_path, tool, {
db_ro_exec!(self.env, tool, {
command.execute(&tool)?;
});
}
Subcommands::Checksum(command) => {
db_ro_exec!(self.chain, &db_path, db_args, static_files_path, tool, {
db_ro_exec!(self.env, tool, {
command.execute(&tool)?;
});
}
Subcommands::Diff(command) => {
db_ro_exec!(self.chain, &db_path, db_args, static_files_path, tool, {
db_ro_exec!(self.env, tool, {
command.execute(&tool)?;
});
}
Subcommands::Get(command) => {
db_ro_exec!(self.chain, &db_path, db_args, static_files_path, tool, {
db_ro_exec!(self.env, tool, {
command.execute(&tool)?;
});
}
@ -146,24 +114,12 @@ impl Command {
}
}
let db = open_db(&db_path, db_args)?;
let provider_factory = ProviderFactory::new(
db,
self.chain.clone(),
StaticFileProvider::read_write(&static_files_path)?,
);
let Environment { provider_factory, .. } = self.env.init(AccessRights::RW)?;
let tool = DbTool::new(provider_factory)?;
tool.drop(db_path, static_files_path)?;
}
Subcommands::Clear(command) => {
let db = open_db(&db_path, db_args)?;
let provider_factory = ProviderFactory::new(
db,
self.chain.clone(),
StaticFileProvider::read_write(static_files_path)?,
);
let Environment { provider_factory, .. } = self.env.init(AccessRights::RW)?;
command.execute(provider_factory)?;
}
Subcommands::Version => {
@ -193,12 +149,13 @@ impl Command {
#[cfg(test)]
mod tests {
use super::*;
use reth_node_core::args::utils::SUPPORTED_CHAINS;
use std::path::Path;
#[test]
fn parse_stats_globals() {
let path = format!("../{}", SUPPORTED_CHAINS[0]);
let cmd = Command::try_parse_from(["reth", "--datadir", &path, "stats"]).unwrap();
assert_eq!(cmd.datadir.resolve_datadir(cmd.chain.chain).as_ref(), Path::new(&path));
assert_eq!(cmd.env.datadir.resolve_datadir(cmd.env.chain.chain).as_ref(), Path::new(&path));
}
}

View File

@ -11,7 +11,7 @@ use reth_fs_util as fs;
use reth_node_core::dirs::{ChainPath, DataDirPath};
use reth_primitives::static_file::{find_fixed_range, SegmentRangeInclusive};
use reth_provider::providers::StaticFileProvider;
use std::time::Duration;
use std::{sync::Arc, time::Duration};
#[derive(Parser, Debug)]
/// The arguments for the `reth db stats` command
@ -39,7 +39,7 @@ impl Command {
pub fn execute(
self,
data_dir: ChainPath<DataDirPath>,
tool: &DbTool<DatabaseEnv>,
tool: &DbTool<Arc<DatabaseEnv>>,
) -> eyre::Result<()> {
if self.checksum {
let checksum_report = self.checksum_report(tool)?;
@ -58,7 +58,7 @@ impl Command {
Ok(())
}
fn db_stats_table(&self, tool: &DbTool<DatabaseEnv>) -> eyre::Result<ComfyTable> {
fn db_stats_table(&self, tool: &DbTool<Arc<DatabaseEnv>>) -> eyre::Result<ComfyTable> {
let mut table = ComfyTable::new();
table.load_preset(comfy_table::presets::ASCII_MARKDOWN);
table.set_header([
@ -306,7 +306,7 @@ impl Command {
Ok(table)
}
fn checksum_report(&self, tool: &DbTool<DatabaseEnv>) -> eyre::Result<ComfyTable> {
fn checksum_report(&self, tool: &DbTool<Arc<DatabaseEnv>>) -> eyre::Result<ComfyTable> {
let mut table = ComfyTable::new();
table.load_preset(comfy_table::presets::ASCII_MARKDOWN);
table.set_header(vec![Cell::new("Table"), Cell::new("Checksum"), Cell::new("Elapsed")]);

View File

@ -1,10 +1,7 @@
//! Command for debugging block building.
use crate::{
args::{
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs, DatadirArgs,
},
commands::common::{AccessRights, Environment, EnvironmentArgs},
macros::block_executor,
};
use alloy_rlp::Decodable;
@ -19,7 +16,7 @@ use reth_blockchain_tree::{
};
use reth_cli_runner::CliContext;
use reth_consensus::Consensus;
use reth_db::{init_db, DatabaseEnv};
use reth_db::DatabaseEnv;
use reth_errors::RethResult;
use reth_evm::execute::{BlockExecutionOutput, BlockExecutorProvider, Executor};
use reth_fs_util as fs;
@ -29,14 +26,13 @@ use reth_primitives::{
constants::eip4844::{LoadKzgSettingsError, MAINNET_KZG_TRUSTED_SETUP},
revm_primitives::KzgSettings,
stage::StageId,
Address, BlobTransaction, BlobTransactionSidecar, Bytes, ChainSpec, PooledTransactionsElement,
Receipts, SealedBlock, SealedBlockWithSenders, Transaction, TransactionSigned, TxEip4844, B256,
U256,
Address, BlobTransaction, BlobTransactionSidecar, Bytes, PooledTransactionsElement, Receipts,
SealedBlock, SealedBlockWithSenders, Transaction, TransactionSigned, TxEip4844, B256, U256,
};
use reth_provider::{
providers::{BlockchainProvider, StaticFileProvider},
BlockHashReader, BlockReader, BlockWriter, BundleStateWithReceipts, ProviderFactory,
StageCheckpointReader, StateProviderFactory,
providers::BlockchainProvider, BlockHashReader, BlockReader, BlockWriter,
BundleStateWithReceipts, ChainSpecProvider, ProviderFactory, StageCheckpointReader,
StateProviderFactory,
};
use reth_revm::database::StateProviderDatabase;
use reth_rpc_types::engine::{BlobsBundleV1, PayloadAttributes};
@ -52,24 +48,8 @@ use tracing::*;
/// The script will then parse the block and attempt to build a similar one.
#[derive(Debug, Parser)]
pub struct Command {
/// The chain this node is running.
///
/// Possible values are either a built-in chain or the path to a chain specification file.
#[arg(
long,
value_name = "CHAIN_OR_PATH",
long_help = chain_help(),
default_value = SUPPORTED_CHAINS[0],
value_parser = genesis_value_parser
)]
chain: Arc<ChainSpec>,
#[command(flatten)]
datadir: DatadirArgs,
/// Database arguments.
#[command(flatten)]
db: DatabaseArgs,
env: EnvironmentArgs,
/// Overrides the KZG trusted setup by reading from the supplied file.
#[arg(long, value_name = "PATH")]
@ -136,23 +116,12 @@ impl Command {
/// Execute `debug in-memory-merkle` command
pub async fn execute(self, ctx: CliContext) -> eyre::Result<()> {
// add network name to data dir
let data_dir = self.datadir.clone().resolve_datadir(self.chain.chain);
let db_path = data_dir.db();
fs::create_dir_all(&db_path)?;
// initialize the database
let db = Arc::new(init_db(db_path, self.db.database_args())?);
let provider_factory = ProviderFactory::new(
Arc::clone(&db),
Arc::clone(&self.chain),
StaticFileProvider::read_write(data_dir.static_files())?,
);
let Environment { provider_factory, .. } = self.env.init(AccessRights::RW)?;
let consensus: Arc<dyn Consensus> =
Arc::new(EthBeaconConsensus::new(Arc::clone(&self.chain)));
Arc::new(EthBeaconConsensus::new(provider_factory.chain_spec()));
let executor = block_executor!(self.chain.clone());
let executor = block_executor!(provider_factory.chain_spec());
// configure blockchain tree
let tree_externals =
@ -169,11 +138,16 @@ impl Command {
BlockchainProvider::new(provider_factory.clone(), blockchain_tree.clone())?;
let blob_store = InMemoryBlobStore::default();
let validator = TransactionValidationTaskExecutor::eth_builder(Arc::clone(&self.chain))
let validator =
TransactionValidationTaskExecutor::eth_builder(provider_factory.chain_spec())
.with_head_timestamp(best_block.timestamp)
.kzg_settings(self.kzg_settings()?)
.with_additional_tasks(1)
.build_with_tasks(blockchain_db.clone(), ctx.task_executor.clone(), blob_store.clone());
.build_with_tasks(
blockchain_db.clone(),
ctx.task_executor.clone(),
blob_store.clone(),
);
let transaction_pool = reth_transaction_pool::Pool::eth_pool(
validator,
@ -259,7 +233,7 @@ impl Command {
best_block.hash(),
payload_attrs,
)?,
self.chain.clone(),
provider_factory.chain_spec(),
);
let args = BuildArguments::new(
@ -273,7 +247,7 @@ impl Command {
#[cfg(feature = "optimism")]
let payload_builder = reth_node_optimism::OptimismPayloadBuilder::new(
self.chain.clone(),
provider_factory.chain_spec(),
reth_node_optimism::OptimismEvmConfig::default(),
)
.compute_pending_block();
@ -295,7 +269,7 @@ impl Command {
SealedBlockWithSenders::new(block.clone(), senders).unwrap();
let db = StateProviderDatabase::new(blockchain_db.latest()?);
let executor = block_executor!(self.chain.clone()).executor(db);
let executor = block_executor!(provider_factory.chain_spec()).executor(db);
let BlockExecutionOutput { state, receipts, .. } =
executor.execute((&block_with_senders.clone().unseal(), U256::MAX).into())?;

View File

@ -1,11 +1,8 @@
//! Command for debugging execution.
use crate::{
args::{
get_secret_key,
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs, DatadirArgs, NetworkArgs,
},
args::{get_secret_key, NetworkArgs},
commands::common::{AccessRights, Environment, EnvironmentArgs},
macros::block_executor,
utils::get_single_header,
};
@ -13,25 +10,20 @@ use clap::Parser;
use futures::{stream::select as stream_select, StreamExt};
use reth_beacon_consensus::EthBeaconConsensus;
use reth_cli_runner::CliContext;
use reth_config::{config::EtlConfig, Config};
use reth_config::Config;
use reth_consensus::Consensus;
use reth_db::{database::Database, init_db, DatabaseEnv};
use reth_db_common::init::init_genesis;
use reth_db::{database::Database, DatabaseEnv};
use reth_downloaders::{
bodies::bodies::BodiesDownloaderBuilder,
headers::reverse_headers::ReverseHeadersDownloaderBuilder,
};
use reth_exex::ExExManagerHandle;
use reth_fs_util as fs;
use reth_network::{NetworkEvents, NetworkHandle};
use reth_network_api::NetworkInfo;
use reth_network_p2p::{bodies::client::BodiesClient, headers::client::HeadersClient};
use reth_primitives::{
stage::StageId, BlockHashOrNumber, BlockNumber, ChainSpec, PruneModes, B256,
};
use reth_primitives::{stage::StageId, BlockHashOrNumber, BlockNumber, PruneModes, B256};
use reth_provider::{
providers::StaticFileProvider, BlockExecutionWriter, HeaderSyncMode, ProviderFactory,
StageCheckpointReader,
BlockExecutionWriter, ChainSpecProvider, HeaderSyncMode, ProviderFactory, StageCheckpointReader,
};
use reth_stages::{
sets::DefaultStages,
@ -47,27 +39,12 @@ use tracing::*;
/// `reth debug execution` command
#[derive(Debug, Parser)]
pub struct Command {
/// The chain this node is running.
///
/// Possible values are either a built-in chain or the path to a chain specification file.
#[arg(
long,
value_name = "CHAIN_OR_PATH",
long_help = chain_help(),
default_value = SUPPORTED_CHAINS[0],
value_parser = genesis_value_parser
)]
chain: Arc<ChainSpec>,
#[command(flatten)]
datadir: DatadirArgs,
env: EnvironmentArgs,
#[command(flatten)]
network: NetworkArgs,
#[command(flatten)]
db: DatabaseArgs,
/// The maximum block height.
#[arg(long)]
pub to: u64,
@ -105,7 +82,7 @@ impl Command {
let prune_modes = config.prune.clone().map(|prune| prune.segments).unwrap_or_default();
let (tip_tx, tip_rx) = watch::channel(B256::ZERO);
let executor = block_executor!(self.chain.clone());
let executor = block_executor!(provider_factory.chain_spec());
let header_mode = HeaderSyncMode::Tip(tip_rx);
let pipeline = Pipeline::builder()
@ -150,7 +127,7 @@ impl Command {
let secret_key = get_secret_key(&network_secret_path)?;
let network = self
.network
.network_config(config, self.chain.clone(), secret_key, default_peers_path)
.network_config(config, provider_factory.chain_spec(), secret_key, default_peers_path)
.with_task_executor(Box::new(task_executor))
.listener_addr(SocketAddr::new(self.network.addr, self.network.port))
.discovery_addr(SocketAddr::new(
@ -186,29 +163,10 @@ impl Command {
/// Execute `execution-debug` command
pub async fn execute(self, ctx: CliContext) -> eyre::Result<()> {
let mut config = Config::default();
let data_dir = self.datadir.clone().resolve_datadir(self.chain.chain);
let db_path = data_dir.db();
// Make sure ETL doesn't default to /tmp/, but to whatever datadir is set to
if config.stages.etl.dir.is_none() {
config.stages.etl.dir = Some(EtlConfig::from_datadir(data_dir.data_dir()));
}
fs::create_dir_all(&db_path)?;
let db = Arc::new(init_db(db_path, self.db.database_args())?);
let provider_factory = ProviderFactory::new(
db.clone(),
self.chain.clone(),
StaticFileProvider::read_write(data_dir.static_files())?,
);
debug!(target: "reth::cli", chain=%self.chain.chain, genesis=?self.chain.genesis_hash(), "Initializing genesis");
init_genesis(provider_factory.clone())?;
let Environment { provider_factory, config, data_dir } = self.env.init(AccessRights::RW)?;
let consensus: Arc<dyn Consensus> =
Arc::new(EthBeaconConsensus::new(Arc::clone(&self.chain)));
Arc::new(EthBeaconConsensus::new(provider_factory.chain_spec()));
// Configure and build network
let network_secret_path =
@ -257,7 +215,7 @@ impl Command {
Some(network.clone()),
latest_block_number,
events,
db.clone(),
provider_factory.db_ref().clone(),
),
);

View File

@ -1,11 +1,8 @@
//! Command for debugging in-memory merkle trie calculation.
use crate::{
args::{
get_secret_key,
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs, DatadirArgs, NetworkArgs,
},
args::{get_secret_key, NetworkArgs},
commands::common::{AccessRights, Environment, EnvironmentArgs},
macros::block_executor,
utils::{get_single_body, get_single_header},
};
@ -13,17 +10,16 @@ use backon::{ConstantBuilder, Retryable};
use clap::Parser;
use reth_cli_runner::CliContext;
use reth_config::Config;
use reth_db::{init_db, DatabaseEnv};
use reth_db::DatabaseEnv;
use reth_errors::BlockValidationError;
use reth_evm::execute::{BlockExecutionOutput, BlockExecutorProvider, Executor};
use reth_fs_util as fs;
use reth_network::NetworkHandle;
use reth_network_api::NetworkInfo;
use reth_primitives::{stage::StageId, BlockHashOrNumber, ChainSpec, Receipts};
use reth_primitives::{stage::StageId, BlockHashOrNumber, Receipts};
use reth_provider::{
providers::StaticFileProvider, AccountExtReader, BundleStateWithReceipts, HashingWriter,
HeaderProvider, LatestStateProviderRef, OriginalValuesKnown, ProviderFactory,
StageCheckpointReader, StateWriter, StaticFileProviderFactory, StorageReader,
AccountExtReader, BundleStateWithReceipts, ChainSpecProvider, HashingWriter, HeaderProvider,
LatestStateProviderRef, OriginalValuesKnown, ProviderFactory, StageCheckpointReader,
StateWriter, StaticFileProviderFactory, StorageReader,
};
use reth_revm::database::StateProviderDatabase;
use reth_tasks::TaskExecutor;
@ -37,23 +33,8 @@ use tracing::*;
/// merkle root for it.
#[derive(Debug, Parser)]
pub struct Command {
/// The chain this node is running.
///
/// Possible values are either a built-in chain or the path to a chain specification file.
#[arg(
long,
value_name = "CHAIN_OR_PATH",
long_help = chain_help(),
default_value = SUPPORTED_CHAINS[0],
value_parser = genesis_value_parser
)]
chain: Arc<ChainSpec>,
#[command(flatten)]
datadir: DatadirArgs,
#[command(flatten)]
db: DatabaseArgs,
env: EnvironmentArgs,
#[command(flatten)]
network: NetworkArgs,
@ -79,7 +60,7 @@ impl Command {
let secret_key = get_secret_key(&network_secret_path)?;
let network = self
.network
.network_config(config, self.chain.clone(), secret_key, default_peers_path)
.network_config(config, provider_factory.chain_spec(), secret_key, default_peers_path)
.with_task_executor(Box::new(task_executor))
.listener_addr(SocketAddr::new(self.network.addr, self.network.port))
.discovery_addr(SocketAddr::new(
@ -96,19 +77,9 @@ impl Command {
/// Execute `debug in-memory-merkle` command
pub async fn execute(self, ctx: CliContext) -> eyre::Result<()> {
let config = Config::default();
let Environment { provider_factory, config, data_dir } = self.env.init(AccessRights::RW)?;
// add network name to data dir
let data_dir = self.datadir.clone().resolve_datadir(self.chain.chain);
let db_path = data_dir.db();
fs::create_dir_all(&db_path)?;
// initialize the database
let db = Arc::new(init_db(db_path, self.db.database_args())?);
let static_file_provider = StaticFileProvider::read_write(data_dir.static_files())?;
let factory =
ProviderFactory::new(db.clone(), self.chain.clone(), static_file_provider.clone());
let provider = factory.provider()?;
let provider = provider_factory.provider()?;
// Look up merkle checkpoint
let merkle_checkpoint = provider
@ -124,7 +95,7 @@ impl Command {
.build_network(
&config,
ctx.task_executor.clone(),
factory.clone(),
provider_factory.clone(),
network_secret_path,
data_dir.known_peers(),
)
@ -147,7 +118,7 @@ impl Command {
.await?;
let client = fetch_client.clone();
let chain = Arc::clone(&self.chain);
let chain = provider_factory.chain_spec();
let block = (move || get_single_body(client.clone(), Arc::clone(&chain), header.clone()))
.retry(&backoff)
.notify(
@ -157,10 +128,10 @@ impl Command {
let db = StateProviderDatabase::new(LatestStateProviderRef::new(
provider.tx_ref(),
factory.static_file_provider(),
provider_factory.static_file_provider(),
));
let executor = block_executor!(self.chain.clone()).executor(db);
let executor = block_executor!(provider_factory.chain_spec()).executor(db);
let merkle_block_td =
provider.header_td_by_number(merkle_block_number)?.unwrap_or_default();
@ -190,7 +161,7 @@ impl Command {
return Ok(())
}
let provider_rw = factory.provider_rw()?;
let provider_rw = provider_factory.provider_rw()?;
// Insert block, state and hashes
provider_rw.insert_historical_block(

View File

@ -1,11 +1,8 @@
//! Command for debugging merkle trie calculation.
use crate::{
args::{
get_secret_key,
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs, DatadirArgs, NetworkArgs,
},
args::{get_secret_key, NetworkArgs},
commands::common::{AccessRights, Environment, EnvironmentArgs},
macros::block_executor,
utils::get_single_header,
};
@ -15,17 +12,15 @@ use reth_beacon_consensus::EthBeaconConsensus;
use reth_cli_runner::CliContext;
use reth_config::Config;
use reth_consensus::Consensus;
use reth_db::{cursor::DbCursorRO, init_db, tables, transaction::DbTx, DatabaseEnv};
use reth_db::{cursor::DbCursorRO, tables, transaction::DbTx, DatabaseEnv};
use reth_evm::execute::{BatchBlockExecutionOutput, BatchExecutor, BlockExecutorProvider};
use reth_fs_util as fs;
use reth_network::NetworkHandle;
use reth_network_api::NetworkInfo;
use reth_network_p2p::full_block::FullBlockClient;
use reth_primitives::{stage::StageCheckpoint, BlockHashOrNumber, ChainSpec, PruneModes};
use reth_primitives::{stage::StageCheckpoint, BlockHashOrNumber, PruneModes};
use reth_provider::{
providers::StaticFileProvider, BlockNumReader, BlockWriter, BundleStateWithReceipts,
HeaderProvider, LatestStateProviderRef, OriginalValuesKnown, ProviderError, ProviderFactory,
StateWriter,
BlockNumReader, BlockWriter, BundleStateWithReceipts, ChainSpecProvider, HeaderProvider,
LatestStateProviderRef, OriginalValuesKnown, ProviderError, ProviderFactory, StateWriter,
};
use reth_revm::database::StateProviderDatabase;
use reth_stages::{
@ -39,23 +34,8 @@ use tracing::*;
/// `reth debug merkle` command
#[derive(Debug, Parser)]
pub struct Command {
/// The chain this node is running.
///
/// Possible values are either a built-in chain or the path to a chain specification file.
#[arg(
long,
value_name = "CHAIN_OR_PATH",
long_help = chain_help(),
default_value = SUPPORTED_CHAINS[0],
value_parser = genesis_value_parser
)]
chain: Arc<ChainSpec>,
#[command(flatten)]
datadir: DatadirArgs,
#[command(flatten)]
db: DatabaseArgs,
env: EnvironmentArgs,
#[command(flatten)]
network: NetworkArgs,
@ -85,7 +65,7 @@ impl Command {
let secret_key = get_secret_key(&network_secret_path)?;
let network = self
.network
.network_config(config, self.chain.clone(), secret_key, default_peers_path)
.network_config(config, provider_factory.chain_spec(), secret_key, default_peers_path)
.with_task_executor(Box::new(task_executor))
.listener_addr(SocketAddr::new(self.network.addr, self.network.port))
.discovery_addr(SocketAddr::new(
@ -102,21 +82,9 @@ impl Command {
/// Execute `merkle-debug` command
pub async fn execute(self, ctx: CliContext) -> eyre::Result<()> {
let config = Config::default();
let Environment { provider_factory, config, data_dir } = self.env.init(AccessRights::RW)?;
// add network name to data dir
let data_dir = self.datadir.clone().resolve_datadir(self.chain.chain);
let db_path = data_dir.db();
fs::create_dir_all(&db_path)?;
// initialize the database
let db = Arc::new(init_db(db_path, self.db.database_args())?);
let factory = ProviderFactory::new(
db.clone(),
self.chain.clone(),
StaticFileProvider::read_write(data_dir.static_files())?,
);
let provider_rw = factory.provider_rw()?;
let provider_rw = provider_factory.provider_rw()?;
// Configure and build network
let network_secret_path =
@ -125,13 +93,13 @@ impl Command {
.build_network(
&config,
ctx.task_executor.clone(),
factory.clone(),
provider_factory.clone(),
network_secret_path,
data_dir.known_peers(),
)
.await?;
let executor_provider = block_executor!(self.chain.clone());
let executor_provider = block_executor!(provider_factory.chain_spec());
// Initialize the fetch client
info!(target: "reth::cli", target_block_number=self.to, "Downloading tip of block range");
@ -151,7 +119,7 @@ impl Command {
// build the full block client
let consensus: Arc<dyn Consensus> =
Arc::new(EthBeaconConsensus::new(Arc::clone(&self.chain)));
Arc::new(EthBeaconConsensus::new(provider_factory.chain_spec()));
let block_range_client = FullBlockClient::new(fetch_client, consensus);
// get best block number

View File

@ -1,9 +1,6 @@
use crate::{
args::{
get_secret_key,
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs, DatadirArgs, NetworkArgs,
},
args::{get_secret_key, NetworkArgs},
commands::common::{AccessRights, Environment, EnvironmentArgs},
macros::block_executor,
};
use clap::Parser;
@ -16,16 +13,15 @@ use reth_blockchain_tree::{
use reth_cli_runner::CliContext;
use reth_config::Config;
use reth_consensus::Consensus;
use reth_db::{init_db, DatabaseEnv};
use reth_db::DatabaseEnv;
use reth_fs_util as fs;
use reth_network::NetworkHandle;
use reth_network_api::NetworkInfo;
use reth_node_core::engine::engine_store::{EngineMessageStore, StoredEngineApiMessage};
use reth_payload_builder::{PayloadBuilderHandle, PayloadBuilderService};
use reth_primitives::{ChainSpec, PruneModes};
use reth_primitives::PruneModes;
use reth_provider::{
providers::{BlockchainProvider, StaticFileProvider},
CanonStateSubscriptions, ProviderFactory,
providers::BlockchainProvider, CanonStateSubscriptions, ChainSpecProvider, ProviderFactory,
};
use reth_stages::Pipeline;
use reth_static_file::StaticFileProducer;
@ -40,23 +36,8 @@ use tracing::*;
/// It does not require
#[derive(Debug, Parser)]
pub struct Command {
/// The chain this node is running.
///
/// Possible values are either a built-in chain or the path to a chain specification file.
#[arg(
long,
value_name = "CHAIN_OR_PATH",
long_help = chain_help(),
default_value = SUPPORTED_CHAINS[0],
value_parser = genesis_value_parser
)]
chain: Arc<ChainSpec>,
#[command(flatten)]
datadir: DatadirArgs,
#[command(flatten)]
db: DatabaseArgs,
env: EnvironmentArgs,
#[command(flatten)]
network: NetworkArgs,
@ -82,7 +63,7 @@ impl Command {
let secret_key = get_secret_key(&network_secret_path)?;
let network = self
.network
.network_config(config, self.chain.clone(), secret_key, default_peers_path)
.network_config(config, provider_factory.chain_spec(), secret_key, default_peers_path)
.with_task_executor(Box::new(task_executor))
.listener_addr(SocketAddr::new(self.network.addr, self.network.port))
.discovery_addr(SocketAddr::new(
@ -99,25 +80,12 @@ impl Command {
/// Execute `debug replay-engine` command
pub async fn execute(self, ctx: CliContext) -> eyre::Result<()> {
let config = Config::default();
// Add network name to data dir
let data_dir = self.datadir.clone().resolve_datadir(self.chain.chain);
let db_path = data_dir.db();
fs::create_dir_all(&db_path)?;
// Initialize the database
let db = Arc::new(init_db(db_path, self.db.database_args())?);
let provider_factory = ProviderFactory::new(
db.clone(),
self.chain.clone(),
StaticFileProvider::read_write(data_dir.static_files())?,
);
let Environment { provider_factory, config, data_dir } = self.env.init(AccessRights::RW)?;
let consensus: Arc<dyn Consensus> =
Arc::new(EthBeaconConsensus::new(Arc::clone(&self.chain)));
Arc::new(EthBeaconConsensus::new(provider_factory.chain_spec()));
let executor = block_executor!(self.chain.clone());
let executor = block_executor!(provider_factory.chain_spec());
// Configure blockchain tree
let tree_externals =
@ -148,7 +116,7 @@ impl Command {
// Optimism's payload builder is implemented on the OptimismPayloadBuilder type.
#[cfg(feature = "optimism")]
let payload_builder = reth_node_optimism::OptimismPayloadBuilder::new(
self.chain.clone(),
provider_factory.chain_spec(),
reth_node_optimism::OptimismEvmConfig::default(),
);
@ -157,7 +125,7 @@ impl Command {
NoopTransactionPool::default(),
ctx.task_executor.clone(),
BasicPayloadJobGeneratorConfig::default(),
self.chain.clone(),
provider_factory.chain_spec(),
payload_builder,
);

View File

@ -1,21 +1,16 @@
//! Command that initializes the node by importing a chain from a file.
use crate::{
args::{
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs, DatadirArgs,
},
commands::common::{AccessRights, Environment, EnvironmentArgs},
macros::block_executor,
version::SHORT_VERSION,
};
use clap::Parser;
use eyre::Context;
use futures::{Stream, StreamExt};
use reth_beacon_consensus::EthBeaconConsensus;
use reth_config::{config::EtlConfig, Config};
use reth_config::Config;
use reth_consensus::Consensus;
use reth_db::{database::Database, init_db, tables, transaction::DbTx};
use reth_db_common::init::init_genesis;
use reth_db::{database::Database, tables, transaction::DbTx};
use reth_downloaders::{
bodies::bodies::BodiesDownloaderBuilder,
file_client::{ChunkedFileReader, FileClient, DEFAULT_BYTE_LEN_CHUNK_CHAIN_FILE},
@ -26,10 +21,10 @@ use reth_network_p2p::{
headers::downloader::{HeaderDownloader, SyncTarget},
};
use reth_node_events::node::NodeEvent;
use reth_primitives::{stage::StageId, ChainSpec, PruneModes, B256};
use reth_primitives::{stage::StageId, PruneModes, B256};
use reth_provider::{
providers::StaticFileProvider, BlockNumReader, ChainSpecProvider, HeaderProvider,
HeaderSyncMode, ProviderError, ProviderFactory, StageCheckpointReader,
BlockNumReader, ChainSpecProvider, HeaderProvider, HeaderSyncMode, ProviderError,
ProviderFactory, StageCheckpointReader,
};
use reth_stages::{prelude::*, Pipeline, StageSet};
use reth_static_file::StaticFileProducer;
@ -40,21 +35,8 @@ use tracing::{debug, error, info};
/// Syncs RLP encoded blocks from a file.
#[derive(Debug, Parser)]
pub struct ImportCommand {
/// The path to the configuration file to use.
#[arg(long, value_name = "FILE", verbatim_doc_comment)]
config: Option<PathBuf>,
/// The chain this node is running.
///
/// Possible values are either a built-in chain or the path to a chain specification file.
#[arg(
long,
value_name = "CHAIN_OR_PATH",
long_help = chain_help(),
default_value = SUPPORTED_CHAINS[0],
value_parser = genesis_value_parser
)]
chain: Arc<ChainSpec>,
#[command(flatten)]
env: EnvironmentArgs,
/// Disables stages that require state.
#[arg(long, verbatim_doc_comment)]
@ -64,12 +46,6 @@ pub struct ImportCommand {
#[arg(long, value_name = "CHUNK_LEN", verbatim_doc_comment)]
chunk_len: Option<u64>,
#[command(flatten)]
datadir: DatadirArgs,
#[command(flatten)]
db: DatabaseArgs,
/// The path to a block file for import.
///
/// The online stages (headers and bodies) are replaced by a file import, after which the
@ -92,34 +68,9 @@ impl ImportCommand {
"Chunking chain import"
);
// add network name to data dir
let data_dir = self.datadir.resolve_datadir(self.chain.chain);
let config_path = self.config.clone().unwrap_or_else(|| data_dir.config());
let Environment { provider_factory, config, .. } = self.env.init(AccessRights::RW)?;
let mut config: Config = load_config(config_path.clone())?;
info!(target: "reth::cli", path = ?config_path, "Configuration loaded");
// Make sure ETL doesn't default to /tmp/, but to whatever datadir is set to
if config.stages.etl.dir.is_none() {
config.stages.etl.dir = Some(EtlConfig::from_datadir(data_dir.data_dir()));
}
let db_path = data_dir.db();
info!(target: "reth::cli", path = ?db_path, "Opening database");
let db = Arc::new(init_db(db_path, self.db.database_args())?);
info!(target: "reth::cli", "Database opened");
let provider_factory = ProviderFactory::new(
db.clone(),
self.chain.clone(),
StaticFileProvider::read_write(data_dir.static_files())?,
);
debug!(target: "reth::cli", chain=%self.chain.chain, genesis=?self.chain.genesis_hash(), "Initializing genesis");
init_genesis(provider_factory.clone())?;
let consensus = Arc::new(EthBeaconConsensus::new(self.chain.clone()));
let consensus = Arc::new(EthBeaconConsensus::new(self.env.chain.clone()));
info!(target: "reth::cli", "Consensus engine initialized");
// open file
@ -162,7 +113,7 @@ impl ImportCommand {
None,
latest_block_number,
events,
db.clone(),
provider_factory.db_ref().clone(),
));
// Run pipeline
@ -273,22 +224,17 @@ where
Ok((pipeline, events))
}
/// Loads the reth config
pub fn load_config(config_path: PathBuf) -> eyre::Result<Config> {
confy::load_path::<Config>(config_path.clone())
.wrap_err_with(|| format!("Could not load config file {config_path:?}"))
}
#[cfg(test)]
mod tests {
use super::*;
use reth_node_core::args::utils::SUPPORTED_CHAINS;
#[test]
fn parse_common_import_command_chain_args() {
for chain in SUPPORTED_CHAINS {
let args: ImportCommand = ImportCommand::parse_from(["reth", "--chain", chain, "."]);
assert_eq!(
Ok(args.chain.chain),
Ok(args.env.chain.chain),
chain.parse::<reth_primitives::Chain>(),
"failed to parse chain {chain}"
);

View File

@ -2,25 +2,21 @@
//! file.
use crate::{
args::{
utils::{genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs,
commands::{
common::{AccessRights, Environment, EnvironmentArgs},
import::build_import_pipeline,
},
commands::import::{build_import_pipeline, load_config},
version::SHORT_VERSION,
};
use clap::Parser;
use reth_config::{config::EtlConfig, Config};
use reth_consensus::noop::NoopConsensus;
use reth_db::{init_db, tables, transaction::DbTx};
use reth_db_common::init::init_genesis;
use reth_db::{tables, transaction::DbTx};
use reth_downloaders::file_client::{
ChunkedFileReader, FileClient, DEFAULT_BYTE_LEN_CHUNK_CHAIN_FILE,
};
use reth_node_core::args::DatadirArgs;
use reth_optimism_primitives::bedrock_import::is_dup_tx;
use reth_primitives::{stage::StageId, PruneModes};
use reth_provider::{providers::StaticFileProvider, ProviderFactory, StageCheckpointReader};
use reth_provider::StageCheckpointReader;
use reth_static_file::StaticFileProducer;
use std::{path::PathBuf, sync::Arc};
use tracing::{debug, error, info};
@ -28,20 +24,13 @@ use tracing::{debug, error, info};
/// Syncs RLP encoded blocks from a file.
#[derive(Debug, Parser)]
pub struct ImportOpCommand {
/// The path to the configuration file to use.
#[arg(long, value_name = "FILE", verbatim_doc_comment)]
config: Option<PathBuf>,
#[command(flatten)]
env: EnvironmentArgs,
/// Chunk byte length to read from file.
#[arg(long, value_name = "CHUNK_LEN", verbatim_doc_comment)]
chunk_len: Option<u64>,
#[command(flatten)]
datadir: DatadirArgs,
#[command(flatten)]
db: DatabaseArgs,
/// The path to a block file for import.
///
/// The online stages (headers and bodies) are replaced by a file import, after which the
@ -64,35 +53,7 @@ impl ImportOpCommand {
"Chunking chain import"
);
let chain_spec = genesis_value_parser(SUPPORTED_CHAINS[0])?;
// add network name to data dir
let data_dir = self.datadir.resolve_datadir(chain_spec.chain);
let config_path = self.config.clone().unwrap_or_else(|| data_dir.config());
let mut config: Config = load_config(config_path.clone())?;
info!(target: "reth::cli", path = ?config_path, "Configuration loaded");
// Make sure ETL doesn't default to /tmp/, but to whatever datadir is set to
if config.stages.etl.dir.is_none() {
config.stages.etl.dir = Some(EtlConfig::from_datadir(data_dir.data_dir()));
}
let db_path = data_dir.db();
info!(target: "reth::cli", path = ?db_path, "Opening database");
let db = Arc::new(init_db(db_path, self.db.database_args())?);
info!(target: "reth::cli", "Database opened");
let provider_factory = ProviderFactory::new(
db.clone(),
chain_spec.clone(),
StaticFileProvider::read_write(data_dir.static_files())?,
);
debug!(target: "reth::cli", chain=%chain_spec.chain, genesis=?chain_spec.genesis_hash(), "Initializing genesis");
init_genesis(provider_factory.clone())?;
let Environment { provider_factory, config, .. } = self.env.init(AccessRights::RW)?;
// we use noop here because we expect the inputs to be valid
let consensus = Arc::new(NoopConsensus::default());
@ -148,7 +109,7 @@ impl ImportOpCommand {
None,
latest_block_number,
events,
db.clone(),
provider_factory.db_ref().clone(),
));
// Run pipeline

View File

@ -1,44 +1,33 @@
//! Command that imports OP mainnet receipts from Bedrock datadir, exported via
//! <https://github.com/testinprod-io/op-geth/pull/1>.
use std::{
path::{Path, PathBuf},
sync::Arc,
};
use crate::commands::common::{AccessRights, Environment, EnvironmentArgs};
use clap::Parser;
use reth_db::{database::Database, init_db, tables, transaction::DbTx};
use reth_db::{database::Database, tables, transaction::DbTx};
use reth_downloaders::{
file_client::{ChunkedFileReader, DEFAULT_BYTE_LEN_CHUNK_CHAIN_FILE},
receipt_file_client::ReceiptFileClient,
};
use reth_node_core::{args::DatadirArgs, version::SHORT_VERSION};
use reth_node_core::version::SHORT_VERSION;
use reth_optimism_primitives::bedrock_import::is_dup_tx;
use reth_primitives::{stage::StageId, Receipts, StaticFileSegment};
use reth_provider::{
providers::StaticFileProvider, BundleStateWithReceipts, OriginalValuesKnown, ProviderFactory,
StageCheckpointReader, StateWriter, StaticFileProviderFactory, StaticFileWriter, StatsReader,
BundleStateWithReceipts, OriginalValuesKnown, ProviderFactory, StageCheckpointReader,
StateWriter, StaticFileProviderFactory, StaticFileWriter, StatsReader,
};
use std::path::{Path, PathBuf};
use tracing::{debug, error, info, trace};
use crate::args::{
utils::{genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs,
};
/// Initializes the database with the genesis block.
#[derive(Debug, Parser)]
pub struct ImportReceiptsOpCommand {
#[command(flatten)]
datadir: DatadirArgs,
env: EnvironmentArgs,
/// Chunk byte length to read from file.
#[arg(long, value_name = "CHUNK_LEN", verbatim_doc_comment)]
chunk_len: Option<u64>,
#[command(flatten)]
db: DatabaseArgs,
/// The path to a receipts file for import. File must use `HackReceiptFileCodec` (used for
/// exporting OP chain segment below Bedrock block via testinprod/op-geth).
///
@ -57,21 +46,7 @@ impl ImportReceiptsOpCommand {
"Chunking receipts import"
);
let chain_spec = genesis_value_parser(SUPPORTED_CHAINS[0])?;
// add network name to data dir
let data_dir = self.datadir.resolve_datadir(chain_spec.chain);
let db_path = data_dir.db();
info!(target: "reth::cli", path = ?db_path, "Opening database");
let db = Arc::new(init_db(db_path, self.db.database_args())?);
info!(target: "reth::cli", "Database opened");
let provider_factory = ProviderFactory::new(
db.clone(),
chain_spec.clone(),
StaticFileProvider::read_write(data_dir.static_files())?,
);
let Environment { provider_factory, .. } = self.env.init(AccessRights::RW)?;
import_receipts_from_file(
provider_factory,

View File

@ -1,37 +1,15 @@
//! Command that initializes the node from a genesis file.
use crate::args::{
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs, DatadirArgs,
};
use crate::commands::common::{AccessRights, Environment, EnvironmentArgs};
use clap::Parser;
use reth_db::init_db;
use reth_db_common::init::init_genesis;
use reth_primitives::ChainSpec;
use reth_provider::{providers::StaticFileProvider, ProviderFactory};
use std::sync::Arc;
use reth_provider::BlockHashReader;
use tracing::info;
/// Initializes the database with the genesis block.
#[derive(Debug, Parser)]
pub struct InitCommand {
/// The chain this node is running.
///
/// Possible values are either a built-in chain or the path to a chain specification file.
#[arg(
long,
value_name = "CHAIN_OR_PATH",
long_help = chain_help(),
default_value = SUPPORTED_CHAINS[0],
value_parser = genesis_value_parser
)]
chain: Arc<ChainSpec>,
#[command(flatten)]
datadir: DatadirArgs,
#[command(flatten)]
db: DatabaseArgs,
env: EnvironmentArgs,
}
impl InitCommand {
@ -39,22 +17,11 @@ impl InitCommand {
pub async fn execute(self) -> eyre::Result<()> {
info!(target: "reth::cli", "reth init starting");
// add network name to data dir
let data_dir = self.datadir.resolve_datadir(self.chain.chain);
let db_path = data_dir.db();
info!(target: "reth::cl", path = ?db_path, "Opening database");
let db = Arc::new(init_db(&db_path, self.db.database_args())?);
info!(target: "reth::cli", "Database opened");
let Environment { provider_factory, .. } = self.env.init(AccessRights::RW)?;
let provider_factory = ProviderFactory::new(
db,
self.chain,
StaticFileProvider::read_write(data_dir.static_files())?,
);
info!(target: "reth::cli", "Writing genesis block");
let hash = init_genesis(provider_factory)?;
let hash = provider_factory
.block_hash(0)?
.ok_or_else(|| eyre::eyre!("Genesis hash not found."))?;
info!(target: "reth::cli", hash = ?hash, "Genesis block written");
Ok(())

View File

@ -1,37 +1,21 @@
//! Command that initializes the node from a genesis file.
use crate::args::{
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs,
};
use crate::commands::common::{AccessRights, Environment, EnvironmentArgs};
use clap::Parser;
use reth_config::config::EtlConfig;
use reth_db::{database::Database, init_db};
use reth_db::database::Database;
use reth_db_common::init::init_from_state_dump;
use reth_node_core::args::DatadirArgs;
use reth_primitives::{ChainSpec, B256};
use reth_provider::{providers::StaticFileProvider, ProviderFactory};
use reth_primitives::B256;
use reth_provider::ProviderFactory;
use std::{fs::File, io::BufReader, path::PathBuf, sync::Arc};
use std::{fs::File, io::BufReader, path::PathBuf};
use tracing::info;
/// Initializes the database with the genesis block.
#[derive(Debug, Parser)]
pub struct InitStateCommand {
/// The chain this node is running.
///
/// Possible values are either a built-in chain or the path to a chain specification file.
#[arg(
long,
value_name = "CHAIN_OR_PATH",
long_help = chain_help(),
default_value = SUPPORTED_CHAINS[0],
value_parser = genesis_value_parser
)]
chain: Arc<ChainSpec>,
#[command(flatten)]
datadir: DatadirArgs,
env: EnvironmentArgs,
/// JSONL file with state dump.
///
@ -52,9 +36,6 @@ pub struct InitStateCommand {
/// and including the non-genesis block to init chain at. See 'import' command.
#[arg(value_name = "STATE_DUMP_FILE", verbatim_doc_comment)]
state: PathBuf,
#[command(flatten)]
db: DatabaseArgs,
}
impl InitStateCommand {
@ -62,26 +43,11 @@ impl InitStateCommand {
pub async fn execute(self) -> eyre::Result<()> {
info!(target: "reth::cli", "Reth init-state starting");
// add network name to data dir
let data_dir = self.datadir.resolve_datadir(self.chain.chain);
let db_path = data_dir.db();
info!(target: "reth::cli", path = ?db_path, "Opening database");
let db = Arc::new(init_db(&db_path, self.db.database_args())?);
info!(target: "reth::cli", "Database opened");
let provider_factory = ProviderFactory::new(
db,
self.chain,
StaticFileProvider::read_write(data_dir.static_files())?,
);
let etl_config = EtlConfig::new(
Some(EtlConfig::from_datadir(data_dir.data_dir())),
EtlConfig::default_file_size(),
);
let Environment { config, provider_factory, .. } = self.env.init(AccessRights::RW)?;
info!(target: "reth::cli", "Initiating state dump");
let hash = init_at_state(self.state, provider_factory, etl_config)?;
let hash = init_at_state(self.state, provider_factory, config.stages.etl)?;
info!(target: "reth::cli", hash = ?hash, "Genesis block written");
Ok(())

View File

@ -16,3 +16,5 @@ pub mod p2p;
pub mod recover;
pub mod stage;
pub mod test_vectors;
pub mod common;

View File

@ -1,62 +1,28 @@
use crate::args::utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS};
use crate::commands::common::{AccessRights, Environment, EnvironmentArgs};
use clap::Parser;
use reth_cli_runner::CliContext;
use reth_db::{
cursor::{DbCursorRO, DbDupCursorRW},
init_db, tables,
tables,
transaction::DbTx,
};
use reth_db_common::init::init_genesis;
use reth_node_core::args::{DatabaseArgs, DatadirArgs};
use reth_primitives::ChainSpec;
use reth_provider::{
providers::StaticFileProvider, BlockNumReader, HeaderProvider, ProviderError, ProviderFactory,
};
use reth_provider::{BlockNumReader, HeaderProvider, ProviderError};
use reth_trie::StateRoot;
use std::{fs, sync::Arc};
use tracing::*;
/// `reth recover storage-tries` command
#[derive(Debug, Parser)]
pub struct Command {
/// The chain this node is running.
///
/// Possible values are either a built-in chain or the path to a chain specification file.
#[arg(
long,
value_name = "CHAIN_OR_PATH",
long_help = chain_help(),
default_value = SUPPORTED_CHAINS[0],
value_parser = genesis_value_parser
)]
chain: Arc<ChainSpec>,
#[command(flatten)]
datadir: DatadirArgs,
/// All database related arguments
#[command(flatten)]
pub db: DatabaseArgs,
env: EnvironmentArgs,
}
impl Command {
/// Execute `storage-tries` recovery command
pub async fn execute(self, _ctx: CliContext) -> eyre::Result<()> {
let data_dir = self.datadir.resolve_datadir(self.chain.chain);
let db_path = data_dir.db();
fs::create_dir_all(&db_path)?;
let db = Arc::new(init_db(db_path, self.db.database_args())?);
let Environment { provider_factory, .. } = self.env.init(AccessRights::RW)?;
let factory = ProviderFactory::new(
&db,
self.chain.clone(),
StaticFileProvider::read_write(data_dir.static_files())?,
);
debug!(target: "reth::cli", chain=%self.chain.chain, genesis=?self.chain.genesis_hash(), "Initializing genesis");
init_genesis(factory.clone())?;
let mut provider = factory.provider_rw()?;
let mut provider = provider_factory.provider_rw()?;
let best_block = provider.best_block_number()?;
let best_header = provider
.sealed_header(best_block)?

View File

@ -1,46 +1,22 @@
//! Database debugging tool
use crate::{
args::{
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs, DatadirArgs, StageEnum,
},
args::StageEnum,
commands::common::{AccessRights, Environment, EnvironmentArgs},
utils::DbTool,
};
use clap::Parser;
use itertools::Itertools;
use reth_db::{open_db, static_file::iter_static_files, tables, transaction::DbTxMut, DatabaseEnv};
use reth_db::{static_file::iter_static_files, tables, transaction::DbTxMut, DatabaseEnv};
use reth_db_common::init::{insert_genesis_header, insert_genesis_history, insert_genesis_state};
use reth_fs_util as fs;
use reth_primitives::{
stage::StageId, static_file::find_fixed_range, ChainSpec, StaticFileSegment,
};
use reth_provider::{
providers::{StaticFileProvider, StaticFileWriter},
ProviderFactory, StaticFileProviderFactory,
};
use std::sync::Arc;
use reth_primitives::{stage::StageId, static_file::find_fixed_range, StaticFileSegment};
use reth_provider::{providers::StaticFileWriter, StaticFileProviderFactory};
/// `reth drop-stage` command
#[derive(Debug, Parser)]
pub struct Command {
/// The chain this node is running.
///
/// Possible values are either a built-in chain or the path to a chain specification file.
#[arg(
long,
value_name = "CHAIN_OR_PATH",
long_help = chain_help(),
default_value = SUPPORTED_CHAINS[0],
value_parser = genesis_value_parser
)]
chain: Arc<ChainSpec>,
#[command(flatten)]
datadir: DatadirArgs,
#[command(flatten)]
db: DatabaseArgs,
env: EnvironmentArgs,
stage: StageEnum,
}
@ -48,17 +24,8 @@ pub struct Command {
impl Command {
/// Execute `db` command
pub async fn execute(self) -> eyre::Result<()> {
// add network name to data dir
let data_dir = self.datadir.resolve_datadir(self.chain.chain);
let db_path = data_dir.db();
fs::create_dir_all(&db_path)?;
let Environment { provider_factory, .. } = self.env.init(AccessRights::RW)?;
let db = open_db(db_path.as_ref(), self.db.database_args())?;
let provider_factory = ProviderFactory::new(
db,
self.chain.clone(),
StaticFileProvider::read_write(data_dir.static_files())?,
);
let static_file_provider = provider_factory.static_file_provider();
let tool = DbTool::new(provider_factory)?;
@ -100,7 +67,7 @@ impl Command {
StageId::Headers.to_string(),
Default::default(),
)?;
insert_genesis_header::<DatabaseEnv>(tx, &static_file_provider, self.chain)?;
insert_genesis_header::<DatabaseEnv>(tx, &static_file_provider, self.env.chain)?;
}
StageEnum::Bodies => {
tx.clear::<tables::BlockBodyIndices>()?;
@ -113,7 +80,7 @@ impl Command {
StageId::Bodies.to_string(),
Default::default(),
)?;
insert_genesis_header::<DatabaseEnv>(tx, &static_file_provider, self.chain)?;
insert_genesis_header::<DatabaseEnv>(tx, &static_file_provider, self.env.chain)?;
}
StageEnum::Senders => {
tx.clear::<tables::TransactionSenders>()?;
@ -133,7 +100,7 @@ impl Command {
StageId::Execution.to_string(),
Default::default(),
)?;
let alloc = &self.chain.genesis().alloc;
let alloc = &self.env.chain.genesis().alloc;
insert_genesis_state::<DatabaseEnv>(tx, alloc.len(), alloc.iter())?;
}
StageEnum::AccountHashing => {
@ -192,7 +159,7 @@ impl Command {
StageId::IndexStorageHistory.to_string(),
Default::default(),
)?;
insert_genesis_history(&provider_rw, self.chain.genesis.alloc.iter())?;
insert_genesis_history(&provider_rw, self.env.chain.genesis.alloc.iter())?;
}
StageEnum::TxLookup => {
tx.clear::<tables::TransactionHashNumbers>()?;
@ -200,7 +167,7 @@ impl Command {
StageId::TransactionLookup.to_string(),
Default::default(),
)?;
insert_genesis_header::<DatabaseEnv>(tx, &static_file_provider, self.chain)?;
insert_genesis_header::<DatabaseEnv>(tx, &static_file_provider, self.env.chain)?;
}
}

View File

@ -1,21 +1,20 @@
//! Database debugging tool
use crate::{dirs::DataDirPath, utils::DbTool};
use crate::args::{
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs, DatadirArgs,
use crate::{
commands::common::{AccessRights, Environment, EnvironmentArgs},
dirs::DataDirPath,
utils::DbTool,
};
use crate::args::DatadirArgs;
use clap::Parser;
use reth_db::{
cursor::DbCursorRO, database::Database, init_db, mdbx::DatabaseArguments,
models::client_version::ClientVersion, open_db_read_only, table::TableImporter, tables,
transaction::DbTx, DatabaseEnv,
models::client_version::ClientVersion, table::TableImporter, tables, transaction::DbTx,
DatabaseEnv,
};
use reth_node_core::dirs::PlatformPath;
use reth_primitives::ChainSpec;
use reth_provider::{providers::StaticFileProvider, ProviderFactory};
use std::{path::PathBuf, sync::Arc};
use std::path::PathBuf;
use tracing::info;
mod hashing_storage;
@ -33,23 +32,8 @@ use merkle::dump_merkle_stage;
/// `reth dump-stage` command
#[derive(Debug, Parser)]
pub struct Command {
/// The chain this node is running.
///
/// Possible values are either a built-in chain or the path to a chain specification file.
#[arg(
long,
value_name = "CHAIN_OR_PATH",
long_help = chain_help(),
default_value = SUPPORTED_CHAINS[0],
value_parser = genesis_value_parser
)]
chain: Arc<ChainSpec>,
#[command(flatten)]
datadir: DatadirArgs,
#[command(flatten)]
db: DatabaseArgs,
env: EnvironmentArgs,
#[command(subcommand)]
command: Stages,
@ -87,65 +71,25 @@ pub struct StageCommand {
dry_run: bool,
}
macro_rules! handle_stage {
($stage_fn:ident, $tool:expr, $command:expr) => {{
let StageCommand { output_datadir, from, to, dry_run, .. } = $command;
let output_datadir = output_datadir.with_chain($tool.chain().chain, DatadirArgs::default());
$stage_fn($tool, *from, *to, output_datadir, *dry_run).await?
}};
}
impl Command {
/// Execute `dump-stage` command
pub async fn execute(self) -> eyre::Result<()> {
// add network name to data dir
let data_dir = self.datadir.clone().resolve_datadir(self.chain.chain);
let db_path = data_dir.db();
info!(target: "reth::cli", path = ?db_path, "Opening database");
let db = Arc::new(open_db_read_only(&db_path, self.db.database_args())?);
let provider_factory = ProviderFactory::new(
db,
self.chain.clone(),
StaticFileProvider::read_only(data_dir.static_files())?,
);
info!(target: "reth::cli", "Database opened");
let Environment { provider_factory, .. } = self.env.init(AccessRights::RO)?;
let tool = DbTool::new(provider_factory)?;
match &self.command {
Stages::Execution(StageCommand { output_datadir, from, to, dry_run, .. }) => {
dump_execution_stage(
&tool,
*from,
*to,
output_datadir.with_chain(self.chain.chain, self.datadir.clone()),
*dry_run,
)
.await?
}
Stages::StorageHashing(StageCommand { output_datadir, from, to, dry_run, .. }) => {
dump_hashing_storage_stage(
&tool,
*from,
*to,
output_datadir.with_chain(self.chain.chain, self.datadir.clone()),
*dry_run,
)
.await?
}
Stages::AccountHashing(StageCommand { output_datadir, from, to, dry_run, .. }) => {
dump_hashing_account_stage(
&tool,
*from,
*to,
output_datadir.with_chain(self.chain.chain, self.datadir.clone()),
*dry_run,
)
.await?
}
Stages::Merkle(StageCommand { output_datadir, from, to, dry_run, .. }) => {
dump_merkle_stage(
&tool,
*from,
*to,
output_datadir.with_chain(self.chain.chain, self.datadir.clone()),
*dry_run,
)
.await?
}
Stages::Execution(cmd) => handle_stage!(dump_execution_stage, &tool, cmd),
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),
}
Ok(())

View File

@ -3,29 +3,19 @@
//! Stage debugging tool
use crate::{
args::{
get_secret_key,
utils::{chain_help, chain_spec_value_parser, SUPPORTED_CHAINS},
DatabaseArgs, DatadirArgs, NetworkArgs, StageEnum,
},
args::{get_secret_key, NetworkArgs, StageEnum},
commands::common::{AccessRights, Environment, EnvironmentArgs},
macros::block_executor,
prometheus_exporter,
version::SHORT_VERSION,
};
use clap::Parser;
use reth_beacon_consensus::EthBeaconConsensus;
use reth_cli_runner::CliContext;
use reth_config::{
config::{EtlConfig, HashingConfig, SenderRecoveryConfig, TransactionLookupConfig},
Config,
};
use reth_db::init_db;
use reth_config::config::{HashingConfig, SenderRecoveryConfig, TransactionLookupConfig};
use reth_downloaders::bodies::bodies::BodiesDownloaderBuilder;
use reth_exex::ExExManagerHandle;
use reth_primitives::ChainSpec;
use reth_provider::{
providers::StaticFileProvider, ProviderFactory, StageCheckpointReader, StageCheckpointWriter,
StaticFileProviderFactory,
ChainSpecProvider, StageCheckpointReader, StageCheckpointWriter, StaticFileProviderFactory,
};
use reth_stages::{
stages::{
@ -35,27 +25,14 @@ use reth_stages::{
},
ExecInput, ExecOutput, Stage, StageExt, UnwindInput, UnwindOutput,
};
use std::{any::Any, net::SocketAddr, path::PathBuf, sync::Arc, time::Instant};
use std::{any::Any, net::SocketAddr, sync::Arc, time::Instant};
use tracing::*;
/// `reth stage` command
#[derive(Debug, Parser)]
pub struct Command {
/// The path to the configuration file to use.
#[arg(long, value_name = "FILE", verbatim_doc_comment)]
config: Option<PathBuf>,
/// The chain this node is running.
///
/// Possible values are either a built-in chain or the path to a chain specification file.
#[arg(
long,
value_name = "CHAIN_OR_PATH",
long_help = chain_help(),
default_value = SUPPORTED_CHAINS[0],
value_parser = chain_spec_value_parser
)]
chain: Arc<ChainSpec>,
#[command(flatten)]
env: EnvironmentArgs,
/// Enable Prometheus metrics.
///
@ -79,14 +56,6 @@ pub struct Command {
#[arg(long)]
batch_size: Option<u64>,
/// The maximum size in bytes of data held in memory before being flushed to disk as a file.
#[arg(long)]
etl_file_size: Option<usize>,
/// Directory where to collect ETL files
#[arg(long)]
etl_dir: Option<PathBuf>,
/// Normally, running the stage requires unwinding for stages that already
/// have been run, in order to not rewrite to the same database slots.
///
@ -107,14 +76,8 @@ pub struct Command {
#[arg(long)]
checkpoints: bool,
#[command(flatten)]
datadir: DatadirArgs,
#[command(flatten)]
network: NetworkArgs,
#[command(flatten)]
db: DatabaseArgs,
}
impl Command {
@ -124,25 +87,8 @@ impl Command {
// Does not do anything on windows.
let _ = fdlimit::raise_fd_limit();
// add network name to data dir
let data_dir = self.datadir.resolve_datadir(self.chain.chain);
let config_path = self.config.clone().unwrap_or_else(|| data_dir.config());
let Environment { provider_factory, config, data_dir } = self.env.init(AccessRights::RW)?;
let config: Config = confy::load_path(config_path).unwrap_or_default();
info!(target: "reth::cli", "reth {} starting stage {:?}", SHORT_VERSION, self.stage);
// use the overridden db path if specified
let db_path = data_dir.db();
info!(target: "reth::cli", path = ?db_path, "Opening database");
let db = Arc::new(init_db(db_path, self.db.database_args())?);
info!(target: "reth::cli", "Database opened");
let provider_factory = ProviderFactory::new(
Arc::clone(&db),
self.chain.clone(),
StaticFileProvider::read_write(data_dir.static_files())?,
);
let mut provider_rw = provider_factory.provider_rw()?;
if let Some(listen_addr) = self.metrics {
@ -150,7 +96,7 @@ impl Command {
prometheus_exporter::serve(
listen_addr,
prometheus_exporter::install_recorder()?,
Arc::clone(&db),
provider_factory.db_ref().clone(),
provider_factory.static_file_provider(),
metrics_process::Collector::default(),
ctx.task_executor,
@ -160,16 +106,14 @@ impl Command {
let batch_size = self.batch_size.unwrap_or(self.to.saturating_sub(self.from) + 1);
let etl_config = EtlConfig::new(
Some(self.etl_dir.unwrap_or_else(|| EtlConfig::from_datadir(data_dir.data_dir()))),
self.etl_file_size.unwrap_or(EtlConfig::default_file_size()),
);
let etl_config = config.stages.etl.clone();
let prune_modes = config.prune.clone().map(|prune| prune.segments).unwrap_or_default();
let (mut exec_stage, mut unwind_stage): (Box<dyn Stage<_>>, Option<Box<dyn Stage<_>>>) =
match self.stage {
StageEnum::Bodies => {
let consensus = Arc::new(EthBeaconConsensus::new(self.chain.clone()));
let consensus =
Arc::new(EthBeaconConsensus::new(provider_factory.chain_spec()));
let mut config = config;
config.peers.trusted_nodes_only = self.network.trusted_only;
@ -192,7 +136,7 @@ impl Command {
.network
.network_config(
&config,
self.chain.clone(),
provider_factory.chain_spec(),
p2p_secret_key,
default_peers_path,
)
@ -223,7 +167,7 @@ impl Command {
None,
),
StageEnum::Execution => {
let executor = block_executor!(self.chain.clone());
let executor = block_executor!(provider_factory.chain_spec());
(
Box::new(ExecutionStage::new(
executor,

View File

@ -4,14 +4,14 @@ use clap::{Parser, Subcommand};
use reth_beacon_consensus::EthBeaconConsensus;
use reth_config::Config;
use reth_consensus::Consensus;
use reth_db::{database::Database, open_db};
use reth_db::database::Database;
use reth_downloaders::{bodies::noop::NoopBodiesDownloader, headers::noop::NoopHeaderDownloader};
use reth_exex::ExExManagerHandle;
use reth_node_core::args::NetworkArgs;
use reth_primitives::{BlockHashOrNumber, ChainSpec, PruneModes, B256};
use reth_primitives::{BlockHashOrNumber, PruneModes, B256};
use reth_provider::{
providers::StaticFileProvider, BlockExecutionWriter, BlockNumReader, ChainSpecProvider,
HeaderSyncMode, ProviderFactory, StaticFileProviderFactory,
BlockExecutionWriter, BlockNumReader, ChainSpecProvider, HeaderSyncMode, ProviderFactory,
StaticFileProviderFactory,
};
use reth_stages::{
sets::DefaultStages,
@ -24,34 +24,15 @@ use tokio::sync::watch;
use tracing::info;
use crate::{
args::{
utils::{chain_help, genesis_value_parser, SUPPORTED_CHAINS},
DatabaseArgs, DatadirArgs,
},
commands::common::{AccessRights, Environment, EnvironmentArgs},
macros::block_executor,
};
/// `reth stage unwind` command
#[derive(Debug, Parser)]
pub struct Command {
/// The chain this node is running.
///
/// Possible values are either a built-in chain or the path to a chain specification file.
#[arg(
long,
value_name = "CHAIN_OR_PATH",
long_help = chain_help(),
default_value = SUPPORTED_CHAINS[0],
value_parser = genesis_value_parser,
global = true
)]
chain: Arc<ChainSpec>,
#[command(flatten)]
datadir: DatadirArgs,
#[command(flatten)]
db: DatabaseArgs,
env: EnvironmentArgs,
#[command(flatten)]
network: NetworkArgs,
@ -63,21 +44,7 @@ pub struct Command {
impl Command {
/// Execute `db stage unwind` command
pub async fn execute(self) -> eyre::Result<()> {
// add network name to data dir
let data_dir = self.datadir.clone().resolve_datadir(self.chain.chain);
let db_path = data_dir.db();
if !db_path.exists() {
eyre::bail!("Database {db_path:?} does not exist.")
}
let config_path = data_dir.config();
let config: Config = confy::load_path(config_path).unwrap_or_default();
let db = Arc::new(open_db(db_path.as_ref(), self.db.database_args())?);
let provider_factory = ProviderFactory::new(
db,
self.chain.clone(),
StaticFileProvider::read_write(data_dir.static_files())?,
);
let Environment { provider_factory, config, .. } = self.env.init(AccessRights::RW)?;
let range = self.command.unwind_range(provider_factory.clone())?;
if *range.start() == 0 {

21
book/cli/reth/db.md vendored
View File

@ -19,15 +19,6 @@ Commands:
help Print this message or the help of the given subcommand(s)
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--instance <INSTANCE>
Add a new instance of a node.
@ -57,6 +48,18 @@ Datadir:
--datadir.static_files <PATH>
The absolute path to store static files in.
--config <FILE>
The path to the configuration file to use
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
Database:
--db.log-level <LOG_LEVEL>
Database logging level. Levels higher than "notice" require a debug build

View File

@ -11,15 +11,6 @@ Arguments:
The table name
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--start-key <START_KEY>
The start of the range to checksum

View File

@ -12,15 +12,6 @@ Commands:
help Print this message or the help of the given subcommand(s)
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--instance <INSTANCE>
Add a new instance of a node.

View File

@ -11,15 +11,6 @@ Arguments:
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--instance <INSTANCE>
Add a new instance of a node.

View File

@ -14,15 +14,6 @@ Arguments:
- receipts: Static File segment responsible for the `Receipts` table
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--instance <INSTANCE>
Add a new instance of a node.

View File

@ -7,15 +7,6 @@ $ reth db diff --help
Usage: reth db diff [OPTIONS] --secondary-datadir <SECONDARY_DATADIR> --output <OUTPUT>
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--secondary-datadir <SECONDARY_DATADIR>
The path to the data dir for all reth files and subdirectories.

View File

@ -7,15 +7,6 @@ $ reth db drop --help
Usage: reth db drop [OPTIONS]
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
-f, --force
Bypasses the interactive confirmation and drops the database directly

View File

@ -12,15 +12,6 @@ Commands:
help Print this message or the help of the given subcommand(s)
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--instance <INSTANCE>
Add a new instance of a node.

View File

@ -17,15 +17,6 @@ Arguments:
The subkey to get content for
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--raw
Output bytes instead of human-readable decoded value

View File

@ -17,15 +17,6 @@ Arguments:
The key to get content for
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--raw
Output bytes instead of human-readable decoded value

View File

@ -11,15 +11,6 @@ Arguments:
The table name
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
-s, --skip <SKIP>
Skip first N entries

View File

@ -7,15 +7,6 @@ $ reth db path --help
Usage: reth db path [OPTIONS]
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--instance <INSTANCE>
Add a new instance of a node.

View File

@ -7,15 +7,6 @@ $ reth db stats --help
Usage: reth db stats [OPTIONS]
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--detailed-sizes
Show only the total size for static files

View File

@ -7,15 +7,6 @@ $ reth db version --help
Usage: reth db version [OPTIONS]
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--instance <INSTANCE>
Add a new instance of a node.

View File

@ -7,24 +7,6 @@ $ reth import --help
Usage: reth import [OPTIONS] <IMPORT_PATH>
Options:
--config <FILE>
The path to the configuration file to use.
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--no-state
Disables stages that require state.
--chunk-len <CHUNK_LEN>
Chunk byte length to read from file.
--instance <INSTANCE>
Add a new instance of a node.
@ -54,6 +36,18 @@ Datadir:
--datadir.static_files <PATH>
The absolute path to store static files in.
--config <FILE>
The path to the configuration file to use
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
Database:
--db.log-level <LOG_LEVEL>
Database logging level. Levels higher than "notice" require a debug build
@ -73,6 +67,12 @@ Database:
[possible values: true, false]
--no-state
Disables stages that require state.
--chunk-len <CHUNK_LEN>
Chunk byte length to read from file.
<IMPORT_PATH>
The path to a block file for import.

View File

@ -7,15 +7,6 @@ $ reth init-state --help
Usage: reth init-state [OPTIONS] <STATE_DUMP_FILE>
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--instance <INSTANCE>
Add a new instance of a node.
@ -45,24 +36,17 @@ Datadir:
--datadir.static_files <PATH>
The absolute path to store static files in.
<STATE_DUMP_FILE>
JSONL file with state dump.
--config <FILE>
The path to the configuration file to use
Must contain accounts in following format, additional account fields are ignored. Must
also contain { "root": \<state-root\> } as first line.
{
"balance": "\<balance\>",
"nonce": \<nonce\>,
"code": "\<bytecode\>",
"storage": {
"\<key\>": "\<value\>",
..
},
"address": "\<address\>",
}
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Allows init at a non-genesis block. Caution! Blocks must be manually imported up until
and including the non-genesis block to init chain at. See 'import' command.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
Database:
--db.log-level <LOG_LEVEL>
@ -83,6 +67,25 @@ Database:
[possible values: true, false]
<STATE_DUMP_FILE>
JSONL file with state dump.
Must contain accounts in following format, additional account fields are ignored. Must
also contain { "root": \<state-root\> } as first line.
{
"balance": "\<balance\>",
"nonce": \<nonce\>,
"code": "\<bytecode\>",
"storage": {
"\<key\>": "\<value\>",
..
},
"address": "\<address\>",
}
Allows init at a non-genesis block. Caution! Blocks must be manually imported up until
and including the non-genesis block to init chain at. See 'import' command.
Logging:
--log.stdout.format <FORMAT>
The format to use for logs written to stdout

21
book/cli/reth/init.md vendored
View File

@ -7,15 +7,6 @@ $ reth init --help
Usage: reth init [OPTIONS]
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--instance <INSTANCE>
Add a new instance of a node.
@ -45,6 +36,18 @@ Datadir:
--datadir.static_files <PATH>
The absolute path to store static files in.
--config <FILE>
The path to the configuration file to use
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
Database:
--db.log-level <LOG_LEVEL>
Database logging level. Levels higher than "notice" require a debug build

View File

@ -7,15 +7,6 @@ $ reth recover storage-tries --help
Usage: reth recover storage-tries [OPTIONS]
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--instance <INSTANCE>
Add a new instance of a node.
@ -45,6 +36,18 @@ Datadir:
--datadir.static_files <PATH>
The absolute path to store static files in.
--config <FILE>
The path to the configuration file to use
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
Database:
--db.log-level <LOG_LEVEL>
Database logging level. Levels higher than "notice" require a debug build

View File

@ -7,15 +7,6 @@ $ reth stage drop --help
Usage: reth stage drop [OPTIONS] <STAGE>
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--instance <INSTANCE>
Add a new instance of a node.
@ -45,6 +36,18 @@ Datadir:
--datadir.static_files <PATH>
The absolute path to store static files in.
--config <FILE>
The path to the configuration file to use
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
Database:
--db.log-level <LOG_LEVEL>
Database logging level. Levels higher than "notice" require a debug build

View File

@ -14,15 +14,6 @@ Commands:
help Print this message or the help of the given subcommand(s)
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--instance <INSTANCE>
Add a new instance of a node.
@ -52,6 +43,18 @@ Datadir:
--datadir.static_files <PATH>
The absolute path to store static files in.
--config <FILE>
The path to the configuration file to use
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
Database:
--db.log-level <LOG_LEVEL>
Database logging level. Levels higher than "notice" require a debug build

View File

@ -6,69 +6,7 @@ Run a single stage.
$ reth stage run --help
Usage: reth stage run [OPTIONS] --from <FROM> --to <TO> <STAGE>
Arguments:
<STAGE>
The name of the stage to run
Possible values:
- headers: The headers stage within the pipeline
- bodies: The bodies stage within the pipeline
- senders: The senders stage within the pipeline
- execution: The execution stage within the pipeline
- account-hashing: The account hashing stage within the pipeline
- storage-hashing: The storage hashing stage within the pipeline
- hashing: The account and storage hashing stages within the pipeline
- merkle: The merkle stage within the pipeline
- tx-lookup: The transaction lookup stage within the pipeline
- account-history: The account history stage within the pipeline
- storage-history: The storage history stage within the pipeline
Options:
--config <FILE>
The path to the configuration file to use.
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--metrics <SOCKET>
Enable Prometheus metrics.
The metrics will be served at the given interface and port.
--from <FROM>
The height to start at
-t, --to <TO>
The end of the stage
--batch-size <BATCH_SIZE>
Batch size for stage execution and unwind
--etl-file-size <ETL_FILE_SIZE>
The maximum size in bytes of data held in memory before being flushed to disk as a file
--etl-dir <ETL_DIR>
Directory where to collect ETL files
-s, --skip-unwind
Normally, running the stage requires unwinding for stages that already have been run, in order to not rewrite to the same database slots.
You can optionally skip the unwinding phase if you're syncing a block range that has not been synced before.
-c, --commit
Commits the changes in the database. WARNING: potentially destructive.
Useful when you want to run diagnostics on the database.
--checkpoints
Save stage checkpoints
--instance <INSTANCE>
Add a new instance of a node.
@ -98,6 +36,80 @@ Datadir:
--datadir.static_files <PATH>
The absolute path to store static files in.
--config <FILE>
The path to the configuration file to use
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
Database:
--db.log-level <LOG_LEVEL>
Database logging level. Levels higher than "notice" require a debug build
Possible values:
- fatal: Enables logging for critical conditions, i.e. assertion failures
- error: Enables logging for error conditions
- warn: Enables logging for warning conditions
- notice: Enables logging for normal but significant condition
- verbose: Enables logging for verbose informational
- debug: Enables logging for debug-level messages
- trace: Enables logging for trace debug-level messages
- extra: Enables logging for extra debug-level messages
--db.exclusive <EXCLUSIVE>
Open environment in exclusive/monopolistic mode. Makes it possible to open a database on an NFS volume
[possible values: true, false]
--metrics <SOCKET>
Enable Prometheus metrics.
The metrics will be served at the given interface and port.
--from <FROM>
The height to start at
-t, --to <TO>
The end of the stage
--batch-size <BATCH_SIZE>
Batch size for stage execution and unwind
-s, --skip-unwind
Normally, running the stage requires unwinding for stages that already have been run, in order to not rewrite to the same database slots.
You can optionally skip the unwinding phase if you're syncing a block range that has not been synced before.
-c, --commit
Commits the changes in the database. WARNING: potentially destructive.
Useful when you want to run diagnostics on the database.
--checkpoints
Save stage checkpoints
<STAGE>
The name of the stage to run
Possible values:
- headers: The headers stage within the pipeline
- bodies: The bodies stage within the pipeline
- senders: The senders stage within the pipeline
- execution: The execution stage within the pipeline
- account-hashing: The account hashing stage within the pipeline
- storage-hashing: The storage hashing stage within the pipeline
- hashing: The account and storage hashing stages within the pipeline
- merkle: The merkle stage within the pipeline
- tx-lookup: The transaction lookup stage within the pipeline
- account-history: The account history stage within the pipeline
- storage-history: The storage history stage within the pipeline
Networking:
-d, --disable-discovery
Disable the discovery service
@ -225,25 +237,6 @@ Networking:
[default: 131072]
Database:
--db.log-level <LOG_LEVEL>
Database logging level. Levels higher than "notice" require a debug build
Possible values:
- fatal: Enables logging for critical conditions, i.e. assertion failures
- error: Enables logging for error conditions
- warn: Enables logging for warning conditions
- notice: Enables logging for normal but significant condition
- verbose: Enables logging for verbose informational
- debug: Enables logging for debug-level messages
- trace: Enables logging for trace debug-level messages
- extra: Enables logging for extra debug-level messages
--db.exclusive <EXCLUSIVE>
Open environment in exclusive/monopolistic mode. Makes it possible to open a database on an NFS volume
[possible values: true, false]
Logging:
--log.stdout.format <FORMAT>
The format to use for logs written to stdout

View File

@ -12,15 +12,6 @@ Commands:
help Print this message or the help of the given subcommand(s)
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--instance <INSTANCE>
Add a new instance of a node.
@ -50,6 +41,18 @@ Datadir:
--datadir.static_files <PATH>
The absolute path to store static files in.
--config <FILE>
The path to the configuration file to use
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
Database:
--db.log-level <LOG_LEVEL>
Database logging level. Levels higher than "notice" require a debug build

View File

@ -11,15 +11,6 @@ Arguments:
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--instance <INSTANCE>
Add a new instance of a node.

View File

@ -11,15 +11,6 @@ Arguments:
Options:
--chain <CHAIN_OR_PATH>
The chain this node is running.
Possible values are either a built-in chain or the path to a chain specification file.
Built-in chains:
mainnet, sepolia, goerli, holesky, dev
[default: mainnet]
--instance <INSTANCE>
Add a new instance of a node.