mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: abstract OP payload builder over transaction (#14153)
This commit is contained in:
1
Cargo.lock
generated
1
Cargo.lock
generated
@ -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",
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -232,7 +232,6 @@ where
|
||||
|
||||
receipts.push(
|
||||
match self.receipt_builder.build_receipt(ReceiptBuilderCtx {
|
||||
header: block.header(),
|
||||
tx: transaction,
|
||||
result,
|
||||
cumulative_gas_used,
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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,17 +398,18 @@ mod tests {
|
||||
|
||||
#[test]
|
||||
fn test_get_extra_data_post_holocene() {
|
||||
let attributes = OpPayloadBuilderAttributes {
|
||||
eip_1559_params: Some(B64::from_str("0x0000000800000008").unwrap()),
|
||||
..Default::default()
|
||||
};
|
||||
let attributes: OpPayloadBuilderAttributes<OpTransactionSigned> =
|
||||
OpPayloadBuilderAttributes {
|
||||
eip_1559_params: Some(B64::from_str("0x0000000800000008").unwrap()),
|
||||
..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, 8, 0, 0, 0, 8]));
|
||||
}
|
||||
|
||||
#[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]));
|
||||
|
||||
30
crates/optimism/payload/src/traits.rs
Normal file
30
crates/optimism/payload/src/traits.rs
Normal 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;
|
||||
}
|
||||
@ -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>,
|
||||
}
|
||||
|
||||
@ -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"] }
|
||||
|
||||
@ -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;
|
||||
|
||||
Reference in New Issue
Block a user