mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 02:49:55 +00:00
refactor: BlockExecutionStrategy API (#14480)
This commit is contained in:
5
Cargo.lock
generated
5
Cargo.lock
generated
@ -7744,12 +7744,12 @@ dependencies = [
|
||||
"alloy-evm",
|
||||
"alloy-primitives",
|
||||
"auto_impl",
|
||||
"derive_more 2.0.1",
|
||||
"futures-util",
|
||||
"metrics",
|
||||
"metrics-util",
|
||||
"parking_lot",
|
||||
"reth-chainspec",
|
||||
"reth-consensus",
|
||||
"reth-consensus-common",
|
||||
"reth-ethereum-forks",
|
||||
"reth-ethereum-primitives",
|
||||
@ -7774,9 +7774,8 @@ dependencies = [
|
||||
"alloy-genesis",
|
||||
"alloy-primitives",
|
||||
"alloy-sol-types",
|
||||
"derive_more 2.0.1",
|
||||
"reth-chainspec",
|
||||
"reth-consensus",
|
||||
"reth-ethereum-consensus",
|
||||
"reth-ethereum-forks",
|
||||
"reth-evm",
|
||||
"reth-execution-types",
|
||||
|
||||
@ -8,7 +8,6 @@ use reth_evm::{
|
||||
state_change::post_block_balance_increments, system_calls::SystemCaller, ConfigureEvmFor, Evm,
|
||||
};
|
||||
use reth_primitives::{NodePrimitives, RecoveredBlock, SealedHeader};
|
||||
use reth_primitives_traits::{BlockBody, SignedTransaction};
|
||||
use reth_provider::{BlockExecutionOutput, ChainSpecProvider, StateProviderFactory};
|
||||
use reth_revm::{
|
||||
database::StateProviderDatabase,
|
||||
@ -85,10 +84,8 @@ where
|
||||
|
||||
// Re-execute all of the transactions in the block to load all touched accounts into
|
||||
// the cache DB.
|
||||
for tx in block.body().transactions() {
|
||||
let signer =
|
||||
tx.recover_signer().map_err(|_| eyre::eyre!("failed to recover sender"))?;
|
||||
evm.transact_commit(self.evm_config.tx_env(tx, signer))?;
|
||||
for tx in block.transactions_recovered() {
|
||||
evm.transact_commit(self.evm_config.tx_env(tx))?;
|
||||
}
|
||||
|
||||
drop(evm);
|
||||
|
||||
@ -2775,7 +2775,7 @@ where
|
||||
let mut evm = evm_config.evm_with_env(state_provider, evm_env);
|
||||
|
||||
// create the tx env and reset nonce
|
||||
let tx_env = evm_config.tx_env(&tx, tx.signer());
|
||||
let tx_env = evm_config.tx_env(&tx);
|
||||
|
||||
// exit early if execution is done
|
||||
if cancel_execution.is_cancelled() {
|
||||
|
||||
@ -312,7 +312,7 @@ where
|
||||
// Configure the environment for the block.
|
||||
let tx_recovered =
|
||||
tx.try_clone_into_recovered().map_err(|_| ProviderError::SenderRecoveryError)?;
|
||||
let tx_env = evm_config.tx_env(&tx_recovered, tx_recovered.signer());
|
||||
let tx_env = evm_config.tx_env(tx_recovered);
|
||||
let exec_result = match evm.transact(tx_env) {
|
||||
Ok(result) => result,
|
||||
Err(err) if err.is_invalid_tx_err() => {
|
||||
|
||||
@ -12,13 +12,12 @@ workspace = true
|
||||
|
||||
[dependencies]
|
||||
# Reth
|
||||
reth-execution-types.workspace = true
|
||||
reth-chainspec.workspace = true
|
||||
reth-ethereum-forks.workspace = true
|
||||
reth-revm.workspace = true
|
||||
reth-evm.workspace = true
|
||||
reth-primitives.workspace = true
|
||||
reth-ethereum-consensus.workspace = true
|
||||
reth-consensus.workspace = true
|
||||
|
||||
# Ethereum
|
||||
reth-primitives-traits.workspace = true
|
||||
@ -30,6 +29,8 @@ alloy-evm.workspace = true
|
||||
alloy-sol-types.workspace = true
|
||||
alloy-consensus.workspace = true
|
||||
|
||||
derive_more.workspace = true
|
||||
|
||||
[dev-dependencies]
|
||||
reth-testing-utils.workspace = true
|
||||
reth-evm = { workspace = true, features = ["test-utils"] }
|
||||
@ -43,7 +44,6 @@ alloy-genesis.workspace = true
|
||||
[features]
|
||||
default = ["std"]
|
||||
std = [
|
||||
"reth-consensus/std",
|
||||
"reth-primitives/std",
|
||||
"reth-revm/std",
|
||||
"alloy-consensus/std",
|
||||
@ -58,4 +58,5 @@ std = [
|
||||
"alloy-evm/std",
|
||||
"reth-execution-types/std",
|
||||
"reth-evm/std",
|
||||
"derive_more/std",
|
||||
]
|
||||
|
||||
@ -8,19 +8,20 @@ use alloc::{boxed::Box, sync::Arc, vec::Vec};
|
||||
use alloy_consensus::{BlockHeader, Transaction};
|
||||
use alloy_eips::{eip6110, eip7685::Requests};
|
||||
use reth_chainspec::{ChainSpec, EthereumHardfork, EthereumHardforks, MAINNET};
|
||||
use reth_consensus::ConsensusError;
|
||||
use reth_ethereum_consensus::validate_block_post_execution;
|
||||
use reth_evm::{
|
||||
execute::{
|
||||
balance_increment_state, BasicBlockExecutorProvider, BlockExecutionError,
|
||||
BlockExecutionStrategy, BlockExecutionStrategyFactory, BlockValidationError, ExecuteOutput,
|
||||
BlockExecutionStrategy, BlockExecutionStrategyFactory, BlockValidationError,
|
||||
},
|
||||
state_change::post_block_balance_increments,
|
||||
system_calls::{OnStateHook, StateChangePostBlockSource, StateChangeSource, SystemCaller},
|
||||
ConfigureEvm, Database, Evm,
|
||||
ConfigureEvm, ConfigureEvmEnv, Database, Evm,
|
||||
};
|
||||
use reth_primitives::{EthPrimitives, Receipt, RecoveredBlock};
|
||||
use reth_primitives_traits::{BlockBody, SignedTransaction};
|
||||
use reth_execution_types::BlockExecutionResult;
|
||||
use reth_primitives::{
|
||||
EthPrimitives, Receipt, Recovered, RecoveredBlock, SealedBlock, TransactionSigned,
|
||||
};
|
||||
use reth_primitives_traits::NodePrimitives;
|
||||
use reth_revm::{context_interface::result::ResultAndState, db::State, DatabaseCommit};
|
||||
|
||||
/// Factory for [`EthExecutionStrategy`].
|
||||
@ -46,7 +47,7 @@ impl EthExecutionStrategyFactory {
|
||||
|
||||
impl<EvmConfig> EthExecutionStrategyFactory<EvmConfig> {
|
||||
/// Creates a new executor strategy factory.
|
||||
pub const fn new(chain_spec: Arc<ChainSpec>, evm_config: EvmConfig) -> Self {
|
||||
pub fn new(chain_spec: Arc<ChainSpec>, evm_config: EvmConfig) -> Self {
|
||||
Self { chain_spec, evm_config }
|
||||
}
|
||||
}
|
||||
@ -65,113 +66,108 @@ where
|
||||
{
|
||||
type Primitives = EthPrimitives;
|
||||
|
||||
type Strategy<DB: Database> = EthExecutionStrategy<DB, EvmConfig>;
|
||||
|
||||
fn create_strategy<DB>(&self, db: DB) -> Self::Strategy<DB>
|
||||
fn create_strategy<'a, DB>(
|
||||
&'a mut self,
|
||||
db: &'a mut State<DB>,
|
||||
block: &'a RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
) -> impl BlockExecutionStrategy<Primitives = Self::Primitives, Error = BlockExecutionError> + 'a
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
let state =
|
||||
State::builder().with_database(db).with_bundle_update().without_state_clear().build();
|
||||
EthExecutionStrategy::new(state, self.chain_spec.clone(), self.evm_config.clone())
|
||||
let evm = self.evm_config.evm_for_block(db, block.header());
|
||||
EthExecutionStrategy::new(evm, block.sealed_block(), self)
|
||||
}
|
||||
}
|
||||
|
||||
/// Block execution strategy for Ethereum.
|
||||
#[allow(missing_debug_implementations)]
|
||||
pub struct EthExecutionStrategy<DB, EvmConfig>
|
||||
where
|
||||
EvmConfig: Clone,
|
||||
{
|
||||
/// The chainspec
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
/// How to create an EVM.
|
||||
evm_config: EvmConfig,
|
||||
/// Current state for block execution.
|
||||
state: State<DB>,
|
||||
#[derive(Debug, derive_more::Deref)]
|
||||
pub struct EthExecutionStrategy<'a, Evm, EvmConfig> {
|
||||
/// Reference to the parent factory providing access to [`ChainSpec`].
|
||||
#[deref]
|
||||
factory: &'a EthExecutionStrategyFactory<EvmConfig>,
|
||||
|
||||
/// Block being executed.
|
||||
block: &'a SealedBlock,
|
||||
/// The EVM used by strategy.
|
||||
evm: Evm,
|
||||
/// Receipts of executed transactions.
|
||||
receipts: Vec<Receipt>,
|
||||
/// Utility to call system smart contracts.
|
||||
system_caller: SystemCaller<Arc<ChainSpec>>,
|
||||
system_caller: SystemCaller<&'a ChainSpec>,
|
||||
}
|
||||
|
||||
impl<DB, EvmConfig> EthExecutionStrategy<DB, EvmConfig>
|
||||
where
|
||||
EvmConfig: Clone,
|
||||
{
|
||||
impl<'a, Evm, EvmConfig> EthExecutionStrategy<'a, Evm, EvmConfig> {
|
||||
/// Creates a new [`EthExecutionStrategy`]
|
||||
pub fn new(state: State<DB>, chain_spec: Arc<ChainSpec>, evm_config: EvmConfig) -> Self {
|
||||
let system_caller = SystemCaller::new(chain_spec.clone());
|
||||
Self { state, chain_spec, evm_config, system_caller }
|
||||
pub fn new(
|
||||
evm: Evm,
|
||||
block: &'a SealedBlock,
|
||||
factory: &'a EthExecutionStrategyFactory<EvmConfig>,
|
||||
) -> Self {
|
||||
Self {
|
||||
evm,
|
||||
factory,
|
||||
block,
|
||||
receipts: Vec::new(),
|
||||
system_caller: SystemCaller::new(&factory.chain_spec),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB, EvmConfig> BlockExecutionStrategy for EthExecutionStrategy<DB, EvmConfig>
|
||||
impl<'db, DB, E, EvmConfig> BlockExecutionStrategy for EthExecutionStrategy<'_, E, EvmConfig>
|
||||
where
|
||||
DB: Database,
|
||||
EvmConfig: ConfigureEvm<
|
||||
Header = alloy_consensus::Header,
|
||||
Transaction = reth_primitives::TransactionSigned,
|
||||
>,
|
||||
DB: Database + 'db,
|
||||
E: Evm<DB = &'db mut State<DB>, Tx = EvmConfig::TxEnv>,
|
||||
EvmConfig: ConfigureEvmEnv<Transaction = TransactionSigned>,
|
||||
{
|
||||
type DB = DB;
|
||||
type Error = BlockExecutionError;
|
||||
type Primitives = EthPrimitives;
|
||||
|
||||
fn apply_pre_execution_changes(
|
||||
&mut self,
|
||||
block: &RecoveredBlock<reth_primitives::Block>,
|
||||
) -> Result<(), Self::Error> {
|
||||
fn apply_pre_execution_changes(&mut self) -> Result<(), Self::Error> {
|
||||
// Set state clear flag if the block is after the Spurious Dragon hardfork.
|
||||
let state_clear_flag = self.chain_spec.is_spurious_dragon_active_at_block(block.number());
|
||||
self.state.set_state_clear_flag(state_clear_flag);
|
||||
|
||||
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
|
||||
|
||||
self.system_caller.apply_pre_execution_changes(block.header(), &mut evm)?;
|
||||
let state_clear_flag =
|
||||
self.chain_spec.is_spurious_dragon_active_at_block(self.block.number());
|
||||
self.evm.db_mut().set_state_clear_flag(state_clear_flag);
|
||||
self.system_caller.apply_pre_execution_changes(self.block.header(), &mut self.evm)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn execute_transactions(
|
||||
fn execute_transactions<'a>(
|
||||
&mut self,
|
||||
block: &RecoveredBlock<reth_primitives::Block>,
|
||||
) -> Result<ExecuteOutput<Receipt>, Self::Error> {
|
||||
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
|
||||
|
||||
transactions: impl IntoIterator<Item = Recovered<&'a TransactionSigned>>,
|
||||
) -> Result<(), Self::Error> {
|
||||
let mut cumulative_gas_used = 0;
|
||||
let mut receipts = Vec::with_capacity(block.body().transaction_count());
|
||||
for (tx_index, (sender, transaction)) in block.transactions_with_sender().enumerate() {
|
||||
for (tx_index, tx) in transactions.into_iter().enumerate() {
|
||||
// The sum of the transaction's gas limit, Tg, and the gas utilized in this block prior,
|
||||
// must be no greater than the block's gasLimit.
|
||||
let block_available_gas = block.gas_limit() - cumulative_gas_used;
|
||||
if transaction.gas_limit() > block_available_gas {
|
||||
let block_available_gas = self.block.gas_limit() - cumulative_gas_used;
|
||||
if tx.gas_limit() > block_available_gas {
|
||||
return Err(BlockValidationError::TransactionGasLimitMoreThanAvailableBlockGas {
|
||||
transaction_gas_limit: transaction.gas_limit(),
|
||||
transaction_gas_limit: tx.gas_limit(),
|
||||
block_available_gas,
|
||||
}
|
||||
.into())
|
||||
}
|
||||
|
||||
let tx_env = self.evm_config.tx_env(transaction, *sender);
|
||||
let tx_env = self.evm_config.tx_env(tx.clone());
|
||||
let hash = tx.hash();
|
||||
|
||||
// Execute transaction.
|
||||
let result_and_state = evm.transact(tx_env).map_err(move |err| {
|
||||
let result_and_state = self.evm.transact(tx_env).map_err(move |err| {
|
||||
// Ensure hash is calculated for error log, if not already done
|
||||
BlockValidationError::EVM {
|
||||
hash: transaction.recalculate_hash(),
|
||||
error: Box::new(err),
|
||||
}
|
||||
BlockValidationError::EVM { hash: *hash, error: Box::new(err) }
|
||||
})?;
|
||||
self.system_caller
|
||||
.on_state(StateChangeSource::Transaction(tx_index), &result_and_state.state);
|
||||
let ResultAndState { result, state } = result_and_state;
|
||||
evm.db_mut().commit(state);
|
||||
self.evm.db_mut().commit(state);
|
||||
|
||||
// append gas used
|
||||
cumulative_gas_used += result.gas_used();
|
||||
|
||||
// Push transaction changeset and calculate header bloom filter for receipt.
|
||||
receipts.push(Receipt {
|
||||
tx_type: transaction.tx_type(),
|
||||
self.receipts.push(Receipt {
|
||||
tx_type: tx.tx_type(),
|
||||
// Success flag was added in `EIP-658: Embedding transaction status code in
|
||||
// receipts`.
|
||||
success: result.is_success(),
|
||||
@ -179,20 +175,16 @@ where
|
||||
logs: result.into_logs(),
|
||||
});
|
||||
}
|
||||
Ok(ExecuteOutput { receipts, gas_used: cumulative_gas_used })
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn apply_post_execution_changes(
|
||||
&mut self,
|
||||
block: &RecoveredBlock<reth_primitives::Block>,
|
||||
receipts: &[Receipt],
|
||||
) -> Result<Requests, Self::Error> {
|
||||
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
|
||||
|
||||
let requests = if self.chain_spec.is_prague_active_at_timestamp(block.timestamp) {
|
||||
mut self,
|
||||
) -> Result<BlockExecutionResult<Receipt>, Self::Error> {
|
||||
let requests = if self.chain_spec.is_prague_active_at_timestamp(self.block.timestamp) {
|
||||
// Collect all EIP-6110 deposits
|
||||
let deposit_requests =
|
||||
crate::eip6110::parse_deposits_from_receipts(&self.chain_spec, receipts)?;
|
||||
crate::eip6110::parse_deposits_from_receipts(&self.chain_spec, &self.receipts)?;
|
||||
|
||||
let mut requests = Requests::default();
|
||||
|
||||
@ -200,20 +192,20 @@ where
|
||||
requests.push_request_with_type(eip6110::DEPOSIT_REQUEST_TYPE, deposit_requests);
|
||||
}
|
||||
|
||||
requests.extend(self.system_caller.apply_post_execution_changes(&mut evm)?);
|
||||
requests.extend(self.system_caller.apply_post_execution_changes(&mut self.evm)?);
|
||||
requests
|
||||
} else {
|
||||
Requests::default()
|
||||
};
|
||||
drop(evm);
|
||||
|
||||
let mut balance_increments = post_block_balance_increments(&self.chain_spec, block);
|
||||
let mut balance_increments = post_block_balance_increments(&self.chain_spec, self.block);
|
||||
|
||||
// Irregular state change at Ethereum DAO hardfork
|
||||
if self.chain_spec.fork(EthereumHardfork::Dao).transitions_at_block(block.number()) {
|
||||
if self.chain_spec.fork(EthereumHardfork::Dao).transitions_at_block(self.block.number()) {
|
||||
// drain balances from hardcoded addresses.
|
||||
let drained_balance: u128 = self
|
||||
.state
|
||||
.evm
|
||||
.db_mut()
|
||||
.drain_balances(DAO_HARDFORK_ACCOUNTS)
|
||||
.map_err(|_| BlockValidationError::IncrementBalanceFailed)?
|
||||
.into_iter()
|
||||
@ -223,43 +215,24 @@ where
|
||||
*balance_increments.entry(DAO_HARDFORK_BENEFICIARY).or_default() += drained_balance;
|
||||
}
|
||||
// increment balances
|
||||
self.state
|
||||
self.evm
|
||||
.db_mut()
|
||||
.increment_balances(balance_increments.clone())
|
||||
.map_err(|_| BlockValidationError::IncrementBalanceFailed)?;
|
||||
// call state hook with changes due to balance increments.
|
||||
let balance_state = balance_increment_state(&balance_increments, &mut self.state)?;
|
||||
let balance_state = balance_increment_state(&balance_increments, self.evm.db_mut())?;
|
||||
self.system_caller.on_state(
|
||||
StateChangeSource::PostBlock(StateChangePostBlockSource::BalanceIncrements),
|
||||
&balance_state,
|
||||
);
|
||||
|
||||
Ok(requests)
|
||||
}
|
||||
|
||||
fn state_ref(&self) -> &State<DB> {
|
||||
&self.state
|
||||
}
|
||||
|
||||
fn state_mut(&mut self) -> &mut State<DB> {
|
||||
&mut self.state
|
||||
}
|
||||
|
||||
fn into_state(self) -> State<Self::DB> {
|
||||
self.state
|
||||
let gas_used = self.receipts.last().map(|r| r.cumulative_gas_used).unwrap_or_default();
|
||||
Ok(BlockExecutionResult { receipts: self.receipts, requests, gas_used })
|
||||
}
|
||||
|
||||
fn with_state_hook(&mut self, hook: Option<Box<dyn OnStateHook>>) {
|
||||
self.system_caller.with_state_hook(hook);
|
||||
}
|
||||
|
||||
fn validate_block_post_execution(
|
||||
&self,
|
||||
block: &RecoveredBlock<reth_primitives::Block>,
|
||||
receipts: &[Receipt],
|
||||
requests: &Requests,
|
||||
) -> Result<(), ConsensusError> {
|
||||
validate_block_post_execution(block, &self.chain_spec.clone(), receipts, requests)
|
||||
}
|
||||
}
|
||||
|
||||
/// Helper type with backwards compatible methods to obtain Ethereum executor
|
||||
|
||||
@ -17,11 +17,11 @@
|
||||
|
||||
extern crate alloc;
|
||||
|
||||
use alloc::sync::Arc;
|
||||
use alloy_consensus::{BlockHeader, Header};
|
||||
use alloc::{borrow::Borrow, sync::Arc};
|
||||
use alloy_consensus::{transaction::Recovered, BlockHeader, Header};
|
||||
pub use alloy_evm::EthEvm;
|
||||
use alloy_evm::EthEvmFactory;
|
||||
use alloy_primitives::{Address, U256};
|
||||
use alloy_primitives::U256;
|
||||
use core::{convert::Infallible, fmt::Debug};
|
||||
use reth_chainspec::{ChainSpec, EthChainSpec, MAINNET};
|
||||
use reth_evm::{ConfigureEvm, ConfigureEvmEnv, EvmEnv, NextBlockEnvAttributes};
|
||||
@ -77,9 +77,14 @@ impl ConfigureEvmEnv for EthEvmConfig {
|
||||
type TxEnv = TxEnv;
|
||||
type Spec = SpecId;
|
||||
|
||||
fn tx_env(&self, transaction: &TransactionSigned, sender: Address) -> Self::TxEnv {
|
||||
fn tx_env<T: Borrow<Self::Transaction>>(
|
||||
&self,
|
||||
transaction: impl Borrow<Recovered<T>>,
|
||||
) -> Self::TxEnv {
|
||||
let transaction = transaction.borrow();
|
||||
|
||||
let mut tx_env = TxEnv::default();
|
||||
transaction.fill_tx_env(&mut tx_env, sender);
|
||||
transaction.tx().borrow().fill_tx_env(&mut tx_env, transaction.signer());
|
||||
tx_env
|
||||
}
|
||||
|
||||
|
||||
@ -279,7 +279,7 @@ where
|
||||
}
|
||||
|
||||
// Configure the environment for the tx.
|
||||
let tx_env = evm_config.tx_env(tx.tx(), tx.signer());
|
||||
let tx_env = evm_config.tx_env(&tx);
|
||||
|
||||
let ResultAndState { result, state } = match evm.transact(tx_env) {
|
||||
Ok(res) => res,
|
||||
|
||||
@ -13,7 +13,6 @@ workspace = true
|
||||
[dependencies]
|
||||
# reth
|
||||
reth-chainspec.workspace = true
|
||||
reth-consensus.workspace = true
|
||||
reth-consensus-common.workspace = true
|
||||
reth-execution-errors.workspace = true
|
||||
reth-execution-types.workspace = true
|
||||
@ -34,6 +33,7 @@ alloy-evm.workspace = true
|
||||
alloy-consensus.workspace = true
|
||||
|
||||
auto_impl.workspace = true
|
||||
derive_more.workspace = true
|
||||
futures-util.workspace = true
|
||||
metrics = { workspace = true, optional = true }
|
||||
parking_lot = { workspace = true, optional = true }
|
||||
@ -47,7 +47,6 @@ metrics-util = { workspace = true, features = ["debugging"] }
|
||||
[features]
|
||||
default = ["std"]
|
||||
std = [
|
||||
"reth-consensus/std",
|
||||
"reth-primitives/std",
|
||||
"reth-primitives-traits/std",
|
||||
"alloy-eips/std",
|
||||
@ -65,6 +64,7 @@ std = [
|
||||
"reth-execution-types/std",
|
||||
"reth-storage-errors/std",
|
||||
"futures-util/std",
|
||||
"derive_more/std",
|
||||
]
|
||||
metrics = [
|
||||
"std",
|
||||
@ -74,7 +74,6 @@ metrics = [
|
||||
test-utils = [
|
||||
"dep:parking_lot",
|
||||
"reth-chainspec/test-utils",
|
||||
"reth-consensus/test-utils",
|
||||
"reth-ethereum-primitives/test-utils",
|
||||
"reth-primitives/test-utils",
|
||||
"reth-primitives-traits/test-utils",
|
||||
|
||||
@ -3,7 +3,7 @@ use alloy_eips::eip7685::Requests;
|
||||
use revm_database::BundleState;
|
||||
|
||||
/// The result of executing a block.
|
||||
#[derive(Debug, Clone, PartialEq, Eq)]
|
||||
#[derive(Debug, Clone, Default, PartialEq, Eq)]
|
||||
pub struct BlockExecutionResult<T> {
|
||||
/// All the receipts of the transactions in the block.
|
||||
pub receipts: Vec<T>,
|
||||
|
||||
@ -2,22 +2,19 @@
|
||||
|
||||
use alloy_consensus::BlockHeader;
|
||||
// Re-export execution types
|
||||
use crate::{system_calls::OnStateHook, Database};
|
||||
use alloc::{boxed::Box, vec::Vec};
|
||||
use alloy_primitives::{
|
||||
map::{DefaultHashBuilder, HashMap},
|
||||
Address,
|
||||
};
|
||||
pub use reth_execution_errors::{
|
||||
BlockExecutionError, BlockValidationError, InternalBlockExecutionError,
|
||||
};
|
||||
use reth_execution_types::BlockExecutionResult;
|
||||
pub use reth_execution_types::{BlockExecutionOutput, ExecutionOutcome};
|
||||
use reth_primitives::{NodePrimitives, Receipt, Recovered, RecoveredBlock};
|
||||
pub use reth_storage_errors::provider::ProviderError;
|
||||
|
||||
use crate::{system_calls::OnStateHook, Database};
|
||||
use alloc::{boxed::Box, vec::Vec};
|
||||
use alloy_eips::eip7685::Requests;
|
||||
use alloy_primitives::{
|
||||
map::{DefaultHashBuilder, HashMap},
|
||||
Address,
|
||||
};
|
||||
use reth_consensus::ConsensusError;
|
||||
use reth_primitives::{NodePrimitives, Receipt, RecoveredBlock};
|
||||
use revm::state::{Account, AccountStatus, EvmState};
|
||||
use revm_database::{states::bundle_state::BundleRetention, State};
|
||||
|
||||
@ -167,9 +164,6 @@ pub struct ExecuteOutput<R = Receipt> {
|
||||
|
||||
/// Defines the strategy for executing a single block.
|
||||
pub trait BlockExecutionStrategy {
|
||||
/// Database this strategy operates on.
|
||||
type DB: revm::Database;
|
||||
|
||||
/// Primitive types used by the strategy.
|
||||
type Primitives: NodePrimitives;
|
||||
|
||||
@ -177,45 +171,23 @@ pub trait BlockExecutionStrategy {
|
||||
type Error: core::error::Error;
|
||||
|
||||
/// Applies any necessary changes before executing the block's transactions.
|
||||
fn apply_pre_execution_changes(
|
||||
&mut self,
|
||||
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
) -> Result<(), Self::Error>;
|
||||
fn apply_pre_execution_changes(&mut self) -> Result<(), Self::Error>;
|
||||
|
||||
/// Executes all transactions in the block.
|
||||
fn execute_transactions(
|
||||
fn execute_transactions<'a>(
|
||||
&mut self,
|
||||
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
) -> Result<ExecuteOutput<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>;
|
||||
transactions: impl IntoIterator<
|
||||
Item = Recovered<&'a <Self::Primitives as NodePrimitives>::SignedTx>,
|
||||
>,
|
||||
) -> Result<(), Self::Error>;
|
||||
|
||||
/// Applies any necessary changes after executing the block's transactions.
|
||||
fn apply_post_execution_changes(
|
||||
&mut self,
|
||||
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
receipts: &[<Self::Primitives as NodePrimitives>::Receipt],
|
||||
) -> Result<Requests, Self::Error>;
|
||||
|
||||
/// Returns a reference to the current state.
|
||||
fn state_ref(&self) -> &State<Self::DB>;
|
||||
|
||||
/// Returns a mutable reference to the current state.
|
||||
fn state_mut(&mut self) -> &mut State<Self::DB>;
|
||||
|
||||
/// Consumes the strategy and returns inner [`State`].
|
||||
fn into_state(self) -> State<Self::DB>;
|
||||
self,
|
||||
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>;
|
||||
|
||||
/// Sets a hook to be called after each state change during execution.
|
||||
fn with_state_hook(&mut self, _hook: Option<Box<dyn OnStateHook>>) {}
|
||||
|
||||
/// Validate a block with regard to execution results.
|
||||
fn validate_block_post_execution(
|
||||
&self,
|
||||
_block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
_receipts: &[<Self::Primitives as NodePrimitives>::Receipt],
|
||||
_requests: &Requests,
|
||||
) -> Result<(), ConsensusError> {
|
||||
Ok(())
|
||||
}
|
||||
fn with_state_hook(&mut self, hook: Option<Box<dyn OnStateHook>>);
|
||||
}
|
||||
|
||||
/// A strategy factory that can create block execution strategies.
|
||||
@ -223,15 +195,12 @@ pub trait BlockExecutionStrategyFactory: Send + Sync + Clone + Unpin + 'static {
|
||||
/// Primitive types used by the strategy.
|
||||
type Primitives: NodePrimitives;
|
||||
|
||||
/// Associated strategy type.
|
||||
type Strategy<DB: Database>: BlockExecutionStrategy<
|
||||
DB = DB,
|
||||
Primitives = Self::Primitives,
|
||||
Error = BlockExecutionError,
|
||||
>;
|
||||
|
||||
/// Creates a strategy using the give database.
|
||||
fn create_strategy<DB>(&self, db: DB) -> Self::Strategy<DB>
|
||||
/// Creates a strategy using the given database.
|
||||
fn create_strategy<'a, DB>(
|
||||
&'a mut self,
|
||||
db: &'a mut State<DB>,
|
||||
block: &'a RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
) -> impl BlockExecutionStrategy<Primitives = Self::Primitives, Error = BlockExecutionError> + 'a
|
||||
where
|
||||
DB: Database;
|
||||
}
|
||||
@ -264,74 +233,85 @@ where
|
||||
{
|
||||
type Primitives = F::Primitives;
|
||||
|
||||
type Executor<DB: Database> = BasicBlockExecutor<F::Strategy<DB>>;
|
||||
type Executor<DB: Database> = BasicBlockExecutor<F, DB>;
|
||||
|
||||
fn executor<DB>(&self, db: DB) -> Self::Executor<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
let strategy = self.strategy_factory.create_strategy(db);
|
||||
BasicBlockExecutor::new(strategy)
|
||||
BasicBlockExecutor::new(self.strategy_factory.clone(), db)
|
||||
}
|
||||
}
|
||||
|
||||
/// A generic block executor that uses a [`BlockExecutionStrategy`] to
|
||||
/// execute blocks.
|
||||
#[allow(missing_debug_implementations, dead_code)]
|
||||
pub struct BasicBlockExecutor<S> {
|
||||
pub struct BasicBlockExecutor<F, DB> {
|
||||
/// Block execution strategy.
|
||||
pub(crate) strategy: S,
|
||||
pub(crate) strategy_factory: F,
|
||||
/// Database.
|
||||
pub(crate) db: State<DB>,
|
||||
}
|
||||
|
||||
impl<S> BasicBlockExecutor<S> {
|
||||
impl<F, DB: Database> BasicBlockExecutor<F, DB> {
|
||||
/// Creates a new `BasicBlockExecutor` with the given strategy.
|
||||
pub const fn new(strategy: S) -> Self {
|
||||
Self { strategy }
|
||||
pub fn new(strategy_factory: F, db: DB) -> Self {
|
||||
let db =
|
||||
State::builder().with_database(db).with_bundle_update().without_state_clear().build();
|
||||
Self { strategy_factory, db }
|
||||
}
|
||||
}
|
||||
|
||||
impl<S, DB> Executor<DB> for BasicBlockExecutor<S>
|
||||
impl<F, DB> Executor<DB> for BasicBlockExecutor<F, DB>
|
||||
where
|
||||
S: BlockExecutionStrategy<DB = DB>,
|
||||
F: BlockExecutionStrategyFactory,
|
||||
DB: Database,
|
||||
{
|
||||
type Primitives = S::Primitives;
|
||||
type Error = S::Error;
|
||||
type Primitives = F::Primitives;
|
||||
type Error = BlockExecutionError;
|
||||
|
||||
fn execute_one(
|
||||
&mut self,
|
||||
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
|
||||
{
|
||||
self.strategy.apply_pre_execution_changes(block)?;
|
||||
let ExecuteOutput { receipts, gas_used } = self.strategy.execute_transactions(block)?;
|
||||
let requests = self.strategy.apply_post_execution_changes(block, &receipts)?;
|
||||
self.strategy.state_mut().merge_transitions(BundleRetention::Reverts);
|
||||
let mut strategy = self.strategy_factory.create_strategy(&mut self.db, block);
|
||||
|
||||
Ok(BlockExecutionResult { receipts, requests, gas_used })
|
||||
strategy.apply_pre_execution_changes()?;
|
||||
strategy.execute_transactions(block.transactions_recovered())?;
|
||||
let result = strategy.apply_post_execution_changes()?;
|
||||
|
||||
self.db.merge_transitions(BundleRetention::Reverts);
|
||||
|
||||
Ok(result)
|
||||
}
|
||||
|
||||
fn execute_one_with_state_hook<F>(
|
||||
fn execute_one_with_state_hook<H>(
|
||||
&mut self,
|
||||
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
state_hook: F,
|
||||
state_hook: H,
|
||||
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
|
||||
where
|
||||
F: OnStateHook + 'static,
|
||||
H: OnStateHook + 'static,
|
||||
{
|
||||
self.strategy.with_state_hook(Some(Box::new(state_hook)));
|
||||
let result = self.execute_one(block);
|
||||
self.strategy.with_state_hook(None);
|
||||
let mut strategy = self.strategy_factory.create_strategy(&mut self.db, block);
|
||||
strategy.with_state_hook(Some(Box::new(state_hook)));
|
||||
|
||||
result
|
||||
strategy.apply_pre_execution_changes()?;
|
||||
strategy.execute_transactions(block.transactions_recovered())?;
|
||||
let result = strategy.apply_post_execution_changes()?;
|
||||
|
||||
self.db.merge_transitions(BundleRetention::Reverts);
|
||||
|
||||
Ok(result)
|
||||
}
|
||||
|
||||
fn into_state(self) -> State<DB> {
|
||||
self.strategy.into_state()
|
||||
self.db
|
||||
}
|
||||
|
||||
fn size_hint(&self) -> usize {
|
||||
self.strategy.state_ref().bundle_state.size_hint()
|
||||
self.db.bundle_state.size_hint()
|
||||
}
|
||||
}
|
||||
|
||||
@ -374,17 +354,14 @@ where
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use alloy_primitives::U256;
|
||||
use alloy_consensus::constants::KECCAK_EMPTY;
|
||||
use alloy_eips::eip7685::Requests;
|
||||
use alloy_primitives::{address, bytes, U256};
|
||||
use core::marker::PhantomData;
|
||||
use reth_chainspec::{ChainSpec, MAINNET};
|
||||
use reth_ethereum_primitives::TransactionSigned;
|
||||
use reth_primitives::EthPrimitives;
|
||||
use revm::{
|
||||
database_interface::EmptyDBTyped,
|
||||
primitives::{address, bytes, KECCAK_EMPTY},
|
||||
state::AccountInfo,
|
||||
};
|
||||
use revm_database::CacheDB;
|
||||
use std::sync::Arc;
|
||||
use revm::state::AccountInfo;
|
||||
use revm_database::{CacheDB, EmptyDBTyped};
|
||||
|
||||
#[derive(Clone, Default)]
|
||||
struct TestExecutorProvider;
|
||||
@ -435,94 +412,55 @@ mod tests {
|
||||
}
|
||||
}
|
||||
|
||||
struct TestExecutorStrategy<DB, EvmConfig> {
|
||||
// chain spec and evm config here only to illustrate how the strategy
|
||||
// factory can use them in a real use case.
|
||||
_chain_spec: Arc<ChainSpec>,
|
||||
_evm_config: EvmConfig,
|
||||
state: State<DB>,
|
||||
execute_transactions_result: ExecuteOutput<Receipt>,
|
||||
apply_post_execution_changes_result: Requests,
|
||||
struct TestExecutorStrategy {
|
||||
result: BlockExecutionResult<Receipt>,
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
struct TestExecutorStrategyFactory {
|
||||
execute_transactions_result: ExecuteOutput<Receipt>,
|
||||
apply_post_execution_changes_result: Requests,
|
||||
result: BlockExecutionResult<Receipt>,
|
||||
}
|
||||
|
||||
impl BlockExecutionStrategyFactory for TestExecutorStrategyFactory {
|
||||
type Primitives = EthPrimitives;
|
||||
type Strategy<DB: Database> = TestExecutorStrategy<DB, TestEvmConfig>;
|
||||
|
||||
fn create_strategy<DB>(&self, db: DB) -> Self::Strategy<DB>
|
||||
fn create_strategy<'a, DB>(
|
||||
&'a mut self,
|
||||
_db: &'a mut State<DB>,
|
||||
_block: &'a RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
) -> impl BlockExecutionStrategy<Primitives = Self::Primitives, Error = BlockExecutionError> + 'a
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
let state = State::builder()
|
||||
.with_database(db)
|
||||
.with_bundle_update()
|
||||
.without_state_clear()
|
||||
.build();
|
||||
|
||||
TestExecutorStrategy {
|
||||
_chain_spec: MAINNET.clone(),
|
||||
_evm_config: TestEvmConfig {},
|
||||
execute_transactions_result: self.execute_transactions_result.clone(),
|
||||
apply_post_execution_changes_result: self
|
||||
.apply_post_execution_changes_result
|
||||
.clone(),
|
||||
state,
|
||||
}
|
||||
TestExecutorStrategy { result: self.result.clone() }
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockExecutionStrategy for TestExecutorStrategy<DB, TestEvmConfig>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
type DB = DB;
|
||||
impl BlockExecutionStrategy for TestExecutorStrategy {
|
||||
type Primitives = EthPrimitives;
|
||||
type Error = BlockExecutionError;
|
||||
|
||||
fn apply_pre_execution_changes(
|
||||
fn apply_pre_execution_changes(&mut self) -> Result<(), Self::Error> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn execute_transactions<'a>(
|
||||
&mut self,
|
||||
_block: &RecoveredBlock<reth_primitives::Block>,
|
||||
_transactions: impl IntoIterator<Item = Recovered<&'a TransactionSigned>>,
|
||||
) -> Result<(), Self::Error> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn execute_transactions(
|
||||
&mut self,
|
||||
_block: &RecoveredBlock<reth_primitives::Block>,
|
||||
) -> Result<ExecuteOutput<Receipt>, Self::Error> {
|
||||
Ok(self.execute_transactions_result.clone())
|
||||
}
|
||||
|
||||
fn apply_post_execution_changes(
|
||||
&mut self,
|
||||
_block: &RecoveredBlock<reth_primitives::Block>,
|
||||
_receipts: &[Receipt],
|
||||
) -> Result<Requests, Self::Error> {
|
||||
Ok(self.apply_post_execution_changes_result.clone())
|
||||
self,
|
||||
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
|
||||
{
|
||||
Ok(self.result)
|
||||
}
|
||||
|
||||
fn state_ref(&self) -> &State<DB> {
|
||||
&self.state
|
||||
}
|
||||
|
||||
fn state_mut(&mut self) -> &mut State<DB> {
|
||||
&mut self.state
|
||||
}
|
||||
|
||||
fn into_state(self) -> State<Self::DB> {
|
||||
self.state
|
||||
}
|
||||
fn with_state_hook(&mut self, _hook: Option<Box<dyn OnStateHook>>) {}
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
struct TestEvmConfig {}
|
||||
|
||||
#[test]
|
||||
fn test_provider() {
|
||||
let provider = TestExecutorProvider;
|
||||
@ -533,19 +471,13 @@ mod tests {
|
||||
|
||||
#[test]
|
||||
fn test_strategy() {
|
||||
let expected_gas_used = 10;
|
||||
let expected_receipts = vec![Receipt::default()];
|
||||
let expected_execute_transactions_result = ExecuteOutput::<Receipt> {
|
||||
receipts: expected_receipts.clone(),
|
||||
gas_used: expected_gas_used,
|
||||
let expected_result = BlockExecutionResult {
|
||||
receipts: vec![Receipt::default()],
|
||||
gas_used: 10,
|
||||
requests: Requests::new(vec![bytes!("deadbeef")]),
|
||||
};
|
||||
let expected_apply_post_execution_changes_result = Requests::new(vec![bytes!("deadbeef")]);
|
||||
|
||||
let strategy_factory = TestExecutorStrategyFactory {
|
||||
execute_transactions_result: expected_execute_transactions_result,
|
||||
apply_post_execution_changes_result: expected_apply_post_execution_changes_result
|
||||
.clone(),
|
||||
};
|
||||
let strategy_factory = TestExecutorStrategyFactory { result: expected_result.clone() };
|
||||
let provider = BasicBlockExecutorProvider::new(strategy_factory);
|
||||
let db = CacheDB::<EmptyDBTyped<ProviderError>>::default();
|
||||
let executor = provider.executor(db);
|
||||
@ -553,9 +485,7 @@ mod tests {
|
||||
|
||||
assert!(result.is_ok());
|
||||
let block_execution_output = result.unwrap();
|
||||
assert_eq!(block_execution_output.gas_used, expected_gas_used);
|
||||
assert_eq!(block_execution_output.receipts, expected_receipts);
|
||||
assert_eq!(block_execution_output.requests, expected_apply_post_execution_changes_result);
|
||||
assert_eq!(block_execution_output.result, expected_result);
|
||||
}
|
||||
|
||||
fn setup_state_with_account(
|
||||
|
||||
@ -17,6 +17,7 @@
|
||||
|
||||
extern crate alloc;
|
||||
|
||||
use alloc::borrow::Borrow;
|
||||
use alloy_consensus::transaction::Recovered;
|
||||
use alloy_eips::eip2930::AccessList;
|
||||
pub use alloy_evm::evm::EvmFactory;
|
||||
@ -182,13 +183,10 @@ pub trait ConfigureEvmEnv: Send + Sync + Unpin + Clone + 'static {
|
||||
type Spec: Debug + Copy + Send + Sync + 'static;
|
||||
|
||||
/// Returns a [`TxEnv`] from a transaction and [`Address`].
|
||||
fn tx_env(&self, transaction: &Self::Transaction, signer: Address) -> Self::TxEnv;
|
||||
|
||||
/// Returns a [`TxEnv`] from a [`Recovered`] transaction.
|
||||
fn tx_env_from_recovered(&self, tx: Recovered<&Self::Transaction>) -> Self::TxEnv {
|
||||
let (tx, address) = tx.into_parts();
|
||||
self.tx_env(tx, address)
|
||||
}
|
||||
fn tx_env<T: Borrow<Self::Transaction>>(
|
||||
&self,
|
||||
transaction: impl Borrow<Recovered<T>>,
|
||||
) -> Self::TxEnv;
|
||||
|
||||
/// Creates a new [`EvmEnv`] for the given header.
|
||||
fn evm_env(&self, header: &Self::Header) -> EvmEnv<Self::Spec>;
|
||||
|
||||
@ -77,10 +77,11 @@ impl OnStateHook for NoopHook {
|
||||
/// An ephemeral helper type for executing system calls.
|
||||
///
|
||||
/// This can be used to chain system transaction calls.
|
||||
#[allow(missing_debug_implementations)]
|
||||
#[derive(derive_more::Debug)]
|
||||
pub struct SystemCaller<ChainSpec> {
|
||||
chain_spec: ChainSpec,
|
||||
/// Optional hook to be called after each state change.
|
||||
#[debug(skip)]
|
||||
hook: Option<Box<dyn OnStateHook>>,
|
||||
}
|
||||
|
||||
@ -149,7 +150,7 @@ where
|
||||
eip2935::transact_blockhashes_contract_call(&self.chain_spec, parent_block_hash, evm)?;
|
||||
|
||||
if let Some(res) = result_and_state {
|
||||
if let Some(ref mut hook) = self.hook {
|
||||
if let Some(hook) = &mut self.hook {
|
||||
hook.on_state(
|
||||
StateChangeSource::PreBlock(StateChangePreBlockSource::BlockHashesContract),
|
||||
&res.state,
|
||||
@ -174,7 +175,7 @@ where
|
||||
)?;
|
||||
|
||||
if let Some(res) = result_and_state {
|
||||
if let Some(ref mut hook) = self.hook {
|
||||
if let Some(hook) = &mut self.hook {
|
||||
hook.on_state(
|
||||
StateChangeSource::PreBlock(StateChangePreBlockSource::BeaconRootContract),
|
||||
&res.state,
|
||||
@ -193,7 +194,7 @@ where
|
||||
) -> Result<Bytes, BlockExecutionError> {
|
||||
let result_and_state = eip7002::transact_withdrawal_requests_contract_call(evm)?;
|
||||
|
||||
if let Some(ref mut hook) = self.hook {
|
||||
if let Some(ref mut hook) = &mut self.hook {
|
||||
hook.on_state(
|
||||
StateChangeSource::PostBlock(
|
||||
StateChangePostBlockSource::WithdrawalRequestsContract,
|
||||
@ -213,7 +214,7 @@ where
|
||||
) -> Result<Bytes, BlockExecutionError> {
|
||||
let result_and_state = eip7251::transact_consolidation_requests_contract_call(evm)?;
|
||||
|
||||
if let Some(ref mut hook) = self.hook {
|
||||
if let Some(ref mut hook) = &mut self.hook {
|
||||
hook.on_state(
|
||||
StateChangeSource::PostBlock(
|
||||
StateChangePostBlockSource::ConsolidationRequestsContract,
|
||||
@ -228,7 +229,7 @@ where
|
||||
|
||||
/// Delegate to stored `OnStateHook`, noop if hook is `None`.
|
||||
pub fn on_state(&mut self, source: StateChangeSource, state: &EvmState) {
|
||||
if let Some(ref mut hook) = &mut self.hook {
|
||||
if let Some(hook) = &mut self.hook {
|
||||
hook.on_state(source, state);
|
||||
}
|
||||
}
|
||||
|
||||
@ -1,10 +1,7 @@
|
||||
//! Helpers for testing.
|
||||
|
||||
use crate::{
|
||||
execute::{
|
||||
BasicBlockExecutor, BlockExecutionOutput, BlockExecutionStrategy, BlockExecutorProvider,
|
||||
Executor,
|
||||
},
|
||||
execute::{BasicBlockExecutor, BlockExecutionOutput, BlockExecutorProvider, Executor},
|
||||
system_calls::OnStateHook,
|
||||
Database,
|
||||
};
|
||||
@ -125,23 +122,20 @@ impl<DB: Database> Executor<DB> for MockExecutorProvider {
|
||||
}
|
||||
}
|
||||
|
||||
impl<S> BasicBlockExecutor<S>
|
||||
where
|
||||
S: BlockExecutionStrategy,
|
||||
{
|
||||
impl<Factory, DB> BasicBlockExecutor<Factory, DB> {
|
||||
/// Provides safe read access to the state
|
||||
pub fn with_state<F, R>(&self, f: F) -> R
|
||||
where
|
||||
F: FnOnce(&State<S::DB>) -> R,
|
||||
F: FnOnce(&State<DB>) -> R,
|
||||
{
|
||||
f(self.strategy.state_ref())
|
||||
f(&self.db)
|
||||
}
|
||||
|
||||
/// Provides safe write access to the state
|
||||
pub fn with_state_mut<F, R>(&mut self, f: F) -> R
|
||||
where
|
||||
F: FnOnce(&mut State<S::DB>) -> R,
|
||||
F: FnOnce(&mut State<DB>) -> R,
|
||||
{
|
||||
f(self.strategy.state_mut())
|
||||
f(&mut self.db)
|
||||
}
|
||||
}
|
||||
|
||||
@ -5,24 +5,24 @@ use crate::{
|
||||
OpReceiptBuilder, ReceiptBuilderCtx,
|
||||
};
|
||||
use alloc::{boxed::Box, sync::Arc, vec::Vec};
|
||||
use alloy_consensus::{BlockHeader, Eip658Value, Receipt, Transaction as _};
|
||||
use alloy_eips::eip7685::Requests;
|
||||
use alloy_consensus::{
|
||||
transaction::Recovered, BlockHeader, Eip658Value, Receipt, Transaction as _, TxReceipt,
|
||||
};
|
||||
use op_alloy_consensus::OpDepositReceipt;
|
||||
use reth_consensus::ConsensusError;
|
||||
use reth_evm::{
|
||||
execute::{
|
||||
balance_increment_state, BasicBlockExecutorProvider, BlockExecutionError,
|
||||
BlockExecutionStrategy, BlockExecutionStrategyFactory, BlockValidationError, ExecuteOutput,
|
||||
BlockExecutionStrategy, BlockExecutionStrategyFactory, BlockValidationError,
|
||||
},
|
||||
state_change::post_block_balance_increments,
|
||||
system_calls::{OnStateHook, StateChangePostBlockSource, StateChangeSource, SystemCaller},
|
||||
ConfigureEvm, ConfigureEvmFor, Database, Evm, HaltReasonFor,
|
||||
};
|
||||
use reth_execution_types::BlockExecutionResult;
|
||||
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 reth_primitives_traits::{NodePrimitives, RecoveredBlock, SealedBlock, SignedTransaction};
|
||||
use revm::{context_interface::result::ResultAndState, DatabaseCommit};
|
||||
use revm_database::State;
|
||||
use tracing::trace;
|
||||
@ -79,43 +79,38 @@ where
|
||||
EvmConfig: ConfigureEvmFor<N> + Clone + Unpin + Sync + Send + 'static,
|
||||
{
|
||||
type Primitives = N;
|
||||
type Strategy<DB: Database> = OpExecutionStrategy<DB, N, ChainSpec, EvmConfig>;
|
||||
|
||||
fn create_strategy<DB>(&self, db: DB) -> Self::Strategy<DB>
|
||||
fn create_strategy<'a, DB>(
|
||||
&'a mut self,
|
||||
db: &'a mut State<DB>,
|
||||
block: &'a RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
) -> impl BlockExecutionStrategy<Primitives = Self::Primitives, Error = BlockExecutionError> + 'a
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
let state =
|
||||
State::builder().with_database(db).with_bundle_update().without_state_clear().build();
|
||||
OpExecutionStrategy::new(
|
||||
state,
|
||||
self.chain_spec.clone(),
|
||||
self.evm_config.clone(),
|
||||
self.receipt_builder.clone(),
|
||||
)
|
||||
let evm = self.evm_config.evm_for_block(db, block.header());
|
||||
OpExecutionStrategy::new(evm, block.sealed_block(), self)
|
||||
}
|
||||
}
|
||||
|
||||
/// Block execution strategy for Optimism.
|
||||
#[allow(missing_debug_implementations)]
|
||||
pub struct OpExecutionStrategy<DB, N: NodePrimitives, ChainSpec, EvmConfig>
|
||||
where
|
||||
EvmConfig: ConfigureEvm,
|
||||
{
|
||||
/// The chainspec
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
/// How to create an EVM.
|
||||
evm_config: EvmConfig,
|
||||
/// Current state for block execution.
|
||||
state: State<DB>,
|
||||
#[derive(Debug, derive_more::Deref)]
|
||||
pub struct OpExecutionStrategy<'a, Evm, N: NodePrimitives, ChainSpec, EvmConfig: ConfigureEvm> {
|
||||
/// Reference to the parent factory.
|
||||
#[deref]
|
||||
factory: &'a OpExecutionStrategyFactory<N, ChainSpec, EvmConfig>,
|
||||
|
||||
/// Block being executed.
|
||||
block: &'a SealedBlock<N::Block>,
|
||||
/// The EVM used by strategy.
|
||||
evm: Evm,
|
||||
/// Receipts of executed transactions.
|
||||
receipts: Vec<N::Receipt>,
|
||||
/// Utility to call system smart contracts.
|
||||
system_caller: SystemCaller<Arc<ChainSpec>>,
|
||||
/// Receipt builder.
|
||||
receipt_builder:
|
||||
Arc<dyn OpReceiptBuilder<N::SignedTx, HaltReasonFor<EvmConfig>, Receipt = N::Receipt>>,
|
||||
system_caller: SystemCaller<&'a ChainSpec>,
|
||||
}
|
||||
|
||||
impl<DB, N, ChainSpec, EvmConfig> OpExecutionStrategy<DB, N, ChainSpec, EvmConfig>
|
||||
impl<'a, Evm, N, ChainSpec, EvmConfig> OpExecutionStrategy<'a, Evm, N, ChainSpec, EvmConfig>
|
||||
where
|
||||
N: NodePrimitives,
|
||||
ChainSpec: OpHardforks,
|
||||
@ -123,73 +118,67 @@ where
|
||||
{
|
||||
/// Creates a new [`OpExecutionStrategy`]
|
||||
pub fn new(
|
||||
state: State<DB>,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
evm_config: EvmConfig,
|
||||
receipt_builder: Arc<
|
||||
dyn OpReceiptBuilder<N::SignedTx, HaltReasonFor<EvmConfig>, Receipt = N::Receipt>,
|
||||
>,
|
||||
evm: Evm,
|
||||
block: &'a SealedBlock<N::Block>,
|
||||
factory: &'a OpExecutionStrategyFactory<N, ChainSpec, EvmConfig>,
|
||||
) -> Self {
|
||||
let system_caller = SystemCaller::new(chain_spec.clone());
|
||||
Self { state, chain_spec, evm_config, system_caller, receipt_builder }
|
||||
Self {
|
||||
evm,
|
||||
factory,
|
||||
block,
|
||||
receipts: Vec::new(),
|
||||
system_caller: SystemCaller::new(&factory.chain_spec),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB, N, ChainSpec, EvmConfig> BlockExecutionStrategy
|
||||
for OpExecutionStrategy<DB, N, ChainSpec, EvmConfig>
|
||||
impl<'db, DB, E, N, ChainSpec, EvmConfig> BlockExecutionStrategy
|
||||
for OpExecutionStrategy<'_, E, N, ChainSpec, EvmConfig>
|
||||
where
|
||||
DB: Database,
|
||||
DB: Database + 'db,
|
||||
E: Evm<DB = &'db mut State<DB>, Tx = EvmConfig::TxEnv, HaltReason = HaltReasonFor<EvmConfig>>,
|
||||
N: NodePrimitives<SignedTx: OpTransaction, Receipt: DepositReceipt>,
|
||||
ChainSpec: OpHardforks,
|
||||
EvmConfig: ConfigureEvmFor<N>,
|
||||
{
|
||||
type DB = DB;
|
||||
type Primitives = N;
|
||||
type Error = BlockExecutionError;
|
||||
|
||||
fn apply_pre_execution_changes(
|
||||
&mut self,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
) -> Result<(), Self::Error> {
|
||||
fn apply_pre_execution_changes(&mut self) -> Result<(), Self::Error> {
|
||||
// Set state clear flag if the block is after the Spurious Dragon hardfork.
|
||||
let state_clear_flag =
|
||||
(*self.chain_spec).is_spurious_dragon_active_at_block(block.number());
|
||||
self.state.set_state_clear_flag(state_clear_flag);
|
||||
(*self.chain_spec).is_spurious_dragon_active_at_block(self.block.number());
|
||||
self.evm.db_mut().set_state_clear_flag(state_clear_flag);
|
||||
|
||||
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
|
||||
|
||||
self.system_caller
|
||||
.apply_beacon_root_contract_call(block.header().parent_beacon_block_root(), &mut evm)?;
|
||||
self.system_caller.apply_beacon_root_contract_call(
|
||||
self.block.parent_beacon_block_root(),
|
||||
&mut self.evm,
|
||||
)?;
|
||||
|
||||
// Ensure that the create2deployer is force-deployed at the canyon transition. Optimism
|
||||
// blocks will always have at least a single transaction in them (the L1 info transaction),
|
||||
// so we can safely assume that this will always be triggered upon the transition and that
|
||||
// the above check for empty blocks will never be hit on OP chains.
|
||||
ensure_create2_deployer(self.chain_spec.clone(), block.header().timestamp(), evm.db_mut())
|
||||
ensure_create2_deployer(self.chain_spec.clone(), self.block.timestamp(), self.evm.db_mut())
|
||||
.map_err(|_| OpBlockExecutionError::ForceCreate2DeployerFail)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn execute_transactions(
|
||||
fn execute_transactions<'a>(
|
||||
&mut self,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
) -> Result<ExecuteOutput<N::Receipt>, Self::Error> {
|
||||
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
|
||||
|
||||
let is_regolith = self.chain_spec.is_regolith_active_at_timestamp(block.timestamp());
|
||||
transactions: impl IntoIterator<Item = Recovered<&'a N::SignedTx>>,
|
||||
) -> Result<(), Self::Error> {
|
||||
let is_regolith = self.chain_spec.is_regolith_active_at_timestamp(self.block.timestamp());
|
||||
|
||||
let mut cumulative_gas_used = 0;
|
||||
let mut receipts = Vec::with_capacity(block.body().transaction_count());
|
||||
for (tx_index, (sender, transaction)) in block.transactions_with_sender().enumerate() {
|
||||
for (tx_index, tx) in transactions.into_iter().enumerate() {
|
||||
// The sum of the transaction’s gas limit, Tg, and the gas utilized in this block prior,
|
||||
// must be no greater than the block’s gasLimit.
|
||||
let block_available_gas = block.gas_limit() - cumulative_gas_used;
|
||||
if transaction.gas_limit() > block_available_gas &&
|
||||
(is_regolith || !transaction.is_deposit())
|
||||
{
|
||||
let block_available_gas = self.block.gas_limit() - cumulative_gas_used;
|
||||
if tx.gas_limit() > block_available_gas && (is_regolith || !tx.is_deposit()) {
|
||||
return Err(BlockValidationError::TransactionGasLimitMoreThanAvailableBlockGas {
|
||||
transaction_gas_limit: transaction.gas_limit(),
|
||||
transaction_gas_limit: tx.gas_limit(),
|
||||
block_available_gas,
|
||||
}
|
||||
.into())
|
||||
@ -200,42 +189,41 @@ 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 = (is_regolith && transaction.is_deposit())
|
||||
let depositor = (is_regolith && tx.is_deposit())
|
||||
.then(|| {
|
||||
evm.db_mut()
|
||||
.load_cache_account(*sender)
|
||||
self.evm
|
||||
.db_mut()
|
||||
.load_cache_account(tx.signer())
|
||||
.map(|acc| acc.account_info().unwrap_or_default())
|
||||
})
|
||||
.transpose()
|
||||
.map_err(|_| OpBlockExecutionError::AccountLoadFailed(*sender))?;
|
||||
.map_err(|_| OpBlockExecutionError::AccountLoadFailed(tx.signer()))?;
|
||||
|
||||
let tx_env = self.evm_config.tx_env(transaction, *sender);
|
||||
let tx_env = self.evm_config.tx_env(&tx);
|
||||
let hash = tx.tx_hash();
|
||||
|
||||
// Execute transaction.
|
||||
let result_and_state = evm.transact(tx_env).map_err(move |err| {
|
||||
let result_and_state = self.evm.transact(tx_env).map_err(move |err| {
|
||||
// Ensure hash is calculated for error log, if not already done
|
||||
BlockValidationError::EVM {
|
||||
hash: transaction.recalculate_hash(),
|
||||
error: Box::new(err),
|
||||
}
|
||||
BlockValidationError::EVM { hash: *hash, error: Box::new(err) }
|
||||
})?;
|
||||
|
||||
trace!(
|
||||
target: "evm",
|
||||
?transaction,
|
||||
?tx,
|
||||
"Executed transaction"
|
||||
);
|
||||
self.system_caller
|
||||
.on_state(StateChangeSource::Transaction(tx_index), &result_and_state.state);
|
||||
let ResultAndState { result, state } = result_and_state;
|
||||
evm.db_mut().commit(state);
|
||||
self.evm.db_mut().commit(state);
|
||||
|
||||
// append gas used
|
||||
cumulative_gas_used += result.gas_used();
|
||||
|
||||
receipts.push(
|
||||
self.receipts.push(
|
||||
match self.receipt_builder.build_receipt(ReceiptBuilderCtx {
|
||||
tx: transaction,
|
||||
tx: tx.tx(),
|
||||
result,
|
||||
cumulative_gas_used,
|
||||
}) {
|
||||
@ -257,9 +245,9 @@ where
|
||||
// when set. The state transition process ensures
|
||||
// this is only set for post-Canyon deposit
|
||||
// transactions.
|
||||
deposit_receipt_version: (transaction.is_deposit() &&
|
||||
deposit_receipt_version: (tx.is_deposit() &&
|
||||
self.chain_spec
|
||||
.is_canyon_active_at_timestamp(block.timestamp()))
|
||||
.is_canyon_active_at_timestamp(self.block.timestamp()))
|
||||
.then_some(1),
|
||||
})
|
||||
}
|
||||
@ -267,53 +255,33 @@ where
|
||||
);
|
||||
}
|
||||
|
||||
Ok(ExecuteOutput { receipts, gas_used: cumulative_gas_used })
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn apply_post_execution_changes(
|
||||
&mut self,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
_receipts: &[N::Receipt],
|
||||
) -> Result<Requests, Self::Error> {
|
||||
let balance_increments = post_block_balance_increments(&self.chain_spec.clone(), block);
|
||||
mut self,
|
||||
) -> Result<BlockExecutionResult<N::Receipt>, Self::Error> {
|
||||
let balance_increments =
|
||||
post_block_balance_increments(&self.chain_spec.clone(), self.block);
|
||||
// increment balances
|
||||
self.state
|
||||
self.evm
|
||||
.db_mut()
|
||||
.increment_balances(balance_increments.clone())
|
||||
.map_err(|_| BlockValidationError::IncrementBalanceFailed)?;
|
||||
// call state hook with changes due to balance increments.
|
||||
let balance_state = balance_increment_state(&balance_increments, &mut self.state)?;
|
||||
let balance_state = balance_increment_state(&balance_increments, self.evm.db_mut())?;
|
||||
self.system_caller.on_state(
|
||||
StateChangeSource::PostBlock(StateChangePostBlockSource::BalanceIncrements),
|
||||
&balance_state,
|
||||
);
|
||||
|
||||
Ok(Requests::default())
|
||||
}
|
||||
|
||||
fn state_ref(&self) -> &State<DB> {
|
||||
&self.state
|
||||
}
|
||||
|
||||
fn state_mut(&mut self) -> &mut State<DB> {
|
||||
&mut self.state
|
||||
}
|
||||
|
||||
fn into_state(self) -> revm_database::State<Self::DB> {
|
||||
self.state
|
||||
let gas_used = self.receipts.last().map(|r| r.cumulative_gas_used()).unwrap_or_default();
|
||||
Ok(BlockExecutionResult { receipts: self.receipts, requests: Default::default(), gas_used })
|
||||
}
|
||||
|
||||
fn with_state_hook(&mut self, hook: Option<Box<dyn OnStateHook>>) {
|
||||
self.system_caller.with_state_hook(hook);
|
||||
}
|
||||
|
||||
fn validate_block_post_execution(
|
||||
&self,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
receipts: &[N::Receipt],
|
||||
_requests: &Requests,
|
||||
) -> Result<(), ConsensusError> {
|
||||
validate_block_post_execution(block.header(), self.chain_spec.clone(), receipts)
|
||||
}
|
||||
}
|
||||
|
||||
/// Helper type with backwards compatible methods to obtain executor providers.
|
||||
|
||||
@ -11,10 +11,10 @@
|
||||
extern crate alloc;
|
||||
|
||||
use alloc::sync::Arc;
|
||||
use alloy_consensus::{BlockHeader, Header};
|
||||
use alloy_consensus::{transaction::Recovered, BlockHeader, Header};
|
||||
use alloy_op_evm::OpEvmFactory;
|
||||
use alloy_primitives::{Address, U256};
|
||||
use core::fmt::Debug;
|
||||
use alloy_primitives::U256;
|
||||
use core::{borrow::Borrow, fmt::Debug};
|
||||
use op_alloy_consensus::EIP1559ParamError;
|
||||
use reth_chainspec::EthChainSpec;
|
||||
use reth_evm::{ConfigureEvm, ConfigureEvmEnv, EvmEnv, NextBlockEnvAttributes};
|
||||
@ -74,9 +74,14 @@ impl<ChainSpec: EthChainSpec + OpHardforks + 'static> ConfigureEvmEnv for OpEvmC
|
||||
type TxEnv = OpTransaction<TxEnv>;
|
||||
type Spec = OpSpecId;
|
||||
|
||||
fn tx_env(&self, transaction: &Self::Transaction, signer: Address) -> Self::TxEnv {
|
||||
fn tx_env<T: Borrow<Self::Transaction>>(
|
||||
&self,
|
||||
transaction: impl Borrow<Recovered<T>>,
|
||||
) -> Self::TxEnv {
|
||||
let transaction = transaction.borrow();
|
||||
|
||||
let mut tx_env = Default::default();
|
||||
transaction.fill_tx_env(&mut tx_env, signer);
|
||||
transaction.tx().borrow().fill_tx_env(&mut tx_env, transaction.signer());
|
||||
tx_env
|
||||
}
|
||||
|
||||
|
||||
@ -934,7 +934,7 @@ where
|
||||
))
|
||||
})?;
|
||||
|
||||
let tx_env = self.evm_config.tx_env(sequencer_tx.tx(), sequencer_tx.signer());
|
||||
let tx_env = self.evm_config.tx_env(&sequencer_tx);
|
||||
|
||||
let ResultAndState { result, state: _ } = match evm.transact_commit(tx_env) {
|
||||
Ok(res) => res,
|
||||
@ -1011,7 +1011,7 @@ where
|
||||
}
|
||||
|
||||
// Configure the environment for the tx.
|
||||
let tx_env = self.evm_config.tx_env(tx.tx(), tx.signer());
|
||||
let tx_env = self.evm_config.tx_env(&tx);
|
||||
|
||||
let ResultAndState { result, state: _ } = match evm.transact_commit(tx_env) {
|
||||
Ok(res) => res,
|
||||
|
||||
@ -2,8 +2,8 @@
|
||||
|
||||
use crate::OpEthApi;
|
||||
use alloy_consensus::{
|
||||
constants::EMPTY_WITHDRAWALS, proofs::calculate_transaction_root, Eip658Value, Header,
|
||||
Transaction as _, TxReceipt, EMPTY_OMMER_ROOT_HASH,
|
||||
constants::EMPTY_WITHDRAWALS, proofs::calculate_transaction_root, transaction::Recovered,
|
||||
Eip658Value, Header, Transaction as _, TxReceipt, EMPTY_OMMER_ROOT_HASH,
|
||||
};
|
||||
use alloy_eips::{eip7685::EMPTY_REQUESTS_HASH, merge::BEACON_NONCE, BlockNumberOrTag};
|
||||
use alloy_primitives::{B256, U256};
|
||||
@ -100,7 +100,7 @@ where
|
||||
block_env: &BlockEnv,
|
||||
parent_hash: B256,
|
||||
state_root: B256,
|
||||
transactions: Vec<ProviderTx<Self::Provider>>,
|
||||
transactions: Vec<Recovered<ProviderTx<Self::Provider>>>,
|
||||
receipts: &[ProviderReceipt<Self::Provider>],
|
||||
) -> reth_provider::ProviderBlock<Self::Provider> {
|
||||
let chain_spec = self.provider().chain_spec();
|
||||
@ -144,7 +144,11 @@ where
|
||||
// seal the block
|
||||
reth_primitives::Block {
|
||||
header,
|
||||
body: BlockBody { transactions, ommers: vec![], withdrawals: None },
|
||||
body: BlockBody {
|
||||
transactions: transactions.into_iter().map(|tx| tx.into_tx()).collect(),
|
||||
ommers: vec![],
|
||||
withdrawals: None,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -7,7 +7,7 @@ use crate::{
|
||||
};
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_eips::{eip1559::calc_next_block_base_fee, eip2930::AccessListResult};
|
||||
use alloy_primitives::{Address, Bytes, B256, U256};
|
||||
use alloy_primitives::{Bytes, B256, U256};
|
||||
use alloy_rpc_types_eth::{
|
||||
simulate::{SimBlock, SimulatePayload, SimulatedBlock},
|
||||
state::{EvmOverrides, StateOverride},
|
||||
@ -22,6 +22,7 @@ use reth_evm::{
|
||||
TransactionEnv,
|
||||
};
|
||||
use reth_node_api::BlockBody;
|
||||
use reth_primitives::Recovered;
|
||||
use reth_primitives_traits::SignedTransaction;
|
||||
use reth_provider::{BlockIdReader, ChainSpecProvider, ProviderHeader};
|
||||
use reth_revm::{database::StateProviderDatabase, db::CacheDB, DatabaseRef};
|
||||
@ -172,8 +173,6 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthA
|
||||
let mut results = Vec::with_capacity(calls.len());
|
||||
|
||||
while let Some(call) = calls.next() {
|
||||
let sender = call.from.unwrap_or_default();
|
||||
|
||||
// Resolve transaction, populate missing fields and enforce calls
|
||||
// correctness.
|
||||
let tx = simulate::resolve_transaction(
|
||||
@ -185,7 +184,7 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthA
|
||||
this.tx_resp_builder(),
|
||||
)?;
|
||||
|
||||
let tx_env = this.evm_config().tx_env(&tx, sender);
|
||||
let tx_env = this.evm_config().tx_env(&tx);
|
||||
|
||||
let (res, (_, tx_env)) = {
|
||||
if trace_transfers {
|
||||
@ -324,9 +323,9 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthA
|
||||
if replay_block_txs {
|
||||
// only need to replay the transactions in the block if not all transactions are
|
||||
// to be replayed
|
||||
let transactions = block.transactions_with_sender().take(num_txs);
|
||||
for (signer, tx) in transactions {
|
||||
let tx_env = RpcNodeCore::evm_config(&this).tx_env(tx, *signer);
|
||||
let transactions = block.transactions_recovered().take(num_txs);
|
||||
for tx in transactions {
|
||||
let tx_env = RpcNodeCore::evm_config(&this).tx_env(tx);
|
||||
let (res, _) = this.transact(&mut db, evm_env.clone(), tx_env)?;
|
||||
db.commit(res.state);
|
||||
}
|
||||
@ -671,12 +670,12 @@ pub trait Call:
|
||||
let this = self.clone();
|
||||
self.spawn_with_state_at_block(parent_block.into(), move |state| {
|
||||
let mut db = CacheDB::new(StateProviderDatabase::new(state));
|
||||
let block_txs = block.transactions_with_sender();
|
||||
let block_txs = block.transactions_recovered();
|
||||
|
||||
// replay all transactions prior to the targeted transaction
|
||||
this.replay_transactions_until(&mut db, evm_env.clone(), block_txs, *tx.tx_hash())?;
|
||||
|
||||
let tx_env = RpcNodeCore::evm_config(&this).tx_env(tx.tx(), tx.signer());
|
||||
let tx_env = RpcNodeCore::evm_config(&this).tx_env(tx);
|
||||
|
||||
let (res, _) = this.transact(&mut db, evm_env, tx_env)?;
|
||||
f(tx_info, res, db)
|
||||
@ -702,18 +701,18 @@ pub trait Call:
|
||||
) -> Result<usize, Self::Error>
|
||||
where
|
||||
DB: Database<Error = ProviderError> + DatabaseCommit,
|
||||
I: IntoIterator<Item = (&'a Address, &'a <Self::Evm as ConfigureEvmEnv>::Transaction)>,
|
||||
I: IntoIterator<Item = Recovered<&'a <Self::Evm as ConfigureEvmEnv>::Transaction>>,
|
||||
<Self::Evm as ConfigureEvmEnv>::Transaction: SignedTransaction,
|
||||
{
|
||||
let mut evm = self.evm_config().evm_with_env(db, evm_env);
|
||||
let mut index = 0;
|
||||
for (sender, tx) in transactions {
|
||||
for tx in transactions {
|
||||
if *tx.tx_hash() == target_tx_hash {
|
||||
// reached the target transaction
|
||||
break
|
||||
}
|
||||
|
||||
let tx_env = self.evm_config().tx_env(tx, *sender);
|
||||
let tx_env = self.evm_config().tx_env(tx);
|
||||
evm.transact_commit(tx_env).map_err(Self::Error::from_evm_err)?;
|
||||
index += 1;
|
||||
}
|
||||
|
||||
@ -3,7 +3,7 @@
|
||||
|
||||
use super::SpawnBlocking;
|
||||
use crate::{types::RpcTypes, EthApiTypes, FromEthApiError, FromEvmError, RpcNodeCore};
|
||||
use alloy_consensus::{BlockHeader, Transaction};
|
||||
use alloy_consensus::{transaction::Recovered, BlockHeader, Transaction};
|
||||
use alloy_eips::eip4844::MAX_DATA_GAS_PER_BLOCK;
|
||||
use alloy_primitives::B256;
|
||||
use alloy_rpc_types_eth::BlockNumberOrTag;
|
||||
@ -209,7 +209,7 @@ pub trait LoadPendingBlock:
|
||||
block_env: &BlockEnv,
|
||||
parent_hash: B256,
|
||||
state_root: B256,
|
||||
transactions: Vec<ProviderTx<Self::Provider>>,
|
||||
transactions: Vec<Recovered<ProviderTx<Self::Provider>>>,
|
||||
receipts: &[ProviderReceipt<Self::Provider>],
|
||||
) -> ProviderBlock<Self::Provider>;
|
||||
|
||||
@ -219,7 +219,7 @@ pub trait LoadPendingBlock:
|
||||
block_env: &BlockEnv,
|
||||
parent_hash: B256,
|
||||
state_root: B256,
|
||||
transactions: Vec<ProviderTx<Self::Provider>>,
|
||||
transactions: Vec<Recovered<ProviderTx<Self::Provider>>>,
|
||||
results: Vec<ExecutionResult<HaltReasonFor<Self::Evm>>>,
|
||||
) -> (ProviderBlock<Self::Provider>, Vec<ProviderReceipt<Self::Provider>>) {
|
||||
let mut cumulative_gas_used = 0;
|
||||
@ -267,7 +267,6 @@ pub trait LoadPendingBlock:
|
||||
let base_fee = evm_env.block_env.basefee;
|
||||
|
||||
let mut executed_txs = Vec::new();
|
||||
let mut senders = Vec::new();
|
||||
let mut best_txs =
|
||||
self.pool().best_transactions_with_attributes(BestTransactionsAttributes::new(
|
||||
base_fee,
|
||||
@ -336,7 +335,7 @@ pub trait LoadPendingBlock:
|
||||
}
|
||||
}
|
||||
|
||||
let tx_env = self.evm_config().tx_env(tx.tx(), tx.signer());
|
||||
let tx_env = self.evm_config().tx_env(&tx);
|
||||
|
||||
let ResultAndState { result, state: _ } = match evm.transact_commit(tx_env) {
|
||||
Ok(res) => res,
|
||||
@ -377,9 +376,7 @@ pub trait LoadPendingBlock:
|
||||
cumulative_gas_used += gas_used;
|
||||
|
||||
// append transaction to the list of executed transactions
|
||||
let (tx, sender) = tx.into_parts();
|
||||
executed_txs.push(tx);
|
||||
senders.push(sender);
|
||||
results.push(result);
|
||||
}
|
||||
|
||||
@ -405,6 +402,8 @@ pub trait LoadPendingBlock:
|
||||
// calculate the state root
|
||||
let state_root = db.database.state_root(hashed_state).map_err(Self::Error::from_eth_err)?;
|
||||
|
||||
let senders = executed_txs.iter().map(|tx| tx.signer()).collect();
|
||||
|
||||
let (block, receipts) = self.assemble_block_and_receipts(
|
||||
&evm_env.block_env,
|
||||
parent_hash,
|
||||
|
||||
@ -204,14 +204,14 @@ pub trait Trace:
|
||||
let this = self.clone();
|
||||
self.spawn_with_state_at_block(parent_block.into(), move |state| {
|
||||
let mut db = CacheDB::new(StateProviderDatabase::new(state));
|
||||
let block_txs = block.transactions_with_sender();
|
||||
let block_txs = block.transactions_recovered();
|
||||
|
||||
this.apply_pre_execution_changes(&block, &mut db, &evm_env)?;
|
||||
|
||||
// replay all transactions prior to the targeted transaction
|
||||
this.replay_transactions_until(&mut db, evm_env.clone(), block_txs, *tx.tx_hash())?;
|
||||
|
||||
let tx_env = this.evm_config().tx_env(tx.tx(), tx.signer());
|
||||
let tx_env = this.evm_config().tx_env(tx);
|
||||
let (res, _) = this.inspect(
|
||||
StateCacheDbRefMutWrapper(&mut db),
|
||||
evm_env,
|
||||
@ -339,10 +339,10 @@ pub trait Trace:
|
||||
let mut results = Vec::with_capacity(max_transactions);
|
||||
|
||||
let mut transactions = block
|
||||
.transactions_with_sender()
|
||||
.transactions_recovered()
|
||||
.take(max_transactions)
|
||||
.enumerate()
|
||||
.map(|(idx, (signer, tx))| {
|
||||
.map(|(idx, tx)| {
|
||||
let tx_info = TransactionInfo {
|
||||
hash: Some(*tx.tx_hash()),
|
||||
index: Some(idx as u64),
|
||||
@ -350,7 +350,7 @@ pub trait Trace:
|
||||
block_number: Some(block_number),
|
||||
base_fee: Some(base_fee),
|
||||
};
|
||||
let tx_env = this.evm_config().tx_env(tx, *signer);
|
||||
let tx_env = this.evm_config().tx_env(tx);
|
||||
(tx_info, tx_env)
|
||||
})
|
||||
.peekable();
|
||||
|
||||
@ -7,7 +7,7 @@ use alloy_rpc_types_eth::{
|
||||
Block, BlockTransactionsKind, Header,
|
||||
};
|
||||
use jsonrpsee_types::ErrorObject;
|
||||
use reth_primitives::RecoveredBlock;
|
||||
use reth_primitives::{Recovered, RecoveredBlock};
|
||||
use reth_primitives_traits::{block::BlockTx, BlockBody as _, SignedTransaction};
|
||||
use reth_rpc_server_types::result::rpc_err;
|
||||
use reth_rpc_types_compat::{block::from_block, TransactionCompat};
|
||||
@ -60,7 +60,7 @@ pub fn resolve_transaction<DB: Database, Tx, T: TransactionCompat<Tx>>(
|
||||
chain_id: u64,
|
||||
db: &mut DB,
|
||||
tx_resp_builder: &T,
|
||||
) -> Result<Tx, EthApiError>
|
||||
) -> Result<Recovered<Tx>, EthApiError>
|
||||
where
|
||||
EthApiError: From<DB::Error>,
|
||||
{
|
||||
@ -108,7 +108,11 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
tx_resp_builder.build_simulate_v1_transaction(tx).map_err(|e| EthApiError::other(e.into()))
|
||||
let tx = tx_resp_builder
|
||||
.build_simulate_v1_transaction(tx)
|
||||
.map_err(|e| EthApiError::other(e.into()))?;
|
||||
|
||||
Ok(Recovered::new_unchecked(tx, from))
|
||||
}
|
||||
|
||||
/// Handles outputs of the calls execution and builds a [`SimulatedBlock`].
|
||||
|
||||
@ -109,12 +109,12 @@ where
|
||||
|
||||
this.eth_api().apply_pre_execution_changes(&block, &mut db, &evm_env)?;
|
||||
|
||||
let mut transactions = block.transactions_with_sender().enumerate().peekable();
|
||||
let mut transactions = block.transactions_recovered().enumerate().peekable();
|
||||
let mut inspector = None;
|
||||
while let Some((index, (signer, tx))) = transactions.next() {
|
||||
while let Some((index, tx)) = transactions.next() {
|
||||
let tx_hash = *tx.tx_hash();
|
||||
|
||||
let tx_env = this.eth_api().evm_config().tx_env(tx, *signer);
|
||||
let tx_env = this.eth_api().evm_config().tx_env(tx);
|
||||
|
||||
let (result, state_changes) = this.trace_transaction(
|
||||
&opts,
|
||||
@ -229,7 +229,7 @@ where
|
||||
let this = self.clone();
|
||||
self.eth_api()
|
||||
.spawn_with_state_at_block(state_at, move |state| {
|
||||
let block_txs = block.transactions_with_sender();
|
||||
let block_txs = block.transactions_recovered();
|
||||
|
||||
// configure env for the target transaction
|
||||
let tx = transaction.into_recovered();
|
||||
@ -246,7 +246,7 @@ where
|
||||
*tx.tx_hash(),
|
||||
)?;
|
||||
|
||||
let tx_env = this.eth_api().evm_config().tx_env(tx.tx(), tx.signer());
|
||||
let tx_env = this.eth_api().evm_config().tx_env(&tx);
|
||||
|
||||
this.trace_transaction(
|
||||
&opts,
|
||||
@ -530,11 +530,11 @@ where
|
||||
if replay_block_txs {
|
||||
// only need to replay the transactions in the block if not all transactions are
|
||||
// to be replayed
|
||||
let transactions = block.transactions_with_sender().take(num_txs);
|
||||
let transactions = block.transactions_recovered().take(num_txs);
|
||||
|
||||
// Execute all transactions until index
|
||||
for (signer, tx) in transactions {
|
||||
let tx_env = this.eth_api().evm_config().tx_env(tx, *signer);
|
||||
for tx in transactions {
|
||||
let tx_env = this.eth_api().evm_config().tx_env(tx);
|
||||
let (res, _) = this.eth_api().transact(&mut db, evm_env.clone(), tx_env)?;
|
||||
db.commit(res.state);
|
||||
}
|
||||
|
||||
@ -176,7 +176,7 @@ where
|
||||
hasher.update(*tx.tx_hash());
|
||||
let gas_price = tx.effective_gas_price(basefee);
|
||||
let ResultAndState { result, state } = evm
|
||||
.transact(eth_api.evm_config().tx_env(&tx, signer))
|
||||
.transact(eth_api.evm_config().tx_env(&tx))
|
||||
.map_err(Eth::Error::from_evm_err)?;
|
||||
|
||||
let gas_used = result.gas_used();
|
||||
|
||||
@ -1,6 +1,9 @@
|
||||
//! Support for building a pending block with transactions from local view of mempool.
|
||||
|
||||
use alloy_consensus::{constants::EMPTY_WITHDRAWALS, Header, Transaction, EMPTY_OMMER_ROOT_HASH};
|
||||
use alloy_consensus::{
|
||||
constants::EMPTY_WITHDRAWALS, transaction::Recovered, Header, Transaction,
|
||||
EMPTY_OMMER_ROOT_HASH,
|
||||
};
|
||||
use alloy_eips::{eip7685::EMPTY_REQUESTS_HASH, merge::BEACON_NONCE};
|
||||
use alloy_primitives::U256;
|
||||
use reth_chainspec::{EthChainSpec, EthereumHardforks};
|
||||
@ -61,7 +64,7 @@ where
|
||||
block_env: &BlockEnv,
|
||||
parent_hash: revm_primitives::B256,
|
||||
state_root: revm_primitives::B256,
|
||||
transactions: Vec<ProviderTx<Self::Provider>>,
|
||||
transactions: Vec<Recovered<ProviderTx<Self::Provider>>>,
|
||||
receipts: &[ProviderReceipt<Self::Provider>],
|
||||
) -> reth_provider::ProviderBlock<Self::Provider> {
|
||||
let chain_spec = self.provider().chain_spec();
|
||||
@ -105,7 +108,11 @@ where
|
||||
// seal the block
|
||||
reth_primitives::Block {
|
||||
header,
|
||||
body: BlockBody { transactions, ommers: vec![], withdrawals: None },
|
||||
body: BlockBody {
|
||||
transactions: transactions.into_iter().map(|tx| tx.into_tx()).collect(),
|
||||
ommers: vec![],
|
||||
withdrawals: None,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -10,6 +10,7 @@ use alloy_rpc_types_mev::{
|
||||
};
|
||||
use jsonrpsee::core::RpcResult;
|
||||
use reth_evm::{ConfigureEvm, ConfigureEvmEnv, Evm};
|
||||
use reth_primitives::Recovered;
|
||||
use reth_provider::ProviderTx;
|
||||
use reth_revm::{database::StateProviderDatabase, db::CacheDB};
|
||||
use reth_rpc_api::MevSimApiServer;
|
||||
@ -21,9 +22,8 @@ use reth_rpc_eth_types::{
|
||||
revm_utils::apply_block_overrides, utils::recover_raw_transaction, EthApiError,
|
||||
};
|
||||
use reth_tasks::pool::BlockingTaskGuard;
|
||||
use reth_transaction_pool::{PoolConsensusTx, PoolPooledTx, PoolTransaction, TransactionPool};
|
||||
use reth_transaction_pool::{PoolPooledTx, PoolTransaction, TransactionPool};
|
||||
use revm::{context_interface::result::ResultAndState, DatabaseCommit, DatabaseRef};
|
||||
use revm_primitives::Address;
|
||||
use std::{sync::Arc, time::Duration};
|
||||
use tracing::info;
|
||||
|
||||
@ -46,9 +46,7 @@ const SBUNDLE_PAYOUT_MAX_COST: u64 = 30_000;
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct FlattenedBundleItem<T> {
|
||||
/// The signed transaction
|
||||
pub tx: T,
|
||||
/// The address that signed the transaction
|
||||
pub signer: Address,
|
||||
pub tx: Recovered<T>,
|
||||
/// Whether the transaction is allowed to revert
|
||||
pub can_revert: bool,
|
||||
/// Item-level inclusion constraints
|
||||
@ -169,10 +167,10 @@ where
|
||||
while idx < body.len() {
|
||||
match &body[idx] {
|
||||
BundleItem::Tx { tx, can_revert } => {
|
||||
let recovered_tx = recover_raw_transaction::<PoolPooledTx<Eth::Pool>>(tx)?;
|
||||
let (tx, signer) = recovered_tx.into_parts();
|
||||
let tx: PoolConsensusTx<Eth::Pool> =
|
||||
<Eth::Pool as TransactionPool>::Transaction::pooled_into_consensus(tx);
|
||||
let tx = recover_raw_transaction::<PoolPooledTx<Eth::Pool>>(tx)?;
|
||||
let tx = tx.map_transaction(
|
||||
<Eth::Pool as TransactionPool>::Transaction::pooled_into_consensus,
|
||||
);
|
||||
|
||||
let refund_percent =
|
||||
validity.as_ref().and_then(|v| v.refund.as_ref()).and_then(|refunds| {
|
||||
@ -186,7 +184,6 @@ where
|
||||
// Create FlattenedBundleItem with current inclusion, validity, and privacy
|
||||
let flattened_item = FlattenedBundleItem {
|
||||
tx,
|
||||
signer,
|
||||
can_revert: *can_revert,
|
||||
inclusion: inclusion.clone(),
|
||||
validity: validity.clone(),
|
||||
@ -282,7 +279,7 @@ where
|
||||
}
|
||||
|
||||
let ResultAndState { result, state } = evm
|
||||
.transact(eth_api.evm_config().tx_env(&item.tx, item.signer))
|
||||
.transact(eth_api.evm_config().tx_env(&item.tx))
|
||||
.map_err(Eth::Error::from_evm_err)?;
|
||||
|
||||
if !result.is_success() && !item.can_revert {
|
||||
@ -330,7 +327,7 @@ where
|
||||
if let Some(refund_percent) = item.refund_percent {
|
||||
// Get refund configurations
|
||||
let refund_configs = item.refund_configs.clone().unwrap_or_else(|| {
|
||||
vec![RefundConfig { address: item.signer, percent: 100 }]
|
||||
vec![RefundConfig { address: item.tx.signer(), percent: 100 }]
|
||||
});
|
||||
|
||||
// Calculate payout transaction fee
|
||||
|
||||
@ -113,7 +113,7 @@ where
|
||||
.map_transaction(<Eth::Pool as TransactionPool>::Transaction::pooled_into_consensus);
|
||||
|
||||
let (evm_env, at) = self.eth_api().evm_env_at(block_id.unwrap_or_default()).await?;
|
||||
let tx_env = self.eth_api().evm_config().tx_env(tx.tx(), tx.signer());
|
||||
let tx_env = self.eth_api().evm_config().tx_env(tx);
|
||||
|
||||
let config = TracingInspectorConfig::from_parity_config(&trace_types);
|
||||
|
||||
|
||||
@ -4,13 +4,14 @@
|
||||
#![cfg_attr(not(test), warn(unused_crate_dependencies))]
|
||||
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_eips::{eip4895::Withdrawal, eip7685::Requests};
|
||||
use alloy_eips::eip4895::Withdrawal;
|
||||
use alloy_sol_macro::sol;
|
||||
use alloy_sol_types::SolCall;
|
||||
use reth::{
|
||||
api::{ConfigureEvm, NodeTypesWithEngine},
|
||||
builder::{components::ExecutorBuilder, BuilderContext, FullNodeTypes},
|
||||
cli::Cli,
|
||||
providers::BlockExecutionResult,
|
||||
revm::{
|
||||
db::State,
|
||||
primitives::{address, Address},
|
||||
@ -20,14 +21,16 @@ use reth::{
|
||||
use reth_chainspec::{ChainSpec, EthereumHardforks};
|
||||
use reth_evm::{
|
||||
execute::{
|
||||
BlockExecutionError, BlockExecutionStrategy, BlockExecutionStrategyFactory, ExecuteOutput,
|
||||
BlockExecutionError, BlockExecutionStrategy, BlockExecutionStrategyFactory,
|
||||
InternalBlockExecutionError,
|
||||
},
|
||||
Database, Evm,
|
||||
};
|
||||
use reth_evm_ethereum::EthEvmConfig;
|
||||
use reth_node_ethereum::{node::EthereumAddOns, BasicBlockExecutorProvider, EthereumNode};
|
||||
use reth_primitives::{EthPrimitives, Receipt, RecoveredBlock};
|
||||
use reth_primitives::{
|
||||
Block, EthPrimitives, Receipt, Recovered, RecoveredBlock, SealedBlock, TransactionSigned,
|
||||
};
|
||||
use std::{fmt::Display, sync::Arc};
|
||||
|
||||
pub const SYSTEM_ADDRESS: Address = address!("fffffffffffffffffffffffffffffffffffffffe");
|
||||
@ -90,86 +93,64 @@ pub struct CustomExecutorStrategyFactory {
|
||||
|
||||
impl BlockExecutionStrategyFactory for CustomExecutorStrategyFactory {
|
||||
type Primitives = EthPrimitives;
|
||||
type Strategy<DB: Database> = CustomExecutorStrategy<DB>;
|
||||
|
||||
fn create_strategy<DB>(&self, db: DB) -> Self::Strategy<DB>
|
||||
fn create_strategy<'a, DB>(
|
||||
&'a mut self,
|
||||
db: &'a mut State<DB>,
|
||||
block: &'a RecoveredBlock<Block>,
|
||||
) -> impl BlockExecutionStrategy<Primitives = Self::Primitives, Error = BlockExecutionError> + 'a
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
let state =
|
||||
State::builder().with_database(db).with_bundle_update().without_state_clear().build();
|
||||
CustomExecutorStrategy {
|
||||
state,
|
||||
chain_spec: self.chain_spec.clone(),
|
||||
evm_config: self.evm_config.clone(),
|
||||
}
|
||||
let evm = self.evm_config.evm_for_block(db, block.header());
|
||||
CustomExecutorStrategy { evm, factory: self, block }
|
||||
}
|
||||
}
|
||||
|
||||
pub struct CustomExecutorStrategy<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
/// The chainspec
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
/// How to create an EVM.
|
||||
evm_config: EthEvmConfig,
|
||||
/// Current state for block execution.
|
||||
state: State<DB>,
|
||||
pub struct CustomExecutorStrategy<'a, Evm> {
|
||||
/// Reference to the parent factory.
|
||||
factory: &'a CustomExecutorStrategyFactory,
|
||||
/// EVM used for execution.
|
||||
evm: Evm,
|
||||
/// Block being executed.
|
||||
block: &'a SealedBlock,
|
||||
}
|
||||
|
||||
impl<DB> BlockExecutionStrategy for CustomExecutorStrategy<DB>
|
||||
impl<'db, DB, E> BlockExecutionStrategy for CustomExecutorStrategy<'_, E>
|
||||
where
|
||||
DB: Database,
|
||||
DB: Database + 'db,
|
||||
E: Evm<DB = &'db mut State<DB>>,
|
||||
{
|
||||
type DB = DB;
|
||||
type Primitives = EthPrimitives;
|
||||
type Error = BlockExecutionError;
|
||||
|
||||
fn apply_pre_execution_changes(
|
||||
&mut self,
|
||||
block: &RecoveredBlock<reth_primitives::Block>,
|
||||
) -> Result<(), Self::Error> {
|
||||
fn apply_pre_execution_changes(&mut self) -> Result<(), Self::Error> {
|
||||
// Set state clear flag if the block is after the Spurious Dragon hardfork.
|
||||
let state_clear_flag =
|
||||
(*self.chain_spec).is_spurious_dragon_active_at_block(block.number());
|
||||
self.state.set_state_clear_flag(state_clear_flag);
|
||||
(*self.factory.chain_spec).is_spurious_dragon_active_at_block(self.block.number());
|
||||
self.evm.db_mut().set_state_clear_flag(state_clear_flag);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn execute_transactions(
|
||||
fn execute_transactions<'a>(
|
||||
&mut self,
|
||||
_block: &RecoveredBlock<reth_primitives::Block>,
|
||||
) -> Result<ExecuteOutput<Receipt>, Self::Error> {
|
||||
Ok(ExecuteOutput { receipts: vec![], gas_used: 0 })
|
||||
_transactions: impl IntoIterator<Item = Recovered<&'a TransactionSigned>>,
|
||||
) -> Result<(), Self::Error> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn apply_post_execution_changes(
|
||||
&mut self,
|
||||
block: &RecoveredBlock<reth_primitives::Block>,
|
||||
_receipts: &[Receipt],
|
||||
) -> Result<Requests, Self::Error> {
|
||||
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
|
||||
|
||||
if let Some(withdrawals) = block.body().withdrawals.as_ref() {
|
||||
apply_withdrawals_contract_call(withdrawals, &mut evm)?;
|
||||
mut self,
|
||||
) -> Result<BlockExecutionResult<Receipt>, Self::Error> {
|
||||
if let Some(withdrawals) = self.block.body().withdrawals.as_ref() {
|
||||
apply_withdrawals_contract_call(withdrawals, &mut self.evm)?;
|
||||
}
|
||||
|
||||
Ok(Requests::default())
|
||||
Ok(Default::default())
|
||||
}
|
||||
|
||||
fn state_ref(&self) -> &State<DB> {
|
||||
&self.state
|
||||
}
|
||||
|
||||
fn state_mut(&mut self) -> &mut State<DB> {
|
||||
&mut self.state
|
||||
}
|
||||
|
||||
fn into_state(self) -> reth::revm::db::State<Self::DB> {
|
||||
self.state
|
||||
}
|
||||
fn with_state_hook(&mut self, _hook: Option<Box<dyn reth_evm::system_calls::OnStateHook>>) {}
|
||||
}
|
||||
|
||||
sol!(
|
||||
|
||||
@ -41,9 +41,10 @@ use reth_node_ethereum::{
|
||||
node::{EthereumAddOns, EthereumPayloadBuilder},
|
||||
BasicBlockExecutorProvider, EthExecutionStrategyFactory, EthereumNode,
|
||||
};
|
||||
use reth_primitives::{EthPrimitives, TransactionSigned};
|
||||
use reth_primitives::{EthPrimitives, Recovered, TransactionSigned};
|
||||
use reth_tracing::{RethTracer, Tracer};
|
||||
use std::{
|
||||
borrow::Borrow,
|
||||
convert::Infallible,
|
||||
sync::{Arc, OnceLock},
|
||||
};
|
||||
@ -105,8 +106,11 @@ impl ConfigureEvmEnv for MyEvmConfig {
|
||||
type TxEnv = TxEnv;
|
||||
type Spec = SpecId;
|
||||
|
||||
fn tx_env(&self, transaction: &Self::Transaction, signer: Address) -> Self::TxEnv {
|
||||
self.inner.tx_env(transaction, signer)
|
||||
fn tx_env<T: Borrow<Self::Transaction>>(
|
||||
&self,
|
||||
transaction: impl Borrow<Recovered<T>>,
|
||||
) -> Self::TxEnv {
|
||||
self.inner.tx_env(transaction)
|
||||
}
|
||||
|
||||
fn evm_env(&self, header: &Self::Header) -> EvmEnv {
|
||||
|
||||
@ -33,10 +33,10 @@ use reth_node_ethereum::{
|
||||
evm::EthEvm, node::EthereumAddOns, BasicBlockExecutorProvider, EthEvmConfig,
|
||||
EthExecutionStrategyFactory, EthereumNode,
|
||||
};
|
||||
use reth_primitives::{EthPrimitives, TransactionSigned};
|
||||
use reth_primitives::{EthPrimitives, Recovered, TransactionSigned};
|
||||
use reth_tracing::{RethTracer, Tracer};
|
||||
use schnellru::{ByLength, LruMap};
|
||||
use std::{collections::HashMap, convert::Infallible, sync::Arc};
|
||||
use std::{borrow::Borrow, collections::HashMap, convert::Infallible, sync::Arc};
|
||||
|
||||
/// Type alias for the LRU cache used within the [`PrecompileCache`].
|
||||
type PrecompileLRUCache = LruMap<(SpecId, Bytes, u64), Result<InterpreterResult, PrecompileErrors>>;
|
||||
@ -186,8 +186,11 @@ impl ConfigureEvmEnv for MyEvmConfig {
|
||||
type TxEnv = TxEnv;
|
||||
type Spec = SpecId;
|
||||
|
||||
fn tx_env(&self, transaction: &Self::Transaction, signer: Address) -> Self::TxEnv {
|
||||
self.inner.tx_env(transaction, signer)
|
||||
fn tx_env<T: Borrow<Self::Transaction>>(
|
||||
&self,
|
||||
transaction: impl Borrow<Recovered<T>>,
|
||||
) -> Self::TxEnv {
|
||||
self.inner.tx_env(transaction)
|
||||
}
|
||||
|
||||
fn evm_env(&self, header: &Self::Header) -> EvmEnv {
|
||||
|
||||
Reference in New Issue
Block a user