mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
refactor: BlockExecutionStrategy API (#14480)
This commit is contained in:
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user