feat: abstract OP payload builder over transaction (#14153)

This commit is contained in:
Arsenii Kulikov
2025-02-03 18:57:41 +04:00
committed by GitHub
parent 7243aaf555
commit 65c3bd7eda
15 changed files with 324 additions and 229 deletions

1
Cargo.lock generated
View File

@ -8530,6 +8530,7 @@ dependencies = [
"alloy-primitives",
"alloy-rpc-types-engine",
"assert_matches",
"auto_impl",
"op-alloy-rpc-types-engine",
"reth-chain-state",
"reth-chainspec",

View File

@ -98,22 +98,6 @@ impl BuiltPayload for EthBuiltPayload {
}
}
impl BuiltPayload for &EthBuiltPayload {
type Primitives = EthPrimitives;
fn block(&self) -> &SealedBlock {
(**self).block()
}
fn fees(&self) -> U256 {
(**self).fees()
}
fn requests(&self) -> Option<Requests> {
self.requests.clone()
}
}
// V1 engine_getPayloadV1 response
impl From<EthBuiltPayload> for ExecutionPayloadV1 {
fn from(value: EthBuiltPayload) -> Self {

View File

@ -1,13 +1,12 @@
//! Helper function for Receipt root calculation for Optimism hardforks.
use alloc::vec::Vec;
use alloy_consensus::TxReceipt;
use alloy_eips::eip2718::Encodable2718;
use alloy_primitives::B256;
use alloy_trie::root::ordered_trie_root_with_encoder;
use reth_chainspec::ChainSpec;
use reth_optimism_forks::{OpHardfork, OpHardforks};
use reth_optimism_primitives::{DepositReceipt, OpReceipt};
use reth_optimism_primitives::DepositReceipt;
use reth_primitives::ReceiptWithBloom;
/// Calculates the receipt root for a header.
@ -41,11 +40,11 @@ pub(crate) fn calculate_receipt_root_optimism<R: DepositReceipt>(
ordered_trie_root_with_encoder(receipts, |r, buf| r.encode_2718(buf))
}
/// Calculates the receipt root for a header for the reference type of [`OpReceipt`].
/// Calculates the receipt root for a header for the reference type of an OP receipt.
///
/// NOTE: Prefer calculate receipt root optimism if you have log blooms memoized.
pub fn calculate_receipt_root_no_memo_optimism(
receipts: &[OpReceipt],
pub fn calculate_receipt_root_no_memo_optimism<R: DepositReceipt>(
receipts: &[R],
chain_spec: impl OpHardforks,
timestamp: u64,
) -> B256 {
@ -61,8 +60,8 @@ pub fn calculate_receipt_root_no_memo_optimism(
.iter()
.map(|r| {
let mut r = (*r).clone();
if let OpReceipt::Deposit(r) = &mut r {
r.deposit_nonce = None;
if let Some(receipt) = r.as_deposit_receipt_mut() {
receipt.deposit_nonce = None;
}
r
})
@ -85,6 +84,7 @@ mod tests {
use alloy_primitives::{b256, bloom, hex, Address, Bloom, Bytes, Log, LogData};
use op_alloy_consensus::OpDepositReceipt;
use reth_optimism_chainspec::BASE_SEPOLIA;
use reth_optimism_primitives::OpReceipt;
use reth_primitives::ReceiptWithBloom;
/// Tests that the receipt root is computed correctly for the regolith block.

View File

@ -232,7 +232,6 @@ where
receipts.push(
match self.receipt_builder.build_receipt(ReceiptBuilderCtx {
header: block.header(),
tx: transaction,
result,
cumulative_gas_used,

View File

@ -1,4 +1,4 @@
use alloy_consensus::{Eip658Value, Header, Receipt};
use alloy_consensus::{Eip658Value, Receipt};
use core::fmt;
use op_alloy_consensus::{OpDepositReceipt, OpTxType};
use reth_optimism_primitives::{OpReceipt, OpTransactionSigned};
@ -7,8 +7,6 @@ use revm_primitives::ExecutionResult;
/// Context for building a receipt.
#[derive(Debug)]
pub struct ReceiptBuilderCtx<'a, T> {
/// Block header.
pub header: &'a Header,
/// Transaction
pub tx: &'a T,
/// Result of transaction execution.

View File

@ -18,7 +18,7 @@ use reth_node_api::{
use reth_optimism_chainspec::OpChainSpec;
use reth_optimism_forks::{OpHardfork, OpHardforks};
use reth_optimism_payload_builder::{OpBuiltPayload, OpPayloadBuilderAttributes};
use reth_optimism_primitives::OpBlock;
use reth_optimism_primitives::{OpBlock, OpPrimitives};
use reth_payload_validator::ExecutionPayloadValidator;
use reth_primitives::SealedBlock;
use std::sync::Arc;
@ -61,12 +61,12 @@ where
/// A default payload type for [`OpEngineTypes`]
#[derive(Debug, Default, Clone, serde::Deserialize, serde::Serialize)]
#[non_exhaustive]
pub struct OpPayloadTypes;
pub struct OpPayloadTypes<N: NodePrimitives = OpPrimitives>(core::marker::PhantomData<N>);
impl PayloadTypes for OpPayloadTypes {
type BuiltPayload = OpBuiltPayload;
impl<N: NodePrimitives> PayloadTypes for OpPayloadTypes<N> {
type BuiltPayload = OpBuiltPayload<N>;
type PayloadAttributes = OpPayloadAttributes;
type PayloadBuilderAttributes = OpPayloadBuilderAttributes;
type PayloadBuilderAttributes = OpPayloadBuilderAttributes<N::SignedTx>;
}
/// Validator for Optimism engine API.

View File

@ -25,7 +25,7 @@ use reth_node_builder::{
};
use reth_optimism_chainspec::OpChainSpec;
use reth_optimism_consensus::OpBeaconConsensus;
use reth_optimism_evm::{OpEvmConfig, OpExecutionStrategyFactory};
use reth_optimism_evm::{BasicOpReceiptBuilder, OpEvmConfig, OpExecutionStrategyFactory};
use reth_optimism_payload_builder::{
builder::OpPayloadTransactions,
config::{OpBuilderConfig, OpDAConfig},
@ -205,11 +205,16 @@ where
ctx: reth_node_api::AddOnsContext<'_, N>,
) -> eyre::Result<Self::Handle> {
let Self { rpc_add_ons, da_config } = self;
let builder = reth_optimism_payload_builder::OpPayloadBuilder::new(
ctx.node.evm_config().clone(),
BasicOpReceiptBuilder::default(),
);
// install additional OP specific rpc methods
let debug_ext = OpDebugWitnessApi::new(
ctx.node.provider().clone(),
ctx.node.evm_config().clone(),
Box::new(ctx.node.task_executor().clone()),
builder,
);
let miner_ext = OpMinerExtApi::new(da_config);
@ -493,9 +498,11 @@ where
+ Unpin
+ 'static,
Evm: ConfigureEvmFor<PrimitivesTy<Node::Types>>,
Txs: OpPayloadTransactions<TxTy<Node::Types>>,
{
let payload_builder = reth_optimism_payload_builder::OpPayloadBuilder::with_builder_config(
evm_config,
BasicOpReceiptBuilder::default(),
OpBuilderConfig { da_config: self.da_config },
)
.with_transactions(self.best_transactions)
@ -535,7 +542,7 @@ where
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
+ Unpin
+ 'static,
Txs: OpPayloadTransactions,
Txs: OpPayloadTransactions<TxTy<Node::Types>>,
{
async fn spawn_payload_service(
self,

View File

@ -7,6 +7,7 @@ use reth_e2e_test_utils::{
};
use reth_node_api::NodeTypesWithDBAdapter;
use reth_optimism_chainspec::OpChainSpecBuilder;
use reth_optimism_primitives::OpTransactionSigned;
use reth_payload_builder::EthPayloadBuilderAttributes;
use reth_provider::providers::BlockchainProvider;
use reth_tasks::TaskManager;
@ -35,7 +36,7 @@ pub async fn advance_chain(
length: usize,
node: &mut OpNode,
wallet: Arc<Mutex<Wallet>>,
) -> eyre::Result<Vec<(OpBuiltPayload, OpPayloadBuilderAttributes)>> {
) -> eyre::Result<Vec<(OpBuiltPayload, OpPayloadBuilderAttributes<OpTransactionSigned>)>> {
node.advance(length as u64, |_| {
let wallet = wallet.clone();
Box::pin(async move {
@ -53,7 +54,7 @@ pub async fn advance_chain(
}
/// Helper function to create a new eth payload attributes
pub fn optimism_payload_attributes(timestamp: u64) -> OpPayloadBuilderAttributes {
pub fn optimism_payload_attributes<T>(timestamp: u64) -> OpPayloadBuilderAttributes<T> {
let attributes = PayloadAttributes {
timestamp,
prev_randao: B256::ZERO,

View File

@ -4,6 +4,7 @@ use crate::{
config::{OpBuilderConfig, OpDAConfig},
error::OpPayloadBuilderError,
payload::{OpBuiltPayload, OpPayloadBuilderAttributes},
OpPayloadPrimitives,
};
use alloy_consensus::{Eip658Value, Header, Transaction, Typed2718, EMPTY_OMMER_ROOT_HASH};
use alloy_eips::{eip4895::Withdrawals, merge::BEACON_NONCE};
@ -11,25 +12,26 @@ use alloy_primitives::{Address, Bytes, B256, U256};
use alloy_rlp::Encodable;
use alloy_rpc_types_debug::ExecutionWitness;
use alloy_rpc_types_engine::PayloadId;
use op_alloy_consensus::{OpDepositReceipt, OpTxType};
use op_alloy_consensus::OpDepositReceipt;
use op_alloy_rpc_types_engine::OpPayloadAttributes;
use reth_basic_payload_builder::*;
use reth_chain_state::{ExecutedBlock, ExecutedBlockWithTrieUpdates};
use reth_chainspec::{ChainSpecProvider, EthereumHardforks};
use reth_evm::{
env::EvmEnv, system_calls::SystemCaller, ConfigureEvm, ConfigureEvmEnv, Database, Evm,
env::EvmEnv, system_calls::SystemCaller, ConfigureEvmEnv, ConfigureEvmFor, Database, Evm,
EvmError, InvalidTxError, NextBlockEnvAttributes,
};
use reth_execution_types::ExecutionOutcome;
use reth_optimism_chainspec::OpChainSpec;
use reth_optimism_consensus::calculate_receipt_root_no_memo_optimism;
use reth_optimism_evm::{OpReceiptBuilder, ReceiptBuilderCtx};
use reth_optimism_forks::OpHardforks;
use reth_optimism_primitives::{OpPrimitives, OpReceipt, OpTransactionSigned};
use reth_optimism_primitives::{transaction::signed::OpTransaction, OpTransactionSigned};
use reth_payload_builder_primitives::PayloadBuilderError;
use reth_payload_primitives::PayloadBuilderAttributes;
use reth_payload_util::{NoopPayloadTransactions, PayloadTransactions};
use reth_primitives::{
transaction::SignedTransactionIntoRecoveredExt, Block, BlockBody, SealedHeader,
transaction::SignedTransactionIntoRecoveredExt, BlockBody, NodePrimitives, SealedHeader,
};
use reth_primitives_traits::{block::Block as _, proofs, RecoveredBlock};
use reth_provider::{
@ -42,7 +44,7 @@ use reth_transaction_pool::{
};
use revm::{
db::{states::bundle_state::BundleRetention, State},
primitives::ResultAndState,
primitives::{ExecutionResult, ResultAndState},
DatabaseCommit,
};
use std::{fmt::Display, sync::Arc};
@ -50,7 +52,7 @@ use tracing::{debug, trace, warn};
/// Optimism's payload builder
#[derive(Debug, Clone)]
pub struct OpPayloadBuilder<EvmConfig, Txs = ()> {
pub struct OpPayloadBuilder<EvmConfig, N: NodePrimitives, Txs = ()> {
/// The rollup's compute pending block configuration option.
// TODO(clabby): Implement this feature.
pub compute_pending_block: bool,
@ -61,23 +63,38 @@ pub struct OpPayloadBuilder<EvmConfig, Txs = ()> {
/// The type responsible for yielding the best transactions for the payload if mempool
/// transactions are allowed.
pub best_transactions: Txs,
/// Node primitive types.
pub receipt_builder: Arc<dyn OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>>,
}
impl<EvmConfig> OpPayloadBuilder<EvmConfig> {
impl<EvmConfig, N: NodePrimitives> OpPayloadBuilder<EvmConfig, N> {
/// `OpPayloadBuilder` constructor.
///
/// Configures the builder with the default settings.
pub fn new(evm_config: EvmConfig) -> Self {
Self::with_builder_config(evm_config, Default::default())
pub fn new(
evm_config: EvmConfig,
receipt_builder: impl OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>,
) -> Self {
Self::with_builder_config(evm_config, receipt_builder, Default::default())
}
/// Configures the builder with the given [`OpBuilderConfig`].
pub const fn with_builder_config(evm_config: EvmConfig, config: OpBuilderConfig) -> Self {
Self { compute_pending_block: true, evm_config, config, best_transactions: () }
pub fn with_builder_config(
evm_config: EvmConfig,
receipt_builder: impl OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>,
config: OpBuilderConfig,
) -> Self {
Self {
compute_pending_block: true,
receipt_builder: Arc::new(receipt_builder),
evm_config,
config,
best_transactions: (),
}
}
}
impl<EvmConfig, Txs> OpPayloadBuilder<EvmConfig, Txs> {
impl<EvmConfig, N: NodePrimitives, Txs> OpPayloadBuilder<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;
@ -89,9 +106,15 @@ impl<EvmConfig, Txs> OpPayloadBuilder<EvmConfig, Txs> {
pub fn with_transactions<T: OpPayloadTransactions>(
self,
best_transactions: T,
) -> OpPayloadBuilder<EvmConfig, T> {
let Self { compute_pending_block, evm_config, config, .. } = self;
OpPayloadBuilder { compute_pending_block, evm_config, best_transactions, config }
) -> OpPayloadBuilder<EvmConfig, N, T> {
let Self { compute_pending_block, evm_config, config, receipt_builder, .. } = self;
OpPayloadBuilder {
compute_pending_block,
evm_config,
best_transactions,
config,
receipt_builder,
}
}
/// Enables the rollup's compute pending block configuration option.
@ -104,9 +127,10 @@ impl<EvmConfig, Txs> OpPayloadBuilder<EvmConfig, Txs> {
self.compute_pending_block
}
}
impl<EvmConfig, T> OpPayloadBuilder<EvmConfig, T>
impl<EvmConfig, N, T> OpPayloadBuilder<EvmConfig, N, T>
where
EvmConfig: ConfigureEvm<Header = Header, Transaction = OpTransactionSigned>,
N: OpPayloadPrimitives,
EvmConfig: ConfigureEvmFor<N>,
{
/// Constructs an Optimism payload from the transactions sent via the
/// Payload attributes by the sequencer. If the `no_tx_pool` argument is passed in
@ -118,12 +142,17 @@ where
/// a result indicating success with the payload or an error in case of failure.
fn build_payload<'a, Client, Pool, Txs>(
&self,
args: BuildArguments<Pool, Client, OpPayloadBuilderAttributes, OpBuiltPayload>,
args: BuildArguments<
Pool,
Client,
OpPayloadBuilderAttributes<N::SignedTx>,
OpBuiltPayload<N>,
>,
best: impl FnOnce(BestTransactionsAttributes) -> Txs + Send + Sync + 'a,
) -> Result<BuildOutcome<OpBuiltPayload>, PayloadBuilderError>
) -> Result<BuildOutcome<OpBuiltPayload<N>>, PayloadBuilderError>
where
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = OpChainSpec>,
Txs: PayloadTransactions<Transaction = OpTransactionSigned>,
Txs: PayloadTransactions<Transaction = N::SignedTx>,
{
let evm_env = self
.evm_env(&args.config.attributes, &args.config.parent_header)
@ -140,6 +169,7 @@ where
evm_env,
cancel,
best_payload,
receipt_builder: self.receipt_builder.clone(),
};
let builder = OpBuilder::new(best);
@ -165,7 +195,7 @@ where
/// (that has the `parent` as its parent).
pub fn evm_env(
&self,
attributes: &OpPayloadBuilderAttributes,
attributes: &OpPayloadBuilderAttributes<N::SignedTx>,
parent: &Header,
) -> Result<EvmEnv<EvmConfig::Spec>, EvmConfig::Error> {
let next_attributes = NextBlockEnvAttributes {
@ -193,7 +223,7 @@ where
let evm_env = self.evm_env(&attributes, &parent).map_err(PayloadBuilderError::other)?;
let config = PayloadConfig { parent_header: Arc::new(parent), attributes };
let ctx = OpPayloadBuilderCtx {
let ctx: OpPayloadBuilderCtx<EvmConfig, N> = OpPayloadBuilderCtx {
evm_config: self.evm_config.clone(),
da_config: self.config.da_config.clone(),
chain_spec: client.chain_spec(),
@ -201,6 +231,7 @@ where
evm_env,
cancel: Default::default(),
best_payload: Default::default(),
receipt_builder: self.receipt_builder.clone(),
};
let state_provider = client.state_by_block_hash(ctx.parent().hash())?;
@ -213,27 +244,29 @@ where
}
/// Implementation of the [`PayloadBuilder`] trait for [`OpPayloadBuilder`].
impl<Pool, Client, EvmConfig, Txs> PayloadBuilder<Pool, Client> for OpPayloadBuilder<EvmConfig, Txs>
impl<Pool, Client, EvmConfig, N, Txs> PayloadBuilder<Pool, Client>
for OpPayloadBuilder<EvmConfig, N, Txs>
where
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = OpChainSpec>,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = EvmConfig::Transaction>>,
EvmConfig: ConfigureEvm<Header = Header, Transaction = OpTransactionSigned>,
Txs: OpPayloadTransactions,
N: OpPayloadPrimitives,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = N::SignedTx>>,
EvmConfig: ConfigureEvmFor<N>,
Txs: OpPayloadTransactions<N::SignedTx>,
{
type Attributes = OpPayloadBuilderAttributes;
type BuiltPayload = OpBuiltPayload;
type Attributes = OpPayloadBuilderAttributes<N::SignedTx>;
type BuiltPayload = OpBuiltPayload<N>;
fn try_build(
&self,
args: BuildArguments<Pool, Client, OpPayloadBuilderAttributes, OpBuiltPayload>,
) -> Result<BuildOutcome<OpBuiltPayload>, PayloadBuilderError> {
args: BuildArguments<Pool, Client, Self::Attributes, Self::BuiltPayload>,
) -> Result<BuildOutcome<Self::BuiltPayload>, PayloadBuilderError> {
let pool = args.pool.clone();
self.build_payload(args, |attrs| self.best_transactions.best_transactions(pool, attrs))
}
fn on_missing_payload(
&self,
_args: BuildArguments<Pool, Client, OpPayloadBuilderAttributes, OpBuiltPayload>,
_args: BuildArguments<Pool, Client, 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
@ -246,7 +279,7 @@ where
&self,
client: &Client,
config: PayloadConfig<Self::Attributes>,
) -> Result<OpBuiltPayload, PayloadBuilderError> {
) -> Result<Self::BuiltPayload, PayloadBuilderError> {
let args = BuildArguments {
client,
config,
@ -292,16 +325,18 @@ impl<'a, Txs> OpBuilder<'a, Txs> {
impl<Txs> OpBuilder<'_, Txs>
where
Txs: PayloadTransactions<Transaction = OpTransactionSigned>,
Txs: PayloadTransactions,
{
/// Executes the payload and returns the outcome.
pub fn execute<EvmConfig, DB>(
pub fn execute<EvmConfig, N, DB>(
self,
state: &mut State<DB>,
ctx: &OpPayloadBuilderCtx<EvmConfig>,
) -> Result<BuildOutcomeKind<ExecutedPayload>, PayloadBuilderError>
ctx: &OpPayloadBuilderCtx<EvmConfig, N>,
) -> Result<BuildOutcomeKind<ExecutedPayload<N>>, PayloadBuilderError>
where
EvmConfig: ConfigureEvm<Header = Header, Transaction = OpTransactionSigned>,
N: OpPayloadPrimitives,
Txs: PayloadTransactions<Transaction = N::SignedTx>,
EvmConfig: ConfigureEvmFor<N>,
DB: Database<Error = ProviderError>,
{
let Self { best } = self;
@ -340,13 +375,15 @@ where
}
/// Builds the payload on top of the state.
pub fn build<EvmConfig, DB, P>(
pub fn build<EvmConfig, N, DB, P>(
self,
mut state: State<DB>,
ctx: OpPayloadBuilderCtx<EvmConfig>,
) -> Result<BuildOutcomeKind<OpBuiltPayload>, PayloadBuilderError>
ctx: OpPayloadBuilderCtx<EvmConfig, N>,
) -> Result<BuildOutcomeKind<OpBuiltPayload<N>>, PayloadBuilderError>
where
EvmConfig: ConfigureEvm<Header = Header, Transaction = OpTransactionSigned>,
EvmConfig: ConfigureEvmFor<N>,
N: OpPayloadPrimitives,
Txs: PayloadTransactions<Transaction = N::SignedTx>,
DB: Database<Error = ProviderError> + AsRef<P>,
P: StateRootProvider + HashedPostStateProvider,
{
@ -422,20 +459,20 @@ where
};
// seal the block
let block = Block {
let block = N::Block::new(
header,
body: BlockBody {
BlockBody {
transactions: info.executed_transactions,
ommers: vec![],
withdrawals: ctx.withdrawals().cloned(),
},
};
);
let sealed_block = Arc::new(block.seal_slow());
debug!(target: "payload_builder", id=%ctx.attributes().payload_id(), sealed_block_header = ?sealed_block.header(), "sealed built block");
// create the executed block data
let executed: ExecutedBlockWithTrieUpdates<OpPrimitives> = ExecutedBlockWithTrieUpdates {
let executed: ExecutedBlockWithTrieUpdates<N> = ExecutedBlockWithTrieUpdates {
block: ExecutedBlock {
recovered_block: Arc::new(RecoveredBlock::new_sealed(
sealed_block.as_ref().clone(),
@ -449,7 +486,8 @@ where
let no_tx_pool = ctx.attributes().no_tx_pool;
let payload = OpBuiltPayload::new(ctx.payload_id(), info.total_fees, executed);
let payload =
OpBuiltPayload::new(ctx.payload_id(), sealed_block, info.total_fees, Some(executed));
if no_tx_pool {
// if `no_tx_pool` is set only transactions from the payload attributes will be included
@ -462,13 +500,15 @@ where
}
/// Builds the payload and returns its [`ExecutionWitness`] based on the state after execution.
pub fn witness<EvmConfig, DB, P>(
pub fn witness<EvmConfig, N, DB, P>(
self,
state: &mut State<DB>,
ctx: &OpPayloadBuilderCtx<EvmConfig>,
ctx: &OpPayloadBuilderCtx<EvmConfig, N>,
) -> Result<ExecutionWitness, PayloadBuilderError>
where
EvmConfig: ConfigureEvm<Header = Header, Transaction = OpTransactionSigned>,
EvmConfig: ConfigureEvmFor<N>,
N: OpPayloadPrimitives,
Txs: PayloadTransactions<Transaction = N::SignedTx>,
DB: Database<Error = ProviderError> + AsRef<P>,
P: StateProofProvider,
{
@ -481,48 +521,48 @@ where
}
/// A type that returns a the [`PayloadTransactions`] that should be included in the pool.
pub trait OpPayloadTransactions: Clone + Send + Sync + Unpin + 'static {
pub trait OpPayloadTransactions<Transaction = OpTransactionSigned>:
Clone + Send + Sync + Unpin + 'static
{
/// Returns an iterator that yields the transaction in the order they should get included in the
/// new payload.
fn best_transactions<
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = OpTransactionSigned>>,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = Transaction>>,
>(
&self,
pool: Pool,
attr: BestTransactionsAttributes,
) -> impl PayloadTransactions<Transaction = OpTransactionSigned>;
) -> impl PayloadTransactions<Transaction = Transaction>;
}
impl OpPayloadTransactions for () {
fn best_transactions<
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = OpTransactionSigned>>,
>(
impl<T> OpPayloadTransactions<T> for () {
fn best_transactions<Pool: TransactionPool<Transaction: PoolTransaction<Consensus = T>>>(
&self,
pool: Pool,
attr: BestTransactionsAttributes,
) -> impl PayloadTransactions<Transaction = OpTransactionSigned> {
) -> impl PayloadTransactions<Transaction = T> {
BestPayloadTransactions::new(pool.best_transactions_with_attributes(attr))
}
}
/// Holds the state after execution
#[derive(Debug)]
pub struct ExecutedPayload {
pub struct ExecutedPayload<N: NodePrimitives> {
/// Tracked execution info
pub info: ExecutionInfo,
pub info: ExecutionInfo<N>,
/// Withdrawal hash.
pub withdrawals_root: Option<B256>,
}
/// This acts as the container for executed transactions and its byproducts (receipts, gas used)
#[derive(Default, Debug)]
pub struct ExecutionInfo {
pub struct ExecutionInfo<N: NodePrimitives> {
/// All executed transactions (unrecovered).
pub executed_transactions: Vec<OpTransactionSigned>,
pub executed_transactions: Vec<N::SignedTx>,
/// The recovered senders for the executed transactions.
pub executed_senders: Vec<Address>,
/// The transaction receipts
pub receipts: Vec<OpReceipt>,
pub receipts: Vec<N::Receipt>,
/// All gas used so far
pub cumulative_gas_used: u64,
/// Estimated DA size
@ -531,7 +571,7 @@ pub struct ExecutionInfo {
pub total_fees: U256,
}
impl ExecutionInfo {
impl<N: NodePrimitives> ExecutionInfo<N> {
/// Create a new instance with allocated slots.
pub fn with_capacity(capacity: usize) -> Self {
Self {
@ -552,7 +592,7 @@ impl ExecutionInfo {
/// maximum allowed DA limit per block.
pub fn is_tx_over_limits(
&self,
tx: &OpTransactionSigned,
tx: &N::SignedTx,
block_gas_limit: u64,
tx_data_limit: Option<u64>,
block_data_limit: Option<u64>,
@ -573,7 +613,7 @@ impl ExecutionInfo {
/// Container type that holds all necessities to build a new payload.
#[derive(Debug)]
pub struct OpPayloadBuilderCtx<EvmConfig: ConfigureEvmEnv> {
pub struct OpPayloadBuilderCtx<EvmConfig: ConfigureEvmEnv, N: NodePrimitives> {
/// The type that knows how to perform system calls and configure the evm.
pub evm_config: EvmConfig,
/// The DA config for the payload builder
@ -581,23 +621,25 @@ pub struct OpPayloadBuilderCtx<EvmConfig: ConfigureEvmEnv> {
/// The chainspec
pub chain_spec: Arc<OpChainSpec>,
/// How to build the payload.
pub config: PayloadConfig<OpPayloadBuilderAttributes>,
pub config: PayloadConfig<OpPayloadBuilderAttributes<N::SignedTx>>,
/// Evm Settings
pub evm_env: EvmEnv<EvmConfig::Spec>,
/// Marker to check whether the job has been cancelled.
pub cancel: Cancelled,
/// The currently best payload.
pub best_payload: Option<OpBuiltPayload>,
pub best_payload: Option<OpBuiltPayload<N>>,
/// Receipt builder.
pub receipt_builder: Arc<dyn OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>>,
}
impl<EvmConfig: ConfigureEvmEnv> OpPayloadBuilderCtx<EvmConfig> {
impl<EvmConfig: ConfigureEvmEnv, N: NodePrimitives> OpPayloadBuilderCtx<EvmConfig, N> {
/// Returns the parent block the payload will be build on.
pub fn parent(&self) -> &SealedHeader {
&self.config.parent_header
}
/// Returns the builder attributes.
pub const fn attributes(&self) -> &OpPayloadBuilderAttributes {
pub const fn attributes(&self) -> &OpPayloadBuilderAttributes<N::SignedTx> {
&self.config.attributes
}
@ -730,9 +772,10 @@ impl<EvmConfig: ConfigureEvmEnv> OpPayloadBuilderCtx<EvmConfig> {
}
}
impl<EvmConfig> OpPayloadBuilderCtx<EvmConfig>
impl<EvmConfig, N> OpPayloadBuilderCtx<EvmConfig, N>
where
EvmConfig: ConfigureEvm<Header = Header, Transaction = OpTransactionSigned>,
EvmConfig: ConfigureEvmFor<N>,
N: OpPayloadPrimitives,
{
/// apply eip-4788 pre block contract call
pub fn apply_pre_beacon_root_contract_call<DB>(
@ -761,11 +804,48 @@ where
Ok(())
}
/// Constructs a receipt for the given transaction.
fn build_receipt(
&self,
info: &ExecutionInfo<N>,
result: ExecutionResult,
deposit_nonce: Option<u64>,
tx: &N::SignedTx,
) -> N::Receipt {
match self.receipt_builder.build_receipt(ReceiptBuilderCtx {
tx,
result,
cumulative_gas_used: info.cumulative_gas_used,
}) {
Ok(receipt) => receipt,
Err(ctx) => {
let receipt = alloy_consensus::Receipt {
// Success flag was added in `EIP-658: Embedding transaction status code
// in receipts`.
status: Eip658Value::Eip658(ctx.result.is_success()),
cumulative_gas_used: ctx.cumulative_gas_used,
logs: ctx.result.into_logs(),
};
self.receipt_builder.build_deposit_receipt(OpDepositReceipt {
inner: receipt,
deposit_nonce,
// The deposit receipt version was introduced in Canyon to indicate an
// update to how receipt hashes should be computed
// when set. The state transition process ensures
// this is only set for post-Canyon deposit
// transactions.
deposit_receipt_version: self.is_canyon_active().then_some(1),
})
}
}
}
/// Executes all sequencer transactions that are included in the payload attributes.
pub fn execute_sequencer_transactions<DB>(
&self,
db: &mut State<DB>,
) -> Result<ExecutionInfo, PayloadBuilderError>
) -> Result<ExecutionInfo<N>, PayloadBuilderError>
where
DB: Database<Error = ProviderError>,
{
@ -793,11 +873,11 @@ where
// Note that this *only* needs to be done post-regolith hardfork, as deposit nonces
// were not introduced in Bedrock. In addition, regular transactions don't have deposit
// nonces, so we don't need to touch the DB for those.
let depositor = (self.is_regolith_active() && sequencer_tx.is_deposit())
let depositor_nonce = (self.is_regolith_active() && sequencer_tx.is_deposit())
.then(|| {
evm.db_mut()
.load_cache_account(sequencer_tx.signer())
.map(|acc| acc.account_info().unwrap_or_default())
.map(|acc| acc.account_info().unwrap_or_default().nonce)
})
.transpose()
.map_err(|_| {
@ -828,28 +908,13 @@ where
// add gas used by the transaction to cumulative gas used, before creating the receipt
info.cumulative_gas_used += gas_used;
let receipt = alloy_consensus::Receipt {
status: Eip658Value::Eip658(result.is_success()),
cumulative_gas_used: info.cumulative_gas_used,
logs: result.into_logs().into_iter().collect(),
};
// Push transaction changeset and calculate header bloom filter for receipt.
info.receipts.push(match sequencer_tx.tx_type() {
OpTxType::Legacy => OpReceipt::Legacy(receipt),
OpTxType::Eip2930 => OpReceipt::Eip2930(receipt),
OpTxType::Eip1559 => OpReceipt::Eip1559(receipt),
OpTxType::Eip7702 => OpReceipt::Eip7702(receipt),
OpTxType::Deposit => OpReceipt::Deposit(OpDepositReceipt {
inner: receipt,
deposit_nonce: depositor.map(|account| account.nonce),
// The deposit receipt version was introduced in Canyon to indicate an update to
// how receipt hashes should be computed when set. The state
// transition process ensures this is only set for
// post-Canyon deposit transactions.
deposit_receipt_version: self.is_canyon_active().then_some(1),
}),
});
info.receipts.push(self.build_receipt(
&info,
result,
depositor_nonce,
sequencer_tx.tx(),
));
// append sender and transaction to the respective lists
info.executed_senders.push(sequencer_tx.signer());
@ -864,7 +929,7 @@ where
/// Returns `Ok(Some(())` if the job was cancelled.
pub fn execute_best_transactions<DB>(
&self,
info: &mut ExecutionInfo,
info: &mut ExecutionInfo<N>,
db: &mut State<DB>,
mut best_txs: impl PayloadTransactions<Transaction = EvmConfig::Transaction>,
) -> Result<Option<()>, PayloadBuilderError>
@ -888,7 +953,7 @@ where
}
// A sequencer's block should never contain blob or deposit transactions from the pool.
if tx.is_eip4844() || tx.tx_type() == OpTxType::Deposit {
if tx.is_eip4844() || tx.is_deposit() {
best_txs.mark_invalid(tx.signer(), tx.nonce());
continue
}
@ -932,24 +997,8 @@ where
info.cumulative_gas_used += gas_used;
info.cumulative_da_bytes_used += tx.length() as u64;
let receipt = alloy_consensus::Receipt {
status: Eip658Value::Eip658(result.is_success()),
cumulative_gas_used: info.cumulative_gas_used,
logs: result.into_logs().into_iter().collect(),
};
// Push transaction changeset and calculate header bloom filter for receipt.
info.receipts.push(match tx.tx_type() {
OpTxType::Legacy => OpReceipt::Legacy(receipt),
OpTxType::Eip2930 => OpReceipt::Eip2930(receipt),
OpTxType::Eip1559 => OpReceipt::Eip1559(receipt),
OpTxType::Eip7702 => OpReceipt::Eip7702(receipt),
OpTxType::Deposit => OpReceipt::Deposit(OpDepositReceipt {
inner: receipt,
deposit_nonce: None,
deposit_receipt_version: None,
}),
});
info.receipts.push(self.build_receipt(info, result, None, &tx));
// update add to total fees
let miner_fee = tx

View File

@ -16,5 +16,7 @@ pub use builder::OpPayloadBuilder;
pub mod error;
pub mod payload;
pub use payload::{OpBuiltPayload, OpPayloadAttributes, OpPayloadBuilderAttributes};
mod traits;
pub use traits::*;
pub mod config;

View File

@ -1,5 +1,7 @@
//! Payload related types
use std::{fmt::Debug, sync::Arc};
use alloy_eips::{
eip1559::BaseFeeParams, eip2718::Decodable2718, eip4895::Withdrawals, eip7685::Requests,
};
@ -14,28 +16,41 @@ use op_alloy_consensus::{encode_holocene_extra_data, EIP1559ParamError};
pub use op_alloy_rpc_types_engine::OpPayloadAttributes;
use op_alloy_rpc_types_engine::{OpExecutionPayloadEnvelopeV3, OpExecutionPayloadEnvelopeV4};
use reth_chain_state::ExecutedBlockWithTrieUpdates;
use reth_optimism_primitives::{OpBlock, OpPrimitives, OpTransactionSigned};
use reth_optimism_primitives::OpPrimitives;
use reth_payload_builder::EthPayloadBuilderAttributes;
use reth_payload_primitives::{BuiltPayload, PayloadBuilderAttributes};
use reth_primitives::{transaction::WithEncoded, SealedBlock};
use reth_primitives::{transaction::WithEncoded, Block, NodePrimitives, SealedBlock};
use reth_primitives_traits::SignedTransaction;
/// Optimism Payload Builder Attributes
#[derive(Debug, Clone, PartialEq, Eq, Default)]
pub struct OpPayloadBuilderAttributes {
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct OpPayloadBuilderAttributes<T> {
/// Inner ethereum payload builder attributes
pub payload_attributes: EthPayloadBuilderAttributes,
/// `NoTxPool` option for the generated payload
pub no_tx_pool: bool,
/// Decoded transactions and the original EIP-2718 encoded bytes as received in the payload
/// attributes.
pub transactions: Vec<WithEncoded<OpTransactionSigned>>,
pub transactions: Vec<WithEncoded<T>>,
/// The gas limit for the generated payload
pub gas_limit: Option<u64>,
/// EIP-1559 parameters for the generated payload
pub eip_1559_params: Option<B64>,
}
impl OpPayloadBuilderAttributes {
impl<T> Default for OpPayloadBuilderAttributes<T> {
fn default() -> Self {
Self {
payload_attributes: Default::default(),
no_tx_pool: Default::default(),
gas_limit: Default::default(),
eip_1559_params: Default::default(),
transactions: Default::default(),
}
}
}
impl<T> OpPayloadBuilderAttributes<T> {
/// Extracts the `eip1559` parameters for the payload.
pub fn get_holocene_extra_data(
&self,
@ -47,7 +62,9 @@ impl OpPayloadBuilderAttributes {
}
}
impl PayloadBuilderAttributes for OpPayloadBuilderAttributes {
impl<T: Decodable2718 + Send + Sync + Debug> PayloadBuilderAttributes
for OpPayloadBuilderAttributes<T>
{
type RpcPayloadAttributes = OpPayloadAttributes;
type Error = alloy_rlp::Error;
@ -127,25 +144,28 @@ impl PayloadBuilderAttributes for OpPayloadBuilderAttributes {
/// Contains the built payload.
#[derive(Debug, Clone)]
pub struct OpBuiltPayload {
pub struct OpBuiltPayload<N: NodePrimitives = OpPrimitives> {
/// Identifier of the payload
pub(crate) id: PayloadId,
/// Sealed block
pub(crate) block: Arc<SealedBlock<N::Block>>,
/// Block execution data for the payload, if any.
pub(crate) block: ExecutedBlockWithTrieUpdates<OpPrimitives>,
pub(crate) executed_block: Option<ExecutedBlockWithTrieUpdates<N>>,
/// The fees of the block
pub(crate) fees: U256,
}
// === impl BuiltPayload ===
impl OpBuiltPayload {
impl<N: NodePrimitives> OpBuiltPayload<N> {
/// Initializes the payload with the given initial block.
pub const fn new(
id: PayloadId,
block: Arc<SealedBlock<N::Block>>,
fees: U256,
block: ExecutedBlockWithTrieUpdates<OpPrimitives>,
executed_block: Option<ExecutedBlockWithTrieUpdates<N>>,
) -> Self {
Self { id, block, fees }
Self { id, block, fees, executed_block }
}
/// Returns the identifier of the payload.
@ -154,8 +174,8 @@ impl OpBuiltPayload {
}
/// Returns the built block(sealed)
pub fn block(&self) -> &SealedBlock<OpBlock> {
self.block.sealed_block()
pub fn block(&self) -> &SealedBlock<N::Block> {
&self.block
}
/// Fees of the block
@ -164,10 +184,10 @@ impl OpBuiltPayload {
}
}
impl BuiltPayload for OpBuiltPayload {
type Primitives = OpPrimitives;
impl<N: NodePrimitives> BuiltPayload for OpBuiltPayload<N> {
type Primitives = N;
fn block(&self) -> &SealedBlock<OpBlock> {
fn block(&self) -> &SealedBlock<N::Block> {
self.block()
}
@ -175,28 +195,8 @@ impl BuiltPayload for OpBuiltPayload {
self.fees
}
fn executed_block(&self) -> Option<ExecutedBlockWithTrieUpdates<OpPrimitives>> {
Some(self.block.clone())
}
fn requests(&self) -> Option<Requests> {
None
}
}
impl BuiltPayload for &OpBuiltPayload {
type Primitives = OpPrimitives;
fn block(&self) -> &SealedBlock<OpBlock> {
(**self).block()
}
fn fees(&self) -> U256 {
(**self).fees()
}
fn executed_block(&self) -> Option<ExecutedBlockWithTrieUpdates<OpPrimitives>> {
Some(self.block.clone())
fn executed_block(&self) -> Option<ExecutedBlockWithTrieUpdates<N>> {
self.executed_block.clone()
}
fn requests(&self) -> Option<Requests> {
@ -205,42 +205,52 @@ impl BuiltPayload for &OpBuiltPayload {
}
// V1 engine_getPayloadV1 response
impl From<OpBuiltPayload> for ExecutionPayloadV1 {
fn from(value: OpBuiltPayload) -> Self {
impl<T, N> From<OpBuiltPayload<N>> for ExecutionPayloadV1
where
T: SignedTransaction,
N: NodePrimitives<Block = Block<T>>,
{
fn from(value: OpBuiltPayload<N>) -> Self {
Self::from_block_unchecked(
value.block().hash(),
&value.block.into_sealed_block().into_block(),
&Arc::unwrap_or_clone(value.block).into_block(),
)
}
}
// V2 engine_getPayloadV2 response
impl From<OpBuiltPayload> for ExecutionPayloadEnvelopeV2 {
fn from(value: OpBuiltPayload) -> Self {
impl<T, N> From<OpBuiltPayload<N>> for ExecutionPayloadEnvelopeV2
where
T: SignedTransaction,
N: NodePrimitives<Block = Block<T>>,
{
fn from(value: OpBuiltPayload<N>) -> Self {
let OpBuiltPayload { block, fees, .. } = value;
let block = block.into_sealed_block();
Self {
block_value: fees,
execution_payload: ExecutionPayloadFieldV2::from_block_unchecked(
block.hash(),
&block.into_block(),
&Arc::unwrap_or_clone(block).into_block(),
),
}
}
}
impl From<OpBuiltPayload> for OpExecutionPayloadEnvelopeV3 {
fn from(value: OpBuiltPayload) -> Self {
impl<T, N> From<OpBuiltPayload<N>> for OpExecutionPayloadEnvelopeV3
where
T: SignedTransaction,
N: NodePrimitives<Block = Block<T>>,
{
fn from(value: OpBuiltPayload<N>) -> Self {
let OpBuiltPayload { block, fees, .. } = value;
let parent_beacon_block_root =
block.sealed_block().parent_beacon_block_root.unwrap_or_default();
let parent_beacon_block_root = block.parent_beacon_block_root.unwrap_or_default();
Self {
execution_payload: ExecutionPayloadV3::from_block_unchecked(
block.sealed_block().hash(),
&block.into_sealed_block().into_block(),
block.hash(),
&Arc::unwrap_or_clone(block).into_block(),
),
block_value: fees,
// From the engine API spec:
@ -258,17 +268,21 @@ impl From<OpBuiltPayload> for OpExecutionPayloadEnvelopeV3 {
}
}
}
impl From<OpBuiltPayload> for OpExecutionPayloadEnvelopeV4 {
fn from(value: OpBuiltPayload) -> Self {
impl<T, N> From<OpBuiltPayload<N>> for OpExecutionPayloadEnvelopeV4
where
T: SignedTransaction,
N: NodePrimitives<Block = Block<T>>,
{
fn from(value: OpBuiltPayload<N>) -> Self {
let OpBuiltPayload { block, fees, .. } = value;
let parent_beacon_block_root =
block.sealed_block().parent_beacon_block_root.unwrap_or_default();
let parent_beacon_block_root = block.parent_beacon_block_root.unwrap_or_default();
Self {
execution_payload: ExecutionPayloadV3::from_block_unchecked(
block.sealed_block().hash(),
&block.into_sealed_block().into_block(),
block.hash(),
&Arc::unwrap_or_clone(block).into_block(),
),
block_value: fees,
// From the engine API spec:
@ -347,6 +361,7 @@ mod tests {
use crate::OpPayloadAttributes;
use alloy_primitives::{address, b256, bytes, FixedBytes};
use alloy_rpc_types_engine::PayloadAttributes;
use reth_optimism_primitives::OpTransactionSigned;
use reth_payload_primitives::EngineApiMessageVersion;
use std::str::FromStr;
@ -383,7 +398,8 @@ mod tests {
#[test]
fn test_get_extra_data_post_holocene() {
let attributes = OpPayloadBuilderAttributes {
let attributes: OpPayloadBuilderAttributes<OpTransactionSigned> =
OpPayloadBuilderAttributes {
eip_1559_params: Some(B64::from_str("0x0000000800000008").unwrap()),
..Default::default()
};
@ -393,7 +409,7 @@ mod tests {
#[test]
fn test_get_extra_data_post_holocene_default() {
let attributes =
let attributes: OpPayloadBuilderAttributes<OpTransactionSigned> =
OpPayloadBuilderAttributes { eip_1559_params: Some(B64::ZERO), ..Default::default() };
let extra_data = attributes.get_holocene_extra_data(BaseFeeParams::new(80, 60));
assert_eq!(extra_data.unwrap(), Bytes::copy_from_slice(&[0, 0, 0, 0, 80, 0, 0, 0, 60]));

View File

@ -0,0 +1,30 @@
use alloy_consensus::{BlockBody, Header};
use reth_optimism_primitives::{transaction::signed::OpTransaction, DepositReceipt};
use reth_primitives::NodePrimitives;
use reth_primitives_traits::SignedTransaction;
/// Helper trait to encapsulate common bounds on [`NodePrimitives`] for OP payload builder.
pub trait OpPayloadPrimitives:
NodePrimitives<
Receipt: DepositReceipt,
SignedTx = Self::_TX,
BlockHeader = Header,
BlockBody = BlockBody<Self::_TX>,
>
{
/// Helper AT to bound [`NodePrimitives::Block`] type without causing bound cycle.
type _TX: SignedTransaction + OpTransaction;
}
impl<Tx, T> OpPayloadPrimitives for T
where
Tx: SignedTransaction + OpTransaction,
T: NodePrimitives<
SignedTx = Tx,
Receipt: DepositReceipt,
BlockHeader = Header,
BlockBody = BlockBody<Tx>,
>,
{
type _TX = Tx;
}

View File

@ -1,17 +1,17 @@
//! Support for optimism specific witness RPCs.
use alloy_consensus::Header;
use alloy_primitives::B256;
use alloy_rpc_types_debug::ExecutionWitness;
use jsonrpsee_core::{async_trait, RpcResult};
use op_alloy_rpc_types_engine::OpPayloadAttributes;
use reth_chainspec::ChainSpecProvider;
use reth_evm::ConfigureEvm;
use reth_evm::ConfigureEvmFor;
use reth_optimism_chainspec::OpChainSpec;
use reth_optimism_payload_builder::OpPayloadBuilder;
use reth_optimism_primitives::OpTransactionSigned;
use reth_optimism_payload_builder::{OpPayloadBuilder, OpPayloadPrimitives};
use reth_primitives::SealedHeader;
use reth_provider::{BlockReaderIdExt, ProviderError, ProviderResult, StateProviderFactory};
use reth_provider::{
BlockReaderIdExt, NodePrimitivesProvider, ProviderError, ProviderResult, StateProviderFactory,
};
pub use reth_rpc_api::DebugExecutionWitnessApiServer;
use reth_rpc_server_types::{result::internal_rpc_err, ToRpcResult};
use reth_tasks::TaskSpawner;
@ -19,18 +19,17 @@ 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, EvmConfig> {
pub struct OpDebugWitnessApi<Provider: NodePrimitivesProvider, EvmConfig> {
inner: Arc<OpDebugWitnessApiInner<Provider, EvmConfig>>,
}
impl<Provider, EvmConfig> OpDebugWitnessApi<Provider, EvmConfig> {
impl<Provider: NodePrimitivesProvider, EvmConfig> OpDebugWitnessApi<Provider, EvmConfig> {
/// Creates a new instance of the `OpDebugWitnessApi`.
pub fn new(
provider: Provider,
evm_config: EvmConfig,
task_spawner: Box<dyn TaskSpawner>,
builder: OpPayloadBuilder<EvmConfig, Provider::Primitives>,
) -> Self {
let builder = OpPayloadBuilder::new(evm_config);
let semaphore = Arc::new(Semaphore::new(3));
let inner = OpDebugWitnessApiInner { provider, builder, task_spawner, semaphore };
Self { inner: Arc::new(inner) }
@ -39,7 +38,7 @@ impl<Provider, EvmConfig> OpDebugWitnessApi<Provider, EvmConfig> {
impl<Provider, EvmConfig> OpDebugWitnessApi<Provider, EvmConfig>
where
Provider: BlockReaderIdExt<Header = reth_primitives::Header>,
Provider: NodePrimitivesProvider + BlockReaderIdExt<Header = reth_primitives::Header>,
{
/// Fetches the parent header by hash.
fn parent_header(&self, parent_block_hash: B256) -> ProviderResult<SealedHeader> {
@ -55,11 +54,12 @@ impl<Provider, EvmConfig> DebugExecutionWitnessApiServer<OpPayloadAttributes>
for OpDebugWitnessApi<Provider, EvmConfig>
where
Provider: BlockReaderIdExt<Header = reth_primitives::Header>
+ NodePrimitivesProvider<Primitives: OpPayloadPrimitives>
+ StateProviderFactory
+ ChainSpecProvider<ChainSpec = OpChainSpec>
+ Clone
+ 'static,
EvmConfig: ConfigureEvm<Header = Header, Transaction = OpTransactionSigned> + 'static,
EvmConfig: ConfigureEvmFor<Provider::Primitives> + 'static,
{
async fn execute_payload(
&self,
@ -84,20 +84,26 @@ where
}
}
impl<Provider, EvmConfig> Clone for OpDebugWitnessApi<Provider, EvmConfig> {
impl<Provider, EvmConfig> Clone for OpDebugWitnessApi<Provider, EvmConfig>
where
Provider: NodePrimitivesProvider,
{
fn clone(&self) -> Self {
Self { inner: Arc::clone(&self.inner) }
}
}
impl<Provider, EvmConfig> Debug for OpDebugWitnessApi<Provider, EvmConfig> {
impl<Provider, EvmConfig> Debug for OpDebugWitnessApi<Provider, EvmConfig>
where
Provider: NodePrimitivesProvider,
{
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
f.debug_struct("OpDebugWitnessApi").finish_non_exhaustive()
}
}
struct OpDebugWitnessApiInner<Provider, EvmConfig> {
struct OpDebugWitnessApiInner<Provider: NodePrimitivesProvider, EvmConfig> {
provider: Provider,
builder: OpPayloadBuilder<EvmConfig>,
builder: OpPayloadBuilder<EvmConfig, Provider::Primitives>,
task_spawner: Box<dyn TaskSpawner>,
semaphore: Arc<Semaphore>,
}

View File

@ -25,6 +25,7 @@ alloy-rpc-types-engine = { workspace = true, features = ["serde"] }
op-alloy-rpc-types-engine = { workspace = true, optional = true }
# misc
auto_impl.workspace = true
serde.workspace = true
thiserror.workspace = true
tokio = { workspace = true, default-features = false, features = ["sync"] }

View File

@ -11,6 +11,7 @@ use reth_primitives::{NodePrimitives, SealedBlock};
/// Represents a built payload type that contains a built `SealedBlock` and can be converted into
/// engine API execution payloads.
#[auto_impl::auto_impl(&, Arc)]
pub trait BuiltPayload: Send + Sync + fmt::Debug {
/// The node's primitive types
type Primitives: NodePrimitives;