feat: extend RethCliExt with payload builder (#4022)

This commit is contained in:
Matthias Seitz
2023-08-04 14:39:07 +02:00
committed by GitHub
parent 3f63a0887a
commit 82a2a6f416
7 changed files with 158 additions and 89 deletions

View File

@ -1,11 +1,13 @@
use crate::{args::utils::parse_duration_from_secs, version::default_extradata};
use crate::{
args::utils::parse_duration_from_secs, cli::config::PayloadBuilderConfig,
version::default_extradata,
};
use clap::{
builder::{RangedU64ValueParser, TypedValueParser},
Arg, Args, Command,
};
use reth_primitives::{bytes::BytesMut, constants::MAXIMUM_EXTRA_DATA_SIZE};
use reth_rlp::Encodable;
use std::{ffi::OsStr, time::Duration};
use reth_primitives::constants::MAXIMUM_EXTRA_DATA_SIZE;
use std::{borrow::Cow, ffi::OsStr, time::Duration};
/// Parameters for configuring the Payload Builder
#[derive(Debug, Args, PartialEq, Default)]
@ -36,12 +38,25 @@ pub struct PayloadBuilderArgs {
pub max_payload_tasks: usize,
}
impl PayloadBuilderArgs {
/// Returns the rlp-encoded extradata bytes.
pub fn extradata_bytes(&self) -> reth_primitives::bytes::Bytes {
let mut extradata = BytesMut::new();
self.extradata.as_bytes().encode(&mut extradata);
extradata.freeze()
impl PayloadBuilderConfig for PayloadBuilderArgs {
fn extradata(&self) -> Cow<'_, str> {
self.extradata.as_str().into()
}
fn interval(&self) -> Duration {
self.interval
}
fn deadline(&self) -> Duration {
self.deadline
}
fn max_gas_limit(&self) -> u64 {
self.max_gas_limit
}
fn max_payload_tasks(&self) -> usize {
self.max_payload_tasks
}
}

View File

@ -2,7 +2,7 @@
use crate::{
args::GasPriceOracleArgs,
cli::ext::{NoopArgsExt, RethRpcConfig, RethRpcServerArgsExt},
cli::{config::RethRpcConfig, ext::RethNodeCommandExt},
};
use clap::{
builder::{PossibleValue, RangedU64ValueParser, TypedValueParser},
@ -57,7 +57,7 @@ pub(crate) const RPC_DEFAULT_MAX_TRACING_REQUESTS: u32 = 25;
/// Parameters for configuring the rpc more granularity via CLI
#[derive(Debug, Args)]
#[command(next_help_heading = "RPC")]
pub struct RpcServerArgs<Ext: RethRpcServerArgsExt = NoopArgsExt> {
pub struct RpcServerArgs {
/// Enable the HTTP-RPC server
#[arg(long, default_value_if("dev", "true", "true"))]
pub http: bool,
@ -163,13 +163,9 @@ pub struct RpcServerArgs<Ext: RethRpcServerArgsExt = NoopArgsExt> {
/// Maximum number of env cache entries.
#[arg(long, default_value_t = DEFAULT_ENV_CACHE_MAX_LEN)]
pub env_cache_len: u32,
/// Additional arguments for rpc.
#[clap(flatten)]
pub ext: Ext,
}
impl<Ext: RethRpcServerArgsExt> RpcServerArgs<Ext> {
impl RpcServerArgs {
/// Returns the max request size in bytes.
pub fn rpc_max_request_size_bytes(&self) -> u32 {
self.rpc_max_request_size * 1024 * 1024
@ -227,7 +223,7 @@ impl<Ext: RethRpcServerArgsExt> RpcServerArgs<Ext> {
/// for the auth server that handles the `engine_` API that's accessed by the consensus
/// layer.
#[allow(clippy::too_many_arguments)]
pub async fn start_servers<Provider, Pool, Network, Tasks, Events, Engine>(
pub async fn start_servers<Provider, Pool, Network, Tasks, Events, Engine, Ext>(
&self,
provider: Provider,
pool: Pool,
@ -236,6 +232,7 @@ impl<Ext: RethRpcServerArgsExt> RpcServerArgs<Ext> {
events: Events,
engine_api: Engine,
jwt_secret: JwtSecret,
ext: &Ext,
) -> eyre::Result<(RpcServerHandle, AuthServerHandle)>
where
Provider: BlockReaderIdExt
@ -252,6 +249,7 @@ impl<Ext: RethRpcServerArgsExt> RpcServerArgs<Ext> {
Tasks: TaskSpawner + Clone + 'static,
Events: CanonStateSubscriptions + Clone + 'static,
Engine: EngineApiServer,
Ext: RethNodeCommandExt,
{
let auth_config = self.auth_server_config(jwt_secret)?;
@ -267,7 +265,7 @@ impl<Ext: RethRpcServerArgsExt> RpcServerArgs<Ext> {
.build_with_auth_server(module_config, engine_api);
// apply configured customization
self.ext.extend_rpc_modules(self, &mut registry, &mut rpc_modules)?;
ext.extend_rpc_modules(self, &mut registry, &mut rpc_modules)?;
let server_config = self.rpc_server_config();
let launch_rpc = rpc_modules.start_server(server_config).map_ok(|handle| {
@ -448,7 +446,7 @@ impl<Ext: RethRpcServerArgsExt> RpcServerArgs<Ext> {
}
}
impl<Ext: RethRpcServerArgsExt> RethRpcConfig for RpcServerArgs<Ext> {
impl RethRpcConfig for RpcServerArgs {
fn is_ipc_enabled(&self) -> bool {
// By default IPC is enabled therefor it is enabled if the `ipcdisable` is false.
!self.ipcdisable

View File

@ -0,0 +1,48 @@
//! Config traits for various node components.
use reth_revm::primitives::bytes::BytesMut;
use reth_rlp::Encodable;
use reth_rpc_builder::EthConfig;
use std::{borrow::Cow, time::Duration};
/// A trait that provides configured RPC server.
///
/// This provides all basic config values for the RPC server and is implemented by the
/// [RpcServerArgs](crate::args::RpcServerArgs) type.
pub trait RethRpcConfig {
/// Returns whether ipc is enabled.
fn is_ipc_enabled(&self) -> bool;
/// The configured ethereum RPC settings.
fn eth_config(&self) -> EthConfig;
// TODO extract more functions from RpcServerArgs
}
/// A trait that provides payload builder settings.
///
/// This provides all basic payload builder settings and is implemented by the
/// [PayloadBuilderArgs](crate::args::PayloadBuilderArgs) type.
pub trait PayloadBuilderConfig {
/// Block extra data set by the payload builder.
fn extradata(&self) -> Cow<'_, str>;
/// Returns the rlp-encoded extradata bytes.
fn extradata_rlp_bytes(&self) -> reth_primitives::bytes::Bytes {
let mut extradata = BytesMut::new();
self.extradata().as_bytes().encode(&mut extradata);
extradata.freeze()
}
/// The interval at which the job should build a new payload after the last.
fn interval(&self) -> Duration;
/// The deadline for when the payload builder job should resolve.
fn deadline(&self) -> Duration;
/// Target gas ceiling for built blocks.
fn max_gas_limit(&self) -> u64;
/// Maximum number of tasks to spawn for building a payload.
fn max_payload_tasks(&self) -> usize;
}

View File

@ -1,73 +1,36 @@
//! Support for integrating customizations into the CLI.
use crate::cli::config::{PayloadBuilderConfig, RethRpcConfig};
use clap::Args;
use reth_basic_payload_builder::{BasicPayloadJobGenerator, BasicPayloadJobGeneratorConfig};
use reth_network_api::{NetworkInfo, Peers};
use reth_payload_builder::{PayloadBuilderHandle, PayloadBuilderService};
use reth_primitives::ChainSpec;
use reth_provider::{
BlockReaderIdExt, CanonStateSubscriptions, ChainSpecProvider, ChangeSetReader, EvmEnvProvider,
StateProviderFactory,
};
use reth_rpc_builder::{EthConfig, RethModuleRegistry, TransportRpcModules};
use reth_rpc_builder::{RethModuleRegistry, TransportRpcModules};
use reth_tasks::TaskSpawner;
use reth_transaction_pool::TransactionPool;
use std::fmt;
use std::{fmt, sync::Arc};
/// A trait that allows for extending parts of the CLI with additional functionality.
pub trait RethCliExt {
/// Extends the rpc arguments for the node
type RpcExt: RethRpcServerArgsExt;
/// Provides additional configuration for the node command.
type Node: RethNodeCommandExt;
}
impl RethCliExt for () {
type RpcExt = NoopArgsExt;
type Node = DefaultRethNodeCommandConfig;
}
/// An [Args] extension that does nothing.
#[derive(Debug, Clone, Copy, Default, Args)]
pub struct NoopArgsExt;
/// A trait that provides configured RPC server.
///
/// This provides all basic config values for the RPC server and is implemented by the
/// [RpcServerArgs](crate::args::RpcServerArgs) type.
pub trait RethRpcConfig {
/// Returns whether ipc is enabled.
fn is_ipc_enabled(&self) -> bool;
/// The configured ethereum RPC settings.
fn eth_config(&self) -> EthConfig;
// TODO extract more functions from RpcServerArgs
}
/// A trait that allows further customization of the RPC server via CLI.
pub trait RethRpcServerArgsExt: fmt::Debug + clap::Args {
/// A trait that allows for extending parts of the CLI with additional functionality.
pub trait RethNodeCommandExt: fmt::Debug + clap::Args {
/// Allows for registering additional RPC modules for the transports.
///
/// This is expected to call the merge functions of [TransportRpcModules], for example
/// [TransportRpcModules::merge_configured]
fn extend_rpc_modules<Conf, Provider, Pool, Network, Tasks, Events>(
&self,
config: &Conf,
registry: &mut RethModuleRegistry<Provider, Pool, Network, Tasks, Events>,
modules: &mut TransportRpcModules<()>,
) -> eyre::Result<()>
where
Conf: RethRpcConfig,
Provider: BlockReaderIdExt
+ StateProviderFactory
+ EvmEnvProvider
+ ChainSpecProvider
+ ChangeSetReader
+ Clone
+ Unpin
+ 'static,
Pool: TransactionPool + Clone + 'static,
Network: NetworkInfo + Peers + Clone + 'static,
Tasks: TaskSpawner + Clone + 'static,
Events: CanonStateSubscriptions + Clone + 'static;
}
impl RethRpcServerArgsExt for NoopArgsExt {
fn extend_rpc_modules<Conf, Provider, Pool, Network, Tasks, Events>(
&self,
_config: &Conf,
@ -91,4 +54,50 @@ impl RethRpcServerArgsExt for NoopArgsExt {
{
Ok(())
}
/// Configures the [PayloadBuilderService] for the node, spawns it and returns the
/// [PayloadBuilderHandle].
///
/// By default this spawns a [BasicPayloadJobGenerator] with the default configuration
/// [BasicPayloadJobGeneratorConfig].
fn spawn_payload_builder_service<Conf, Provider, Pool, Tasks>(
&self,
conf: &Conf,
provider: Provider,
pool: Pool,
executor: Tasks,
chain_spec: Arc<ChainSpec>,
) -> eyre::Result<PayloadBuilderHandle>
where
Conf: PayloadBuilderConfig,
Provider: StateProviderFactory + BlockReaderIdExt + Clone + Unpin + 'static,
Pool: TransactionPool + Unpin + 'static,
Tasks: TaskSpawner + Clone + Unpin + 'static,
{
let payload_generator = BasicPayloadJobGenerator::new(
provider,
pool,
executor.clone(),
BasicPayloadJobGeneratorConfig::default()
.interval(conf.interval())
.deadline(conf.deadline())
.max_payload_tasks(conf.max_payload_tasks())
.extradata(conf.extradata_rlp_bytes())
.max_gas_limit(conf.max_gas_limit()),
chain_spec,
);
let (payload_service, payload_builder) = PayloadBuilderService::new(payload_generator);
executor.spawn_critical("payload builder service", Box::pin(payload_service));
Ok(payload_builder)
}
// TODO move network related functions here
}
/// The default configuration for the reth node command [Command](crate::node::Command).
#[derive(Debug, Clone, Copy, Default, Args)]
pub struct DefaultRethNodeCommandConfig;
impl RethNodeCommandExt for DefaultRethNodeCommandConfig {}

View File

@ -3,7 +3,7 @@ use crate::{
args::utils::genesis_value_parser,
chain,
cli::ext::RethCliExt,
config, db, debug_cmd,
db, debug_cmd,
dirs::{LogsDir, PlatformPath},
node, p2p,
runner::CliRunner,
@ -19,6 +19,7 @@ use reth_tracing::{
};
use std::sync::Arc;
pub mod config;
pub mod ext;
/// The main reth cli interface.
@ -127,7 +128,7 @@ pub enum Commands<Ext: RethCliExt = ()> {
TestVectors(test_vectors::Command),
/// Write config to stdout
#[command(name = "config")]
Config(config::Command),
Config(crate::config::Command),
/// Various debug routines
#[command(name = "debug")]
Debug(debug_cmd::Command),

View File

@ -8,7 +8,7 @@ use crate::{
DatabaseArgs, DebugArgs, DevArgs, NetworkArgs, PayloadBuilderArgs, PruningArgs,
RpcServerArgs, TxPoolArgs,
},
cli::ext::RethCliExt,
cli::ext::{RethCliExt, RethNodeCommandExt},
dirs::{DataDirPath, MaybePlatformPath},
init::init_genesis,
node::cl_events::ConsensusLayerHealthEvents,
@ -22,7 +22,7 @@ use eyre::Context;
use fdlimit::raise_fd_limit;
use futures::{future::Either, pin_mut, stream, stream_select, StreamExt};
use reth_auto_seal_consensus::{AutoSealBuilder, AutoSealConsensus, MiningMode};
use reth_basic_payload_builder::{BasicPayloadJobGenerator, BasicPayloadJobGeneratorConfig};
use reth_beacon_consensus::{BeaconConsensus, BeaconConsensusEngine, MIN_BLOCKS_FOR_PIPELINE_RUN};
use reth_blockchain_tree::{
config::BlockchainTreeConfig, externals::TreeExternals, BlockchainTree, ShareableBlockchainTree,
@ -44,7 +44,7 @@ use reth_interfaces::{
};
use reth_network::{error::NetworkError, NetworkConfig, NetworkHandle, NetworkManager};
use reth_network_api::NetworkInfo;
use reth_payload_builder::PayloadBuilderService;
use reth_primitives::{
stage::StageId, BlockHashOrNumber, BlockNumber, ChainSpec, DisplayHardforks, Head,
SealedHeader, H256,
@ -127,7 +127,7 @@ pub struct Command<Ext: RethCliExt = ()> {
network: NetworkArgs,
#[clap(flatten)]
rpc: RpcServerArgs<Ext::RpcExt>,
rpc: RpcServerArgs,
#[clap(flatten)]
txpool: TxPoolArgs,
@ -146,6 +146,10 @@ pub struct Command<Ext: RethCliExt = ()> {
#[clap(flatten)]
pruning: PruningArgs,
/// Additional cli arguments
#[clap(flatten)]
pub ext: Ext::Node,
}
impl<Ext: RethCliExt> Command<Ext> {
@ -276,22 +280,14 @@ impl<Ext: RethCliExt> Command<Ext> {
let (consensus_engine_tx, consensus_engine_rx) = unbounded_channel();
let payload_generator = BasicPayloadJobGenerator::new(
debug!(target: "reth::cli", "Spawning payload builder service");
let payload_builder = self.ext.spawn_payload_builder_service(
&self.builder,
blockchain_db.clone(),
transaction_pool.clone(),
ctx.task_executor.clone(),
BasicPayloadJobGeneratorConfig::default()
.interval(self.builder.interval)
.deadline(self.builder.deadline)
.max_payload_tasks(self.builder.max_payload_tasks)
.extradata(self.builder.extradata_bytes())
.max_gas_limit(self.builder.max_gas_limit),
Arc::clone(&self.chain),
);
let (payload_service, payload_builder) = PayloadBuilderService::new(payload_generator);
debug!(target: "reth::cli", "Spawning payload builder service");
ctx.task_executor.spawn_critical("payload builder service", payload_service);
)?;
let max_block = if let Some(block) = self.debug.max_block {
Some(block)
@ -451,6 +447,7 @@ impl<Ext: RethCliExt> Command<Ext> {
blockchain_tree,
engine_api,
jwt_secret,
&self.ext,
)
.await?;

View File

@ -15,7 +15,8 @@ use clap::Parser;
use jsonrpsee::{core::RpcResult, proc_macros::rpc};
use reth::{
cli::{
ext::{RethCliExt, RethRpcConfig, RethRpcServerArgsExt},
config::RethRpcConfig,
ext::{RethCliExt, RethNodeCommandExt},
Cli,
},
network::{NetworkInfo, Peers},
@ -37,7 +38,7 @@ struct MyRethCliExt;
impl RethCliExt for MyRethCliExt {
/// This tells the reth CLI to install the `txpool` rpc namespace via `RethCliTxpoolExt`
type RpcExt = RethCliTxpoolExt;
type Node = RethCliTxpoolExt;
}
/// Our custom cli args extension that adds one flag to reth default CLI.
@ -48,7 +49,7 @@ struct RethCliTxpoolExt {
pub enable_ext: bool,
}
impl RethRpcServerArgsExt for RethCliTxpoolExt {
impl RethNodeCommandExt for RethCliTxpoolExt {
// This is the entrypoint for the CLI to extend the RPC server with custom rpc namespaces.
fn extend_rpc_modules<Conf, Provider, Pool, Network, Tasks, Events>(
&self,