feat: alloy-evm and new revm integration (#14021)

Co-authored-by: Matthias Seitz <matthias.seitz@outlook.de>
Co-authored-by: rakita <rakita@users.noreply.github.com>
This commit is contained in:
Arsenii Kulikov
2025-02-17 23:59:23 +04:00
committed by GitHub
parent bb6dec7ceb
commit 336c3d1fac
142 changed files with 1841 additions and 1929 deletions

View File

@ -14,7 +14,7 @@ workspace = true
# Reth
reth-chainspec.workspace = true
reth-ethereum-forks.workspace = true
reth-evm.workspace = true
reth-evm = { workspace = true, features = ["op"] }
reth-primitives-traits.workspace = true
reth-execution-errors.workspace = true
reth-execution-types.workspace = true
@ -23,7 +23,9 @@ reth-consensus-common.workspace = true
# ethereum
alloy-eips.workspace = true
alloy-evm.workspace = true
alloy-primitives.workspace = true
alloy-op-evm.workspace = true
op-alloy-consensus.workspace = true
alloy-consensus.workspace = true
@ -35,7 +37,9 @@ reth-optimism-primitives.workspace = true
# revm
revm.workspace = true
revm-database.workspace = true
revm-primitives.workspace = true
revm-optimism.workspace = true
# misc
derive_more.workspace = true
@ -74,11 +78,13 @@ std = [
"reth-optimism-chainspec/std",
"reth-execution-errors/std",
"reth-execution-types/std",
"alloy-evm/std",
"alloy-op-evm/std",
"revm-database/std",
"revm-optimism/std",
]
optimism = [
"reth-execution-types/optimism",
"reth-optimism-consensus/optimism",
"revm/optimism",
"revm-primitives/optimism",
"reth-optimism-primitives/optimism",
]

View File

@ -1,13 +1,13 @@
use alloy_consensus::Header;
use reth_optimism_forks::OpHardforks;
use revm_optimism::OpSpecId;
/// Map the latest active hardfork at the given header to a revm
/// [`SpecId`](revm_primitives::SpecId).
pub fn revm_spec(chain_spec: impl OpHardforks, header: &Header) -> revm_primitives::SpecId {
/// Map the latest active hardfork at the given header to a revm [`OpSpecId`].
pub fn revm_spec(chain_spec: impl OpHardforks, header: &Header) -> OpSpecId {
revm_spec_by_timestamp_after_bedrock(chain_spec, header.timestamp)
}
/// Returns the revm [`SpecId`](revm_primitives::SpecId) at the given timestamp.
/// Returns the revm [`OpSpecId`] at the given timestamp.
///
/// # Note
///
@ -16,23 +16,23 @@ pub fn revm_spec(chain_spec: impl OpHardforks, header: &Header) -> revm_primitiv
pub fn revm_spec_by_timestamp_after_bedrock(
chain_spec: impl OpHardforks,
timestamp: u64,
) -> revm_primitives::SpecId {
) -> OpSpecId {
if chain_spec.is_isthmus_active_at_timestamp(timestamp) {
revm_primitives::ISTHMUS
OpSpecId::ISTHMUS
} else if chain_spec.is_holocene_active_at_timestamp(timestamp) {
revm_primitives::HOLOCENE
OpSpecId::HOLOCENE
} else if chain_spec.is_granite_active_at_timestamp(timestamp) {
revm_primitives::GRANITE
OpSpecId::GRANITE
} else if chain_spec.is_fjord_active_at_timestamp(timestamp) {
revm_primitives::FJORD
OpSpecId::FJORD
} else if chain_spec.is_ecotone_active_at_timestamp(timestamp) {
revm_primitives::ECOTONE
OpSpecId::ECOTONE
} else if chain_spec.is_canyon_active_at_timestamp(timestamp) {
revm_primitives::CANYON
OpSpecId::CANYON
} else if chain_spec.is_regolith_active_at_timestamp(timestamp) {
revm_primitives::REGOLITH
OpSpecId::REGOLITH
} else {
revm_primitives::BEDROCK
OpSpecId::BEDROCK
}
}
@ -51,35 +51,35 @@ mod tests {
}
assert_eq!(
revm_spec_by_timestamp_after_bedrock(op_cs(|cs| cs.isthmus_activated()), 0),
revm_primitives::ISTHMUS
OpSpecId::ISTHMUS
);
assert_eq!(
revm_spec_by_timestamp_after_bedrock(op_cs(|cs| cs.holocene_activated()), 0),
revm_primitives::HOLOCENE
OpSpecId::HOLOCENE
);
assert_eq!(
revm_spec_by_timestamp_after_bedrock(op_cs(|cs| cs.granite_activated()), 0),
revm_primitives::GRANITE
OpSpecId::GRANITE
);
assert_eq!(
revm_spec_by_timestamp_after_bedrock(op_cs(|cs| cs.fjord_activated()), 0),
revm_primitives::FJORD
OpSpecId::FJORD
);
assert_eq!(
revm_spec_by_timestamp_after_bedrock(op_cs(|cs| cs.ecotone_activated()), 0),
revm_primitives::ECOTONE
OpSpecId::ECOTONE
);
assert_eq!(
revm_spec_by_timestamp_after_bedrock(op_cs(|cs| cs.canyon_activated()), 0),
revm_primitives::CANYON
OpSpecId::CANYON
);
assert_eq!(
revm_spec_by_timestamp_after_bedrock(op_cs(|cs| cs.bedrock_activated()), 0),
revm_primitives::BEDROCK
OpSpecId::BEDROCK
);
assert_eq!(
revm_spec_by_timestamp_after_bedrock(op_cs(|cs| cs.regolith_activated()), 0),
revm_primitives::REGOLITH
OpSpecId::REGOLITH
);
}
@ -92,35 +92,35 @@ mod tests {
}
assert_eq!(
revm_spec(op_cs(|cs| cs.isthmus_activated()), &Default::default()),
revm_primitives::ISTHMUS
OpSpecId::ISTHMUS
);
assert_eq!(
revm_spec(op_cs(|cs| cs.holocene_activated()), &Default::default()),
revm_primitives::HOLOCENE
OpSpecId::HOLOCENE
);
assert_eq!(
revm_spec(op_cs(|cs| cs.granite_activated()), &Default::default()),
revm_primitives::GRANITE
OpSpecId::GRANITE
);
assert_eq!(
revm_spec(op_cs(|cs| cs.fjord_activated()), &Default::default()),
revm_primitives::FJORD
OpSpecId::FJORD
);
assert_eq!(
revm_spec(op_cs(|cs| cs.ecotone_activated()), &Default::default()),
revm_primitives::ECOTONE
OpSpecId::ECOTONE
);
assert_eq!(
revm_spec(op_cs(|cs| cs.canyon_activated()), &Default::default()),
revm_primitives::CANYON
OpSpecId::CANYON
);
assert_eq!(
revm_spec(op_cs(|cs| cs.bedrock_activated()), &Default::default()),
revm_primitives::BEDROCK
OpSpecId::BEDROCK
);
assert_eq!(
revm_spec(op_cs(|cs| cs.regolith_activated()), &Default::default()),
revm_primitives::REGOLITH
OpSpecId::REGOLITH
);
}
}

