refactor: BlockExecutionStrategy API (#14480)

This commit is contained in:
Arsenii Kulikov
2025-02-19 17:10:50 +04:00
committed by GitHub
parent 1ae733a7d1
commit 4f454c91d6
30 changed files with 423 additions and 555 deletions

View File

@ -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",

View File

@ -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>,

View File

@ -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(

View File

@ -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>;

View File

@ -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);
}
}

View File

@ -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)
}
}