mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
chore: simplify PayloadBuilder trait (#14246)
This commit is contained in:
1
Cargo.lock
generated
1
Cargo.lock
generated
@ -6510,7 +6510,6 @@ dependencies = [
|
||||
"reth-provider",
|
||||
"reth-revm",
|
||||
"reth-tasks",
|
||||
"reth-transaction-pool",
|
||||
"revm",
|
||||
"tokio",
|
||||
"tracing",
|
||||
|
||||
@ -216,8 +216,6 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
);
|
||||
|
||||
let args = BuildArguments::new(
|
||||
blockchain_db.clone(),
|
||||
transaction_pool,
|
||||
CachedReads::default(),
|
||||
payload_config,
|
||||
CancelOnDrop::default(),
|
||||
@ -225,6 +223,8 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
);
|
||||
|
||||
let payload_builder = reth_ethereum_payload_builder::EthereumPayloadBuilder::new(
|
||||
blockchain_db.clone(),
|
||||
transaction_pool,
|
||||
EthEvmConfig::new(provider_factory.chain_spec()),
|
||||
EthereumBuilderConfig::new(Default::default()),
|
||||
);
|
||||
|
||||
@ -45,6 +45,8 @@ impl EthereumPayloadBuilder {
|
||||
{
|
||||
let conf = ctx.payload_builder_config();
|
||||
let payload_builder = reth_ethereum_payload_builder::EthereumPayloadBuilder::new(
|
||||
ctx.provider().clone(),
|
||||
pool,
|
||||
evm_config,
|
||||
EthereumBuilderConfig::new(conf.extra_data_bytes()).with_gas_limit(conf.gas_limit()),
|
||||
);
|
||||
@ -56,7 +58,6 @@ impl EthereumPayloadBuilder {
|
||||
|
||||
let payload_generator = BasicPayloadJobGenerator::with_builder(
|
||||
ctx.provider().clone(),
|
||||
pool,
|
||||
ctx.task_executor().clone(),
|
||||
payload_job_config,
|
||||
payload_builder,
|
||||
|
||||
@ -37,8 +37,8 @@ use reth_primitives_traits::{
|
||||
use reth_revm::database::StateProviderDatabase;
|
||||
use reth_storage_api::StateProviderFactory;
|
||||
use reth_transaction_pool::{
|
||||
error::InvalidPoolTransactionError, noop::NoopTransactionPool, BestTransactions,
|
||||
BestTransactionsAttributes, PoolTransaction, TransactionPool, ValidPoolTransaction,
|
||||
error::InvalidPoolTransactionError, BestTransactions, BestTransactionsAttributes,
|
||||
PoolTransaction, TransactionPool, ValidPoolTransaction,
|
||||
};
|
||||
use revm::{
|
||||
db::{states::bundle_state::BundleRetention, State},
|
||||
@ -58,21 +58,30 @@ type BestTransactionsIter<Pool> = Box<
|
||||
|
||||
/// Ethereum payload builder
|
||||
#[derive(Debug, Clone, PartialEq, Eq)]
|
||||
pub struct EthereumPayloadBuilder<EvmConfig = EthEvmConfig> {
|
||||
pub struct EthereumPayloadBuilder<Pool, Client, EvmConfig = EthEvmConfig> {
|
||||
/// Client providing access to node state.
|
||||
client: Client,
|
||||
/// Transaction pool.
|
||||
pool: Pool,
|
||||
/// The type responsible for creating the evm.
|
||||
evm_config: EvmConfig,
|
||||
/// Payload builder configuration.
|
||||
builder_config: EthereumBuilderConfig,
|
||||
}
|
||||
|
||||
impl<EvmConfig> EthereumPayloadBuilder<EvmConfig> {
|
||||
impl<Pool, Client, EvmConfig> EthereumPayloadBuilder<Pool, Client, EvmConfig> {
|
||||
/// `EthereumPayloadBuilder` constructor.
|
||||
pub const fn new(evm_config: EvmConfig, builder_config: EthereumBuilderConfig) -> Self {
|
||||
Self { evm_config, builder_config }
|
||||
pub const fn new(
|
||||
client: Client,
|
||||
pool: Pool,
|
||||
evm_config: EvmConfig,
|
||||
builder_config: EthereumBuilderConfig,
|
||||
) -> Self {
|
||||
Self { client, pool, evm_config, builder_config }
|
||||
}
|
||||
}
|
||||
|
||||
impl<EvmConfig> EthereumPayloadBuilder<EvmConfig>
|
||||
impl<Pool, Client, EvmConfig> EthereumPayloadBuilder<Pool, Client, EvmConfig>
|
||||
where
|
||||
EvmConfig: ConfigureEvm<Header = Header>,
|
||||
{
|
||||
@ -94,10 +103,10 @@ where
|
||||
}
|
||||
|
||||
// Default implementation of [PayloadBuilder] for unit type
|
||||
impl<EvmConfig, Pool, Client> PayloadBuilder<Pool, Client> for EthereumPayloadBuilder<EvmConfig>
|
||||
impl<Pool, Client, EvmConfig> PayloadBuilder for EthereumPayloadBuilder<Pool, Client, EvmConfig>
|
||||
where
|
||||
EvmConfig: ConfigureEvm<Header = Header, Transaction = TransactionSigned>,
|
||||
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = ChainSpec>,
|
||||
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = ChainSpec> + Clone,
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>,
|
||||
{
|
||||
type Attributes = EthPayloadBuilderAttributes;
|
||||
@ -105,49 +114,41 @@ where
|
||||
|
||||
fn try_build(
|
||||
&self,
|
||||
args: BuildArguments<Pool, Client, EthPayloadBuilderAttributes, EthBuiltPayload>,
|
||||
args: BuildArguments<EthPayloadBuilderAttributes, EthBuiltPayload>,
|
||||
) -> Result<BuildOutcome<EthBuiltPayload>, PayloadBuilderError> {
|
||||
let evm_env = self
|
||||
.evm_env(&args.config, &args.config.parent_header)
|
||||
.map_err(PayloadBuilderError::other)?;
|
||||
|
||||
let pool = args.pool.clone();
|
||||
default_ethereum_payload(
|
||||
self.evm_config.clone(),
|
||||
self.client.clone(),
|
||||
self.pool.clone(),
|
||||
self.builder_config.clone(),
|
||||
args,
|
||||
evm_env,
|
||||
|attributes| pool.best_transactions_with_attributes(attributes),
|
||||
|attributes| self.pool.best_transactions_with_attributes(attributes),
|
||||
)
|
||||
}
|
||||
|
||||
fn build_empty_payload(
|
||||
&self,
|
||||
client: &Client,
|
||||
config: PayloadConfig<Self::Attributes>,
|
||||
) -> Result<EthBuiltPayload, PayloadBuilderError> {
|
||||
let args = BuildArguments::new(
|
||||
client,
|
||||
// we use defaults here because for the empty payload we don't need to execute anything
|
||||
NoopTransactionPool::default(),
|
||||
Default::default(),
|
||||
config,
|
||||
Default::default(),
|
||||
None,
|
||||
);
|
||||
let args = BuildArguments::new(Default::default(), config, Default::default(), None);
|
||||
|
||||
let evm_env = self
|
||||
.evm_env(&args.config, &args.config.parent_header)
|
||||
.map_err(PayloadBuilderError::other)?;
|
||||
|
||||
let pool = args.pool.clone();
|
||||
|
||||
default_ethereum_payload(
|
||||
self.evm_config.clone(),
|
||||
self.client.clone(),
|
||||
self.pool.clone(),
|
||||
self.builder_config.clone(),
|
||||
args,
|
||||
evm_env,
|
||||
|attributes| pool.best_transactions_with_attributes(attributes),
|
||||
|attributes| self.pool.best_transactions_with_attributes(attributes),
|
||||
)?
|
||||
.into_payload()
|
||||
.ok_or_else(|| PayloadBuilderError::MissingPayload)
|
||||
@ -160,10 +161,12 @@ where
|
||||
/// and configuration, this function creates a transaction payload. Returns
|
||||
/// a result indicating success with the payload or an error in case of failure.
|
||||
#[inline]
|
||||
pub fn default_ethereum_payload<EvmConfig, Pool, Client, F>(
|
||||
pub fn default_ethereum_payload<EvmConfig, Client, Pool, F>(
|
||||
evm_config: EvmConfig,
|
||||
client: Client,
|
||||
pool: Pool,
|
||||
builder_config: EthereumBuilderConfig,
|
||||
args: BuildArguments<Pool, Client, EthPayloadBuilderAttributes, EthBuiltPayload>,
|
||||
args: BuildArguments<EthPayloadBuilderAttributes, EthBuiltPayload>,
|
||||
evm_env: EvmEnv<EvmConfig::Spec>,
|
||||
best_txs: F,
|
||||
) -> Result<BuildOutcome<EthBuiltPayload>, PayloadBuilderError>
|
||||
@ -173,7 +176,7 @@ where
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>,
|
||||
F: FnOnce(BestTransactionsAttributes) -> BestTransactionsIter<Pool>,
|
||||
{
|
||||
let BuildArguments { client, pool, mut cached_reads, config, cancel, best_payload } = args;
|
||||
let BuildArguments { mut cached_reads, config, cancel, best_payload } = args;
|
||||
|
||||
let chain_spec = client.chain_spec();
|
||||
let state_provider = client.state_by_block_hash(config.parent_header.hash())?;
|
||||
|
||||
@ -210,6 +210,8 @@ where
|
||||
let Self { rpc_add_ons, da_config } = self;
|
||||
|
||||
let builder = reth_optimism_payload_builder::OpPayloadBuilder::new(
|
||||
ctx.node.pool().clone(),
|
||||
ctx.node.provider().clone(),
|
||||
ctx.node.evm_config().clone(),
|
||||
BasicOpReceiptBuilder::default(),
|
||||
);
|
||||
@ -507,6 +509,8 @@ where
|
||||
Txs: OpPayloadTransactions<TxTy<Node::Types>>,
|
||||
{
|
||||
let payload_builder = reth_optimism_payload_builder::OpPayloadBuilder::with_builder_config(
|
||||
pool,
|
||||
ctx.provider().clone(),
|
||||
evm_config,
|
||||
BasicOpReceiptBuilder::default(),
|
||||
OpBuilderConfig { da_config: self.da_config },
|
||||
@ -522,7 +526,6 @@ where
|
||||
|
||||
let payload_generator = BasicPayloadJobGenerator::with_builder(
|
||||
ctx.provider().clone(),
|
||||
pool,
|
||||
ctx.task_executor().clone(),
|
||||
payload_job_config,
|
||||
payload_builder,
|
||||
|
||||
@ -59,12 +59,16 @@ use tracing::{debug, trace, warn};
|
||||
|
||||
/// Optimism's payload builder
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct OpPayloadBuilder<EvmConfig, N: NodePrimitives, Txs = ()> {
|
||||
pub struct OpPayloadBuilder<Pool, Client, EvmConfig, N: NodePrimitives, Txs = ()> {
|
||||
/// The rollup's compute pending block configuration option.
|
||||
// TODO(clabby): Implement this feature.
|
||||
pub compute_pending_block: bool,
|
||||
/// The type responsible for creating the evm.
|
||||
pub evm_config: EvmConfig,
|
||||
/// Transaction pool.
|
||||
pub pool: Pool,
|
||||
/// Node client.
|
||||
pub client: Client,
|
||||
/// Settings for the builder, e.g. DA settings.
|
||||
pub config: OpBuilderConfig,
|
||||
/// The type responsible for yielding the best transactions for the payload if mempool
|
||||
@ -74,24 +78,30 @@ pub struct OpPayloadBuilder<EvmConfig, N: NodePrimitives, Txs = ()> {
|
||||
pub receipt_builder: Arc<dyn OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>>,
|
||||
}
|
||||
|
||||
impl<EvmConfig, N: NodePrimitives> OpPayloadBuilder<EvmConfig, N> {
|
||||
impl<Pool, Client, EvmConfig, N: NodePrimitives> OpPayloadBuilder<Pool, Client, EvmConfig, N> {
|
||||
/// `OpPayloadBuilder` constructor.
|
||||
///
|
||||
/// Configures the builder with the default settings.
|
||||
pub fn new(
|
||||
pool: Pool,
|
||||
client: Client,
|
||||
evm_config: EvmConfig,
|
||||
receipt_builder: impl OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>,
|
||||
) -> Self {
|
||||
Self::with_builder_config(evm_config, receipt_builder, Default::default())
|
||||
Self::with_builder_config(pool, client, evm_config, receipt_builder, Default::default())
|
||||
}
|
||||
|
||||
/// Configures the builder with the given [`OpBuilderConfig`].
|
||||
pub fn with_builder_config(
|
||||
pool: Pool,
|
||||
client: Client,
|
||||
evm_config: EvmConfig,
|
||||
receipt_builder: impl OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>,
|
||||
config: OpBuilderConfig,
|
||||
) -> Self {
|
||||
Self {
|
||||
pool,
|
||||
client,
|
||||
compute_pending_block: true,
|
||||
receipt_builder: Arc::new(receipt_builder),
|
||||
evm_config,
|
||||
@ -101,7 +111,9 @@ impl<EvmConfig, N: NodePrimitives> OpPayloadBuilder<EvmConfig, N> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<EvmConfig, N: NodePrimitives, Txs> OpPayloadBuilder<EvmConfig, N, Txs> {
|
||||
impl<Pool, Client, EvmConfig, N: NodePrimitives, Txs>
|
||||
OpPayloadBuilder<Pool, Client, EvmConfig, N, Txs>
|
||||
{
|
||||
/// Sets the rollup's compute pending block configuration option.
|
||||
pub const fn set_compute_pending_block(mut self, compute_pending_block: bool) -> Self {
|
||||
self.compute_pending_block = compute_pending_block;
|
||||
@ -113,9 +125,13 @@ impl<EvmConfig, N: NodePrimitives, Txs> OpPayloadBuilder<EvmConfig, N, Txs> {
|
||||
pub fn with_transactions<T: OpPayloadTransactions>(
|
||||
self,
|
||||
best_transactions: T,
|
||||
) -> OpPayloadBuilder<EvmConfig, N, T> {
|
||||
let Self { compute_pending_block, evm_config, config, receipt_builder, .. } = self;
|
||||
) -> OpPayloadBuilder<Pool, Client, EvmConfig, N, T> {
|
||||
let Self {
|
||||
pool, client, compute_pending_block, evm_config, config, receipt_builder, ..
|
||||
} = self;
|
||||
OpPayloadBuilder {
|
||||
pool,
|
||||
client,
|
||||
compute_pending_block,
|
||||
evm_config,
|
||||
best_transactions,
|
||||
@ -134,8 +150,9 @@ impl<EvmConfig, N: NodePrimitives, Txs> OpPayloadBuilder<EvmConfig, N, Txs> {
|
||||
self.compute_pending_block
|
||||
}
|
||||
}
|
||||
impl<EvmConfig, N, T> OpPayloadBuilder<EvmConfig, N, T>
|
||||
impl<Pool, Client, EvmConfig, N, T> OpPayloadBuilder<Pool, Client, EvmConfig, N, T>
|
||||
where
|
||||
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = OpChainSpec>,
|
||||
N: OpPayloadPrimitives,
|
||||
EvmConfig: ConfigureEvmFor<N>,
|
||||
{
|
||||
@ -147,31 +164,24 @@ where
|
||||
/// Given build arguments including an Optimism client, transaction pool,
|
||||
/// and configuration, this function creates a transaction payload. Returns
|
||||
/// a result indicating success with the payload or an error in case of failure.
|
||||
fn build_payload<'a, Client, Pool, Txs>(
|
||||
fn build_payload<'a, Txs>(
|
||||
&self,
|
||||
args: BuildArguments<
|
||||
Pool,
|
||||
Client,
|
||||
OpPayloadBuilderAttributes<N::SignedTx>,
|
||||
OpBuiltPayload<N>,
|
||||
>,
|
||||
args: BuildArguments<OpPayloadBuilderAttributes<N::SignedTx>, OpBuiltPayload<N>>,
|
||||
best: impl FnOnce(BestTransactionsAttributes) -> Txs + Send + Sync + 'a,
|
||||
) -> Result<BuildOutcome<OpBuiltPayload<N>>, PayloadBuilderError>
|
||||
where
|
||||
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = OpChainSpec>,
|
||||
Txs: PayloadTransactions<Transaction = N::SignedTx>,
|
||||
{
|
||||
let evm_env = self
|
||||
.evm_env(&args.config.attributes, &args.config.parent_header)
|
||||
.map_err(PayloadBuilderError::other)?;
|
||||
|
||||
let BuildArguments { client, pool: _, mut cached_reads, config, cancel, best_payload } =
|
||||
args;
|
||||
let BuildArguments { mut cached_reads, config, cancel, best_payload } = args;
|
||||
|
||||
let ctx = OpPayloadBuilderCtx {
|
||||
evm_config: self.evm_config.clone(),
|
||||
da_config: self.config.da_config.clone(),
|
||||
chain_spec: client.chain_spec(),
|
||||
chain_spec: self.client.chain_spec(),
|
||||
config,
|
||||
evm_env,
|
||||
cancel,
|
||||
@ -181,7 +191,7 @@ where
|
||||
|
||||
let builder = OpBuilder::new(best);
|
||||
|
||||
let state_provider = client.state_by_block_hash(ctx.parent().hash())?;
|
||||
let state_provider = self.client.state_by_block_hash(ctx.parent().hash())?;
|
||||
let state = StateProviderDatabase::new(state_provider);
|
||||
|
||||
if ctx.attributes().no_tx_pool {
|
||||
@ -215,15 +225,11 @@ where
|
||||
}
|
||||
|
||||
/// Computes the witness for the payload.
|
||||
pub fn payload_witness<Client>(
|
||||
pub fn payload_witness(
|
||||
&self,
|
||||
client: &Client,
|
||||
parent: SealedHeader,
|
||||
attributes: OpPayloadAttributes,
|
||||
) -> Result<ExecutionWitness, PayloadBuilderError>
|
||||
where
|
||||
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = OpChainSpec>,
|
||||
{
|
||||
) -> Result<ExecutionWitness, PayloadBuilderError> {
|
||||
let attributes = OpPayloadBuilderAttributes::try_new(parent.hash(), attributes, 3)
|
||||
.map_err(PayloadBuilderError::other)?;
|
||||
|
||||
@ -233,7 +239,7 @@ where
|
||||
let ctx: OpPayloadBuilderCtx<EvmConfig, N> = OpPayloadBuilderCtx {
|
||||
evm_config: self.evm_config.clone(),
|
||||
da_config: self.config.da_config.clone(),
|
||||
chain_spec: client.chain_spec(),
|
||||
chain_spec: self.client.chain_spec(),
|
||||
config,
|
||||
evm_env,
|
||||
cancel: Default::default(),
|
||||
@ -241,7 +247,7 @@ where
|
||||
receipt_builder: self.receipt_builder.clone(),
|
||||
};
|
||||
|
||||
let state_provider = client.state_by_block_hash(ctx.parent().hash())?;
|
||||
let state_provider = self.client.state_by_block_hash(ctx.parent().hash())?;
|
||||
let state = StateProviderDatabase::new(state_provider);
|
||||
let mut state = State::builder().with_database(state).with_bundle_update().build();
|
||||
|
||||
@ -251,10 +257,10 @@ where
|
||||
}
|
||||
|
||||
/// Implementation of the [`PayloadBuilder`] trait for [`OpPayloadBuilder`].
|
||||
impl<Pool, Client, EvmConfig, N, Txs> PayloadBuilder<Pool, Client>
|
||||
for OpPayloadBuilder<EvmConfig, N, Txs>
|
||||
impl<Pool, Client, EvmConfig, N, Txs> PayloadBuilder
|
||||
for OpPayloadBuilder<Pool, Client, EvmConfig, N, Txs>
|
||||
where
|
||||
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = OpChainSpec>,
|
||||
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = OpChainSpec> + Clone,
|
||||
N: OpPayloadPrimitives,
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = N::SignedTx>>,
|
||||
EvmConfig: ConfigureEvmFor<N>,
|
||||
@ -265,15 +271,15 @@ where
|
||||
|
||||
fn try_build(
|
||||
&self,
|
||||
args: BuildArguments<Pool, Client, Self::Attributes, Self::BuiltPayload>,
|
||||
args: BuildArguments<Self::Attributes, Self::BuiltPayload>,
|
||||
) -> Result<BuildOutcome<Self::BuiltPayload>, PayloadBuilderError> {
|
||||
let pool = args.pool.clone();
|
||||
let pool = self.pool.clone();
|
||||
self.build_payload(args, |attrs| self.best_transactions.best_transactions(pool, attrs))
|
||||
}
|
||||
|
||||
fn on_missing_payload(
|
||||
&self,
|
||||
_args: BuildArguments<Pool, Client, Self::Attributes, Self::BuiltPayload>,
|
||||
_args: BuildArguments<Self::Attributes, Self::BuiltPayload>,
|
||||
) -> MissingPayloadBehaviour<Self::BuiltPayload> {
|
||||
// we want to await the job that's already in progress because that should be returned as
|
||||
// is, there's no benefit in racing another job
|
||||
@ -284,14 +290,10 @@ where
|
||||
// system txs, hence on_missing_payload we return [MissingPayloadBehaviour::AwaitInProgress].
|
||||
fn build_empty_payload(
|
||||
&self,
|
||||
client: &Client,
|
||||
config: PayloadConfig<Self::Attributes>,
|
||||
) -> Result<Self::BuiltPayload, PayloadBuilderError> {
|
||||
let args = BuildArguments {
|
||||
client,
|
||||
config,
|
||||
// we use defaults here because for the empty payload we don't need to execute anything
|
||||
pool: (),
|
||||
cached_reads: Default::default(),
|
||||
cancel: Default::default(),
|
||||
best_payload: None,
|
||||
|
||||
@ -19,16 +19,18 @@ use std::{fmt::Debug, sync::Arc};
|
||||
use tokio::sync::{oneshot, Semaphore};
|
||||
|
||||
/// An extension to the `debug_` namespace of the RPC API.
|
||||
pub struct OpDebugWitnessApi<Provider: NodePrimitivesProvider, EvmConfig> {
|
||||
inner: Arc<OpDebugWitnessApiInner<Provider, EvmConfig>>,
|
||||
pub struct OpDebugWitnessApi<Pool, Provider: NodePrimitivesProvider, EvmConfig> {
|
||||
inner: Arc<OpDebugWitnessApiInner<Pool, Provider, EvmConfig>>,
|
||||
}
|
||||
|
||||
impl<Provider: NodePrimitivesProvider, EvmConfig> OpDebugWitnessApi<Provider, EvmConfig> {
|
||||
impl<Pool, Provider: NodePrimitivesProvider, EvmConfig>
|
||||
OpDebugWitnessApi<Pool, Provider, EvmConfig>
|
||||
{
|
||||
/// Creates a new instance of the `OpDebugWitnessApi`.
|
||||
pub fn new(
|
||||
provider: Provider,
|
||||
task_spawner: Box<dyn TaskSpawner>,
|
||||
builder: OpPayloadBuilder<EvmConfig, Provider::Primitives>,
|
||||
builder: OpPayloadBuilder<Pool, Provider, EvmConfig, Provider::Primitives>,
|
||||
) -> Self {
|
||||
let semaphore = Arc::new(Semaphore::new(3));
|
||||
let inner = OpDebugWitnessApiInner { provider, builder, task_spawner, semaphore };
|
||||
@ -36,7 +38,7 @@ impl<Provider: NodePrimitivesProvider, EvmConfig> OpDebugWitnessApi<Provider, Ev
|
||||
}
|
||||
}
|
||||
|
||||
impl<Provider, EvmConfig> OpDebugWitnessApi<Provider, EvmConfig>
|
||||
impl<Pool, Provider, EvmConfig> OpDebugWitnessApi<Pool, Provider, EvmConfig>
|
||||
where
|
||||
Provider: NodePrimitivesProvider + BlockReaderIdExt<Header = reth_primitives::Header>,
|
||||
{
|
||||
@ -50,9 +52,10 @@ where
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl<Provider, EvmConfig> DebugExecutionWitnessApiServer<OpPayloadAttributes>
|
||||
for OpDebugWitnessApi<Provider, EvmConfig>
|
||||
impl<Pool, Provider, EvmConfig> DebugExecutionWitnessApiServer<OpPayloadAttributes>
|
||||
for OpDebugWitnessApi<Pool, Provider, EvmConfig>
|
||||
where
|
||||
Pool: Send + Sync + 'static,
|
||||
Provider: BlockReaderIdExt<Header = reth_primitives::Header>
|
||||
+ NodePrimitivesProvider<Primitives: OpPayloadPrimitives>
|
||||
+ StateProviderFactory
|
||||
@ -73,8 +76,7 @@ where
|
||||
let (tx, rx) = oneshot::channel();
|
||||
let this = self.clone();
|
||||
self.inner.task_spawner.spawn_blocking(Box::pin(async move {
|
||||
let res =
|
||||
this.inner.builder.payload_witness(&this.inner.provider, parent_header, attributes);
|
||||
let res = this.inner.builder.payload_witness(parent_header, attributes);
|
||||
let _ = tx.send(res);
|
||||
}));
|
||||
|
||||
@ -84,7 +86,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<Provider, EvmConfig> Clone for OpDebugWitnessApi<Provider, EvmConfig>
|
||||
impl<Pool, Provider, EvmConfig> Clone for OpDebugWitnessApi<Pool, Provider, EvmConfig>
|
||||
where
|
||||
Provider: NodePrimitivesProvider,
|
||||
{
|
||||
@ -92,7 +94,7 @@ where
|
||||
Self { inner: Arc::clone(&self.inner) }
|
||||
}
|
||||
}
|
||||
impl<Provider, EvmConfig> Debug for OpDebugWitnessApi<Provider, EvmConfig>
|
||||
impl<Pool, Provider, EvmConfig> Debug for OpDebugWitnessApi<Pool, Provider, EvmConfig>
|
||||
where
|
||||
Provider: NodePrimitivesProvider,
|
||||
{
|
||||
@ -101,9 +103,9 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
struct OpDebugWitnessApiInner<Provider: NodePrimitivesProvider, EvmConfig> {
|
||||
struct OpDebugWitnessApiInner<Pool, Provider: NodePrimitivesProvider, EvmConfig> {
|
||||
provider: Provider,
|
||||
builder: OpPayloadBuilder<EvmConfig, Provider::Primitives>,
|
||||
builder: OpPayloadBuilder<Pool, Provider, EvmConfig, Provider::Primitives>,
|
||||
task_spawner: Box<dyn TaskSpawner>,
|
||||
semaphore: Arc<Semaphore>,
|
||||
}
|
||||
|
||||
@ -16,7 +16,6 @@ workspace = true
|
||||
reth-chainspec.workspace = true
|
||||
reth-primitives.workspace = true
|
||||
reth-primitives-traits.workspace = true
|
||||
reth-transaction-pool.workspace = true
|
||||
reth-provider.workspace = true
|
||||
reth-payload-builder.workspace = true
|
||||
reth-payload-builder-primitives.workspace = true
|
||||
|
||||
@ -24,7 +24,6 @@ use reth_primitives_traits::proofs;
|
||||
use reth_provider::{BlockReaderIdExt, CanonStateNotification, StateProviderFactory};
|
||||
use reth_revm::{cached::CachedReads, cancelled::CancelOnDrop};
|
||||
use reth_tasks::TaskSpawner;
|
||||
use reth_transaction_pool::TransactionPool;
|
||||
use revm::{Database, State};
|
||||
use std::{
|
||||
fmt,
|
||||
@ -48,11 +47,9 @@ pub use stack::PayloadBuilderStack;
|
||||
|
||||
/// The [`PayloadJobGenerator`] that creates [`BasicPayloadJob`]s.
|
||||
#[derive(Debug)]
|
||||
pub struct BasicPayloadJobGenerator<Client, Pool, Tasks, Builder> {
|
||||
pub struct BasicPayloadJobGenerator<Client, Tasks, Builder> {
|
||||
/// The client that can interact with the chain.
|
||||
client: Client,
|
||||
/// The transaction pool to pull transactions from.
|
||||
pool: Pool,
|
||||
/// The task executor to spawn payload building tasks on.
|
||||
executor: Tasks,
|
||||
/// The configuration for the job generator.
|
||||
@ -69,19 +66,17 @@ pub struct BasicPayloadJobGenerator<Client, Pool, Tasks, Builder> {
|
||||
|
||||
// === impl BasicPayloadJobGenerator ===
|
||||
|
||||
impl<Client, Pool, Tasks, Builder> BasicPayloadJobGenerator<Client, Pool, Tasks, Builder> {
|
||||
impl<Client, Tasks, Builder> BasicPayloadJobGenerator<Client, Tasks, Builder> {
|
||||
/// Creates a new [`BasicPayloadJobGenerator`] with the given config and custom
|
||||
/// [`PayloadBuilder`]
|
||||
pub fn with_builder(
|
||||
client: Client,
|
||||
pool: Pool,
|
||||
executor: Tasks,
|
||||
config: BasicPayloadJobGeneratorConfig,
|
||||
builder: Builder,
|
||||
) -> Self {
|
||||
Self {
|
||||
client,
|
||||
pool,
|
||||
executor,
|
||||
payload_task_guard: PayloadTaskGuard::new(config.max_payload_tasks),
|
||||
config,
|
||||
@ -129,21 +124,20 @@ impl<Client, Pool, Tasks, Builder> BasicPayloadJobGenerator<Client, Pool, Tasks,
|
||||
|
||||
// === impl BasicPayloadJobGenerator ===
|
||||
|
||||
impl<Client, Pool, Tasks, Builder> PayloadJobGenerator
|
||||
for BasicPayloadJobGenerator<Client, Pool, Tasks, Builder>
|
||||
impl<Client, Tasks, Builder> PayloadJobGenerator
|
||||
for BasicPayloadJobGenerator<Client, Tasks, Builder>
|
||||
where
|
||||
Client: StateProviderFactory
|
||||
+ BlockReaderIdExt<Header = alloy_consensus::Header>
|
||||
+ Clone
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
Pool: TransactionPool + Unpin + 'static,
|
||||
Tasks: TaskSpawner + Clone + Unpin + 'static,
|
||||
Builder: PayloadBuilder<Pool, Client> + Unpin + 'static,
|
||||
<Builder as PayloadBuilder<Pool, Client>>::Attributes: Unpin + Clone,
|
||||
<Builder as PayloadBuilder<Pool, Client>>::BuiltPayload: Unpin + Clone,
|
||||
Builder: PayloadBuilder + Unpin + 'static,
|
||||
Builder::Attributes: Unpin + Clone,
|
||||
Builder::BuiltPayload: Unpin + Clone,
|
||||
{
|
||||
type Job = BasicPayloadJob<Client, Pool, Tasks, Builder>;
|
||||
type Job = BasicPayloadJob<Tasks, Builder>;
|
||||
|
||||
fn new_payload_job(
|
||||
&self,
|
||||
@ -172,8 +166,6 @@ where
|
||||
|
||||
let mut job = BasicPayloadJob {
|
||||
config,
|
||||
client: self.client.clone(),
|
||||
pool: self.pool.clone(),
|
||||
executor: self.executor.clone(),
|
||||
deadline,
|
||||
// ticks immediately
|
||||
@ -306,16 +298,12 @@ impl Default for BasicPayloadJobGeneratorConfig {
|
||||
/// built and this future will wait to be resolved: [`PayloadJob::resolve`] or terminated if the
|
||||
/// deadline is reached..
|
||||
#[derive(Debug)]
|
||||
pub struct BasicPayloadJob<Client, Pool, Tasks, Builder>
|
||||
pub struct BasicPayloadJob<Tasks, Builder>
|
||||
where
|
||||
Builder: PayloadBuilder<Pool, Client>,
|
||||
Builder: PayloadBuilder,
|
||||
{
|
||||
/// The configuration for how the payload will be created.
|
||||
config: PayloadConfig<Builder::Attributes>,
|
||||
/// The client that can interact with the chain.
|
||||
client: Client,
|
||||
/// The transaction pool.
|
||||
pool: Pool,
|
||||
/// How to spawn building tasks
|
||||
executor: Tasks,
|
||||
/// The deadline when this job should resolve.
|
||||
@ -341,21 +329,17 @@ where
|
||||
builder: Builder,
|
||||
}
|
||||
|
||||
impl<Client, Pool, Tasks, Builder> BasicPayloadJob<Client, Pool, Tasks, Builder>
|
||||
impl<Tasks, Builder> BasicPayloadJob<Tasks, Builder>
|
||||
where
|
||||
Client: StateProviderFactory + Clone + Unpin + 'static,
|
||||
Pool: TransactionPool + Unpin + 'static,
|
||||
Tasks: TaskSpawner + Clone + 'static,
|
||||
Builder: PayloadBuilder<Pool, Client> + Unpin + 'static,
|
||||
<Builder as PayloadBuilder<Pool, Client>>::Attributes: Unpin + Clone,
|
||||
<Builder as PayloadBuilder<Pool, Client>>::BuiltPayload: Unpin + Clone,
|
||||
Builder: PayloadBuilder + Unpin + 'static,
|
||||
Builder::Attributes: Unpin + Clone,
|
||||
Builder::BuiltPayload: Unpin + Clone,
|
||||
{
|
||||
/// Spawns a new payload build task.
|
||||
fn spawn_build_job(&mut self) {
|
||||
trace!(target: "payload_builder", id = %self.config.payload_id(), "spawn new payload build task");
|
||||
let (tx, rx) = oneshot::channel();
|
||||
let client = self.client.clone();
|
||||
let pool = self.pool.clone();
|
||||
let cancel = CancelOnDrop::default();
|
||||
let _cancel = cancel.clone();
|
||||
let guard = self.payload_task_guard.clone();
|
||||
@ -367,14 +351,8 @@ where
|
||||
self.executor.spawn_blocking(Box::pin(async move {
|
||||
// acquire the permit for executing the task
|
||||
let _permit = guard.acquire().await;
|
||||
let args = BuildArguments {
|
||||
client,
|
||||
pool,
|
||||
cached_reads,
|
||||
config: payload_config,
|
||||
cancel,
|
||||
best_payload,
|
||||
};
|
||||
let args =
|
||||
BuildArguments { cached_reads, config: payload_config, cancel, best_payload };
|
||||
let result = builder.try_build(args);
|
||||
let _ = tx.send(result);
|
||||
}));
|
||||
@ -383,14 +361,12 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<Client, Pool, Tasks, Builder> Future for BasicPayloadJob<Client, Pool, Tasks, Builder>
|
||||
impl<Tasks, Builder> Future for BasicPayloadJob<Tasks, Builder>
|
||||
where
|
||||
Client: StateProviderFactory + Clone + Unpin + 'static,
|
||||
Pool: TransactionPool + Unpin + 'static,
|
||||
Tasks: TaskSpawner + Clone + 'static,
|
||||
Builder: PayloadBuilder<Pool, Client> + Unpin + 'static,
|
||||
<Builder as PayloadBuilder<Pool, Client>>::Attributes: Unpin + Clone,
|
||||
<Builder as PayloadBuilder<Pool, Client>>::BuiltPayload: Unpin + Clone,
|
||||
Builder: PayloadBuilder + Unpin + 'static,
|
||||
Builder::Attributes: Unpin + Clone,
|
||||
Builder::BuiltPayload: Unpin + Clone,
|
||||
{
|
||||
type Output = Result<(), PayloadBuilderError>;
|
||||
|
||||
@ -448,14 +424,12 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<Client, Pool, Tasks, Builder> PayloadJob for BasicPayloadJob<Client, Pool, Tasks, Builder>
|
||||
impl<Tasks, Builder> PayloadJob for BasicPayloadJob<Tasks, Builder>
|
||||
where
|
||||
Client: StateProviderFactory + Clone + Unpin + 'static,
|
||||
Pool: TransactionPool + Unpin + 'static,
|
||||
Tasks: TaskSpawner + Clone + 'static,
|
||||
Builder: PayloadBuilder<Pool, Client> + Unpin + 'static,
|
||||
<Builder as PayloadBuilder<Pool, Client>>::Attributes: Unpin + Clone,
|
||||
<Builder as PayloadBuilder<Pool, Client>>::BuiltPayload: Unpin + Clone,
|
||||
Builder: PayloadBuilder + Unpin + 'static,
|
||||
Builder::Attributes: Unpin + Clone,
|
||||
Builder::BuiltPayload: Unpin + Clone,
|
||||
{
|
||||
type PayloadAttributes = Builder::Attributes;
|
||||
type ResolvePayloadFuture = ResolveBestPayload<Self::BuiltPayload>;
|
||||
@ -472,7 +446,7 @@ where
|
||||
// started right away and the first full block should have been
|
||||
// built by the time CL is requesting the payload.
|
||||
self.metrics.inc_requested_empty_payload();
|
||||
self.builder.build_empty_payload(&self.client, self.config.clone())
|
||||
self.builder.build_empty_payload(self.config.clone())
|
||||
}
|
||||
}
|
||||
|
||||
@ -497,8 +471,6 @@ where
|
||||
debug!(target: "payload_builder", id=%self.config.payload_id(), "no best payload yet to resolve, building empty payload");
|
||||
|
||||
let args = BuildArguments {
|
||||
client: self.client.clone(),
|
||||
pool: self.pool.clone(),
|
||||
cached_reads: self.cached_reads.take().unwrap_or_default(),
|
||||
config: self.config.clone(),
|
||||
cancel: CancelOnDrop::default(),
|
||||
@ -516,11 +488,10 @@ where
|
||||
self.metrics.inc_requested_empty_payload();
|
||||
// no payload built yet, so we need to return an empty payload
|
||||
let (tx, rx) = oneshot::channel();
|
||||
let client = self.client.clone();
|
||||
let config = self.config.clone();
|
||||
let builder = self.builder.clone();
|
||||
self.executor.spawn_blocking(Box::pin(async move {
|
||||
let res = builder.build_empty_payload(&client, config);
|
||||
let res = builder.build_empty_payload(config);
|
||||
let _ = tx.send(res);
|
||||
}));
|
||||
|
||||
@ -806,13 +777,7 @@ impl<Payload> BuildOutcomeKind<Payload> {
|
||||
/// building process. It holds references to the Ethereum client, transaction pool, cached reads,
|
||||
/// payload configuration, cancellation status, and the best payload achieved so far.
|
||||
#[derive(Debug)]
|
||||
pub struct BuildArguments<Pool, Client, Attributes, Payload> {
|
||||
/// How to interact with the chain.
|
||||
pub client: Client,
|
||||
/// The transaction pool.
|
||||
///
|
||||
/// Or the type that provides the transactions to build the payload.
|
||||
pub pool: Pool,
|
||||
pub struct BuildArguments<Attributes, Payload> {
|
||||
/// Previously cached disk reads
|
||||
pub cached_reads: CachedReads,
|
||||
/// How to configure the payload.
|
||||
@ -823,44 +788,15 @@ pub struct BuildArguments<Pool, Client, Attributes, Payload> {
|
||||
pub best_payload: Option<Payload>,
|
||||
}
|
||||
|
||||
impl<Pool, Client, Attributes, Payload> BuildArguments<Pool, Client, Attributes, Payload> {
|
||||
impl<Attributes, Payload> BuildArguments<Attributes, Payload> {
|
||||
/// Create new build arguments.
|
||||
pub const fn new(
|
||||
client: Client,
|
||||
pool: Pool,
|
||||
cached_reads: CachedReads,
|
||||
config: PayloadConfig<Attributes>,
|
||||
cancel: CancelOnDrop,
|
||||
best_payload: Option<Payload>,
|
||||
) -> Self {
|
||||
Self { client, pool, cached_reads, config, cancel, best_payload }
|
||||
}
|
||||
|
||||
/// Maps the transaction pool to a new type.
|
||||
pub fn with_pool<P>(self, pool: P) -> BuildArguments<P, Client, Attributes, Payload> {
|
||||
BuildArguments {
|
||||
client: self.client,
|
||||
pool,
|
||||
cached_reads: self.cached_reads,
|
||||
config: self.config,
|
||||
cancel: self.cancel,
|
||||
best_payload: self.best_payload,
|
||||
}
|
||||
}
|
||||
|
||||
/// Maps the transaction pool to a new type using a closure with the current pool type as input.
|
||||
pub fn map_pool<F, P>(self, f: F) -> BuildArguments<P, Client, Attributes, Payload>
|
||||
where
|
||||
F: FnOnce(Pool) -> P,
|
||||
{
|
||||
BuildArguments {
|
||||
client: self.client,
|
||||
pool: f(self.pool),
|
||||
cached_reads: self.cached_reads,
|
||||
config: self.config,
|
||||
cancel: self.cancel,
|
||||
best_payload: self.best_payload,
|
||||
}
|
||||
Self { cached_reads, config, cancel, best_payload }
|
||||
}
|
||||
}
|
||||
|
||||
@ -872,7 +808,7 @@ impl<Pool, Client, Attributes, Payload> BuildArguments<Pool, Client, Attributes,
|
||||
///
|
||||
/// Generic parameters `Pool` and `Client` represent the transaction pool and
|
||||
/// Ethereum client types.
|
||||
pub trait PayloadBuilder<Pool, Client>: Send + Sync + Clone {
|
||||
pub trait PayloadBuilder: Send + Sync + Clone {
|
||||
/// The payload attributes type to accept for building.
|
||||
type Attributes: PayloadBuilderAttributes;
|
||||
/// The type of the built payload.
|
||||
@ -892,7 +828,7 @@ pub trait PayloadBuilder<Pool, Client>: Send + Sync + Clone {
|
||||
/// A `Result` indicating the build outcome or an error.
|
||||
fn try_build(
|
||||
&self,
|
||||
args: BuildArguments<Pool, Client, Self::Attributes, Self::BuiltPayload>,
|
||||
args: BuildArguments<Self::Attributes, Self::BuiltPayload>,
|
||||
) -> Result<BuildOutcome<Self::BuiltPayload>, PayloadBuilderError>;
|
||||
|
||||
/// Invoked when the payload job is being resolved and there is no payload yet.
|
||||
@ -900,7 +836,7 @@ pub trait PayloadBuilder<Pool, Client>: Send + Sync + Clone {
|
||||
/// This can happen if the CL requests a payload before the first payload has been built.
|
||||
fn on_missing_payload(
|
||||
&self,
|
||||
_args: BuildArguments<Pool, Client, Self::Attributes, Self::BuiltPayload>,
|
||||
_args: BuildArguments<Self::Attributes, Self::BuiltPayload>,
|
||||
) -> MissingPayloadBehaviour<Self::BuiltPayload> {
|
||||
MissingPayloadBehaviour::RaceEmptyPayload
|
||||
}
|
||||
@ -908,7 +844,6 @@ pub trait PayloadBuilder<Pool, Client>: Send + Sync + Clone {
|
||||
/// Builds an empty payload without any transaction.
|
||||
fn build_empty_payload(
|
||||
&self,
|
||||
client: &Client,
|
||||
config: PayloadConfig<Self::Attributes>,
|
||||
) -> Result<Self::BuiltPayload, PayloadBuilderError>;
|
||||
}
|
||||
|
||||
@ -177,33 +177,26 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<L, R, Pool, Client> PayloadBuilder<Pool, Client> for PayloadBuilderStack<L, R>
|
||||
impl<L, R> PayloadBuilder for PayloadBuilderStack<L, R>
|
||||
where
|
||||
L: PayloadBuilder<Pool, Client> + Unpin + 'static,
|
||||
R: PayloadBuilder<Pool, Client> + Unpin + 'static,
|
||||
Client: Clone,
|
||||
Pool: Clone,
|
||||
L: PayloadBuilder + Unpin + 'static,
|
||||
R: PayloadBuilder + Unpin + 'static,
|
||||
L::Attributes: Unpin + Clone,
|
||||
R::Attributes: Unpin + Clone,
|
||||
L::BuiltPayload: Unpin + Clone,
|
||||
R::BuiltPayload:
|
||||
BuiltPayload<Primitives = <L::BuiltPayload as BuiltPayload>::Primitives> + Unpin + Clone,
|
||||
<<L as PayloadBuilder<Pool, Client>>::Attributes as PayloadBuilderAttributes>::Error: 'static,
|
||||
<<R as PayloadBuilder<Pool, Client>>::Attributes as PayloadBuilderAttributes>::Error: 'static,
|
||||
{
|
||||
type Attributes = Either<L::Attributes, R::Attributes>;
|
||||
type BuiltPayload = Either<L::BuiltPayload, R::BuiltPayload>;
|
||||
|
||||
fn try_build(
|
||||
&self,
|
||||
args: BuildArguments<Pool, Client, Self::Attributes, Self::BuiltPayload>,
|
||||
args: BuildArguments<Self::Attributes, Self::BuiltPayload>,
|
||||
) -> Result<BuildOutcome<Self::BuiltPayload>, PayloadBuilderError> {
|
||||
match args.config.attributes {
|
||||
Either::Left(ref left_attr) => {
|
||||
let left_args: BuildArguments<Pool, Client, L::Attributes, L::BuiltPayload> =
|
||||
BuildArguments {
|
||||
client: args.client.clone(),
|
||||
pool: args.pool.clone(),
|
||||
let left_args: BuildArguments<L::Attributes, L::BuiltPayload> = BuildArguments {
|
||||
cached_reads: args.cached_reads.clone(),
|
||||
config: PayloadConfig {
|
||||
parent_header: args.config.parent_header.clone(),
|
||||
@ -223,8 +216,6 @@ where
|
||||
}
|
||||
Either::Right(ref right_attr) => {
|
||||
let right_args = BuildArguments {
|
||||
client: args.client.clone(),
|
||||
pool: args.pool.clone(),
|
||||
cached_reads: args.cached_reads.clone(),
|
||||
config: PayloadConfig {
|
||||
parent_header: args.config.parent_header.clone(),
|
||||
@ -247,7 +238,6 @@ where
|
||||
|
||||
fn build_empty_payload(
|
||||
&self,
|
||||
client: &Client,
|
||||
config: PayloadConfig<Self::Attributes>,
|
||||
) -> Result<Self::BuiltPayload, PayloadBuilderError> {
|
||||
match config.attributes {
|
||||
@ -256,14 +246,14 @@ where
|
||||
parent_header: config.parent_header.clone(),
|
||||
attributes: left_attr,
|
||||
};
|
||||
self.left.build_empty_payload(client, left_config).map(Either::Left)
|
||||
self.left.build_empty_payload(left_config).map(Either::Left)
|
||||
}
|
||||
Either::Right(right_attr) => {
|
||||
let right_config = PayloadConfig {
|
||||
parent_header: config.parent_header.clone(),
|
||||
attributes: right_attr,
|
||||
};
|
||||
self.right.build_empty_payload(client, right_config).map(Either::Right)
|
||||
self.right.build_empty_payload(right_config).map(Either::Right)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -372,7 +372,14 @@ where
|
||||
ctx: &BuilderContext<Node>,
|
||||
pool: Pool,
|
||||
) -> eyre::Result<PayloadBuilderHandle<<Node::Types as NodeTypesWithEngine>::Engine>> {
|
||||
let payload_builder = CustomPayloadBuilder::default();
|
||||
let payload_builder = CustomPayloadBuilder {
|
||||
inner: reth_ethereum_payload_builder::EthereumPayloadBuilder::new(
|
||||
ctx.provider().clone(),
|
||||
pool,
|
||||
EthEvmConfig::new(ctx.provider().chain_spec().clone()),
|
||||
EthereumBuilderConfig::new(default_extra_data_bytes()),
|
||||
),
|
||||
};
|
||||
let conf = ctx.payload_builder_config();
|
||||
|
||||
let payload_job_config = BasicPayloadJobGeneratorConfig::default()
|
||||
@ -382,7 +389,6 @@ where
|
||||
|
||||
let payload_generator = BasicPayloadJobGenerator::with_builder(
|
||||
ctx.provider().clone(),
|
||||
pool,
|
||||
ctx.task_executor().clone(),
|
||||
payload_job_config,
|
||||
payload_builder,
|
||||
@ -397,13 +403,15 @@ where
|
||||
}
|
||||
|
||||
/// The type responsible for building custom payloads
|
||||
#[derive(Debug, Default, Clone)]
|
||||
#[derive(Debug, Clone)]
|
||||
#[non_exhaustive]
|
||||
pub struct CustomPayloadBuilder;
|
||||
pub struct CustomPayloadBuilder<Pool, Client> {
|
||||
inner: reth_ethereum_payload_builder::EthereumPayloadBuilder<Pool, Client>,
|
||||
}
|
||||
|
||||
impl<Pool, Client> PayloadBuilder<Pool, Client> for CustomPayloadBuilder
|
||||
impl<Pool, Client> PayloadBuilder for CustomPayloadBuilder<Pool, Client>
|
||||
where
|
||||
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = ChainSpec>,
|
||||
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = ChainSpec> + Clone,
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>,
|
||||
{
|
||||
type Attributes = CustomPayloadBuilderAttributes;
|
||||
@ -411,22 +419,14 @@ where
|
||||
|
||||
fn try_build(
|
||||
&self,
|
||||
args: BuildArguments<Pool, Client, Self::Attributes, Self::BuiltPayload>,
|
||||
args: BuildArguments<Self::Attributes, Self::BuiltPayload>,
|
||||
) -> Result<BuildOutcome<Self::BuiltPayload>, PayloadBuilderError> {
|
||||
let BuildArguments { client, pool, cached_reads, config, cancel, best_payload } = args;
|
||||
let BuildArguments { cached_reads, config, cancel, best_payload } = args;
|
||||
let PayloadConfig { parent_header, attributes } = config;
|
||||
|
||||
let chain_spec = client.chain_spec();
|
||||
|
||||
// This reuses the default EthereumPayloadBuilder to build the payload
|
||||
// but any custom logic can be implemented here
|
||||
reth_ethereum_payload_builder::EthereumPayloadBuilder::new(
|
||||
EthEvmConfig::new(chain_spec.clone()),
|
||||
EthereumBuilderConfig::new(default_extra_data_bytes()),
|
||||
)
|
||||
.try_build(BuildArguments {
|
||||
client,
|
||||
pool,
|
||||
self.inner.try_build(BuildArguments {
|
||||
cached_reads,
|
||||
config: PayloadConfig { parent_header, attributes: attributes.0 },
|
||||
cancel,
|
||||
@ -436,19 +436,10 @@ where
|
||||
|
||||
fn build_empty_payload(
|
||||
&self,
|
||||
client: &Client,
|
||||
config: PayloadConfig<Self::Attributes>,
|
||||
) -> Result<Self::BuiltPayload, PayloadBuilderError> {
|
||||
let PayloadConfig { parent_header, attributes } = config;
|
||||
let chain_spec = client.chain_spec();
|
||||
<reth_ethereum_payload_builder::EthereumPayloadBuilder as PayloadBuilder<Pool, Client>>::build_empty_payload(
|
||||
&reth_ethereum_payload_builder::EthereumPayloadBuilder::new(
|
||||
EthEvmConfig::new(chain_spec.clone()),
|
||||
EthereumBuilderConfig::new(default_extra_data_bytes())
|
||||
),
|
||||
client,
|
||||
PayloadConfig { parent_header, attributes: attributes.0}
|
||||
)
|
||||
self.inner.build_empty_payload(PayloadConfig { parent_header, attributes: attributes.0 })
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -4,7 +4,6 @@ use reth::{
|
||||
api::Block,
|
||||
providers::{BlockReaderIdExt, BlockSource, StateProviderFactory},
|
||||
tasks::TaskSpawner,
|
||||
transaction_pool::TransactionPool,
|
||||
};
|
||||
use reth_basic_payload_builder::{BasicPayloadJobGeneratorConfig, PayloadBuilder, PayloadConfig};
|
||||
use reth_node_api::PayloadBuilderAttributes;
|
||||
@ -14,11 +13,9 @@ use std::sync::Arc;
|
||||
|
||||
/// The generator type that creates new jobs that builds empty blocks.
|
||||
#[derive(Debug)]
|
||||
pub struct EmptyBlockPayloadJobGenerator<Client, Pool, Tasks, Builder> {
|
||||
pub struct EmptyBlockPayloadJobGenerator<Client, Tasks, Builder> {
|
||||
/// The client that can interact with the chain.
|
||||
client: Client,
|
||||
/// txpool
|
||||
pool: Pool,
|
||||
/// How to spawn building tasks
|
||||
executor: Tasks,
|
||||
/// The configuration for the job generator.
|
||||
@ -31,41 +28,39 @@ pub struct EmptyBlockPayloadJobGenerator<Client, Pool, Tasks, Builder> {
|
||||
|
||||
// === impl EmptyBlockPayloadJobGenerator ===
|
||||
|
||||
impl<Client, Pool, Tasks, Builder> EmptyBlockPayloadJobGenerator<Client, Pool, Tasks, Builder> {
|
||||
impl<Client, Tasks, Builder> EmptyBlockPayloadJobGenerator<Client, Tasks, Builder> {
|
||||
/// Creates a new [EmptyBlockPayloadJobGenerator] with the given config and custom
|
||||
/// [PayloadBuilder]
|
||||
pub fn with_builder(
|
||||
client: Client,
|
||||
pool: Pool,
|
||||
executor: Tasks,
|
||||
config: BasicPayloadJobGeneratorConfig,
|
||||
builder: Builder,
|
||||
) -> Self {
|
||||
Self { client, pool, executor, _config: config, builder }
|
||||
Self { client, executor, _config: config, builder }
|
||||
}
|
||||
}
|
||||
|
||||
impl<Client, Pool, Tasks, Builder> PayloadJobGenerator
|
||||
for EmptyBlockPayloadJobGenerator<Client, Pool, Tasks, Builder>
|
||||
impl<Client, Tasks, Builder> PayloadJobGenerator
|
||||
for EmptyBlockPayloadJobGenerator<Client, Tasks, Builder>
|
||||
where
|
||||
Client: StateProviderFactory
|
||||
+ BlockReaderIdExt<Block = reth_primitives::Block>
|
||||
+ Clone
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
Pool: TransactionPool + Unpin + 'static,
|
||||
Tasks: TaskSpawner + Clone + Unpin + 'static,
|
||||
Builder: PayloadBuilder<Pool, Client> + Unpin + 'static,
|
||||
<Builder as PayloadBuilder<Pool, Client>>::Attributes: Unpin + Clone,
|
||||
<Builder as PayloadBuilder<Pool, Client>>::BuiltPayload: Unpin + Clone,
|
||||
Builder: PayloadBuilder + Unpin + 'static,
|
||||
Builder::Attributes: Unpin + Clone,
|
||||
Builder::BuiltPayload: Unpin + Clone,
|
||||
{
|
||||
type Job = EmptyBlockPayloadJob<Client, Pool, Tasks, Builder>;
|
||||
type Job = EmptyBlockPayloadJob<Tasks, Builder>;
|
||||
|
||||
/// This is invoked when the node receives payload attributes from the beacon node via
|
||||
/// `engine_forkchoiceUpdatedV1`
|
||||
fn new_payload_job(
|
||||
&self,
|
||||
attributes: <Builder as PayloadBuilder<Pool, Client>>::Attributes,
|
||||
attributes: Builder::Attributes,
|
||||
) -> Result<Self::Job, PayloadBuilderError> {
|
||||
let parent_block = if attributes.parent().is_zero() {
|
||||
// use latest block if parent is zero: genesis block
|
||||
@ -87,8 +82,6 @@ where
|
||||
|
||||
let config = PayloadConfig::new(Arc::new(header), attributes);
|
||||
Ok(EmptyBlockPayloadJob {
|
||||
client: self.client.clone(),
|
||||
_pool: self.pool.clone(),
|
||||
_executor: self.executor.clone(),
|
||||
builder: self.builder.clone(),
|
||||
config,
|
||||
|
||||
@ -1,7 +1,5 @@
|
||||
use futures_util::Future;
|
||||
use reth::{
|
||||
providers::StateProviderFactory, tasks::TaskSpawner, transaction_pool::TransactionPool,
|
||||
};
|
||||
use reth::tasks::TaskSpawner;
|
||||
use reth_basic_payload_builder::{PayloadBuilder, PayloadConfig};
|
||||
use reth_node_api::PayloadKind;
|
||||
use reth_payload_builder::{KeepPayloadJobAlive, PayloadBuilderError, PayloadJob};
|
||||
@ -12,16 +10,12 @@ use std::{
|
||||
};
|
||||
|
||||
/// A [PayloadJob] that builds empty blocks.
|
||||
pub struct EmptyBlockPayloadJob<Client, Pool, Tasks, Builder>
|
||||
pub struct EmptyBlockPayloadJob<Tasks, Builder>
|
||||
where
|
||||
Builder: PayloadBuilder<Pool, Client>,
|
||||
Builder: PayloadBuilder,
|
||||
{
|
||||
/// The configuration for how the payload will be created.
|
||||
pub(crate) config: PayloadConfig<Builder::Attributes>,
|
||||
/// The client that can interact with the chain.
|
||||
pub(crate) client: Client,
|
||||
/// The transaction pool.
|
||||
pub(crate) _pool: Pool,
|
||||
/// How to spawn building tasks
|
||||
pub(crate) _executor: Tasks,
|
||||
/// The type responsible for building payloads.
|
||||
@ -30,14 +24,12 @@ where
|
||||
pub(crate) builder: Builder,
|
||||
}
|
||||
|
||||
impl<Client, Pool, Tasks, Builder> PayloadJob for EmptyBlockPayloadJob<Client, Pool, Tasks, Builder>
|
||||
impl<Tasks, Builder> PayloadJob for EmptyBlockPayloadJob<Tasks, Builder>
|
||||
where
|
||||
Client: StateProviderFactory + Clone + Unpin + 'static,
|
||||
Pool: TransactionPool + Unpin + 'static,
|
||||
Tasks: TaskSpawner + Clone + 'static,
|
||||
Builder: PayloadBuilder<Pool, Client> + Unpin + 'static,
|
||||
<Builder as PayloadBuilder<Pool, Client>>::Attributes: Unpin + Clone,
|
||||
<Builder as PayloadBuilder<Pool, Client>>::BuiltPayload: Unpin + Clone,
|
||||
Builder: PayloadBuilder + Unpin + 'static,
|
||||
Builder::Attributes: Unpin + Clone,
|
||||
Builder::BuiltPayload: Unpin + Clone,
|
||||
{
|
||||
type PayloadAttributes = Builder::Attributes;
|
||||
type ResolvePayloadFuture =
|
||||
@ -45,7 +37,7 @@ where
|
||||
type BuiltPayload = Builder::BuiltPayload;
|
||||
|
||||
fn best_payload(&self) -> Result<Self::BuiltPayload, PayloadBuilderError> {
|
||||
let payload = self.builder.build_empty_payload(&self.client, self.config.clone())?;
|
||||
let payload = self.builder.build_empty_payload(self.config.clone())?;
|
||||
Ok(payload)
|
||||
}
|
||||
|
||||
@ -63,14 +55,12 @@ where
|
||||
}
|
||||
|
||||
/// A [PayloadJob] is a a future that's being polled by the `PayloadBuilderService`
|
||||
impl<Client, Pool, Tasks, Builder> Future for EmptyBlockPayloadJob<Client, Pool, Tasks, Builder>
|
||||
impl<Tasks, Builder> Future for EmptyBlockPayloadJob<Tasks, Builder>
|
||||
where
|
||||
Client: StateProviderFactory + Clone + Unpin + 'static,
|
||||
Pool: TransactionPool + Unpin + 'static,
|
||||
Tasks: TaskSpawner + Clone + 'static,
|
||||
Builder: PayloadBuilder<Pool, Client> + Unpin + 'static,
|
||||
<Builder as PayloadBuilder<Pool, Client>>::Attributes: Unpin + Clone,
|
||||
<Builder as PayloadBuilder<Pool, Client>>::BuiltPayload: Unpin + Clone,
|
||||
Builder: PayloadBuilder + Unpin + 'static,
|
||||
Builder::Attributes: Unpin + Clone,
|
||||
Builder::BuiltPayload: Unpin + Clone,
|
||||
{
|
||||
type Output = Result<(), PayloadBuilderError>;
|
||||
|
||||
|
||||
@ -62,10 +62,11 @@ where
|
||||
|
||||
let payload_generator = EmptyBlockPayloadJobGenerator::with_builder(
|
||||
ctx.provider().clone(),
|
||||
pool,
|
||||
ctx.task_executor().clone(),
|
||||
payload_job_config,
|
||||
reth_ethereum_payload_builder::EthereumPayloadBuilder::new(
|
||||
ctx.provider().clone(),
|
||||
pool,
|
||||
EthEvmConfig::new(ctx.chain_spec()),
|
||||
EthereumBuilderConfig::new(conf.extra_data_bytes()),
|
||||
),
|
||||
|
||||
Reference in New Issue
Block a user