refactor: Add more low-level methods to Executor (#14440)

This commit is contained in:
Arsenii Kulikov
2025-02-12 15:15:01 +04:00
committed by GitHub
parent 9a805b06db
commit 48bffd418f
9 changed files with 292 additions and 192 deletions

View File

@ -2,6 +2,17 @@ use alloc::vec::Vec;
use alloy_eips::eip7685::Requests;
use revm::db::BundleState;
/// The result of executing a block.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct BlockExecutionResult<T> {
/// All the receipts of the transactions in the block.
pub receipts: Vec<T>,
/// All the EIP-7685 requests in the block.
pub requests: Requests,
/// The total gas used by the block.
pub gas_used: u64,
}
/// The output of an ethereum block.
///
/// Contains the state changes, transaction receipts, and total gas used in the block.

View File

@ -8,7 +8,8 @@ use crate::{
// re-export Either
pub use futures_util::future::Either;
use revm::State;
use reth_execution_types::{BlockExecutionOutput, BlockExecutionResult};
use reth_primitives::{NodePrimitives, RecoveredBlock};
impl<A, B> BlockExecutorProvider for Either<A, B>
where
@ -45,45 +46,73 @@ where
impl<A, B, DB> Executor<DB> for Either<A, B>
where
A: Executor<DB>,
B: for<'a> Executor<DB, Input<'a> = A::Input<'a>, Output = A::Output, Error = A::Error>,
B: Executor<DB, Primitives = A::Primitives, Error = A::Error>,
DB: Database,
{
type Input<'a> = A::Input<'a>;
type Output = A::Output;
type Primitives = A::Primitives;
type Error = A::Error;
fn execute(self, input: Self::Input<'_>) -> Result<Self::Output, Self::Error> {
fn execute_one(
&mut self,
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
{
match self {
Self::Left(a) => a.execute(input),
Self::Right(b) => b.execute(input),
Self::Left(a) => a.execute_one(block),
Self::Right(b) => b.execute_one(block),
}
}
fn execute_one_with_state_hook<F>(
&mut self,
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
state_hook: F,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
where
F: OnStateHook + 'static,
{
match self {
Self::Left(a) => a.execute_one_with_state_hook(block, state_hook),
Self::Right(b) => b.execute_one_with_state_hook(block, state_hook),
}
}
fn execute(
self,
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> Result<BlockExecutionOutput<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
{
match self {
Self::Left(a) => a.execute(block),
Self::Right(b) => b.execute(block),
}
}
fn execute_with_state_closure<F>(
self,
input: Self::Input<'_>,
witness: F,
) -> Result<Self::Output, Self::Error>
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
state: F,
) -> Result<BlockExecutionOutput<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
where
F: FnMut(&State<DB>),
F: FnMut(&revm::db::State<DB>),
{
match self {
Self::Left(a) => a.execute_with_state_closure(input, witness),
Self::Right(b) => b.execute_with_state_closure(input, witness),
Self::Left(a) => a.execute_with_state_closure(block, state),
Self::Right(b) => b.execute_with_state_closure(block, state),
}
}
fn execute_with_state_hook<F>(
self,
input: Self::Input<'_>,
state_hook: F,
) -> Result<Self::Output, Self::Error>
where
F: OnStateHook + 'static,
{
fn into_state(self) -> revm::db::State<DB> {
match self {
Self::Left(a) => a.execute_with_state_hook(input, state_hook),
Self::Right(b) => b.execute_with_state_hook(input, state_hook),
Self::Left(a) => a.into_state(),
Self::Right(b) => b.into_state(),
}
}
fn size_hint(&self) -> usize {
match self {
Self::Left(a) => a.size_hint(),
Self::Right(b) => b.size_hint(),
}
}
}

View File

@ -5,6 +5,7 @@ use alloy_consensus::BlockHeader;
pub use reth_execution_errors::{
BlockExecutionError, BlockValidationError, InternalBlockExecutionError,
};
use reth_execution_types::BlockExecutionResult;
pub use reth_execution_types::{BlockExecutionOutput, ExecutionOutcome};
pub use reth_storage_errors::provider::ProviderError;
@ -17,24 +18,35 @@ use alloy_primitives::{
};
use reth_consensus::ConsensusError;
use reth_primitives::{NodePrimitives, Receipt, RecoveredBlock};
use revm::{
db::{states::bundle_state::BundleRetention, BundleState},
State,
};
use revm::db::{states::bundle_state::BundleRetention, State};
use revm_primitives::{Account, AccountStatus, EvmState};
/// A general purpose executor trait that executes an input (e.g. block) and produces an output
/// (e.g. state changes and receipts).
///
/// This executor does not validate the output, see [`BatchExecutor`] for that.
pub trait Executor<DB> {
/// The input type for the executor.
type Input<'a>;
/// The output type for the executor.
type Output;
pub trait Executor<DB: Database>: Sized {
/// The primitive types used by the executor.
type Primitives: NodePrimitives;
/// The error type returned by the executor.
type Error;
/// Executes a single block and returns [`BlockExecutionResult`], without the state changes.
fn execute_one(
&mut self,
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>;
/// Executes the EVM with the given input and accepts a state hook closure that is invoked with
/// the EVM state after execution.
fn execute_one_with_state_hook<F>(
&mut self,
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
state_hook: F,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
where
F: OnStateHook + 'static;
/// Consumes the type and executes the block.
///
/// # Note
@ -43,27 +55,55 @@ pub trait Executor<DB> {
///
/// # Returns
/// The output of the block execution.
fn execute(self, input: Self::Input<'_>) -> Result<Self::Output, Self::Error>;
fn execute(
mut self,
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> Result<BlockExecutionOutput<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
{
let BlockExecutionResult { receipts, requests, gas_used } = self.execute_one(block)?;
let mut state = self.into_state();
Ok(BlockExecutionOutput { state: state.take_bundle(), receipts, requests, gas_used })
}
/// Executes the EVM with the given input and accepts a state closure that is invoked with
/// the EVM state after execution.
fn execute_with_state_closure<F>(
self,
input: Self::Input<'_>,
state: F,
) -> Result<Self::Output, Self::Error>
mut self,
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
mut f: F,
) -> Result<BlockExecutionOutput<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
where
F: FnMut(&State<DB>);
F: FnMut(&State<DB>),
{
let BlockExecutionResult { receipts, requests, gas_used } = self.execute_one(block)?;
let mut state = self.into_state();
f(&state);
Ok(BlockExecutionOutput { state: state.take_bundle(), receipts, requests, gas_used })
}
/// Executes the EVM with the given input and accepts a state hook closure that is invoked with
/// the EVM state after execution.
fn execute_with_state_hook<F>(
self,
input: Self::Input<'_>,
mut self,
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
state_hook: F,
) -> Result<Self::Output, Self::Error>
) -> Result<BlockExecutionOutput<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
where
F: OnStateHook + 'static;
F: OnStateHook + 'static,
{
let BlockExecutionResult { receipts, requests, gas_used } =
self.execute_one_with_state_hook(block, state_hook)?;
let mut state = self.into_state();
Ok(BlockExecutionOutput { state: state.take_bundle(), receipts, requests, gas_used })
}
/// Consumes the executor and returns the [`State`] containing all state changes.
fn into_state(self) -> State<DB>;
/// The size hint of the batch's tracked state size.
///
/// This is used to optimize DB commits depending on the size of the state.
fn size_hint(&self) -> usize;
}
/// A general purpose executor that can execute multiple inputs in sequence, validate the outputs,
@ -132,10 +172,9 @@ pub trait BlockExecutorProvider: Send + Sync + Clone + Unpin + 'static {
///
/// It is not expected to validate the state trie root, this must be done by the caller using
/// the returned state.
type Executor<DB: Database>: for<'a> Executor<
type Executor<DB: Database>: Executor<
DB,
Input<'a> = &'a RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
Output = BlockExecutionOutput<<Self::Primitives as NodePrimitives>::Receipt>,
Primitives = Self::Primitives,
Error = BlockExecutionError,
>;
@ -208,15 +247,12 @@ pub trait BlockExecutionStrategy {
/// 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>;
/// Sets a hook to be called after each state change during execution.
fn with_state_hook(&mut self, _hook: Option<Box<dyn OnStateHook>>) {}
/// Returns the final bundle state.
fn finish(&mut self) -> BundleState {
self.state_mut().merge_transitions(BundleRetention::Reverts);
self.state_mut().take_bundle()
}
/// Validate a block with regard to execution results.
fn validate_block_post_execution(
&self,
@ -316,55 +352,46 @@ where
S: BlockExecutionStrategy<DB = DB>,
DB: Database,
{
type Input<'a> = &'a RecoveredBlock<<S::Primitives as NodePrimitives>::Block>;
type Output = BlockExecutionOutput<<S::Primitives as NodePrimitives>::Receipt>;
type Primitives = S::Primitives;
type Error = S::Error;
fn execute(mut self, block: Self::Input<'_>) -> Result<Self::Output, 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)?;
let state = self.strategy.finish();
Ok(BlockExecutionOutput { state, receipts, requests, gas_used })
}
fn execute_with_state_closure<F>(
mut self,
block: Self::Input<'_>,
mut state: F,
) -> Result<Self::Output, Self::Error>
where
F: FnMut(&State<DB>),
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);
state(self.strategy.state_ref());
let state = self.strategy.finish();
Ok(BlockExecutionOutput { state, receipts, requests, gas_used })
Ok(BlockExecutionResult { receipts, requests, gas_used })
}
fn execute_with_state_hook<H>(
mut self,
block: Self::Input<'_>,
state_hook: H,
) -> Result<Self::Output, Self::Error>
fn execute_one_with_state_hook<F>(
&mut self,
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
state_hook: F,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
where
H: OnStateHook + 'static,
F: OnStateHook + 'static,
{
self.strategy.with_state_hook(Some(Box::new(state_hook)));
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 state = self.strategy.finish();
Ok(BlockExecutionResult { receipts, requests, gas_used })
}
Ok(BlockExecutionOutput { state, receipts, requests, gas_used })
fn into_state(self) -> State<DB> {
self.strategy.into_state()
}
fn size_hint(&self) -> usize {
self.strategy.state_ref().bundle_state.size_hint()
}
}
@ -511,36 +538,36 @@ mod tests {
struct TestExecutor<DB>(PhantomData<DB>);
impl<DB> Executor<DB> for TestExecutor<DB> {
type Input<'a> = &'a RecoveredBlock<reth_primitives::Block>;
type Output = BlockExecutionOutput<Receipt>;
impl<DB: Database> Executor<DB> for TestExecutor<DB> {
type Primitives = EthPrimitives;
type Error = BlockExecutionError;
fn execute(self, _input: Self::Input<'_>) -> Result<Self::Output, Self::Error> {
Err(BlockExecutionError::msg("execution unavailable for tests"))
}
fn execute_with_state_closure<F>(
self,
_: Self::Input<'_>,
_: F,
) -> Result<Self::Output, Self::Error>
where
F: FnMut(&State<DB>),
fn execute_one(
&mut self,
_block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
{
Err(BlockExecutionError::msg("execution unavailable for tests"))
}
fn execute_with_state_hook<F>(
self,
_: Self::Input<'_>,
_: F,
) -> Result<Self::Output, Self::Error>
fn execute_one_with_state_hook<F>(
&mut self,
_block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
_state_hook: F,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
where
F: OnStateHook,
F: OnStateHook + 'static,
{
Err(BlockExecutionError::msg("execution unavailable for tests"))
}
fn into_state(self) -> State<DB> {
unreachable!()
}
fn size_hint(&self) -> usize {
0
}
}
impl<DB> BatchExecutor<DB> for TestExecutor<DB> {
@ -569,14 +596,12 @@ mod tests {
state: State<DB>,
execute_transactions_result: ExecuteOutput<Receipt>,
apply_post_execution_changes_result: Requests,
finish_result: BundleState,
}
#[derive(Clone)]
struct TestExecutorStrategyFactory {
execute_transactions_result: ExecuteOutput<Receipt>,
apply_post_execution_changes_result: Requests,
finish_result: BundleState,
}
impl BlockExecutionStrategyFactory for TestExecutorStrategyFactory {
@ -600,7 +625,6 @@ mod tests {
apply_post_execution_changes_result: self
.apply_post_execution_changes_result
.clone(),
finish_result: self.finish_result.clone(),
state,
}
}
@ -644,8 +668,8 @@ mod tests {
&mut self.state
}
fn finish(&mut self) -> BundleState {
self.finish_result.clone()
fn into_state(self) -> State<Self::DB> {
self.state
}
}
@ -669,13 +693,11 @@ mod tests {
gas_used: expected_gas_used,
};
let expected_apply_post_execution_changes_result = Requests::new(vec![bytes!("deadbeef")]);
let expected_finish_result = BundleState::default();
let strategy_factory = TestExecutorStrategyFactory {
execute_transactions_result: expected_execute_transactions_result,
apply_post_execution_changes_result: expected_apply_post_execution_changes_result
.clone(),
finish_result: expected_finish_result.clone(),
};
let provider = BasicBlockExecutorProvider::new(strategy_factory);
let db = CacheDB::<EmptyDBTyped<ProviderError>>::default();
@ -687,7 +709,6 @@ mod tests {
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.state, expected_finish_result);
}
fn setup_state_with_account(

View File

@ -2,12 +2,12 @@
//!
//! Block processing related to syncing should take care to update the metrics by using either
//! [`ExecutorMetrics::execute_metered`] or [`ExecutorMetrics::metered_one`].
use crate::{execute::Executor, system_calls::OnStateHook};
use crate::{execute::Executor, system_calls::OnStateHook, Database};
use alloy_consensus::BlockHeader;
use metrics::{Counter, Gauge, Histogram};
use reth_execution_types::BlockExecutionOutput;
use reth_metrics::Metrics;
use reth_primitives::RecoveredBlock;
use reth_primitives::{NodePrimitives, RecoveredBlock};
use revm_primitives::EvmState;
use std::time::Instant;
@ -94,20 +94,15 @@ impl ExecutorMetrics {
/// of accounts, storage slots and bytecodes loaded and updated.
/// Execute the given block using the provided [`Executor`] and update metrics for the
/// execution.
pub fn execute_metered<'a, E, DB, O, Error, B>(
pub fn execute_metered<E, DB>(
&self,
executor: E,
input: &'a RecoveredBlock<B>,
input: &RecoveredBlock<<E::Primitives as NodePrimitives>::Block>,
state_hook: Box<dyn OnStateHook>,
) -> Result<BlockExecutionOutput<O>, Error>
) -> Result<BlockExecutionOutput<<E::Primitives as NodePrimitives>::Receipt>, E::Error>
where
E: Executor<
DB,
Input<'a> = &'a RecoveredBlock<B>,
Output = BlockExecutionOutput<O>,
Error = Error,
>,
B: reth_primitives_traits::Block,
DB: Database,
E: Executor<DB>,
{
// clone here is cheap, all the metrics are Option<Arc<_>>. additionally
// they are gloally registered so that the data recorded in the hook will
@ -145,7 +140,9 @@ mod tests {
use super::*;
use alloy_eips::eip7685::Requests;
use metrics_util::debugging::{DebugValue, DebuggingRecorder, Snapshotter};
use revm::db::BundleState;
use reth_execution_types::BlockExecutionResult;
use reth_primitives::EthPrimitives;
use revm::db::{EmptyDB, State};
use revm_primitives::{
Account, AccountInfo, AccountStatus, EvmState, EvmStorage, EvmStorageSlot, B256, U256,
};
@ -156,55 +153,47 @@ mod tests {
state: EvmState,
}
impl Executor<()> for MockExecutor {
type Input<'a>
= &'a RecoveredBlock<reth_primitives::Block>
where
Self: 'a;
type Output = BlockExecutionOutput<()>;
impl<DB: Database + Default> Executor<DB> for MockExecutor {
type Primitives = EthPrimitives;
type Error = std::convert::Infallible;
fn execute(self, _input: Self::Input<'_>) -> Result<Self::Output, Self::Error> {
Ok(BlockExecutionOutput {
state: BundleState::default(),
receipts: vec![],
requests: Requests::default(),
gas_used: 0,
})
}
fn execute_with_state_closure<F>(
self,
_input: Self::Input<'_>,
_state: F,
) -> Result<Self::Output, Self::Error>
where
F: FnMut(&revm::State<()>),
fn execute_one(
&mut self,
_block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
{
Ok(BlockExecutionOutput {
state: BundleState::default(),
Ok(BlockExecutionResult {
receipts: vec![],
requests: Requests::default(),
gas_used: 0,
})
}
fn execute_with_state_hook<F>(
self,
_input: Self::Input<'_>,
fn execute_one_with_state_hook<F>(
&mut self,
_block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
mut hook: F,
) -> Result<Self::Output, Self::Error>
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
where
F: OnStateHook + 'static,
{
// Call hook with our mock state
hook.on_state(&self.state);
Ok(BlockExecutionOutput {
state: BundleState::default(),
Ok(BlockExecutionResult {
receipts: vec![],
requests: Requests::default(),
gas_used: 0,
})
}
fn into_state(self) -> revm::db::State<DB> {
State::builder().with_database(Default::default()).build()
}
fn size_hint(&self) -> usize {
0
}
}
struct ChannelStateHook {
@ -255,7 +244,7 @@ mod tests {
state
};
let executor = MockExecutor { state };
let _result = metrics.execute_metered(executor, &input, state_hook).unwrap();
let _result = metrics.execute_metered::<_, EmptyDB>(executor, &input, state_hook).unwrap();
let snapshot = snapshotter.snapshot().into_vec();
@ -287,7 +276,7 @@ mod tests {
let state = EvmState::default();
let executor = MockExecutor { state };
let _result = metrics.execute_metered(executor, &input, state_hook).unwrap();
let _result = metrics.execute_metered::<_, EmptyDB>(executor, &input, state_hook).unwrap();
let actual_output = rx.try_recv().unwrap();
assert_eq!(actual_output, expected_output);

View File

@ -1,9 +1,8 @@
//! A no operation block executor implementation.
use reth_execution_errors::BlockExecutionError;
use reth_execution_types::{BlockExecutionOutput, ExecutionOutcome};
use reth_execution_types::{BlockExecutionResult, ExecutionOutcome};
use reth_primitives::{NodePrimitives, RecoveredBlock};
use revm::State;
use crate::{
execute::{BatchExecutor, BlockExecutorProvider, Executor},
@ -40,36 +39,36 @@ impl<P: NodePrimitives> BlockExecutorProvider for NoopBlockExecutorProvider<P> {
}
}
impl<DB, P: NodePrimitives> Executor<DB> for NoopBlockExecutorProvider<P> {
type Input<'a> = &'a RecoveredBlock<P::Block>;
type Output = BlockExecutionOutput<P::Receipt>;
impl<DB: Database, P: NodePrimitives> Executor<DB> for NoopBlockExecutorProvider<P> {
type Primitives = P;
type Error = BlockExecutionError;
fn execute(self, _: Self::Input<'_>) -> Result<Self::Output, Self::Error> {
Err(BlockExecutionError::msg(UNAVAILABLE_FOR_NOOP))
}
fn execute_with_state_closure<F>(
self,
_: Self::Input<'_>,
_: F,
) -> Result<Self::Output, Self::Error>
where
F: FnMut(&State<DB>),
fn execute_one(
&mut self,
_block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
{
Err(BlockExecutionError::msg(UNAVAILABLE_FOR_NOOP))
}
fn execute_with_state_hook<F>(
self,
_: Self::Input<'_>,
_: F,
) -> Result<Self::Output, Self::Error>
fn execute_one_with_state_hook<F>(
&mut self,
_block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
_state_hook: F,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
where
F: OnStateHook,
F: OnStateHook + 'static,
{
Err(BlockExecutionError::msg(UNAVAILABLE_FOR_NOOP))
}
fn into_state(self) -> revm::db::State<DB> {
unreachable!()
}
fn size_hint(&self) -> usize {
0
}
}
impl<DB, P: NodePrimitives> BatchExecutor<DB> for NoopBlockExecutorProvider<P> {

View File

@ -11,8 +11,8 @@ use crate::{
use alloy_eips::eip7685::Requests;
use parking_lot::Mutex;
use reth_execution_errors::BlockExecutionError;
use reth_execution_types::ExecutionOutcome;
use reth_primitives::{EthPrimitives, NodePrimitives, Receipt, RecoveredBlock};
use reth_execution_types::{BlockExecutionResult, ExecutionOutcome};
use reth_primitives::{EthPrimitives, NodePrimitives, RecoveredBlock};
use revm::State;
use std::sync::Arc;
@ -51,12 +51,43 @@ impl BlockExecutorProvider for MockExecutorProvider {
}
}
impl<DB> Executor<DB> for MockExecutorProvider {
type Input<'a> = &'a RecoveredBlock<reth_primitives::Block>;
type Output = BlockExecutionOutput<Receipt>;
impl<DB: Database> Executor<DB> for MockExecutorProvider {
type Primitives = EthPrimitives;
type Error = BlockExecutionError;
fn execute(self, _: Self::Input<'_>) -> Result<Self::Output, Self::Error> {
fn execute_one(
&mut self,
_block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
{
let ExecutionOutcome { bundle: _, receipts, requests, first_block: _ } =
self.exec_results.lock().pop().unwrap();
Ok(BlockExecutionResult {
receipts: receipts.into_iter().flatten().collect(),
requests: requests.into_iter().fold(Requests::default(), |mut reqs, req| {
reqs.extend(req);
reqs
}),
gas_used: 0,
})
}
fn execute_one_with_state_hook<F>(
&mut self,
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
_state_hook: F,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
where
F: OnStateHook + 'static,
{
<Self as Executor<DB>>::execute_one(self, block)
}
fn execute(
self,
_block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> Result<BlockExecutionOutput<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
{
let ExecutionOutcome { bundle, receipts, requests, first_block: _ } =
self.exec_results.lock().pop().unwrap();
Ok(BlockExecutionOutput {
@ -72,24 +103,32 @@ impl<DB> Executor<DB> for MockExecutorProvider {
fn execute_with_state_closure<F>(
self,
input: Self::Input<'_>,
_: F,
) -> Result<Self::Output, Self::Error>
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
_f: F,
) -> Result<BlockExecutionOutput<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
where
F: FnMut(&State<DB>),
F: FnMut(&revm::db::State<DB>),
{
<Self as Executor<DB>>::execute(self, input)
<Self as Executor<DB>>::execute(self, block)
}
fn execute_with_state_hook<F>(
self,
input: Self::Input<'_>,
_: F,
) -> Result<Self::Output, Self::Error>
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
_state_hook: F,
) -> Result<BlockExecutionOutput<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
where
F: OnStateHook,
F: OnStateHook + 'static,
{
<Self as Executor<DB>>::execute(self, input)
<Self as Executor<DB>>::execute(self, block)
}
fn into_state(self) -> revm::db::State<DB> {
unreachable!()
}
fn size_hint(&self) -> usize {
0
}
}