View File

@ -16,15 +16,15 @@ use reth_evm::{
},
state_change::post_block_balance_increments,
system_calls::{OnStateHook, StateChangePostBlockSource, StateChangeSource, SystemCaller},
ConfigureEvmFor, Database, Evm,
ConfigureEvm, ConfigureEvmFor, Database, Evm, HaltReasonFor,
};
use reth_optimism_chainspec::OpChainSpec;
use reth_optimism_consensus::validate_block_post_execution;
use reth_optimism_forks::OpHardforks;
use reth_optimism_primitives::{transaction::signed::OpTransaction, DepositReceipt, OpPrimitives};
use reth_primitives_traits::{BlockBody, NodePrimitives, RecoveredBlock, SignedTransaction};
use revm::State;
use revm_primitives::{db::DatabaseCommit, ResultAndState};
use revm::{context_interface::result::ResultAndState, DatabaseCommit};
use revm_database::State;
use tracing::trace;
/// Factory for [`OpExecutionStrategy`].
@ -32,14 +32,15 @@ use tracing::trace;
pub struct OpExecutionStrategyFactory<
N: NodePrimitives = OpPrimitives,
ChainSpec = OpChainSpec,
EvmConfig = OpEvmConfig<ChainSpec>,
EvmConfig: ConfigureEvm = OpEvmConfig<ChainSpec>,
> {
/// The chainspec
chain_spec: Arc<ChainSpec>,
/// How to create an EVM.
evm_config: EvmConfig,
/// Receipt builder.
receipt_builder: Arc<dyn OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>>,
receipt_builder:
Arc<dyn OpReceiptBuilder<N::SignedTx, HaltReasonFor<EvmConfig>, Receipt = N::Receipt>>,
}
impl OpExecutionStrategyFactory<OpPrimitives> {
@ -53,12 +54,18 @@ impl OpExecutionStrategyFactory<OpPrimitives> {
}
}
impl<N: NodePrimitives, ChainSpec, EvmConfig> OpExecutionStrategyFactory<N, ChainSpec, EvmConfig> {
impl<N: NodePrimitives, ChainSpec, EvmConfig: ConfigureEvm>
OpExecutionStrategyFactory<N, ChainSpec, EvmConfig>
{
/// Creates a new executor strategy factory.
pub fn new(
chain_spec: Arc<ChainSpec>,
evm_config: EvmConfig,
receipt_builder: impl OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>,
receipt_builder: impl OpReceiptBuilder<
N::SignedTx,
HaltReasonFor<EvmConfig>,
Receipt = N::Receipt,
>,
) -> Self {
Self { chain_spec, evm_config, receipt_builder: Arc::new(receipt_builder) }
}
@ -93,7 +100,7 @@ where
#[allow(missing_debug_implementations)]
pub struct OpExecutionStrategy<DB, N: NodePrimitives, ChainSpec, EvmConfig>
where
EvmConfig: Clone,
EvmConfig: ConfigureEvm,
{
/// The chainspec
chain_spec: Arc<ChainSpec>,
@ -104,21 +111,24 @@ where
/// Utility to call system smart contracts.
system_caller: SystemCaller<EvmConfig, ChainSpec>,
/// Receipt builder.
receipt_builder: Arc<dyn OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>>,
receipt_builder:
Arc<dyn OpReceiptBuilder<N::SignedTx, HaltReasonFor<EvmConfig>, Receipt = N::Receipt>>,
}
impl<DB, N, ChainSpec, EvmConfig> OpExecutionStrategy<DB, N, ChainSpec, EvmConfig>
where
N: NodePrimitives,
ChainSpec: OpHardforks,
EvmConfig: Clone,
EvmConfig: ConfigureEvm,
{
/// Creates a new [`OpExecutionStrategy`]
pub fn new(
state: State<DB>,
chain_spec: Arc<ChainSpec>,
evm_config: EvmConfig,
receipt_builder: Arc<dyn OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>>,
receipt_builder: Arc<
dyn OpReceiptBuilder<N::SignedTx, HaltReasonFor<EvmConfig>, Receipt = N::Receipt>,
>,
) -> Self {
let system_caller = SystemCaller::new(evm_config.clone(), chain_spec.clone());
Self { state, chain_spec, evm_config, system_caller, receipt_builder }
@ -292,7 +302,7 @@ where
&mut self.state
}
fn into_state(self) -> revm::db::State<Self::DB> {
fn into_state(self) -> revm_database::State<Self::DB> {
self.state
}
@ -337,9 +347,8 @@ mod tests {
use reth_optimism_chainspec::OpChainSpecBuilder;
use reth_optimism_primitives::{OpReceipt, OpTransactionSigned};
use reth_primitives_traits::Account;
use reth_revm::{
database::StateProviderDatabase, test_utils::StateProviderTest, L1_BLOCK_CONTRACT,
};
use reth_revm::{database::StateProviderDatabase, test_utils::StateProviderTest};
use revm_optimism::constants::L1_BLOCK_CONTRACT;
use std::{collections::HashMap, str::FromStr};
fn create_op_state_provider() -> StateProviderTest {

View File

@ -6,10 +6,8 @@ use alloy_primitives::{address, b256, hex, Address, Bytes, B256, U256};
use reth_execution_errors::BlockExecutionError;
use reth_optimism_forks::OpHardforks;
use reth_primitives_traits::BlockBody;
use revm::{
primitives::{Bytecode, HashMap, SpecId},
DatabaseCommit, L1BlockInfo,
};
use revm::{primitives::HashMap, state::Bytecode, DatabaseCommit};
use revm_optimism::{L1BlockInfo, OpSpecId};
use tracing::trace;
/// The address of the create2 deployer
@ -272,13 +270,13 @@ impl RethL1BlockInfo for L1BlockInfo {
}
let spec_id = if chain_spec.is_fjord_active_at_timestamp(timestamp) {
SpecId::FJORD
OpSpecId::FJORD
} else if chain_spec.is_ecotone_active_at_timestamp(timestamp) {
SpecId::ECOTONE
OpSpecId::ECOTONE
} else if chain_spec.is_regolith_active_at_timestamp(timestamp) {
SpecId::REGOLITH
OpSpecId::REGOLITH
} else if chain_spec.is_bedrock_active_at_block(block_number) {
SpecId::BEDROCK
OpSpecId::BEDROCK
} else {
return Err(
OpBlockExecutionError::L1BlockInfo(L1BlockInfoError::HardforksNotActive).into()
@ -295,11 +293,11 @@ impl RethL1BlockInfo for L1BlockInfo {
input: &[u8],
) -> Result<U256, BlockExecutionError> {
let spec_id = if chain_spec.is_fjord_active_at_timestamp(timestamp) {
SpecId::FJORD
OpSpecId::FJORD
} else if chain_spec.is_regolith_active_at_timestamp(timestamp) {
SpecId::REGOLITH
OpSpecId::REGOLITH
} else if chain_spec.is_bedrock_active_at_block(block_number) {
SpecId::BEDROCK
OpSpecId::BEDROCK
} else {
return Err(
OpBlockExecutionError::L1BlockInfo(L1BlockInfoError::HardforksNotActive).into()
@ -315,7 +313,7 @@ impl RethL1BlockInfo for L1BlockInfo {
pub fn ensure_create2_deployer<DB>(
chain_spec: impl OpHardforks,
timestamp: u64,
db: &mut revm::State<DB>,
db: &mut revm_database::State<DB>,
) -> Result<(), DB::Error>
where
DB: revm::Database,
@ -337,7 +335,7 @@ where
acc_info.code = Some(Bytecode::new_raw(Bytes::from_static(&CREATE_2_DEPLOYER_BYTECODE)));
// Convert the cache account back into a revm account and mark it as touched.
let mut revm_acc: revm::primitives::Account = acc_info.into();
let mut revm_acc: revm::state::Account = acc_info.into();
revm_acc.mark_touch();
// Commit the create2 deployer account to the database.

View File

@ -12,23 +12,25 @@
extern crate alloc;
use alloc::{sync::Arc, vec::Vec};
use alloc::sync::Arc;
use alloy_consensus::{BlockHeader, Header};
use alloy_op_evm::OpEvmFactory;
use alloy_primitives::{Address, U256};
use core::fmt::Debug;
use op_alloy_consensus::EIP1559ParamError;
use reth_chainspec::EthChainSpec;
use reth_evm::{env::EvmEnv, ConfigureEvm, ConfigureEvmEnv, Database, Evm, NextBlockEnvAttributes};
use reth_evm::{ConfigureEvm, ConfigureEvmEnv, EvmEnv, NextBlockEnvAttributes};
use reth_optimism_chainspec::OpChainSpec;
use reth_optimism_consensus::next_block_base_fee;
use reth_optimism_forks::OpHardforks;
use reth_optimism_primitives::OpTransactionSigned;
use reth_primitives_traits::FillTxEnv;
use revm::{
inspector_handle_register,
primitives::{AnalysisKind, CfgEnvWithHandlerCfg, TxEnv},
EvmBuilder, GetInspector,
context::{BlockEnv, CfgEnv, TxEnv},
context_interface::block::BlobExcessGasAndPrice,
specification::hardfork::SpecId,
};
use revm_optimism::{OpSpecId, OpTransaction};
mod config;
pub use config::{revm_spec, revm_spec_by_timestamp_after_bedrock};
@ -41,108 +43,24 @@ pub use receipts::*;
mod error;
pub use error::OpBlockExecutionError;
use revm_primitives::{
BlobExcessGasAndPrice, BlockEnv, Bytes, CfgEnv, EVMError, HaltReason, HandlerCfg,
OptimismFields, ResultAndState, SpecId, TxKind,
};
/// OP EVM implementation.
#[derive(derive_more::Debug, derive_more::Deref, derive_more::DerefMut, derive_more::From)]
#[debug(bound(DB::Error: Debug))]
pub struct OpEvm<'a, EXT, DB: Database>(revm::Evm<'a, EXT, DB>);
impl<EXT, DB: Database> Evm for OpEvm<'_, EXT, DB> {
type DB = DB;
type Tx = TxEnv;
type Error = EVMError<DB::Error>;
type HaltReason = HaltReason;
fn block(&self) -> &BlockEnv {
self.0.block()
}
fn transact(&mut self, tx: Self::Tx) -> Result<ResultAndState, Self::Error> {
*self.tx_mut() = tx;
self.0.transact()
}
fn transact_system_call(
&mut self,
caller: Address,
contract: Address,
data: Bytes,
) -> Result<ResultAndState, Self::Error> {
#[allow(clippy::needless_update)] // side-effect of optimism fields
let tx_env = TxEnv {
caller,
transact_to: TxKind::Call(contract),
// Explicitly set nonce to None so revm does not do any nonce checks
nonce: None,
gas_limit: 30_000_000,
value: U256::ZERO,
data,
// Setting the gas price to zero enforces that no value is transferred as part of the
// call, and that the call will not count against the block's gas limit
gas_price: U256::ZERO,
// The chain ID check is not relevant here and is disabled if set to None
chain_id: None,
// Setting the gas priority fee to None ensures the effective gas price is derived from
// the `gas_price` field, which we need to be zero
gas_priority_fee: None,
access_list: Vec::new(),
// blob fields can be None for this tx
blob_hashes: Vec::new(),
max_fee_per_blob_gas: None,
authorization_list: None,
optimism: OptimismFields {
source_hash: None,
mint: None,
is_system_transaction: Some(false),
// The L1 fee is not charged for the EIP-4788 transaction, submit zero bytes for the
// enveloped tx size.
enveloped_tx: Some(Bytes::default()),
},
};
*self.tx_mut() = tx_env;
let prev_block_env = self.block().clone();
// ensure the block gas limit is >= the tx
self.block_mut().gas_limit = U256::from(self.tx().gas_limit);
// disable the base fee check for this call by setting the base fee to zero
self.block_mut().basefee = U256::ZERO;
let res = self.0.transact();
// re-set the block env
*self.block_mut() = prev_block_env;
res
}
fn db_mut(&mut self) -> &mut Self::DB {
&mut self.context.evm.db
}
}
/// Optimism-related EVM configuration.
#[derive(Debug)]
pub struct OpEvmConfig<ChainSpec = OpChainSpec> {
chain_spec: Arc<ChainSpec>,
evm_factory: OpEvmFactory,
}
impl<ChainSpec> Clone for OpEvmConfig<ChainSpec> {
fn clone(&self) -> Self {
Self { chain_spec: self.chain_spec.clone() }
Self { chain_spec: self.chain_spec.clone(), evm_factory: OpEvmFactory::default() }
}
}
impl<ChainSpec> OpEvmConfig<ChainSpec> {
/// Creates a new [`OpEvmConfig`] with the given chain spec.
pub const fn new(chain_spec: Arc<ChainSpec>) -> Self {
Self { chain_spec }
pub fn new(chain_spec: Arc<ChainSpec>) -> Self {
Self { chain_spec, evm_factory: OpEvmFactory::default() }
}
/// Returns the chain spec associated with this configuration.
@ -155,130 +73,87 @@ impl<ChainSpec: EthChainSpec + OpHardforks + 'static> ConfigureEvmEnv for OpEvmC
type Header = Header;
type Transaction = OpTransactionSigned;
type Error = EIP1559ParamError;
type TxEnv = TxEnv;
type Spec = SpecId;
type TxEnv = OpTransaction<TxEnv>;
type Spec = OpSpecId;
fn tx_env(&self, transaction: &Self::Transaction, signer: Address) -> Self::TxEnv {
let mut tx_env = TxEnv::default();
let mut tx_env = Default::default();
transaction.fill_tx_env(&mut tx_env, signer);
tx_env
}
fn evm_env(&self, header: &Self::Header) -> EvmEnv {
fn evm_env(&self, header: &Self::Header) -> EvmEnv<Self::Spec> {
let spec = config::revm_spec(self.chain_spec(), header);
let mut cfg_env = CfgEnv::default();
cfg_env.chain_id = self.chain_spec.chain().id();
cfg_env.perf_analyse_created_bytecodes = AnalysisKind::default();
let cfg_env = CfgEnv::new().with_chain_id(self.chain_spec.chain().id()).with_spec(spec);
let block_env = BlockEnv {
number: U256::from(header.number()),
coinbase: header.beneficiary(),
timestamp: U256::from(header.timestamp()),
difficulty: if spec >= SpecId::MERGE { U256::ZERO } else { header.difficulty() },
prevrandao: if spec >= SpecId::MERGE { header.mix_hash() } else { None },
gas_limit: U256::from(header.gas_limit()),
basefee: U256::from(header.base_fee_per_gas().unwrap_or_default()),
number: header.number(),
beneficiary: header.beneficiary(),
timestamp: header.timestamp(),
difficulty: if spec.into_eth_spec() >= SpecId::MERGE {
U256::ZERO
} else {
header.difficulty()
},
prevrandao: if spec.into_eth_spec() >= SpecId::MERGE {
header.mix_hash()
} else {
None
},
gas_limit: header.gas_limit(),
basefee: header.base_fee_per_gas().unwrap_or_default(),
// EIP-4844 excess blob gas of this block, introduced in Cancun
blob_excess_gas_and_price: header.excess_blob_gas().map(|excess_blob_gas| {
BlobExcessGasAndPrice::new(excess_blob_gas, spec >= SpecId::PRAGUE)
BlobExcessGasAndPrice::new(excess_blob_gas, spec.into_eth_spec() >= SpecId::PRAGUE)
}),
};
EvmEnv { cfg_env, block_env, spec }
EvmEnv { cfg_env, block_env }
}
fn next_evm_env(
&self,
parent: &Self::Header,
attributes: NextBlockEnvAttributes,
) -> Result<EvmEnv, Self::Error> {
// configure evm env based on parent block
let cfg = CfgEnv::default().with_chain_id(self.chain_spec.chain().id());
) -> Result<EvmEnv<Self::Spec>, Self::Error> {
// ensure we're not missing any timestamp based hardforks
let spec_id = revm_spec_by_timestamp_after_bedrock(&self.chain_spec, attributes.timestamp);
// configure evm env based on parent block
let cfg_env = CfgEnv::new().with_chain_id(self.chain_spec.chain().id()).with_spec(spec_id);
// if the parent block did not have excess blob gas (i.e. it was pre-cancun), but it is
// cancun now, we need to set the excess blob gas to the default value(0)
let blob_excess_gas_and_price = parent
.maybe_next_block_excess_blob_gas(
self.chain_spec().blob_params_at_timestamp(attributes.timestamp),
)
.or_else(|| (spec_id.is_enabled_in(SpecId::CANCUN)).then_some(0))
.or_else(|| (spec_id.into_eth_spec().is_enabled_in(SpecId::CANCUN)).then_some(0))
.map(|gas| BlobExcessGasAndPrice::new(gas, false));
let block_env = BlockEnv {
number: U256::from(parent.number + 1),
coinbase: attributes.suggested_fee_recipient,
timestamp: U256::from(attributes.timestamp),
number: parent.number + 1,
beneficiary: attributes.suggested_fee_recipient,
timestamp: attributes.timestamp,
difficulty: U256::ZERO,
prevrandao: Some(attributes.prev_randao),
gas_limit: U256::from(attributes.gas_limit),
gas_limit: attributes.gas_limit,
// calculate basefee based on parent block's gas usage
basefee: U256::from(next_block_base_fee(
&self.chain_spec,
parent,
attributes.timestamp,
)?),
basefee: next_block_base_fee(&self.chain_spec, parent, attributes.timestamp)?,
// calculate excess gas based on parent block's blob gas usage
blob_excess_gas_and_price,
};
let cfg_env_with_handler_cfg;
{
cfg_env_with_handler_cfg = CfgEnvWithHandlerCfg {
cfg_env: cfg,
handler_cfg: HandlerCfg { spec_id, is_optimism: true },
};
}
Ok((cfg_env_with_handler_cfg, block_env).into())
Ok(EvmEnv { cfg_env, block_env })
}
}
impl<ChainSpec: EthChainSpec + OpHardforks + 'static> ConfigureEvm for OpEvmConfig<ChainSpec> {
type Evm<'a, DB: Database + 'a, I: 'a> = OpEvm<'a, I, DB>;
type EvmError<DBError: core::error::Error + Send + Sync + 'static> = EVMError<DBError>;
type HaltReason = HaltReason;
type EvmFactory = OpEvmFactory;
fn evm_with_env<DB: Database>(&self, db: DB, evm_env: EvmEnv) -> Self::Evm<'_, DB, ()> {
let cfg_env_with_handler_cfg = CfgEnvWithHandlerCfg {
cfg_env: evm_env.cfg_env,
handler_cfg: HandlerCfg { spec_id: evm_env.spec, is_optimism: true },
};
EvmBuilder::default()
.with_db(db)
.with_cfg_env_with_handler_cfg(cfg_env_with_handler_cfg)
.with_block_env(evm_env.block_env)
.build()
.into()
}
fn evm_with_env_and_inspector<DB, I>(
&self,
db: DB,
evm_env: EvmEnv,
inspector: I,
) -> Self::Evm<'_, DB, I>
where
DB: Database,
I: GetInspector<DB>,
{
let cfg_env_with_handler_cfg = CfgEnvWithHandlerCfg {
cfg_env: evm_env.cfg_env,
handler_cfg: HandlerCfg { spec_id: evm_env.spec, is_optimism: true },
};
EvmBuilder::default()
.with_db(db)
.with_external_context(inspector)
.with_cfg_env_with_handler_cfg(cfg_env_with_handler_cfg)
.with_block_env(evm_env.block_env)
.append_handler_register(inspector_handle_register)
.build()
.into()
fn evm_factory(&self) -> &Self::EvmFactory {
&self.evm_factory
}
}
@ -288,7 +163,7 @@ mod tests {
use alloy_consensus::{Header, Receipt};
use alloy_eips::eip7685::Requests;
use alloy_genesis::Genesis;
use alloy_primitives::{bytes, map::HashMap, Address, LogData, B256, U256};
use alloy_primitives::{bytes, map::HashMap, Address, LogData, B256};
use reth_chainspec::ChainSpec;
use reth_evm::execute::ProviderError;
use reth_execution_types::{
@ -296,13 +171,11 @@ mod tests {
};
use reth_optimism_chainspec::BASE_MAINNET;
use reth_optimism_primitives::{OpBlock, OpPrimitives, OpReceipt};
use reth_primitives_traits::{Account, Log, RecoveredBlock};
use revm::{
db::{BundleState, CacheDB, EmptyDBTyped},
inspectors::NoOpInspector,
primitives::{AccountInfo, BlockEnv, CfgEnv, SpecId},
};
use revm_primitives::HandlerCfg;
use reth_primitives_traits::{Account, RecoveredBlock};
use revm::{database_interface::EmptyDBTyped, inspector::NoOpInspector, state::AccountInfo};
use revm_database::{BundleState, CacheDB};
use revm_optimism::OpSpecId;
use revm_primitives::Log;
use std::sync::Arc;
fn test_evm_config() -> OpEvmConfig {
@ -345,13 +218,7 @@ mod tests {
let evm = evm_config.evm_with_env(db, evm_env.clone());
// Check that the EVM environment
assert_eq!(evm.context.evm.env.cfg, evm_env.cfg_env);
// Default spec ID
assert_eq!(evm.handler.spec_id(), SpecId::LATEST);
// Optimism in handler
assert_eq!(evm.handler.cfg, HandlerCfg { spec_id: SpecId::LATEST, is_optimism: true });
assert_eq!(evm.cfg, evm_env.cfg_env);
}
#[test]
@ -361,20 +228,14 @@ mod tests {
let db = CacheDB::<EmptyDBTyped<ProviderError>>::default();
// Create a custom configuration environment with a chain ID of 111
let cfg = CfgEnv::default().with_chain_id(111);
let cfg = CfgEnv::new().with_chain_id(111).with_spec(OpSpecId::default());
let evm_env = EvmEnv { cfg_env: cfg.clone(), ..Default::default() };
let evm = evm_config.evm_with_env(db, evm_env);
// Check that the EVM environment is initialized with the custom environment
assert_eq!(evm.context.evm.inner.env.cfg, cfg);
// Default spec ID
assert_eq!(evm.handler.spec_id(), SpecId::LATEST);
// Optimism in handler
assert_eq!(evm.handler.cfg, HandlerCfg { spec_id: SpecId::LATEST, is_optimism: true });
assert_eq!(evm.cfg, cfg);
}
#[test]
@ -384,25 +245,15 @@ mod tests {
let db = CacheDB::<EmptyDBTyped<ProviderError>>::default();
// Create customs block and tx env
let block = BlockEnv {
basefee: U256::from(1000),
gas_limit: U256::from(10_000_000),
number: U256::from(42),
..Default::default()
};
let block =
BlockEnv { basefee: 1000, gas_limit: 10_000_000, number: 42, ..Default::default() };
let evm_env = EvmEnv { block_env: block, ..Default::default() };
let evm = evm_config.evm_with_env(db, evm_env.clone());
// Verify that the block and transaction environments are set correctly
assert_eq!(evm.context.evm.env.block, evm_env.block_env);
// Default spec ID
assert_eq!(evm.handler.spec_id(), SpecId::LATEST);
// Optimism in handler
assert_eq!(evm.handler.cfg, HandlerCfg { spec_id: SpecId::LATEST, is_optimism: true });
assert_eq!(evm.block, evm_env.block_env);
}
#[test]
@ -411,15 +262,12 @@ mod tests {
let db = CacheDB::<EmptyDBTyped<ProviderError>>::default();
let evm_env = EvmEnv { spec: SpecId::ECOTONE, ..Default::default() };
let evm_env =
EvmEnv { cfg_env: CfgEnv::new().with_spec(OpSpecId::ECOTONE), ..Default::default() };
let evm = evm_config.evm_with_env(db, evm_env);
let evm = evm_config.evm_with_env(db, evm_env.clone());
// Check that the spec ID is setup properly
assert_eq!(evm.handler.spec_id(), SpecId::ECOTONE);
// Optimism in handler
assert_eq!(evm.handler.cfg, HandlerCfg { spec_id: SpecId::ECOTONE, is_optimism: true });
assert_eq!(evm.cfg, evm_env.cfg_env);
}
#[test]
@ -429,17 +277,11 @@ mod tests {
let evm_env = EvmEnv { cfg_env: Default::default(), ..Default::default() };
let evm = evm_config.evm_with_env_and_inspector(db, evm_env.clone(), NoOpInspector);
let evm = evm_config.evm_with_env_and_inspector(db, evm_env.clone(), NoOpInspector {});
// Check that the EVM environment is set to default values
assert_eq!(evm.context.evm.env.block, evm_env.block_env);
assert_eq!(evm.context.evm.env.cfg, evm_env.cfg_env);
assert_eq!(evm.context.evm.env.tx, Default::default());
assert_eq!(evm.context.external, NoOpInspector);
assert_eq!(evm.handler.spec_id(), SpecId::LATEST);
// Optimism in handler
assert_eq!(evm.handler.cfg, HandlerCfg { spec_id: SpecId::LATEST, is_optimism: true });
assert_eq!(evm.block, evm_env.block_env);
assert_eq!(evm.cfg, evm_env.cfg_env);
}
#[test]
@ -447,20 +289,15 @@ mod tests {
let evm_config = test_evm_config();
let db = CacheDB::<EmptyDBTyped<ProviderError>>::default();
let cfg = CfgEnv::default().with_chain_id(111);
let cfg = CfgEnv::new().with_chain_id(111).with_spec(OpSpecId::default());
let block = BlockEnv::default();
let evm_env = EvmEnv { block_env: block, cfg_env: cfg.clone(), ..Default::default() };
let evm_env = EvmEnv { block_env: block, cfg_env: cfg.clone() };
let evm = evm_config.evm_with_env_and_inspector(db, evm_env.clone(), NoOpInspector);
let evm = evm_config.evm_with_env_and_inspector(db, evm_env.clone(), NoOpInspector {});
// Check that the EVM environment is set with custom configuration
assert_eq!(evm.context.evm.env.cfg, cfg);
assert_eq!(evm.context.evm.env.block, evm_env.block_env);
assert_eq!(evm.context.external, NoOpInspector);
assert_eq!(evm.handler.spec_id(), SpecId::LATEST);
// Optimism in handler
assert_eq!(evm.handler.cfg, HandlerCfg { spec_id: SpecId::LATEST, is_optimism: true });
assert_eq!(evm.cfg, cfg);
assert_eq!(evm.block, evm_env.block_env);
}
#[test]
@ -469,23 +306,14 @@ mod tests {
let db = CacheDB::<EmptyDBTyped<ProviderError>>::default();
// Create custom block and tx environment
let block = BlockEnv {
basefee: U256::from(1000),
gas_limit: U256::from(10_000_000),
number: U256::from(42),
..Default::default()
};
let block =
BlockEnv { basefee: 1000, gas_limit: 10_000_000, number: 42, ..Default::default() };
let evm_env = EvmEnv { block_env: block, ..Default::default() };
let evm = evm_config.evm_with_env_and_inspector(db, evm_env.clone(), NoOpInspector);
let evm = evm_config.evm_with_env_and_inspector(db, evm_env.clone(), NoOpInspector {});
// Verify that the block and transaction environments are set correctly
assert_eq!(evm.context.evm.env.block, evm_env.block_env);
assert_eq!(evm.context.external, NoOpInspector);
assert_eq!(evm.handler.spec_id(), SpecId::LATEST);
// Optimism in handler
assert_eq!(evm.handler.cfg, HandlerCfg { spec_id: SpecId::LATEST, is_optimism: true });
assert_eq!(evm.block, evm_env.block_env);
}
#[test]
@ -493,21 +321,14 @@ mod tests {
let evm_config = test_evm_config();
let db = CacheDB::<EmptyDBTyped<ProviderError>>::default();
let evm_env = EvmEnv { spec: SpecId::ECOTONE, ..Default::default() };
let evm_env =
EvmEnv { cfg_env: CfgEnv::new().with_spec(OpSpecId::ECOTONE), ..Default::default() };
let evm = evm_config.evm_with_env_and_inspector(db, evm_env.clone(), NoOpInspector);
let evm = evm_config.evm_with_env_and_inspector(db, evm_env.clone(), NoOpInspector {});
// Check that the spec ID is set properly
assert_eq!(evm.handler.spec_id(), SpecId::ECOTONE);
assert_eq!(evm.context.evm.env.cfg, evm_env.cfg_env);
assert_eq!(evm.context.evm.env.block, evm_env.block_env);
assert_eq!(evm.context.external, NoOpInspector);
// Check that the spec ID is setup properly
assert_eq!(evm.handler.spec_id(), SpecId::ECOTONE);
// Optimism in handler
assert_eq!(evm.handler.cfg, HandlerCfg { spec_id: SpecId::ECOTONE, is_optimism: true });
assert_eq!(evm.cfg, evm_env.cfg_env);
assert_eq!(evm.block, evm_env.block_env);
}
#[test]

View File

@ -2,21 +2,21 @@ use alloy_consensus::{Eip658Value, Receipt};
use core::fmt;
use op_alloy_consensus::{OpDepositReceipt, OpTxType};
use reth_optimism_primitives::{OpReceipt, OpTransactionSigned};
use revm_primitives::ExecutionResult;
use revm::context_interface::result::ExecutionResult;
/// Context for building a receipt.
#[derive(Debug)]
pub struct ReceiptBuilderCtx<'a, T> {
pub struct ReceiptBuilderCtx<'a, T, Halt> {
/// Transaction
pub tx: &'a T,
/// Result of transaction execution.
pub result: ExecutionResult,
pub result: ExecutionResult<Halt>,
/// Cumulative gas used.
pub cumulative_gas_used: u64,
}
/// Type that knows how to build a receipt based on execution result.
pub trait OpReceiptBuilder<T>: fmt::Debug + Send + Sync + Unpin + 'static {
pub trait OpReceiptBuilder<T, Halt>: fmt::Debug + Send + Sync + Unpin + 'static {
/// Receipt type.
type Receipt: Send + Sync + Clone + Unpin + 'static;
@ -26,8 +26,8 @@ pub trait OpReceiptBuilder<T>: fmt::Debug + Send + Sync + Unpin + 'static {
/// case, the `build_deposit_receipt` method will be called.
fn build_receipt<'a>(
&self,
ctx: ReceiptBuilderCtx<'a, T>,
) -> Result<Self::Receipt, ReceiptBuilderCtx<'a, T>>;
ctx: ReceiptBuilderCtx<'a, T, Halt>,
) -> Result<Self::Receipt, ReceiptBuilderCtx<'a, T, Halt>>;
/// Builds receipt for a deposit transaction.
fn build_deposit_receipt(&self, inner: OpDepositReceipt) -> Self::Receipt;
@ -38,13 +38,13 @@ pub trait OpReceiptBuilder<T>: fmt::Debug + Send + Sync + Unpin + 'static {
#[non_exhaustive]
pub struct BasicOpReceiptBuilder;
impl OpReceiptBuilder<OpTransactionSigned> for BasicOpReceiptBuilder {
impl<Halt> OpReceiptBuilder<OpTransactionSigned, Halt> for BasicOpReceiptBuilder {
type Receipt = OpReceipt;
fn build_receipt<'a>(
&self,
ctx: ReceiptBuilderCtx<'a, OpTransactionSigned>,
) -> Result<Self::Receipt, ReceiptBuilderCtx<'a, OpTransactionSigned>> {
ctx: ReceiptBuilderCtx<'a, OpTransactionSigned, Halt>,
) -> Result<Self::Receipt, ReceiptBuilderCtx<'a, OpTransactionSigned, Halt>> {
match ctx.tx.tx_type() {
OpTxType::Deposit => Err(ctx),
ty => {