feat: split nodebuilder generics into separate states (#7847)

This commit is contained in:
Matthias Seitz
2024-04-24 22:18:07 +02:00
committed by GitHub
parent ddc5ed3263
commit 659059c67f
12 changed files with 1492 additions and 1538 deletions

View File

@ -14,7 +14,7 @@ use crate::{
use clap::{value_parser, Parser, Subcommand};
use reth_cli_runner::CliRunner;
use reth_db::DatabaseEnv;
use reth_node_builder::{InitState, WithLaunchContext};
use reth_node_builder::{NodeBuilder, WithLaunchContext};
use reth_primitives::ChainSpec;
use reth_tracing::FileWorkerGuard;
use std::{ffi::OsString, fmt, future::Future, sync::Arc};
@ -130,7 +130,7 @@ impl<Ext: clap::Args + fmt::Debug> Cli<Ext> {
/// ````
pub fn run<L, Fut>(mut self, launcher: L) -> eyre::Result<()>
where
L: FnOnce(WithLaunchContext<Arc<DatabaseEnv>, InitState>, Ext) -> Fut,
L: FnOnce(WithLaunchContext<NodeBuilder<Arc<DatabaseEnv>>>, Ext) -> Fut,
Fut: Future<Output = eyre::Result<()>>,
{
// add network name to logs dir

View File

@ -11,7 +11,7 @@ use crate::{
use clap::{value_parser, Args, Parser};
use reth_cli_runner::CliContext;
use reth_db::{init_db, DatabaseEnv};
use reth_node_builder::{InitState, NodeBuilder, WithLaunchContext};
use reth_node_builder::{NodeBuilder, WithLaunchContext};
use reth_node_core::{node_config::NodeConfig, version};
use reth_primitives::ChainSpec;
use std::{ffi::OsString, fmt, future::Future, net::SocketAddr, path::PathBuf, sync::Arc};
@ -136,7 +136,7 @@ impl<Ext: clap::Args + fmt::Debug> NodeCommand<Ext> {
/// closure.
pub async fn execute<L, Fut>(self, ctx: CliContext, launcher: L) -> eyre::Result<()>
where
L: FnOnce(WithLaunchContext<Arc<DatabaseEnv>, InitState>, Ext) -> Fut,
L: FnOnce(WithLaunchContext<NodeBuilder<Arc<DatabaseEnv>>>, Ext) -> Fut,
Fut: Future<Output = eyre::Result<()>>,
{
tracing::info!(target: "reth::cli", version = ?version::SHORT_VERSION, "Starting reth");

View File

@ -6,8 +6,8 @@ use reth::{
};
use reth_db::{test_utils::TempDatabase, DatabaseEnv};
use reth_node_builder::{
components::{NetworkBuilder, PayloadServiceBuilder, PoolBuilder},
FullNodeComponentsAdapter, FullNodeTypesAdapter,
components::{Components, NetworkBuilder, PayloadServiceBuilder, PoolBuilder},
FullNodeTypesAdapter, NodeAdapter,
};
use reth_primitives::ChainSpec;
use reth_provider::providers::BlockchainProvider;
@ -59,7 +59,7 @@ where
};
// Create nodes and peer them
let mut nodes: Vec<NodeHelperType<N>> = Vec::with_capacity(num_nodes);
let mut nodes: Vec<NodeTestContext<_>> = Vec::with_capacity(num_nodes);
for idx in 0..num_nodes {
let mut node_config = NodeConfig::test()
@ -110,4 +110,4 @@ type TmpNodeAdapter<N> = FullNodeTypesAdapter<N, TmpDB, BlockchainProvider<TmpDB
/// Type alias for a type of NodeHelper
pub type NodeHelperType<N> =
NodeTestContext<FullNodeComponentsAdapter<TmpNodeAdapter<N>, TmpPool<N>>>;
NodeTestContext<NodeAdapter<TmpNodeAdapter<N>, Components<TmpNodeAdapter<N>, TmpPool<N>>>>;

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,619 @@
//! Customizable node builder.
#![allow(clippy::type_complexity, missing_debug_implementations)]
use crate::{
components::{Components, ComponentsBuilder, NodeComponentsBuilder, PoolBuilder},
node::FullNode,
rpc::{RethRpcServerHandles, RpcContext},
DefaultNodeLauncher, Node, NodeHandle,
};
use futures::Future;
use reth_db::{
database::Database,
database_metrics::{DatabaseMetadata, DatabaseMetrics},
test_utils::{create_test_rw_db, TempDatabase},
DatabaseEnv,
};
use reth_exex::ExExContext;
use reth_network::{NetworkBuilder, NetworkConfig, NetworkHandle};
use reth_node_api::{FullNodeTypes, FullNodeTypesAdapter, NodeTypes};
use reth_node_core::{
cli::config::{PayloadBuilderConfig, RethTransactionPoolConfig},
dirs::{ChainPath, DataDirPath, MaybePlatformPath},
node_config::NodeConfig,
primitives::{kzg::KzgSettings, Head},
utils::write_peers_to_file,
};
use reth_primitives::{constants::eip4844::MAINNET_KZG_TRUSTED_SETUP, ChainSpec};
use reth_provider::{providers::BlockchainProvider, ChainSpecProvider};
use reth_tasks::TaskExecutor;
use reth_transaction_pool::{PoolConfig, TransactionPool};
pub use states::*;
use std::{str::FromStr, sync::Arc};
mod states;
/// The builtin provider type of the reth node.
// Note: we need to hardcode this because custom components might depend on it in associated types.
pub type RethFullProviderType<DB> = BlockchainProvider<DB>;
/// The adapter type for a reth node with the given types
pub type RethFullAdapter<DB, Types> = FullNodeTypesAdapter<Types, DB, RethFullProviderType<DB>>;
#[cfg_attr(doc, aquamarine::aquamarine)]
/// Declaratively construct a node.
///
/// [`NodeBuilder`] provides a [builder-like interface][builder] for composing
/// components of a node.
///
/// ## Order
///
/// Configuring a node starts out with a [`NodeConfig`] (this can be obtained from cli arguments for
/// example) and then proceeds to configure the core static types of the node: [NodeTypes], these
/// include the node's primitive types and the node's engine types.
///
/// Next all stateful components of the node are configured, these include the
/// [ConfigureEvm](reth_node_api::evm::ConfigureEvm), the database [Database] and all the
/// components of the node that are downstream of those types, these include:
///
/// - The transaction pool: [PoolBuilder]
/// - The network: [NetworkBuilder](crate::components::NetworkBuilder)
/// - The payload builder: [PayloadBuilder](crate::components::PayloadServiceBuilder)
///
/// Once all the components are configured, the node is ready to be launched.
///
/// On launch the builder returns a fully type aware [NodeHandle] that has access to all the
/// configured components and can interact with the node.
///
/// There are convenience functions for networks that come with a preset of types and components via
/// the [Node] trait, see `reth_node_ethereum::EthereumNode` or `reth_node_optimism::OptimismNode`.
///
/// The [NodeBuilder::node] function configures the node's types and components in one step.
///
/// ## Components
///
/// All components are configured with a [NodeComponentsBuilder] that is responsible for actually
/// creating the node components during the launch process. The [ComponentsBuilder] is a general
/// purpose implementation of the [NodeComponentsBuilder] trait that can be used to configure the
/// network, transaction pool and payload builder of the node. It enforces the correct order of
/// configuration, for example the network and the payload builder depend on the transaction pool
/// type that is configured first.
///
/// All builder traits are generic over the node types and are invoked with the [BuilderContext]
/// that gives access to internals of the that are needed to configure the components. This include
/// the original config, chain spec, the database provider and the task executor,
///
/// ## Hooks
///
/// Once all the components are configured, the builder can be used to set hooks that are run at
/// specific points in the node's lifecycle. This way custom services can be spawned before the node
/// is launched [NodeBuilder::on_component_initialized], or once the rpc server(s) are launched
/// [NodeBuilder::on_rpc_started]. The [NodeBuilder::extend_rpc_modules] can be used to inject
/// custom rpc modules into the rpc server before it is launched. See also [RpcContext]
/// All hooks accept a closure that is then invoked at the appropriate time in the node's launch
/// process.
///
/// ## Flow
///
/// The [NodeBuilder] is intended to sit behind a CLI that provides the necessary [NodeConfig]
/// input: [NodeBuilder::new]
///
/// From there the builder is configured with the node's types, components, and hooks, then launched
/// with the [NodeBuilder::launch] method. On launch all the builtin internals, such as the
/// `Database` and its providers [BlockchainProvider] are initialized before the configured
/// [NodeComponentsBuilder] is invoked with the [BuilderContext] to create the transaction pool,
/// network, and payload builder components. When the RPC is configured, the corresponding hooks are
/// invoked to allow for custom rpc modules to be injected into the rpc server:
/// [NodeBuilder::extend_rpc_modules]
///
/// Finally all components are created and all services are launched and a [NodeHandle] is returned
/// that can be used to interact with the node: [FullNode]
///
/// The following diagram shows the flow of the node builder from CLI to a launched node.
///
/// include_mmd!("docs/mermaid/builder.mmd")
///
/// ## Internals
///
/// The node builder is fully type safe, it uses the [NodeTypes] trait to enforce that all
/// components are configured with the correct types. However the database types and with that the
/// provider trait implementations are currently created by the builder itself during the launch
/// process, hence the database type is not part of the [NodeTypes] trait and the node's components,
/// that depend on the database, are configured separately. In order to have a nice trait that
/// encapsulates the entire node the [FullNodeComponents] trait was introduced. This trait has
/// convenient associated types for all the components of the node. After [NodeBuilder::launch] the
/// [NodeHandle] contains an instance of [FullNode] that implements the [FullNodeComponents] trait
/// and has access to all the components of the node. Internally the node builder uses several
/// generic adapter types that are then map to traits with associated types for ease of use.
///
/// ### Limitations
///
/// Currently the launch process is limited to ethereum nodes and requires all the components
/// specified above. It also expect beacon consensus with the ethereum engine API that is configured
/// by the builder itself during launch. This might change in the future.
///
/// [builder]: https://doc.rust-lang.org/1.0.0/style/ownership/builders.html
pub struct NodeBuilder<DB> {
/// All settings for how the node should be configured.
config: NodeConfig,
/// The configured database for the node.
database: DB,
}
impl NodeBuilder<()> {
/// Create a new [`NodeBuilder`].
pub fn new(config: NodeConfig) -> Self {
Self { config, database: () }
}
}
impl<DB> NodeBuilder<DB> {
/// Returns a reference to the node builder's config.
pub fn config(&self) -> &NodeConfig {
&self.config
}
/// Configures the underlying database that the node will use.
pub fn with_database<D>(self, database: D) -> NodeBuilder<D> {
NodeBuilder { config: self.config, database }
}
/// Preconfigure the builder with the context to launch the node.
///
/// This provides the task executor and the data directory for the node.
pub fn with_launch_context(
self,
task_executor: TaskExecutor,
data_dir: ChainPath<DataDirPath>,
) -> WithLaunchContext<NodeBuilder<DB>> {
WithLaunchContext { builder: self, task_executor, data_dir }
}
/// Creates an _ephemeral_ preconfigured node for testing purposes.
pub fn testing_node(
self,
task_executor: TaskExecutor,
) -> WithLaunchContext<NodeBuilder<Arc<TempDatabase<DatabaseEnv>>>> {
let db = create_test_rw_db();
let db_path_str = db.path().to_str().expect("Path is not valid unicode");
let path =
MaybePlatformPath::<DataDirPath>::from_str(db_path_str).expect("Path is not valid");
let data_dir = path.unwrap_or_chain_default(self.config.chain.chain);
WithLaunchContext { builder: self.with_database(db), task_executor, data_dir }
}
}
impl<DB> NodeBuilder<DB>
where
DB: Database + Unpin + Clone + 'static,
{
/// Configures the types of the node.
pub fn with_types<T>(self, types: T) -> NodeBuilderWithTypes<RethFullAdapter<DB, T>>
where
T: NodeTypes,
{
let types = FullNodeTypesAdapter::new(types);
NodeBuilderWithTypes::new(self.config, types, self.database)
}
/// Preconfigures the node with a specific node implementation.
///
/// This is a convenience method that sets the node's types and components in one call.
pub fn node<N>(
self,
node: N,
) -> NodeBuilderWithComponents<
RethFullAdapter<DB, N>,
ComponentsBuilder<
RethFullAdapter<DB, N>,
N::PoolBuilder,
N::PayloadBuilder,
N::NetworkBuilder,
>,
>
where
N: Node<RethFullAdapter<DB, N>>,
N::PoolBuilder: PoolBuilder<RethFullAdapter<DB, N>>,
N::NetworkBuilder: crate::components::NetworkBuilder<
RethFullAdapter<DB, N>,
<N::PoolBuilder as PoolBuilder<RethFullAdapter<DB, N>>>::Pool,
>,
N::PayloadBuilder: crate::components::PayloadServiceBuilder<
RethFullAdapter<DB, N>,
<N::PoolBuilder as PoolBuilder<RethFullAdapter<DB, N>>>::Pool,
>,
{
self.with_types(node.clone()).with_components(node.components())
}
}
/// A [NodeBuilder] with it's launch context already configured.
///
/// This exposes the same methods as [NodeBuilder] but with the launch context already configured,
/// See [WithLaunchContext::launch]
pub struct WithLaunchContext<Builder> {
builder: Builder,
task_executor: TaskExecutor,
data_dir: ChainPath<DataDirPath>,
}
impl<Builder> WithLaunchContext<Builder> {
/// Returns a reference to the task executor.
pub fn task_executor(&self) -> &TaskExecutor {
&self.task_executor
}
/// Returns a reference to the data directory.
pub fn data_dir(&self) -> &ChainPath<DataDirPath> {
&self.data_dir
}
}
impl<DB> WithLaunchContext<NodeBuilder<DB>>
where
DB: Database + DatabaseMetrics + DatabaseMetadata + Clone + Unpin + 'static,
{
/// Configures the types of the node.
pub fn with_types<T>(
self,
types: T,
) -> WithLaunchContext<NodeBuilderWithTypes<RethFullAdapter<DB, T>>>
where
T: NodeTypes,
{
WithLaunchContext {
builder: self.builder.with_types(types),
task_executor: self.task_executor,
data_dir: self.data_dir,
}
}
/// Preconfigures the node with a specific node implementation.
pub fn node<N>(
self,
node: N,
) -> WithLaunchContext<
NodeBuilderWithComponents<
RethFullAdapter<DB, N>,
ComponentsBuilder<
RethFullAdapter<DB, N>,
N::PoolBuilder,
N::PayloadBuilder,
N::NetworkBuilder,
>,
>,
>
where
N: Node<RethFullAdapter<DB, N>>,
N::PoolBuilder: PoolBuilder<RethFullAdapter<DB, N>>,
N::NetworkBuilder: crate::components::NetworkBuilder<
RethFullAdapter<DB, N>,
<N::PoolBuilder as PoolBuilder<RethFullAdapter<DB, N>>>::Pool,
>,
N::PayloadBuilder: crate::components::PayloadServiceBuilder<
RethFullAdapter<DB, N>,
<N::PoolBuilder as PoolBuilder<RethFullAdapter<DB, N>>>::Pool,
>,
{
self.with_types(node.clone()).with_components(node.components())
}
/// Launches a preconfigured [Node]
///
/// This bootstraps the node internals, creates all the components with the given [Node]
///
/// Returns a [NodeHandle] that can be used to interact with the node.
pub async fn launch_node<N>(
self,
node: N,
) -> eyre::Result<
NodeHandle<
NodeAdapter<
RethFullAdapter<DB, N>,
Components<
RethFullAdapter<DB, N>,
<N::PoolBuilder as PoolBuilder<RethFullAdapter<DB, N>>>::Pool,
>,
>,
>,
>
where
N: Node<RethFullAdapter<DB, N>>,
N::PoolBuilder: PoolBuilder<RethFullAdapter<DB, N>>,
N::NetworkBuilder: crate::components::NetworkBuilder<
RethFullAdapter<DB, N>,
<N::PoolBuilder as PoolBuilder<RethFullAdapter<DB, N>>>::Pool,
>,
N::PayloadBuilder: crate::components::PayloadServiceBuilder<
RethFullAdapter<DB, N>,
<N::PoolBuilder as PoolBuilder<RethFullAdapter<DB, N>>>::Pool,
>,
{
self.node(node).launch().await
}
}
impl<T, DB> WithLaunchContext<NodeBuilderWithTypes<RethFullAdapter<DB, T>>>
where
DB: Database + DatabaseMetrics + DatabaseMetadata + Clone + Unpin + 'static,
T: NodeTypes,
{
/// Advances the state of the node builder to the next state where all components are configured
pub fn with_components<CB>(
self,
components_builder: CB,
) -> WithLaunchContext<NodeBuilderWithComponents<RethFullAdapter<DB, T>, CB>>
where
CB: NodeComponentsBuilder<RethFullAdapter<DB, T>>,
{
WithLaunchContext {
builder: self.builder.with_components(components_builder),
task_executor: self.task_executor,
data_dir: self.data_dir,
}
}
}
impl<T, DB, CB> WithLaunchContext<NodeBuilderWithComponents<RethFullAdapter<DB, T>, CB>>
where
DB: Database + DatabaseMetrics + DatabaseMetadata + Clone + Unpin + 'static,
T: NodeTypes,
CB: NodeComponentsBuilder<RethFullAdapter<DB, T>>,
{
/// Sets the hook that is run once the node's components are initialized.
pub fn on_component_initialized<F>(self, hook: F) -> Self
where
F: Fn(NodeAdapter<RethFullAdapter<DB, T>, CB::Components>) -> eyre::Result<()>
+ Send
+ 'static,
{
Self {
builder: self.builder.on_component_initialized(hook),
task_executor: self.task_executor,
data_dir: self.data_dir,
}
}
/// Sets the hook that is run once the node has started.
pub fn on_node_started<F>(self, hook: F) -> Self
where
F: Fn(FullNode<NodeAdapter<RethFullAdapter<DB, T>, CB::Components>>) -> eyre::Result<()>
+ Send
+ 'static,
{
Self {
builder: self.builder.on_node_started(hook),
task_executor: self.task_executor,
data_dir: self.data_dir,
}
}
/// Sets the hook that is run once the rpc server is started.
pub fn on_rpc_started<F>(self, hook: F) -> Self
where
F: Fn(
RpcContext<'_, NodeAdapter<RethFullAdapter<DB, T>, CB::Components>>,
RethRpcServerHandles,
) -> eyre::Result<()>
+ Send
+ 'static,
{
Self {
builder: self.builder.on_rpc_started(hook),
task_executor: self.task_executor,
data_dir: self.data_dir,
}
}
/// Sets the hook that is run to configure the rpc modules.
pub fn extend_rpc_modules<F>(self, hook: F) -> Self
where
F: Fn(
RpcContext<'_, NodeAdapter<RethFullAdapter<DB, T>, CB::Components>>,
) -> eyre::Result<()>
+ Send
+ 'static,
{
Self {
builder: self.builder.extend_rpc_modules(hook),
task_executor: self.task_executor,
data_dir: self.data_dir,
}
}
/// Installs an ExEx (Execution Extension) in the node.
///
/// # Note
///
/// The ExEx ID must be unique.
pub fn install_exex<F, R, E>(self, exex_id: impl Into<String>, exex: F) -> Self
where
F: Fn(ExExContext<NodeAdapter<RethFullAdapter<DB, T>, CB::Components>>) -> R
+ Send
+ 'static,
R: Future<Output = eyre::Result<E>> + Send,
E: Future<Output = eyre::Result<()>> + Send,
{
Self {
builder: self.builder.install_exex(exex_id, exex),
task_executor: self.task_executor,
data_dir: self.data_dir,
}
}
/// Launches the node and returns a handle to it.
pub async fn launch(
self,
) -> eyre::Result<NodeHandle<NodeAdapter<RethFullAdapter<DB, T>, CB::Components>>> {
let Self { builder, task_executor, data_dir } = self;
let launcher = DefaultNodeLauncher { task_executor, data_dir };
builder.launch_with(launcher).await
}
/// Check that the builder can be launched
///
/// This is useful when writing tests to ensure that the builder is configured correctly.
pub fn check_launch(self) -> Self {
self
}
}
/// Captures the necessary context for building the components of the node.
pub struct BuilderContext<Node: FullNodeTypes> {
/// The current head of the blockchain at launch.
pub(crate) head: Head,
/// The configured provider to interact with the blockchain.
pub(crate) provider: Node::Provider,
/// The executor of the node.
pub(crate) executor: TaskExecutor,
/// The data dir of the node.
pub(crate) data_dir: ChainPath<DataDirPath>,
/// The config of the node
pub(crate) config: NodeConfig,
/// loaded config
pub(crate) reth_config: reth_config::Config,
/// EVM config of the node
pub(crate) evm_config: Node::Evm,
}
impl<Node: FullNodeTypes> BuilderContext<Node> {
/// Create a new instance of [BuilderContext]
pub fn new(
head: Head,
provider: Node::Provider,
executor: TaskExecutor,
data_dir: ChainPath<DataDirPath>,
config: NodeConfig,
reth_config: reth_config::Config,
evm_config: Node::Evm,
) -> Self {
Self { head, provider, executor, data_dir, config, reth_config, evm_config }
}
/// Returns the configured provider to interact with the blockchain.
pub fn provider(&self) -> &Node::Provider {
&self.provider
}
/// Returns the configured evm.
pub fn evm_config(&self) -> &Node::Evm {
&self.evm_config
}
/// Returns the current head of the blockchain at launch.
pub fn head(&self) -> Head {
self.head
}
/// Returns the config of the node.
pub fn config(&self) -> &NodeConfig {
&self.config
}
/// Returns the data dir of the node.
///
/// This gives access to all relevant files and directories of the node's datadir.
pub fn data_dir(&self) -> &ChainPath<DataDirPath> {
&self.data_dir
}
/// Returns the executor of the node.
///
/// This can be used to execute async tasks or functions during the setup.
pub fn task_executor(&self) -> &TaskExecutor {
&self.executor
}
/// Returns the chain spec of the node.
pub fn chain_spec(&self) -> Arc<ChainSpec> {
self.provider().chain_spec()
}
/// Returns the transaction pool config of the node.
pub fn pool_config(&self) -> PoolConfig {
self.config().txpool.pool_config()
}
/// Loads `MAINNET_KZG_TRUSTED_SETUP`.
pub fn kzg_settings(&self) -> eyre::Result<Arc<KzgSettings>> {
Ok(Arc::clone(&MAINNET_KZG_TRUSTED_SETUP))
}
/// Returns the config for payload building.
pub fn payload_builder_config(&self) -> impl PayloadBuilderConfig {
self.config.builder.clone()
}
/// Returns the default network config for the node.
pub fn network_config(&self) -> eyre::Result<NetworkConfig<Node::Provider>> {
self.config.network_config(
&self.reth_config,
self.provider.clone(),
self.executor.clone(),
self.head,
self.data_dir(),
)
}
/// Creates the [NetworkBuilder] for the node.
pub async fn network_builder(&self) -> eyre::Result<NetworkBuilder<Node::Provider, (), ()>> {
self.config
.build_network(
&self.reth_config,
self.provider.clone(),
self.executor.clone(),
self.head,
self.data_dir(),
)
.await
}
/// Convenience function to start the network.
///
/// Spawns the configured network and associated tasks and returns the [NetworkHandle] connected
/// to that network.
pub fn start_network<Pool>(
&self,
builder: NetworkBuilder<Node::Provider, (), ()>,
pool: Pool,
) -> NetworkHandle
where
Pool: TransactionPool + Unpin + 'static,
{
let (handle, network, txpool, eth) = builder
.transactions(pool, Default::default())
.request_handler(self.provider().clone())
.split_with_handle();
self.executor.spawn_critical("p2p txpool", txpool);
self.executor.spawn_critical("p2p eth request handler", eth);
let default_peers_path = self.data_dir().known_peers_path();
let known_peers_file = self.config.network.persistent_peers_file(default_peers_path);
self.executor.spawn_critical_with_graceful_shutdown_signal(
"p2p network task",
|shutdown| {
network.run_until_graceful_shutdown(shutdown, |network| {
write_peers_to_file(network, known_peers_file)
})
},
);
handle
}
}
impl<Node: FullNodeTypes> std::fmt::Debug for BuilderContext<Node> {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
f.debug_struct("BuilderContext")
.field("head", &self.head)
.field("provider", &std::any::type_name::<Node::Provider>())
.field("executor", &self.executor)
.field("data_dir", &self.data_dir)
.field("config", &self.config)
.finish()
}
}

View File

@ -0,0 +1,237 @@
//! Node builder states and helper traits.
//!
//! Keeps track of the current state of the node builder.
//!
//! The node builder process is essentially a state machine that transitions through various states
//! before the node can be launched.
use crate::{
components::{NodeComponents, NodeComponentsBuilder},
exex::BoxedLaunchExEx,
hooks::NodeHooks,
launch::LaunchNode,
rpc::{RethRpcServerHandles, RpcContext, RpcHooks},
FullNode,
};
use reth_exex::ExExContext;
use reth_network::NetworkHandle;
use reth_node_api::{FullNodeComponents, FullNodeTypes, NodeTypes};
use reth_node_core::node_config::NodeConfig;
use reth_payload_builder::PayloadBuilderHandle;
use reth_tasks::TaskExecutor;
use std::{fmt, future::Future};
/// A node builder that also has the configured types.
pub struct NodeBuilderWithTypes<T: FullNodeTypes> {
/// All settings for how the node should be configured.
config: NodeConfig,
/// The configured database for the node.
adapter: NodeTypesAdapter<T>,
}
impl<T: FullNodeTypes> NodeBuilderWithTypes<T> {
/// Creates a new instance of the node builder with the given configuration and types.
pub fn new(config: NodeConfig, types: T, database: T::DB) -> Self {
Self { config, adapter: NodeTypesAdapter::new(types, database) }
}
/// Advances the state of the node builder to the next state where all components are configured
pub fn with_components<CB>(self, components_builder: CB) -> NodeBuilderWithComponents<T, CB>
where
CB: NodeComponentsBuilder<T>,
{
let Self { config, adapter } = self;
NodeBuilderWithComponents {
config,
adapter,
components_builder,
add_ons: NodeAddOns {
hooks: NodeHooks::default(),
rpc: RpcHooks::new(),
exexs: Vec::new(),
},
}
}
}
/// Container for the node's types and the database the node uses.
pub(crate) struct NodeTypesAdapter<T: FullNodeTypes> {
/// The database type used by the node.
pub(crate) database: T::DB,
// TODO(mattsse): make this stateless
pub(crate) types: T,
}
impl<T: FullNodeTypes> NodeTypesAdapter<T> {
/// Create a new adapter from the given node types.
pub(crate) fn new(types: T, database: T::DB) -> Self {
Self { types, database }
}
}
impl<T: FullNodeTypes> fmt::Debug for NodeTypesAdapter<T> {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
f.debug_struct("NodeTypesAdapter").field("db", &"...").field("types", &"...").finish()
}
}
/// Container for the node's types and the components and other internals that can be used by addons
/// of the node.
pub struct NodeAdapter<T: FullNodeTypes, C: NodeComponents<T>> {
/// The components of the node.
pub components: C,
/// The task executor for the node.
pub task_executor: TaskExecutor,
/// The provider of the node.
pub provider: T::Provider,
/// EVM config
pub evm: T::Evm,
}
impl<T: FullNodeTypes, C: NodeComponents<T>> NodeTypes for NodeAdapter<T, C> {
type Primitives = T::Primitives;
type Engine = T::Engine;
type Evm = T::Evm;
fn evm_config(&self) -> Self::Evm {
self.evm.clone()
}
}
impl<T: FullNodeTypes, C: NodeComponents<T>> FullNodeTypes for NodeAdapter<T, C> {
type DB = T::DB;
type Provider = T::Provider;
}
impl<T: FullNodeTypes, C: NodeComponents<T>> FullNodeComponents for NodeAdapter<T, C> {
type Pool = C::Pool;
fn pool(&self) -> &Self::Pool {
self.components.pool()
}
fn provider(&self) -> &Self::Provider {
&self.provider
}
fn network(&self) -> &NetworkHandle {
self.components.network()
}
fn payload_builder(&self) -> &PayloadBuilderHandle<T::Engine> {
self.components.payload_builder()
}
fn task_executor(&self) -> &TaskExecutor {
&self.task_executor
}
}
impl<T: FullNodeTypes, C: NodeComponents<T>> Clone for NodeAdapter<T, C> {
fn clone(&self) -> Self {
Self {
components: self.components.clone(),
task_executor: self.task_executor.clone(),
provider: self.provider.clone(),
evm: self.evm.clone(),
}
}
}
/// A fully type configured node builder.
///
/// Supports adding additional addons to the node.
pub struct NodeBuilderWithComponents<T: FullNodeTypes, CB: NodeComponentsBuilder<T>> {
/// All settings for how the node should be configured.
pub(crate) config: NodeConfig,
/// Adapter for the underlying node types and database
pub(crate) adapter: NodeTypesAdapter<T>,
/// container for type specific components
pub(crate) components_builder: CB,
/// Additional node extensions.
pub(crate) add_ons: NodeAddOns<NodeAdapter<T, CB::Components>>,
}
impl<T: FullNodeTypes, CB: NodeComponentsBuilder<T>> NodeBuilderWithComponents<T, CB> {
/// Sets the hook that is run once the node's components are initialized.
pub fn on_component_initialized<F>(mut self, hook: F) -> Self
where
F: Fn(NodeAdapter<T, CB::Components>) -> eyre::Result<()> + Send + 'static,
{
self.add_ons.hooks.set_on_component_initialized(hook);
self
}
/// Sets the hook that is run once the node has started.
pub fn on_node_started<F>(mut self, hook: F) -> Self
where
F: Fn(FullNode<NodeAdapter<T, CB::Components>>) -> eyre::Result<()> + Send + 'static,
{
self.add_ons.hooks.set_on_node_started(hook);
self
}
/// Sets the hook that is run once the rpc server is started.
pub fn on_rpc_started<F>(mut self, hook: F) -> Self
where
F: Fn(
RpcContext<'_, NodeAdapter<T, CB::Components>>,
RethRpcServerHandles,
) -> eyre::Result<()>
+ Send
+ 'static,
{
self.add_ons.rpc.set_on_rpc_started(hook);
self
}
/// Sets the hook that is run to configure the rpc modules.
pub fn extend_rpc_modules<F>(mut self, hook: F) -> Self
where
F: Fn(RpcContext<'_, NodeAdapter<T, CB::Components>>) -> eyre::Result<()> + Send + 'static,
{
self.add_ons.rpc.set_extend_rpc_modules(hook);
self
}
/// Installs an ExEx (Execution Extension) in the node.
///
/// # Note
///
/// The ExEx ID must be unique.
pub fn install_exex<F, R, E>(mut self, exex_id: impl Into<String>, exex: F) -> Self
where
F: Fn(ExExContext<NodeAdapter<T, CB::Components>>) -> R + Send + 'static,
R: Future<Output = eyre::Result<E>> + Send,
E: Future<Output = eyre::Result<()>> + Send,
{
self.add_ons.exexs.push((exex_id.into(), Box::new(exex)));
self
}
/// Launches the node with the given launcher.
pub async fn launch_with<L>(self, launcher: L) -> eyre::Result<L::Node>
where
L: LaunchNode<Self>,
{
launcher.launch_node(self).await
}
/// Check that the builder can be launched
///
/// This is useful when writing tests to ensure that the builder is configured correctly.
pub fn check_launch(self) -> Self {
self
}
}
/// Additional node extensions.
pub(crate) struct NodeAddOns<Node: FullNodeComponents> {
/// Additional NodeHooks that are called at specific points in the node's launch lifecycle.
pub(crate) hooks: NodeHooks<Node>,
/// Additional RPC hooks.
pub(crate) rpc: RpcHooks<Node>,
/// The ExExs (execution extensions) of the node.
pub(crate) exexs: Vec<(String, Box<dyn BoxedLaunchExEx<Node>>)>,
}

View File

@ -1,11 +1,11 @@
//! A generic [NodeComponentsBuilder]
use crate::{
components::{NetworkBuilder, NodeComponents, PayloadServiceBuilder, PoolBuilder},
components::{Components, NetworkBuilder, NodeComponents, PayloadServiceBuilder, PoolBuilder},
BuilderContext, FullNodeTypes,
};
use reth_transaction_pool::TransactionPool;
use std::marker::PhantomData;
use std::{future::Future, marker::PhantomData};
/// A generic, customizable [`NodeComponentsBuilder`].
///
@ -135,19 +135,19 @@ where
NetworkB: NetworkBuilder<Node, PoolB::Pool>,
PayloadB: PayloadServiceBuilder<Node, PoolB::Pool>,
{
type Pool = PoolB::Pool;
type Components = Components<Node, PoolB::Pool>;
async fn build_components(
self,
context: &BuilderContext<Node>,
) -> eyre::Result<NodeComponents<Node, Self::Pool>> {
) -> eyre::Result<Self::Components> {
let Self { pool_builder, payload_builder, network_builder, _marker } = self;
let pool = pool_builder.build_pool(context).await?;
let network = network_builder.build_network(context, pool.clone()).await?;
let payload_builder = payload_builder.spawn_payload_service(context, pool.clone()).await?;
Ok(NodeComponents { transaction_pool: pool, network, payload_builder })
Ok(Components { transaction_pool: pool, network, payload_builder })
}
}
@ -170,31 +170,31 @@ impl Default for ComponentsBuilder<(), (), (), ()> {
/// The [ComponentsBuilder] is a generic implementation of this trait that can be used to customize
/// certain components of the node using the builder pattern and defaults, e.g. Ethereum and
/// Optimism.
pub trait NodeComponentsBuilder<Node: FullNodeTypes> {
/// The transaction pool to use.
type Pool: TransactionPool + Unpin + 'static;
/// A type that's responsible for building the components of the node.
pub trait NodeComponentsBuilder<Node: FullNodeTypes>: Send {
/// The components for the node with the given types
type Components: NodeComponents<Node>;
/// Builds the components of the node.
/// Consumes the type and returns the crated components.
fn build_components(
self,
context: &BuilderContext<Node>,
) -> impl std::future::Future<Output = eyre::Result<NodeComponents<Node, Self::Pool>>> + Send;
ctx: &BuilderContext<Node>,
) -> impl Future<Output = eyre::Result<Self::Components>> + Send;
}
impl<Node, F, Fut, Pool> NodeComponentsBuilder<Node> for F
where
Node: FullNodeTypes,
F: FnOnce(&BuilderContext<Node>) -> Fut + Send,
Fut: std::future::Future<Output = eyre::Result<NodeComponents<Node, Pool>>> + Send,
Fut: Future<Output = eyre::Result<Components<Node, Pool>>> + Send,
Pool: TransactionPool + Unpin + 'static,
{
type Pool = Pool;
type Components = Components<Node, Pool>;
fn build_components(
self,
ctx: &BuilderContext<Node>,
) -> impl std::future::Future<Output = eyre::Result<NodeComponents<Node, Self::Pool>>> + Send
{
) -> impl Future<Output = eyre::Result<Self::Components>> + Send {
self(ctx)
}
}

View File

@ -14,17 +14,36 @@ pub use payload::*;
pub use pool::*;
use reth_network::NetworkHandle;
use reth_payload_builder::PayloadBuilderHandle;
use reth_transaction_pool::TransactionPool;
mod builder;
mod network;
mod payload;
mod pool;
/// An abstraction over the components of a node, consisting of:
/// - transaction pool
/// - network
/// - payload builder.
pub trait NodeComponents<NodeTypes: FullNodeTypes>: Clone + Send + Sync + 'static {
/// The transaction pool of the node.
type Pool: TransactionPool + Unpin;
/// Returns the transaction pool of the node.
fn pool(&self) -> &Self::Pool;
/// Returns the handle to the network
fn network(&self) -> &NetworkHandle;
/// Returns the handle to the payload builder service.
fn payload_builder(&self) -> &PayloadBuilderHandle<NodeTypes::Engine>;
}
/// All the components of the node.
///
/// This provides access to all the components of the node.
#[derive(Debug)]
pub struct NodeComponents<Node: FullNodeTypes, Pool> {
pub struct Components<Node: FullNodeTypes, Pool> {
/// The transaction pool of the node.
pub transaction_pool: Pool,
/// The network implementation of the node.
@ -33,9 +52,36 @@ pub struct NodeComponents<Node: FullNodeTypes, Pool> {
pub payload_builder: PayloadBuilderHandle<Node::Engine>,
}
impl<Node: FullNodeTypes, Pool> NodeComponents<Node, Pool> {
/// Returns the handle to the payload builder service.
pub fn payload_builder(&self) -> PayloadBuilderHandle<Node::Engine> {
self.payload_builder.clone()
impl<Node, Pool> NodeComponents<Node> for Components<Node, Pool>
where
Node: FullNodeTypes,
Pool: TransactionPool + Unpin + 'static,
{
type Pool = Pool;
fn pool(&self) -> &Self::Pool {
&self.transaction_pool
}
fn network(&self) -> &NetworkHandle {
&self.network
}
fn payload_builder(&self) -> &PayloadBuilderHandle<Node::Engine> {
&self.payload_builder
}
}
impl<Node, Pool> Clone for Components<Node, Pool>
where
Node: FullNodeTypes,
Pool: TransactionPool,
{
fn clone(&self) -> Self {
Self {
transaction_pool: self.transaction_pool.clone(),
network: self.network.clone(),
payload_builder: self.payload_builder.clone(),
}
}
}

View File

@ -0,0 +1,558 @@
//! Abstraction for launching a node.
use crate::{
builder::{NodeAdapter, NodeAddOns, NodeTypesAdapter},
components::{NodeComponents, NodeComponentsBuilder},
hooks::NodeHooks,
node::FullNode,
BuilderContext, NodeBuilderWithComponents, NodeHandle, RethFullAdapter,
};
use eyre::Context;
use futures::{future, future::Either, stream, stream_select, StreamExt};
use rayon::ThreadPoolBuilder;
use reth_auto_seal_consensus::{AutoSealConsensus, MiningMode};
use reth_beacon_consensus::{
hooks::{EngineHooks, PruneHook, StaticFileHook},
BeaconConsensus, BeaconConsensusEngine,
};
use reth_blockchain_tree::{
BlockchainTree, BlockchainTreeConfig, ShareableBlockchainTree, TreeExternals,
};
use reth_config::config::EtlConfig;
use reth_consensus::Consensus;
use reth_db::{
database::Database,
database_metrics::{DatabaseMetadata, DatabaseMetrics},
};
use reth_exex::{ExExContext, ExExHandle, ExExManager, ExExManagerHandle};
use reth_interfaces::p2p::either::EitherDownloader;
use reth_network::NetworkEvents;
use reth_node_api::{FullNodeComponents, NodeTypes};
use reth_node_core::{
cli::config::RethRpcConfig,
dirs::{ChainPath, DataDirPath},
engine_api_store::EngineApiStore,
engine_skip_fcu::EngineApiSkipFcu,
exit::NodeExitFuture,
init::init_genesis,
node_config::NodeConfig,
};
use reth_node_events::{cl::ConsensusLayerHealthEvents, node};
use reth_primitives::format_ether;
use reth_provider::{providers::BlockchainProvider, CanonStateSubscriptions, ProviderFactory};
use reth_prune::PrunerBuilder;
use reth_revm::EvmProcessorFactory;
use reth_rpc_engine_api::EngineApi;
use reth_static_file::StaticFileProducer;
use reth_tasks::TaskExecutor;
use reth_tracing::tracing::{debug, error, info};
use reth_transaction_pool::TransactionPool;
use std::{cmp::max, future::Future, sync::Arc, thread::available_parallelism};
use tokio::sync::{mpsc::unbounded_channel, oneshot};
/// Launches a new node.
///
/// Acts as a node factory.
///
/// This is essentially the launch logic for a node.
pub trait LaunchNode<Target> {
/// The node type that is created.
type Node;
/// Create and return a new node asynchronously.
fn launch_node(self, target: Target) -> impl Future<Output = eyre::Result<Self::Node>> + Send;
}
/// The default launcher for a node.
#[derive(Debug)]
pub struct DefaultNodeLauncher {
/// The task executor for the node.
pub task_executor: TaskExecutor,
/// The data directory for the node.
pub data_dir: ChainPath<DataDirPath>,
}
impl DefaultNodeLauncher {
/// Create a new instance of the default node launcher.
pub fn new(task_executor: TaskExecutor, data_dir: ChainPath<DataDirPath>) -> Self {
Self { task_executor, data_dir }
}
/// Loads the reth config with the given datadir root
fn load_toml_config(&self, config: &NodeConfig) -> eyre::Result<reth_config::Config> {
let config_path = config.config.clone().unwrap_or_else(|| self.data_dir.config_path());
let mut toml_config = confy::load_path::<reth_config::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
toml_config.peers.trusted_nodes_only = config.network.trusted_only;
if !config.network.trusted_peers.is_empty() {
info!(target: "reth::cli", "Adding trusted nodes");
config.network.trusted_peers.iter().for_each(|peer| {
toml_config.peers.trusted_nodes.insert(*peer);
});
}
Ok(toml_config)
}
}
impl<T, DB, CB> LaunchNode<NodeBuilderWithComponents<RethFullAdapter<DB, T>, CB>>
for DefaultNodeLauncher
where
DB: Database + DatabaseMetrics + DatabaseMetadata + Clone + Unpin + 'static,
T: NodeTypes,
CB: NodeComponentsBuilder<RethFullAdapter<DB, T>>,
{
type Node = NodeHandle<NodeAdapter<RethFullAdapter<DB, T>, CB::Components>>;
async fn launch_node(
self,
target: NodeBuilderWithComponents<RethFullAdapter<DB, T>, CB>,
) -> eyre::Result<Self::Node> {
let NodeBuilderWithComponents {
adapter: NodeTypesAdapter { types, database },
components_builder,
add_ons: NodeAddOns { hooks, rpc, exexs: installed_exex },
config,
} = target;
// get config from file
let reth_config = self.load_toml_config(&config)?;
let Self { task_executor, data_dir } = self;
// Raise the fd limit of the process.
// Does not do anything on windows.
fdlimit::raise_fd_limit()?;
// Limit the global rayon thread pool, reserving 2 cores for the rest of the system
let _ = ThreadPoolBuilder::new()
.num_threads(
available_parallelism().map_or(25, |cpus| max(cpus.get().saturating_sub(2), 2)),
)
.build_global()
.map_err(|e| error!("Failed to build global thread pool: {:?}", e));
let provider_factory = ProviderFactory::new(
database.clone(),
Arc::clone(&config.chain),
data_dir.static_files_path(),
)?
.with_static_files_metrics();
info!(target: "reth::cli", "Database opened");
let prometheus_handle = config.install_prometheus_recorder()?;
config
.start_metrics_endpoint(
prometheus_handle,
database.clone(),
provider_factory.static_file_provider(),
task_executor.clone(),
)
.await?;
debug!(target: "reth::cli", chain=%config.chain.chain,
genesis=?config.chain.genesis_hash(), "Initializing genesis");
let genesis_hash = init_genesis(provider_factory.clone())?;
info!(target: "reth::cli", "\n{}", config.chain.display_hardforks());
// setup the consensus instance
let consensus: Arc<dyn Consensus> = if config.dev.dev {
Arc::new(AutoSealConsensus::new(Arc::clone(&config.chain)))
} else {
Arc::new(BeaconConsensus::new(Arc::clone(&config.chain)))
};
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);
task_executor.spawn_critical("stages metrics listener task", sync_metrics_listener);
let prune_config = config.prune_config()?.or_else(|| reth_config.prune.clone());
// Configure the blockchain tree for the node
let evm_config = types.evm_config();
let tree_config = BlockchainTreeConfig::default();
let tree_externals = TreeExternals::new(
provider_factory.clone(),
consensus.clone(),
EvmProcessorFactory::new(config.chain.clone(), evm_config.clone()),
);
let tree = BlockchainTree::new(
tree_externals,
tree_config,
prune_config.as_ref().map(|config| config.segments.clone()),
)?
.with_sync_metrics_tx(sync_metrics_tx.clone());
let canon_state_notification_sender = tree.canon_state_notification_sender();
let blockchain_tree = Arc::new(ShareableBlockchainTree::new(tree));
debug!(target: "reth::cli", "configured blockchain tree");
// fetch the head block from the database
let head =
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())?;
let ctx = BuilderContext::new(
head,
blockchain_db,
task_executor,
data_dir,
config,
reth_config,
evm_config.clone(),
);
debug!(target: "reth::cli", "creating components");
let components = components_builder.build_components(&ctx).await?;
let BuilderContext {
provider: blockchain_db,
executor,
data_dir,
mut config,
mut reth_config,
..
} = ctx;
let NodeHooks { on_component_initialized, on_node_started, .. } = hooks;
let node_adapter = NodeAdapter {
components,
task_executor: executor.clone(),
provider: blockchain_db.clone(),
evm: evm_config.clone(),
};
debug!(target: "reth::cli", "calling on_component_initialized hook");
on_component_initialized.on_event(node_adapter.clone())?;
// spawn exexs
let mut exex_handles = Vec::with_capacity(installed_exex.len());
let mut exexs = Vec::with_capacity(installed_exex.len());
for (id, exex) in installed_exex {
// create a new exex handle
let (handle, events, notifications) = ExExHandle::new(id.clone());
exex_handles.push(handle);
// create the launch context for the exex
let context = ExExContext {
head,
provider: blockchain_db.clone(),
task_executor: executor.clone(),
data_dir: data_dir.clone(),
config: config.clone(),
reth_config: reth_config.clone(),
pool: node_adapter.components.pool().clone(),
events,
notifications,
};
let executor = executor.clone();
exexs.push(async move {
debug!(target: "reth::cli", id, "spawning exex");
let span = reth_tracing::tracing::info_span!("exex", id);
let _enter = span.enter();
// init the exex
let exex = exex.launch(context).await.unwrap();
// spawn it as a crit task
executor.spawn_critical("exex", async move {
info!(target: "reth::cli", "ExEx started");
match exex.await {
Ok(_) => panic!("ExEx {id} finished. ExEx's should run indefinitely"),
Err(err) => panic!("ExEx {id} crashed: {err}"),
}
});
});
}
future::join_all(exexs).await;
// spawn exex manager
let exex_manager_handle = if !exex_handles.is_empty() {
debug!(target: "reth::cli", "spawning exex manager");
// todo(onbjerg): rm magic number
let exex_manager = ExExManager::new(exex_handles, 1024);
let exex_manager_handle = exex_manager.handle();
executor.spawn_critical("exex manager", async move {
exex_manager.await.expect("exex manager crashed");
});
// send notifications from the blockchain tree to exex manager
let mut canon_state_notifications = blockchain_tree.subscribe_to_canonical_state();
let mut handle = exex_manager_handle.clone();
executor.spawn_critical("exex manager blockchain tree notifications", async move {
while let Ok(notification) = canon_state_notifications.recv().await {
handle.send_async(notification.into()).await.expect(
"blockchain tree notification could not be sent to exex
manager",
);
}
});
info!(target: "reth::cli", "ExEx Manager started");
Some(exex_manager_handle)
} else {
None
};
// create pipeline
let network_client = node_adapter.network().fetch_client().await?;
let (consensus_engine_tx, mut consensus_engine_rx) = unbounded_channel();
if let Some(skip_fcu_threshold) = config.debug.skip_fcu {
debug!(target: "reth::cli", "spawning skip FCU task");
let (skip_fcu_tx, skip_fcu_rx) = unbounded_channel();
let engine_skip_fcu = EngineApiSkipFcu::new(skip_fcu_threshold);
executor.spawn_critical(
"skip FCU interceptor",
engine_skip_fcu.intercept(consensus_engine_rx, skip_fcu_tx),
);
consensus_engine_rx = skip_fcu_rx;
}
if let Some(store_path) = config.debug.engine_api_store.clone() {
debug!(target: "reth::cli", "spawning engine API store");
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 = config.max_block(network_client.clone(), provider_factory.clone()).await?;
let mut hooks = EngineHooks::new();
let 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.lock().events();
hooks.add(StaticFileHook::new(static_file_producer.clone(), Box::new(executor.clone())));
info!(target: "reth::cli", "StaticFileProducer initialized");
// Make sure ETL doesn't default to /tmp/, but to whatever datadir is set to
if reth_config.stages.etl.dir.is_none() {
reth_config.stages.etl.dir = Some(EtlConfig::from_datadir(&data_dir.data_dir_path()));
}
// Configure the pipeline
let pipeline_exex_handle =
exex_manager_handle.clone().unwrap_or_else(ExExManagerHandle::empty);
let (mut pipeline, client) = if config.dev.dev {
info!(target: "reth::cli", "Starting Reth in dev mode");
for (idx, (address, alloc)) in config.chain.genesis.alloc.iter().enumerate() {
info!(target: "reth::cli", "Allocated Genesis Account: {:02}. {} ({} ETH)", idx,
address.to_string(), format_ether(alloc.balance));
}
// install auto-seal
let pending_transactions_listener =
node_adapter.components.pool().pending_transactions_listener();
let mining_mode = if let Some(interval) = config.dev.block_time {
MiningMode::interval(interval)
} else if let Some(max_transactions) = config.dev.block_max_transactions {
MiningMode::instant(max_transactions, pending_transactions_listener)
} else {
info!(target: "reth::cli", "No mining mode specified, defaulting to
ReadyTransaction");
MiningMode::instant(1, pending_transactions_listener)
};
let (_, client, mut task) = reth_auto_seal_consensus::AutoSealBuilder::new(
Arc::clone(&config.chain),
blockchain_db.clone(),
node_adapter.components.pool().clone(),
consensus_engine_tx.clone(),
canon_state_notification_sender,
mining_mode,
evm_config.clone(),
)
.build();
let mut pipeline = crate::setup::build_networked_pipeline(
&config,
&reth_config.stages,
client.clone(),
Arc::clone(&consensus),
provider_factory.clone(),
&executor,
sync_metrics_tx,
prune_config.clone(),
max_block,
static_file_producer,
evm_config,
pipeline_exex_handle,
)
.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 = crate::setup::build_networked_pipeline(
&config,
&reth_config.stages,
network_client.clone(),
Arc::clone(&consensus),
provider_factory.clone(),
&executor,
sync_metrics_tx,
prune_config.clone(),
max_block,
static_file_producer,
evm_config,
pipeline_exex_handle,
)
.await?;
(pipeline, EitherDownloader::Right(network_client.clone()))
};
let pipeline_events = pipeline.events();
let initial_target = config.initial_pipeline_target(genesis_hash);
let prune_config = prune_config.unwrap_or_default();
let mut pruner_builder = PrunerBuilder::new(prune_config.clone())
.max_reorg_depth(tree_config.max_reorg_depth() as usize)
.prune_delete_limit(config.chain.prune_delete_limit)
.timeout(PrunerBuilder::DEFAULT_TIMEOUT);
if let Some(exex_manager_handle) = &exex_manager_handle {
pruner_builder =
pruner_builder.finished_exex_height(exex_manager_handle.finished_height());
}
let mut pruner = pruner_builder.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(node_adapter.components.network().clone()),
max_block,
config.debug.continuous,
node_adapter.components.payload_builder().clone(),
initial_target,
reth_beacon_consensus::MIN_BLOCKS_FOR_PIPELINE_RUN,
consensus_engine_tx,
consensus_engine_rx,
hooks,
)?;
info!(target: "reth::cli", "Consensus engine initialized");
let events = stream_select!(
node_adapter.components.network().event_listener().map(Into::into),
beacon_engine_handle.event_listener().map(Into::into),
pipeline_events.map(Into::into),
if config.debug.tip.is_none() && !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",
node::handle_events(
Some(node_adapter.components.network().clone()),
Some(head.number),
events,
database.clone(),
),
);
let engine_api = EngineApi::new(
blockchain_db.clone(),
config.chain.clone(),
beacon_engine_handle,
node_adapter.components.payload_builder().clone().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 = data_dir.jwt_path();
let jwt_secret = config.rpc.auth_jwt_secret(default_jwt_path)?;
// adjust rpc port numbers based on instance number
config.adjust_instance_ports();
// Start RPC servers
let (rpc_server_handles, mut rpc_registry) = crate::rpc::launch_rpc_servers(
node_adapter.clone(),
engine_api,
&config,
jwt_secret,
rpc,
)
.await?;
// in dev mode we generate 20 random dev-signer accounts
if config.dev.dev {
rpc_registry.eth_api().with_dev_accounts();
}
// 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);
});
let full_node = FullNode {
evm_config: node_adapter.evm.clone(),
pool: node_adapter.components.pool().clone(),
network: node_adapter.components.network().clone(),
provider: node_adapter.provider.clone(),
payload_builder: node_adapter.components.payload_builder().clone(),
task_executor: executor,
rpc_server_handles,
rpc_registry,
config,
data_dir,
};
// Notify on node started
on_node_started.on_event(full_node.clone())?;
let handle = NodeHandle {
node_exit_future: NodeExitFuture::new(rx, full_node.config.debug.terminate),
node: full_node,
};
Ok(handle)
}
}

View File

@ -21,6 +21,9 @@ pub mod components;
mod builder;
pub use builder::*;
mod launch;
pub use launch::*;
mod handle;
pub use handle::NodeHandle;

View File

@ -270,7 +270,7 @@ impl NodeConfig {
/// necessary
pub async fn max_block<Provider, Client>(
&self,
network_client: &Client,
network_client: Client,
provider: Provider,
) -> eyre::Result<Option<BlockNumber>>
where

View File

@ -97,87 +97,3 @@ pub trait FullNodeComponents: FullNodeTypes + 'static {
/// Returns the task executor.
fn task_executor(&self) -> &TaskExecutor;
}
/// A type that encapsulates all the components of the node.
#[derive(Debug)]
pub struct FullNodeComponentsAdapter<Node: FullNodeTypes, Pool> {
/// The EVM configuration of the node.
pub evm_config: Node::Evm,
/// The transaction pool of the node.
pub pool: Pool,
/// The network handle of the node.
pub network: NetworkHandle,
/// The provider of the node.
pub provider: Node::Provider,
/// The payload builder service handle of the node.
pub payload_builder: PayloadBuilderHandle<Node::Engine>,
/// The task executor of the node.
pub executor: TaskExecutor,
}
impl<Node, Pool> FullNodeTypes for FullNodeComponentsAdapter<Node, Pool>
where
Node: FullNodeTypes,
Pool: TransactionPool + 'static,
{
type DB = Node::DB;
type Provider = Node::Provider;
}
impl<Node, Pool> NodeTypes for FullNodeComponentsAdapter<Node, Pool>
where
Node: FullNodeTypes,
Pool: TransactionPool + 'static,
{
type Primitives = Node::Primitives;
type Engine = Node::Engine;
type Evm = Node::Evm;
fn evm_config(&self) -> Self::Evm {
self.evm_config.clone()
}
}
impl<Node, Pool> FullNodeComponents for FullNodeComponentsAdapter<Node, Pool>
where
Node: FullNodeTypes,
Pool: TransactionPool + 'static,
{
type Pool = Pool;
fn pool(&self) -> &Self::Pool {
&self.pool
}
fn provider(&self) -> &Self::Provider {
&self.provider
}
fn network(&self) -> &NetworkHandle {
&self.network
}
fn payload_builder(&self) -> &PayloadBuilderHandle<Self::Engine> {
&self.payload_builder
}
fn task_executor(&self) -> &TaskExecutor {
&self.executor
}
}
impl<Node: FullNodeTypes, Pool> Clone for FullNodeComponentsAdapter<Node, Pool>
where
Pool: Clone,
{
fn clone(&self) -> Self {
Self {
evm_config: self.evm_config.clone(),
pool: self.pool.clone(),
network: self.network.clone(),
provider: self.provider.clone(),
payload_builder: self.payload_builder.clone(),
executor: self.executor.clone(),
}
}
}