mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: simplify PayloadBuilder setup (#14276)
This commit is contained in:
@ -12,12 +12,14 @@ workspace = true
|
||||
|
||||
[dependencies]
|
||||
# reth
|
||||
reth-basic-payload-builder.workspace = true
|
||||
reth-db-api.workspace = true
|
||||
reth-consensus.workspace = true
|
||||
reth-evm.workspace = true
|
||||
reth-provider.workspace = true
|
||||
reth-engine-primitives.workspace = true
|
||||
reth-transaction-pool.workspace = true
|
||||
reth-payload-builder.workspace = true
|
||||
reth-payload-builder-primitives.workspace = true
|
||||
reth-payload-primitives.workspace = true
|
||||
reth-tasks.workspace = true
|
||||
|
||||
@ -1,6 +1,8 @@
|
||||
//! Traits for configuring a node.
|
||||
|
||||
use crate::PayloadTypes;
|
||||
use alloy_rpc_types_engine::JwtSecret;
|
||||
use reth_basic_payload_builder::PayloadBuilder;
|
||||
use reth_consensus::{ConsensusError, FullConsensus};
|
||||
use reth_db_api::{database_metrics::DatabaseMetrics, Database};
|
||||
use reth_engine_primitives::BeaconConsensusEngineHandle;
|
||||
@ -8,7 +10,7 @@ use reth_evm::{execute::BlockExecutorProvider, ConfigureEvmFor};
|
||||
use reth_network_api::FullNetwork;
|
||||
use reth_node_core::node_config::NodeConfig;
|
||||
use reth_node_types::{NodeTypes, NodeTypesWithDBAdapter, NodeTypesWithEngine, TxTy};
|
||||
use reth_payload_builder_primitives::PayloadBuilder;
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_provider::FullProvider;
|
||||
use reth_tasks::TaskExecutor;
|
||||
use reth_transaction_pool::{PoolTransaction, TransactionPool};
|
||||
@ -42,6 +44,23 @@ where
|
||||
type Provider = Provider;
|
||||
}
|
||||
|
||||
/// Helper trait to bound [`PayloadBuilder`] to the node's engine types.
|
||||
pub trait PayloadBuilderFor<N: NodeTypesWithEngine>:
|
||||
PayloadBuilder<
|
||||
Attributes = <N::Engine as PayloadTypes>::PayloadBuilderAttributes,
|
||||
BuiltPayload = <N::Engine as PayloadTypes>::BuiltPayload,
|
||||
>
|
||||
{
|
||||
}
|
||||
|
||||
impl<T, N: NodeTypesWithEngine> PayloadBuilderFor<N> for T where
|
||||
T: PayloadBuilder<
|
||||
Attributes = <N::Engine as PayloadTypes>::PayloadBuilderAttributes,
|
||||
BuiltPayload = <N::Engine as PayloadTypes>::BuiltPayload,
|
||||
>
|
||||
{
|
||||
}
|
||||
|
||||
/// Encapsulates all types and components of the node.
|
||||
pub trait FullNodeComponents: FullNodeTypes + Clone + 'static {
|
||||
/// The transaction pool of the node.
|
||||
@ -63,8 +82,7 @@ pub trait FullNodeComponents: FullNodeTypes + Clone + 'static {
|
||||
type Network: FullNetwork;
|
||||
|
||||
/// Builds new blocks.
|
||||
type PayloadBuilder: PayloadBuilder<PayloadType = <Self::Types as NodeTypesWithEngine>::Engine>
|
||||
+ Clone;
|
||||
type PayloadBuilder: PayloadBuilderFor<Self::Types>;
|
||||
|
||||
/// Returns the transaction pool of the node.
|
||||
fn pool(&self) -> &Self::Pool;
|
||||
@ -81,9 +99,15 @@ pub trait FullNodeComponents: FullNodeTypes + Clone + 'static {
|
||||
/// Returns the handle to the network
|
||||
fn network(&self) -> &Self::Network;
|
||||
|
||||
/// Returns the handle to the payload builder service.
|
||||
/// Returns the configured payload builder.
|
||||
fn payload_builder(&self) -> &Self::PayloadBuilder;
|
||||
|
||||
/// Returns the handle to the payload builder service handling payload building requests from
|
||||
/// the engine.
|
||||
fn payload_builder_handle(
|
||||
&self,
|
||||
) -> &PayloadBuilderHandle<<Self::Types as NodeTypesWithEngine>::Engine>;
|
||||
|
||||
/// Returns the provider of the node.
|
||||
fn provider(&self) -> &Self::Provider;
|
||||
|
||||
|
||||
@ -54,6 +54,7 @@ reth-tasks.workspace = true
|
||||
reth-tokio-util.workspace = true
|
||||
reth-tracing.workspace = true
|
||||
reth-transaction-pool.workspace = true
|
||||
reth-basic-payload-builder.workspace = true
|
||||
|
||||
## ethereum
|
||||
alloy-consensus.workspace = true
|
||||
|
||||
@ -119,6 +119,14 @@ impl<T: FullNodeTypes, C: NodeComponents<T>> FullNodeComponents for NodeAdapter<
|
||||
self.components.payload_builder()
|
||||
}
|
||||
|
||||
fn payload_builder_handle(
|
||||
&self,
|
||||
) -> &reth_payload_builder::PayloadBuilderHandle<
|
||||
<Self::Types as reth_node_api::NodeTypesWithEngine>::Engine,
|
||||
> {
|
||||
self.components.payload_builder_handle()
|
||||
}
|
||||
|
||||
fn provider(&self) -> &Self::Provider {
|
||||
&self.provider
|
||||
}
|
||||
|
||||
@ -10,11 +10,12 @@ use crate::{
|
||||
use reth_consensus::{ConsensusError, FullConsensus};
|
||||
use reth_evm::{execute::BlockExecutorProvider, ConfigureEvmFor};
|
||||
use reth_network::NetworkPrimitives;
|
||||
use reth_node_api::{BlockTy, BodyTy, HeaderTy, NodeTypesWithEngine, PrimitivesTy, TxTy};
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_node_api::{BlockTy, BodyTy, HeaderTy, PrimitivesTy, TxTy};
|
||||
use reth_transaction_pool::{PoolTransaction, TransactionPool};
|
||||
use std::{future::Future, marker::PhantomData};
|
||||
|
||||
use super::PayloadBuilderFor;
|
||||
|
||||
/// A generic, general purpose and customizable [`NodeComponentsBuilder`] implementation.
|
||||
///
|
||||
/// This type is stateful and captures the configuration of the node's components.
|
||||
@ -324,6 +325,7 @@ where
|
||||
ExecB::EVM,
|
||||
ExecB::Executor,
|
||||
ConsB::Consensus,
|
||||
PayloadB::PayloadBuilder,
|
||||
>;
|
||||
|
||||
async fn build_components(
|
||||
@ -332,7 +334,7 @@ where
|
||||
) -> eyre::Result<Self::Components> {
|
||||
let Self {
|
||||
pool_builder,
|
||||
payload_builder,
|
||||
payload_builder: payload_builder_builder,
|
||||
network_builder,
|
||||
executor_builder: evm_builder,
|
||||
consensus_builder,
|
||||
@ -342,7 +344,10 @@ where
|
||||
let (evm_config, executor) = evm_builder.build_evm(context).await?;
|
||||
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?;
|
||||
let payload_builder =
|
||||
payload_builder_builder.build_payload_builder(context, pool.clone()).await?;
|
||||
let payload_builder_handle =
|
||||
payload_builder_builder.spawn_payload_builder_service(context, payload_builder.clone());
|
||||
let consensus = consensus_builder.build_consensus(context).await?;
|
||||
|
||||
Ok(Components {
|
||||
@ -350,6 +355,7 @@ where
|
||||
evm_config,
|
||||
network,
|
||||
payload_builder,
|
||||
payload_builder_handle,
|
||||
executor,
|
||||
consensus,
|
||||
})
|
||||
@ -380,10 +386,7 @@ impl Default for ComponentsBuilder<(), (), (), (), (), ()> {
|
||||
/// 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,
|
||||
PayloadBuilder = PayloadBuilderHandle<<Node::Types as NodeTypesWithEngine>::Engine>,
|
||||
>;
|
||||
type Components: NodeComponents<Node>;
|
||||
|
||||
/// Consumes the type and returns the created components.
|
||||
fn build_components(
|
||||
@ -392,7 +395,7 @@ pub trait NodeComponentsBuilder<Node: FullNodeTypes>: Send {
|
||||
) -> impl Future<Output = eyre::Result<Self::Components>> + Send;
|
||||
}
|
||||
|
||||
impl<Node, N, F, Fut, Pool, EVM, Executor, Cons> NodeComponentsBuilder<Node> for F
|
||||
impl<Node, N, F, Fut, Pool, EVM, Executor, Cons, Payload> NodeComponentsBuilder<Node> for F
|
||||
where
|
||||
N: NetworkPrimitives<
|
||||
BlockHeader = HeaderTy<Node::Types>,
|
||||
@ -401,7 +404,8 @@ where
|
||||
>,
|
||||
Node: FullNodeTypes,
|
||||
F: FnOnce(&BuilderContext<Node>) -> Fut + Send,
|
||||
Fut: Future<Output = eyre::Result<Components<Node, N, Pool, EVM, Executor, Cons>>> + Send,
|
||||
Fut: Future<Output = eyre::Result<Components<Node, N, Pool, EVM, Executor, Cons, Payload>>>
|
||||
+ Send,
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
@ -409,8 +413,9 @@ where
|
||||
Executor: BlockExecutorProvider<Primitives = PrimitivesTy<Node::Types>>,
|
||||
Cons:
|
||||
FullConsensus<PrimitivesTy<Node::Types>, Error = ConsensusError> + Clone + Unpin + 'static,
|
||||
Payload: PayloadBuilderFor<Node::Types> + Unpin + 'static,
|
||||
{
|
||||
type Components = Components<Node, N, Pool, EVM, Executor, Cons>;
|
||||
type Components = Components<Node, N, Pool, EVM, Executor, Cons, Payload>;
|
||||
|
||||
fn build_components(
|
||||
self,
|
||||
|
||||
@ -21,6 +21,7 @@ pub use network::*;
|
||||
pub use payload::*;
|
||||
pub use pool::*;
|
||||
use reth_network_p2p::BlockClient;
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
|
||||
use crate::{ConfigureEvm, FullNodeTypes};
|
||||
use reth_consensus::{ConsensusError, FullConsensus};
|
||||
@ -28,9 +29,9 @@ use reth_evm::{execute::BlockExecutorProvider, ConfigureEvmFor};
|
||||
use reth_network::{NetworkHandle, NetworkPrimitives};
|
||||
use reth_network_api::FullNetwork;
|
||||
use reth_node_api::{
|
||||
BlockTy, BodyTy, HeaderTy, NodeTypes, NodeTypesWithEngine, PayloadBuilder, PrimitivesTy, TxTy,
|
||||
BlockTy, BodyTy, HeaderTy, NodeTypes, NodeTypesWithEngine, PayloadBuilderFor, PrimitivesTy,
|
||||
TxTy,
|
||||
};
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_transaction_pool::{PoolTransaction, TransactionPool};
|
||||
|
||||
/// An abstraction over the components of a node, consisting of:
|
||||
@ -58,8 +59,7 @@ pub trait NodeComponents<T: FullNodeTypes>: Clone + Unpin + Send + Sync + 'stati
|
||||
type Network: FullNetwork<Client: BlockClient<Block = BlockTy<T::Types>>>;
|
||||
|
||||
/// Builds new blocks.
|
||||
type PayloadBuilder: PayloadBuilder<PayloadType = <T::Types as NodeTypesWithEngine>::Engine>
|
||||
+ Clone;
|
||||
type PayloadBuilder: PayloadBuilderFor<T::Types> + Clone + Unpin + 'static;
|
||||
|
||||
/// Returns the transaction pool of the node.
|
||||
fn pool(&self) -> &Self::Pool;
|
||||
@ -76,15 +76,29 @@ pub trait NodeComponents<T: FullNodeTypes>: Clone + Unpin + Send + Sync + 'stati
|
||||
/// Returns the handle to the network
|
||||
fn network(&self) -> &Self::Network;
|
||||
|
||||
/// Returns the handle to the payload builder service.
|
||||
/// Returns the payload builder that knows how to build blocks.
|
||||
fn payload_builder(&self) -> &Self::PayloadBuilder;
|
||||
|
||||
/// Returns the handle to the payload builder service handling payload building requests from
|
||||
/// the engine.
|
||||
fn payload_builder_handle(
|
||||
&self,
|
||||
) -> &PayloadBuilderHandle<<T::Types as NodeTypesWithEngine>::Engine>;
|
||||
}
|
||||
|
||||
/// All the components of the node.
|
||||
///
|
||||
/// This provides access to all the components of the node.
|
||||
#[derive(Debug)]
|
||||
pub struct Components<Node: FullNodeTypes, N: NetworkPrimitives, Pool, EVM, Executor, Consensus> {
|
||||
pub struct Components<
|
||||
Node: FullNodeTypes,
|
||||
N: NetworkPrimitives,
|
||||
Pool,
|
||||
EVM,
|
||||
Executor,
|
||||
Consensus,
|
||||
Payload,
|
||||
> {
|
||||
/// The transaction pool of the node.
|
||||
pub transaction_pool: Pool,
|
||||
/// The node's EVM configuration, defining settings for the Ethereum Virtual Machine.
|
||||
@ -95,19 +109,21 @@ pub struct Components<Node: FullNodeTypes, N: NetworkPrimitives, Pool, EVM, Exec
|
||||
pub consensus: Consensus,
|
||||
/// The network implementation of the node.
|
||||
pub network: NetworkHandle<N>,
|
||||
/// The payload builder.
|
||||
pub payload_builder: Payload,
|
||||
/// The handle to the payload builder service.
|
||||
pub payload_builder: PayloadBuilderHandle<<Node::Types as NodeTypesWithEngine>::Engine>,
|
||||
pub payload_builder_handle: PayloadBuilderHandle<<Node::Types as NodeTypesWithEngine>::Engine>,
|
||||
}
|
||||
|
||||
impl<Node, Pool, EVM, Executor, Cons, N> NodeComponents<Node>
|
||||
for Components<Node, N, Pool, EVM, Executor, Cons>
|
||||
impl<Node, Pool, EVM, Executor, Cons, N, Payload> NodeComponents<Node>
|
||||
for Components<Node, N, Pool, EVM, Executor, Cons, Payload>
|
||||
where
|
||||
Node: FullNodeTypes,
|
||||
N: NetworkPrimitives<
|
||||
BlockHeader = HeaderTy<Node::Types>,
|
||||
BlockBody = BodyTy<Node::Types>,
|
||||
Block = BlockTy<Node::Types>,
|
||||
>,
|
||||
Node: FullNodeTypes,
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
@ -115,13 +131,14 @@ where
|
||||
Executor: BlockExecutorProvider<Primitives = PrimitivesTy<Node::Types>>,
|
||||
Cons:
|
||||
FullConsensus<PrimitivesTy<Node::Types>, Error = ConsensusError> + Clone + Unpin + 'static,
|
||||
Payload: PayloadBuilderFor<Node::Types> + Clone + Unpin + 'static,
|
||||
{
|
||||
type Pool = Pool;
|
||||
type Evm = EVM;
|
||||
type Executor = Executor;
|
||||
type Consensus = Cons;
|
||||
type Network = NetworkHandle<N>;
|
||||
type PayloadBuilder = PayloadBuilderHandle<<Node::Types as NodeTypesWithEngine>::Engine>;
|
||||
type PayloadBuilder = Payload;
|
||||
|
||||
fn pool(&self) -> &Self::Pool {
|
||||
&self.transaction_pool
|
||||
@ -146,16 +163,24 @@ where
|
||||
fn payload_builder(&self) -> &Self::PayloadBuilder {
|
||||
&self.payload_builder
|
||||
}
|
||||
|
||||
fn payload_builder_handle(
|
||||
&self,
|
||||
) -> &PayloadBuilderHandle<<Node::Types as NodeTypesWithEngine>::Engine> {
|
||||
&self.payload_builder_handle
|
||||
}
|
||||
}
|
||||
|
||||
impl<Node, N, Pool, EVM, Executor, Cons> Clone for Components<Node, N, Pool, EVM, Executor, Cons>
|
||||
impl<Node, N, Pool, EVM, Executor, Cons, Payload> Clone
|
||||
for Components<Node, N, Pool, EVM, Executor, Cons, Payload>
|
||||
where
|
||||
N: NetworkPrimitives,
|
||||
Node: FullNodeTypes,
|
||||
Pool: TransactionPool,
|
||||
EVM: ConfigureEvm<Header = HeaderTy<Node::Types>, Transaction = TxTy<Node::Types>>,
|
||||
EVM: ConfigureEvm,
|
||||
Executor: BlockExecutorProvider,
|
||||
Cons: Clone,
|
||||
Payload: Clone,
|
||||
{
|
||||
fn clone(&self) -> Self {
|
||||
Self {
|
||||
@ -165,6 +190,7 @@ where
|
||||
consensus: self.consensus.clone(),
|
||||
network: self.network.clone(),
|
||||
payload_builder: self.payload_builder.clone(),
|
||||
payload_builder_handle: self.payload_builder_handle.clone(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -1,45 +1,73 @@
|
||||
//! Payload service component for the node builder.
|
||||
|
||||
use crate::{BuilderContext, FullNodeTypes, NodeTypesWithEngine};
|
||||
use reth_basic_payload_builder::{BasicPayloadJobGenerator, BasicPayloadJobGeneratorConfig};
|
||||
use reth_chain_state::CanonStateSubscriptions;
|
||||
use reth_node_api::PayloadBuilderFor;
|
||||
use reth_payload_builder::{PayloadBuilderHandle, PayloadBuilderService};
|
||||
use reth_transaction_pool::TransactionPool;
|
||||
use std::future::Future;
|
||||
|
||||
use reth_node_api::NodeTypesWithEngine;
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_transaction_pool::TransactionPool;
|
||||
|
||||
use crate::{BuilderContext, FullNodeTypes};
|
||||
|
||||
/// A type that knows how to spawn the payload service.
|
||||
pub trait PayloadServiceBuilder<Node: FullNodeTypes, Pool: TransactionPool>: Send {
|
||||
pub trait PayloadServiceBuilder<Node: FullNodeTypes, Pool: TransactionPool>: Send + Sized {
|
||||
/// Payload builder implementation.
|
||||
type PayloadBuilder: PayloadBuilderFor<Node::Types> + Unpin + 'static;
|
||||
|
||||
/// Spawns the payload service and returns the handle to it.
|
||||
///
|
||||
/// The [`BuilderContext`] is provided to allow access to the node's configuration.
|
||||
fn spawn_payload_service(
|
||||
self,
|
||||
fn build_payload_builder(
|
||||
&self,
|
||||
ctx: &BuilderContext<Node>,
|
||||
pool: Pool,
|
||||
) -> impl Future<
|
||||
Output = eyre::Result<PayloadBuilderHandle<<Node::Types as NodeTypesWithEngine>::Engine>>,
|
||||
> + Send;
|
||||
) -> impl Future<Output = eyre::Result<Self::PayloadBuilder>> + Send;
|
||||
|
||||
/// Spawns the [`PayloadBuilderService`] and returns the handle to it for use by the engine.
|
||||
///
|
||||
/// We provide default implementation via [`BasicPayloadJobGenerator`] but it can be overridden
|
||||
/// for custom job orchestration logic,
|
||||
fn spawn_payload_builder_service(
|
||||
self,
|
||||
ctx: &BuilderContext<Node>,
|
||||
payload_builder: Self::PayloadBuilder,
|
||||
) -> PayloadBuilderHandle<<Node::Types as NodeTypesWithEngine>::Engine> {
|
||||
let conf = ctx.config().builder.clone();
|
||||
|
||||
let payload_job_config = BasicPayloadJobGeneratorConfig::default()
|
||||
.interval(conf.interval)
|
||||
.deadline(conf.deadline)
|
||||
.max_payload_tasks(conf.max_payload_tasks);
|
||||
|
||||
let payload_generator = BasicPayloadJobGenerator::with_builder(
|
||||
ctx.provider().clone(),
|
||||
ctx.task_executor().clone(),
|
||||
payload_job_config,
|
||||
payload_builder,
|
||||
);
|
||||
let (payload_service, payload_service_handle) =
|
||||
PayloadBuilderService::new(payload_generator, ctx.provider().canonical_state_stream());
|
||||
|
||||
ctx.task_executor().spawn_critical("payload builder service", Box::pin(payload_service));
|
||||
|
||||
payload_service_handle
|
||||
}
|
||||
}
|
||||
|
||||
impl<Node, F, Fut, Pool> PayloadServiceBuilder<Node, Pool> for F
|
||||
impl<Node, F, Fut, Pool, Builder> PayloadServiceBuilder<Node, Pool> for F
|
||||
where
|
||||
Node: FullNodeTypes,
|
||||
Pool: TransactionPool,
|
||||
F: Fn(&BuilderContext<Node>, Pool) -> Fut + Send,
|
||||
Fut: Future<
|
||||
Output = eyre::Result<
|
||||
PayloadBuilderHandle<<Node::Types as NodeTypesWithEngine>::Engine>,
|
||||
>,
|
||||
> + Send,
|
||||
Fut: Future<Output = eyre::Result<Builder>> + Send,
|
||||
Builder: PayloadBuilderFor<Node::Types> + Unpin + 'static,
|
||||
{
|
||||
fn spawn_payload_service(
|
||||
self,
|
||||
type PayloadBuilder = Builder;
|
||||
|
||||
fn build_payload_builder(
|
||||
&self,
|
||||
ctx: &BuilderContext<Node>,
|
||||
pool: Pool,
|
||||
) -> impl Future<
|
||||
Output = eyre::Result<PayloadBuilderHandle<<Node::Types as NodeTypesWithEngine>::Engine>>,
|
||||
> + Send {
|
||||
) -> impl Future<Output = eyre::Result<Self::PayloadBuilder>> + Send {
|
||||
self(ctx, pool)
|
||||
}
|
||||
}
|
||||
|
||||
@ -17,7 +17,7 @@ use reth_network::{NetworkSyncUpdater, SyncState};
|
||||
use reth_network_api::BlockDownloaderProvider;
|
||||
use reth_node_api::{
|
||||
BeaconConsensusEngineHandle, BuiltPayload, FullNodeTypes, NodeTypesWithDBAdapter,
|
||||
NodeTypesWithEngine, PayloadAttributesBuilder, PayloadBuilder, PayloadTypes,
|
||||
NodeTypesWithEngine, PayloadAttributesBuilder, PayloadTypes,
|
||||
};
|
||||
use reth_node_core::{
|
||||
dirs::{ChainPath, DataDirPath},
|
||||
@ -218,7 +218,7 @@ where
|
||||
ctx.provider_factory().clone(),
|
||||
ctx.blockchain_db().clone(),
|
||||
pruner,
|
||||
ctx.components().payload_builder().clone(),
|
||||
ctx.components().payload_builder_handle().clone(),
|
||||
engine_payload_validator,
|
||||
engine_tree_config,
|
||||
ctx.invalid_block_hook()?,
|
||||
@ -243,7 +243,7 @@ where
|
||||
ctx.provider_factory().clone(),
|
||||
ctx.blockchain_db().clone(),
|
||||
pruner,
|
||||
ctx.components().payload_builder().clone(),
|
||||
ctx.components().payload_builder_handle().clone(),
|
||||
engine_payload_validator,
|
||||
engine_tree_config,
|
||||
ctx.invalid_block_hook()?,
|
||||
@ -317,7 +317,7 @@ where
|
||||
let network_handle = ctx.components().network().clone();
|
||||
let mut built_payloads = ctx
|
||||
.components()
|
||||
.payload_builder()
|
||||
.payload_builder_handle()
|
||||
.subscribe()
|
||||
.await
|
||||
.map_err(|e| eyre::eyre!("Failed to subscribe to payload builder events: {:?}", e))?
|
||||
@ -399,6 +399,7 @@ where
|
||||
network: ctx.components().network().clone(),
|
||||
provider: ctx.node_adapter().provider.clone(),
|
||||
payload_builder: ctx.components().payload_builder().clone(),
|
||||
payload_builder_handle: ctx.components().payload_builder_handle().clone(),
|
||||
task_executor: ctx.task_executor().clone(),
|
||||
config: ctx.node_config().clone(),
|
||||
data_dir: ctx.data_dir().clone(),
|
||||
|
||||
@ -1,5 +1,6 @@
|
||||
// re-export the node api types
|
||||
pub use reth_node_api::{FullNodeTypes, NodeTypes, NodeTypesWithEngine};
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
|
||||
use std::{
|
||||
marker::PhantomData,
|
||||
@ -12,7 +13,6 @@ use reth_node_core::{
|
||||
dirs::{ChainPath, DataDirPath},
|
||||
node_config::NodeConfig,
|
||||
};
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_provider::ChainSpecProvider;
|
||||
use reth_rpc_api::EngineApiClient;
|
||||
use reth_rpc_builder::{auth::AuthServerHandle, RpcServerHandle};
|
||||
@ -117,8 +117,10 @@ pub struct FullNode<Node: FullNodeComponents, AddOns: NodeAddOns<Node>> {
|
||||
pub network: Node::Network,
|
||||
/// Provider to interact with the node's database
|
||||
pub provider: Node::Provider,
|
||||
/// Node's configured payload builder.
|
||||
pub payload_builder: Node::PayloadBuilder,
|
||||
/// Handle to the node's payload builder service.
|
||||
pub payload_builder: PayloadBuilderHandle<<Node::Types as NodeTypesWithEngine>::Engine>,
|
||||
pub payload_builder_handle: PayloadBuilderHandle<<Node::Types as NodeTypesWithEngine>::Engine>,
|
||||
/// Task executor for the node.
|
||||
pub task_executor: TaskExecutor,
|
||||
/// The initial node config.
|
||||
@ -138,6 +140,7 @@ impl<Node: FullNodeComponents, AddOns: NodeAddOns<Node>> Clone for FullNode<Node
|
||||
network: self.network.clone(),
|
||||
provider: self.provider.clone(),
|
||||
payload_builder: self.payload_builder.clone(),
|
||||
payload_builder_handle: self.payload_builder_handle.clone(),
|
||||
task_executor: self.task_executor.clone(),
|
||||
config: self.config.clone(),
|
||||
data_dir: self.data_dir.clone(),
|
||||
|
||||
@ -437,7 +437,7 @@ where
|
||||
node.provider().clone(),
|
||||
config.chain.clone(),
|
||||
beacon_engine_handle,
|
||||
PayloadStore::new(node.payload_builder().clone()),
|
||||
PayloadStore::new(node.payload_builder_handle().clone()),
|
||||
node.pool().clone(),
|
||||
Box::new(node.task_executor().clone()),
|
||||
client,
|
||||
|
||||
Reference in New Issue
Block a user