mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: add Primitives AT to BlockExecutorProvider (#12994)
This commit is contained in:
@ -8,9 +8,6 @@ use crate::{
|
||||
};
|
||||
use alloc::boxed::Box;
|
||||
use alloy_primitives::BlockNumber;
|
||||
use reth_execution_errors::BlockExecutionError;
|
||||
use reth_execution_types::{BlockExecutionInput, BlockExecutionOutput, ExecutionOutcome};
|
||||
use reth_primitives::{BlockWithSenders, Receipt};
|
||||
use reth_prune_types::PruneModes;
|
||||
use reth_storage_errors::provider::ProviderError;
|
||||
use revm_primitives::db::Database;
|
||||
@ -22,8 +19,10 @@ use revm::State;
|
||||
impl<A, B> BlockExecutorProvider for Either<A, B>
|
||||
where
|
||||
A: BlockExecutorProvider,
|
||||
B: BlockExecutorProvider,
|
||||
B: BlockExecutorProvider<Primitives = A::Primitives>,
|
||||
{
|
||||
type Primitives = A::Primitives;
|
||||
|
||||
type Executor<DB: Database<Error: Into<ProviderError> + Display>> =
|
||||
Either<A::Executor<DB>, B::Executor<DB>>;
|
||||
|
||||
@ -53,23 +52,13 @@ where
|
||||
|
||||
impl<A, B, DB> Executor<DB> for Either<A, B>
|
||||
where
|
||||
A: for<'a> Executor<
|
||||
DB,
|
||||
Input<'a> = BlockExecutionInput<'a, BlockWithSenders>,
|
||||
Output = BlockExecutionOutput<Receipt>,
|
||||
Error = BlockExecutionError,
|
||||
>,
|
||||
B: for<'a> Executor<
|
||||
DB,
|
||||
Input<'a> = BlockExecutionInput<'a, BlockWithSenders>,
|
||||
Output = BlockExecutionOutput<Receipt>,
|
||||
Error = BlockExecutionError,
|
||||
>,
|
||||
A: Executor<DB>,
|
||||
B: for<'a> Executor<DB, Input<'a> = A::Input<'a>, Output = A::Output, Error = A::Error>,
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
{
|
||||
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
|
||||
type Output = BlockExecutionOutput<Receipt>;
|
||||
type Error = BlockExecutionError;
|
||||
type Input<'a> = A::Input<'a>;
|
||||
type Output = A::Output;
|
||||
type Error = A::Error;
|
||||
|
||||
fn init(&mut self, tx_env_overrides: Box<dyn crate::TxEnvOverrides>) {
|
||||
match self {
|
||||
@ -116,23 +105,13 @@ where
|
||||
|
||||
impl<A, B, DB> BatchExecutor<DB> for Either<A, B>
|
||||
where
|
||||
A: for<'a> BatchExecutor<
|
||||
DB,
|
||||
Input<'a> = BlockExecutionInput<'a, BlockWithSenders>,
|
||||
Output = ExecutionOutcome,
|
||||
Error = BlockExecutionError,
|
||||
>,
|
||||
B: for<'a> BatchExecutor<
|
||||
DB,
|
||||
Input<'a> = BlockExecutionInput<'a, BlockWithSenders>,
|
||||
Output = ExecutionOutcome,
|
||||
Error = BlockExecutionError,
|
||||
>,
|
||||
A: BatchExecutor<DB>,
|
||||
B: for<'a> BatchExecutor<DB, Input<'a> = A::Input<'a>, Output = A::Output, Error = A::Error>,
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
{
|
||||
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
|
||||
type Output = ExecutionOutcome;
|
||||
type Error = BlockExecutionError;
|
||||
type Input<'a> = A::Input<'a>;
|
||||
type Output = A::Output;
|
||||
type Error = A::Error;
|
||||
|
||||
fn execute_and_verify_one(&mut self, input: Self::Input<'_>) -> Result<(), Self::Error> {
|
||||
match self {
|
||||
|
||||
@ -1,19 +1,21 @@
|
||||
//! Traits for execution.
|
||||
|
||||
use alloy_consensus::BlockHeader;
|
||||
// Re-export execution types
|
||||
pub use reth_execution_errors::{
|
||||
BlockExecutionError, BlockValidationError, InternalBlockExecutionError,
|
||||
};
|
||||
pub use reth_execution_types::{BlockExecutionInput, BlockExecutionOutput, ExecutionOutcome};
|
||||
use reth_primitives_traits::Block as _;
|
||||
pub use reth_storage_errors::provider::ProviderError;
|
||||
|
||||
use crate::{system_calls::OnStateHook, TxEnvOverrides};
|
||||
use alloc::{boxed::Box, vec::Vec};
|
||||
use alloy_eips::eip7685::Requests;
|
||||
use alloy_primitives::BlockNumber;
|
||||
use core::{fmt::Display, marker::PhantomData};
|
||||
use core::fmt::Display;
|
||||
use reth_consensus::ConsensusError;
|
||||
use reth_primitives::{BlockWithSenders, Receipt};
|
||||
use reth_primitives::{BlockWithSenders, NodePrimitives, Receipt};
|
||||
use reth_prune_types::PruneModes;
|
||||
use reth_revm::batch::BlockBatchRecord;
|
||||
use revm::{
|
||||
@ -130,6 +132,9 @@ pub trait BatchExecutor<DB> {
|
||||
|
||||
/// A type that can create a new executor for block execution.
|
||||
pub trait BlockExecutorProvider: Send + Sync + Clone + Unpin + 'static {
|
||||
/// Receipt type.
|
||||
type Primitives: NodePrimitives;
|
||||
|
||||
/// An executor that can execute a single block given a database.
|
||||
///
|
||||
/// # Verification
|
||||
@ -143,16 +148,22 @@ pub trait BlockExecutorProvider: Send + Sync + Clone + Unpin + 'static {
|
||||
/// the returned state.
|
||||
type Executor<DB: Database<Error: Into<ProviderError> + Display>>: for<'a> Executor<
|
||||
DB,
|
||||
Input<'a> = BlockExecutionInput<'a, BlockWithSenders>,
|
||||
Output = BlockExecutionOutput<Receipt>,
|
||||
Input<'a> = BlockExecutionInput<
|
||||
'a,
|
||||
BlockWithSenders<<Self::Primitives as NodePrimitives>::Block>,
|
||||
>,
|
||||
Output = BlockExecutionOutput<<Self::Primitives as NodePrimitives>::Receipt>,
|
||||
Error = BlockExecutionError,
|
||||
>;
|
||||
|
||||
/// An executor that can execute a batch of blocks given a database.
|
||||
type BatchExecutor<DB: Database<Error: Into<ProviderError> + Display>>: for<'a> BatchExecutor<
|
||||
DB,
|
||||
Input<'a> = BlockExecutionInput<'a, BlockWithSenders>,
|
||||
Output = ExecutionOutcome,
|
||||
Input<'a> = BlockExecutionInput<
|
||||
'a,
|
||||
BlockWithSenders<<Self::Primitives as NodePrimitives>::Block>,
|
||||
>,
|
||||
Output = ExecutionOutcome<<Self::Primitives as NodePrimitives>::Receipt>,
|
||||
Error = BlockExecutionError,
|
||||
>;
|
||||
|
||||
@ -174,18 +185,21 @@ pub trait BlockExecutorProvider: Send + Sync + Clone + Unpin + 'static {
|
||||
|
||||
/// Helper type for the output of executing a block.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct ExecuteOutput {
|
||||
pub struct ExecuteOutput<R = Receipt> {
|
||||
/// Receipts obtained after executing a block.
|
||||
pub receipts: Vec<Receipt>,
|
||||
pub receipts: Vec<R>,
|
||||
/// Cumulative gas used in the block execution.
|
||||
pub gas_used: u64,
|
||||
}
|
||||
|
||||
/// Defines the strategy for executing a single block.
|
||||
pub trait BlockExecutionStrategy<DB>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
pub trait BlockExecutionStrategy {
|
||||
/// Database this strategy operates on.
|
||||
type DB: Database;
|
||||
|
||||
/// Primitive types used by the strategy.
|
||||
type Primitives: NodePrimitives;
|
||||
|
||||
/// The error type returned by this strategy's methods.
|
||||
type Error: From<ProviderError> + core::error::Error;
|
||||
|
||||
@ -195,30 +209,30 @@ where
|
||||
/// Applies any necessary changes before executing the block's transactions.
|
||||
fn apply_pre_execution_changes(
|
||||
&mut self,
|
||||
block: &BlockWithSenders,
|
||||
block: &BlockWithSenders<<Self::Primitives as NodePrimitives>::Block>,
|
||||
total_difficulty: U256,
|
||||
) -> Result<(), Self::Error>;
|
||||
|
||||
/// Executes all transactions in the block.
|
||||
fn execute_transactions(
|
||||
&mut self,
|
||||
block: &BlockWithSenders,
|
||||
block: &BlockWithSenders<<Self::Primitives as NodePrimitives>::Block>,
|
||||
total_difficulty: U256,
|
||||
) -> Result<ExecuteOutput, Self::Error>;
|
||||
) -> Result<ExecuteOutput<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>;
|
||||
|
||||
/// Applies any necessary changes after executing the block's transactions.
|
||||
fn apply_post_execution_changes(
|
||||
&mut self,
|
||||
block: &BlockWithSenders,
|
||||
block: &BlockWithSenders<<Self::Primitives as NodePrimitives>::Block>,
|
||||
total_difficulty: U256,
|
||||
receipts: &[Receipt],
|
||||
receipts: &[<Self::Primitives as NodePrimitives>::Receipt],
|
||||
) -> Result<Requests, Self::Error>;
|
||||
|
||||
/// Returns a reference to the current state.
|
||||
fn state_ref(&self) -> &State<DB>;
|
||||
fn state_ref(&self) -> &State<Self::DB>;
|
||||
|
||||
/// Returns a mutable reference to the current state.
|
||||
fn state_mut(&mut self) -> &mut State<DB>;
|
||||
fn state_mut(&mut self) -> &mut 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>>) {}
|
||||
@ -232,8 +246,8 @@ where
|
||||
/// Validate a block with regard to execution results.
|
||||
fn validate_block_post_execution(
|
||||
&self,
|
||||
_block: &BlockWithSenders,
|
||||
_receipts: &[Receipt],
|
||||
_block: &BlockWithSenders<<Self::Primitives as NodePrimitives>::Block>,
|
||||
_receipts: &[<Self::Primitives as NodePrimitives>::Receipt],
|
||||
_requests: &Requests,
|
||||
) -> Result<(), ConsensusError> {
|
||||
Ok(())
|
||||
@ -242,9 +256,13 @@ where
|
||||
|
||||
/// A strategy factory that can create block execution strategies.
|
||||
pub trait BlockExecutionStrategyFactory: Send + Sync + Clone + Unpin + 'static {
|
||||
/// Primitive types used by the strategy.
|
||||
type Primitives: NodePrimitives;
|
||||
|
||||
/// Associated strategy type.
|
||||
type Strategy<DB: Database<Error: Into<ProviderError> + Display>>: BlockExecutionStrategy<
|
||||
DB,
|
||||
DB = DB,
|
||||
Primitives = Self::Primitives,
|
||||
Error = BlockExecutionError,
|
||||
>;
|
||||
|
||||
@ -280,11 +298,13 @@ impl<F> BlockExecutorProvider for BasicBlockExecutorProvider<F>
|
||||
where
|
||||
F: BlockExecutionStrategyFactory,
|
||||
{
|
||||
type Primitives = F::Primitives;
|
||||
|
||||
type Executor<DB: Database<Error: Into<ProviderError> + Display>> =
|
||||
BasicBlockExecutor<F::Strategy<DB>, DB>;
|
||||
BasicBlockExecutor<F::Strategy<DB>>;
|
||||
|
||||
type BatchExecutor<DB: Database<Error: Into<ProviderError> + Display>> =
|
||||
BasicBatchExecutor<F::Strategy<DB>, DB>;
|
||||
BasicBatchExecutor<F::Strategy<DB>>;
|
||||
|
||||
fn executor<DB>(&self, db: DB) -> Self::Executor<DB>
|
||||
where
|
||||
@ -307,34 +327,26 @@ where
|
||||
/// A generic block executor that uses a [`BlockExecutionStrategy`] to
|
||||
/// execute blocks.
|
||||
#[allow(missing_debug_implementations, dead_code)]
|
||||
pub struct BasicBlockExecutor<S, DB>
|
||||
where
|
||||
S: BlockExecutionStrategy<DB>,
|
||||
DB: Database,
|
||||
{
|
||||
pub struct BasicBlockExecutor<S> {
|
||||
/// Block execution strategy.
|
||||
pub(crate) strategy: S,
|
||||
_phantom: PhantomData<DB>,
|
||||
}
|
||||
|
||||
impl<S, DB> BasicBlockExecutor<S, DB>
|
||||
where
|
||||
S: BlockExecutionStrategy<DB>,
|
||||
DB: Database,
|
||||
{
|
||||
impl<S> BasicBlockExecutor<S> {
|
||||
/// Creates a new `BasicBlockExecutor` with the given strategy.
|
||||
pub const fn new(strategy: S) -> Self {
|
||||
Self { strategy, _phantom: PhantomData }
|
||||
Self { strategy }
|
||||
}
|
||||
}
|
||||
|
||||
impl<S, DB> Executor<DB> for BasicBlockExecutor<S, DB>
|
||||
impl<S, DB> Executor<DB> for BasicBlockExecutor<S>
|
||||
where
|
||||
S: BlockExecutionStrategy<DB>,
|
||||
S: BlockExecutionStrategy<DB = DB>,
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
{
|
||||
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
|
||||
type Output = BlockExecutionOutput<Receipt>;
|
||||
type Input<'a> =
|
||||
BlockExecutionInput<'a, BlockWithSenders<<S::Primitives as NodePrimitives>::Block>>;
|
||||
type Output = BlockExecutionOutput<<S::Primitives as NodePrimitives>::Receipt>;
|
||||
type Error = S::Error;
|
||||
|
||||
fn init(&mut self, env_overrides: Box<dyn TxEnvOverrides>) {
|
||||
@ -404,43 +416,44 @@ where
|
||||
/// A generic batch executor that uses a [`BlockExecutionStrategy`] to
|
||||
/// execute batches.
|
||||
#[allow(missing_debug_implementations)]
|
||||
pub struct BasicBatchExecutor<S, DB>
|
||||
pub struct BasicBatchExecutor<S>
|
||||
where
|
||||
S: BlockExecutionStrategy<DB>,
|
||||
DB: Database,
|
||||
S: BlockExecutionStrategy,
|
||||
{
|
||||
/// Batch execution strategy.
|
||||
pub(crate) strategy: S,
|
||||
/// Keeps track of batch execution receipts and requests.
|
||||
pub(crate) batch_record: BlockBatchRecord,
|
||||
_phantom: PhantomData<DB>,
|
||||
pub(crate) batch_record: BlockBatchRecord<<S::Primitives as NodePrimitives>::Receipt>,
|
||||
}
|
||||
|
||||
impl<S, DB> BasicBatchExecutor<S, DB>
|
||||
impl<S> BasicBatchExecutor<S>
|
||||
where
|
||||
S: BlockExecutionStrategy<DB>,
|
||||
DB: Database,
|
||||
S: BlockExecutionStrategy,
|
||||
{
|
||||
/// Creates a new `BasicBatchExecutor` with the given strategy.
|
||||
pub const fn new(strategy: S, batch_record: BlockBatchRecord) -> Self {
|
||||
Self { strategy, batch_record, _phantom: PhantomData }
|
||||
pub const fn new(
|
||||
strategy: S,
|
||||
batch_record: BlockBatchRecord<<S::Primitives as NodePrimitives>::Receipt>,
|
||||
) -> Self {
|
||||
Self { strategy, batch_record }
|
||||
}
|
||||
}
|
||||
|
||||
impl<S, DB> BatchExecutor<DB> for BasicBatchExecutor<S, DB>
|
||||
impl<S, DB> BatchExecutor<DB> for BasicBatchExecutor<S>
|
||||
where
|
||||
S: BlockExecutionStrategy<DB, Error = BlockExecutionError>,
|
||||
S: BlockExecutionStrategy<DB = DB, Error = BlockExecutionError>,
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
{
|
||||
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
|
||||
type Output = ExecutionOutcome;
|
||||
type Input<'a> =
|
||||
BlockExecutionInput<'a, BlockWithSenders<<S::Primitives as NodePrimitives>::Block>>;
|
||||
type Output = ExecutionOutcome<<S::Primitives as NodePrimitives>::Receipt>;
|
||||
type Error = BlockExecutionError;
|
||||
|
||||
fn execute_and_verify_one(&mut self, input: Self::Input<'_>) -> Result<(), Self::Error> {
|
||||
let BlockExecutionInput { block, total_difficulty } = input;
|
||||
|
||||
if self.batch_record.first_block().is_none() {
|
||||
self.batch_record.set_first_block(block.number);
|
||||
self.batch_record.set_first_block(block.header().number());
|
||||
}
|
||||
|
||||
self.strategy.apply_pre_execution_changes(block, total_difficulty)?;
|
||||
@ -452,7 +465,7 @@ where
|
||||
self.strategy.validate_block_post_execution(block, &receipts, &requests)?;
|
||||
|
||||
// prepare the state according to the prune mode
|
||||
let retention = self.batch_record.bundle_retention(block.number);
|
||||
let retention = self.batch_record.bundle_retention(block.header().number());
|
||||
self.strategy.state_mut().merge_transitions(retention);
|
||||
|
||||
// store receipts in the set
|
||||
@ -490,7 +503,9 @@ where
|
||||
mod tests {
|
||||
use super::*;
|
||||
use alloy_primitives::U256;
|
||||
use core::marker::PhantomData;
|
||||
use reth_chainspec::{ChainSpec, MAINNET};
|
||||
use reth_primitives::EthPrimitives;
|
||||
use revm::db::{CacheDB, EmptyDBTyped};
|
||||
use revm_primitives::{bytes, TxEnv};
|
||||
use std::sync::Arc;
|
||||
@ -499,6 +514,7 @@ mod tests {
|
||||
struct TestExecutorProvider;
|
||||
|
||||
impl BlockExecutorProvider for TestExecutorProvider {
|
||||
type Primitives = EthPrimitives;
|
||||
type Executor<DB: Database<Error: Into<ProviderError> + Display>> = TestExecutor<DB>;
|
||||
type BatchExecutor<DB: Database<Error: Into<ProviderError> + Display>> = TestExecutor<DB>;
|
||||
|
||||
@ -596,6 +612,7 @@ mod tests {
|
||||
}
|
||||
|
||||
impl BlockExecutionStrategyFactory for TestExecutorStrategyFactory {
|
||||
type Primitives = EthPrimitives;
|
||||
type Strategy<DB: Database<Error: Into<ProviderError> + Display>> =
|
||||
TestExecutorStrategy<DB, TestEvmConfig>;
|
||||
|
||||
@ -622,10 +639,12 @@ mod tests {
|
||||
}
|
||||
}
|
||||
|
||||
impl<DB> BlockExecutionStrategy<DB> for TestExecutorStrategy<DB, TestEvmConfig>
|
||||
impl<DB> BlockExecutionStrategy for TestExecutorStrategy<DB, TestEvmConfig>
|
||||
where
|
||||
DB: Database,
|
||||
{
|
||||
type DB = DB;
|
||||
type Primitives = EthPrimitives;
|
||||
type Error = BlockExecutionError;
|
||||
|
||||
fn apply_pre_execution_changes(
|
||||
|
||||
@ -3,6 +3,7 @@
|
||||
//! 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 alloy_consensus::BlockHeader;
|
||||
use metrics::{Counter, Gauge, Histogram};
|
||||
use reth_execution_types::{BlockExecutionInput, BlockExecutionOutput};
|
||||
use reth_metrics::Metrics;
|
||||
@ -69,9 +70,10 @@ pub struct ExecutorMetrics {
|
||||
}
|
||||
|
||||
impl ExecutorMetrics {
|
||||
fn metered<F, R>(&self, block: &BlockWithSenders, f: F) -> R
|
||||
fn metered<F, R, B>(&self, block: &BlockWithSenders<B>, f: F) -> R
|
||||
where
|
||||
F: FnOnce() -> R,
|
||||
B: reth_primitives_traits::Block,
|
||||
{
|
||||
// Execute the block and record the elapsed time.
|
||||
let execute_start = Instant::now();
|
||||
@ -79,8 +81,8 @@ impl ExecutorMetrics {
|
||||
let execution_duration = execute_start.elapsed().as_secs_f64();
|
||||
|
||||
// Update gas metrics.
|
||||
self.gas_processed_total.increment(block.gas_used);
|
||||
self.gas_per_second.set(block.gas_used as f64 / execution_duration);
|
||||
self.gas_processed_total.increment(block.header().gas_used());
|
||||
self.gas_per_second.set(block.header().gas_used() as f64 / execution_duration);
|
||||
self.execution_histogram.record(execution_duration);
|
||||
self.execution_duration.set(execution_duration);
|
||||
|
||||
@ -94,19 +96,20 @@ 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>(
|
||||
pub fn execute_metered<'a, E, DB, O, Error, B>(
|
||||
&self,
|
||||
executor: E,
|
||||
input: BlockExecutionInput<'a, BlockWithSenders>,
|
||||
input: BlockExecutionInput<'a, BlockWithSenders<B>>,
|
||||
state_hook: Box<dyn OnStateHook>,
|
||||
) -> Result<BlockExecutionOutput<O>, Error>
|
||||
where
|
||||
E: Executor<
|
||||
DB,
|
||||
Input<'a> = BlockExecutionInput<'a, BlockWithSenders>,
|
||||
Input<'a> = BlockExecutionInput<'a, BlockWithSenders<B>>,
|
||||
Output = BlockExecutionOutput<O>,
|
||||
Error = Error,
|
||||
>,
|
||||
B: reth_primitives_traits::Block,
|
||||
{
|
||||
// clone here is cheap, all the metrics are Option<Arc<_>>. additionally
|
||||
// they are gloally registered so that the data recorded in the hook will
|
||||
@ -133,9 +136,14 @@ impl ExecutorMetrics {
|
||||
}
|
||||
|
||||
/// Execute the given block and update metrics for the execution.
|
||||
pub fn metered_one<F, R>(&self, input: BlockExecutionInput<'_, BlockWithSenders>, f: F) -> R
|
||||
pub fn metered_one<F, R, B>(
|
||||
&self,
|
||||
input: BlockExecutionInput<'_, BlockWithSenders<B>>,
|
||||
f: F,
|
||||
) -> R
|
||||
where
|
||||
F: FnOnce(BlockExecutionInput<'_, BlockWithSenders>) -> R,
|
||||
F: FnOnce(BlockExecutionInput<'_, BlockWithSenders<B>>) -> R,
|
||||
B: reth_primitives_traits::Block,
|
||||
{
|
||||
self.metered(input.block, || f(input))
|
||||
}
|
||||
|
||||
@ -4,7 +4,7 @@ use alloy_primitives::BlockNumber;
|
||||
use core::fmt::Display;
|
||||
use reth_execution_errors::BlockExecutionError;
|
||||
use reth_execution_types::{BlockExecutionInput, BlockExecutionOutput, ExecutionOutcome};
|
||||
use reth_primitives::{BlockWithSenders, Receipt};
|
||||
use reth_primitives::{BlockWithSenders, EthPrimitives, Receipt};
|
||||
use reth_prune_types::PruneModes;
|
||||
use reth_storage_errors::provider::ProviderError;
|
||||
use revm::State;
|
||||
@ -23,6 +23,8 @@ const UNAVAILABLE_FOR_NOOP: &str = "execution unavailable for noop";
|
||||
pub struct NoopBlockExecutorProvider;
|
||||
|
||||
impl BlockExecutorProvider for NoopBlockExecutorProvider {
|
||||
type Primitives = EthPrimitives;
|
||||
|
||||
type Executor<DB: Database<Error: Into<ProviderError> + Display>> = Self;
|
||||
|
||||
type BatchExecutor<DB: Database<Error: Into<ProviderError> + Display>> = Self;
|
||||
|
||||
@ -12,7 +12,7 @@ use alloy_primitives::BlockNumber;
|
||||
use parking_lot::Mutex;
|
||||
use reth_execution_errors::BlockExecutionError;
|
||||
use reth_execution_types::ExecutionOutcome;
|
||||
use reth_primitives::{BlockWithSenders, Receipt, Receipts};
|
||||
use reth_primitives::{BlockWithSenders, EthPrimitives, NodePrimitives, Receipt, Receipts};
|
||||
use reth_prune_types::PruneModes;
|
||||
use reth_storage_errors::provider::ProviderError;
|
||||
use revm::State;
|
||||
@ -33,6 +33,8 @@ impl MockExecutorProvider {
|
||||
}
|
||||
|
||||
impl BlockExecutorProvider for MockExecutorProvider {
|
||||
type Primitives = EthPrimitives;
|
||||
|
||||
type Executor<DB: Database<Error: Into<ProviderError> + Display>> = Self;
|
||||
|
||||
type BatchExecutor<DB: Database<Error: Into<ProviderError> + Display>> = Self;
|
||||
@ -116,15 +118,14 @@ impl<DB> BatchExecutor<DB> for MockExecutorProvider {
|
||||
}
|
||||
}
|
||||
|
||||
impl<S, DB> BasicBlockExecutor<S, DB>
|
||||
impl<S> BasicBlockExecutor<S>
|
||||
where
|
||||
S: BlockExecutionStrategy<DB>,
|
||||
DB: Database,
|
||||
S: BlockExecutionStrategy,
|
||||
{
|
||||
/// Provides safe read access to the state
|
||||
pub fn with_state<F, R>(&self, f: F) -> R
|
||||
where
|
||||
F: FnOnce(&State<DB>) -> R,
|
||||
F: FnOnce(&State<S::DB>) -> R,
|
||||
{
|
||||
f(self.strategy.state_ref())
|
||||
}
|
||||
@ -132,21 +133,20 @@ where
|
||||
/// Provides safe write access to the state
|
||||
pub fn with_state_mut<F, R>(&mut self, f: F) -> R
|
||||
where
|
||||
F: FnOnce(&mut State<DB>) -> R,
|
||||
F: FnOnce(&mut State<S::DB>) -> R,
|
||||
{
|
||||
f(self.strategy.state_mut())
|
||||
}
|
||||
}
|
||||
|
||||
impl<S, DB> BasicBatchExecutor<S, DB>
|
||||
impl<S> BasicBatchExecutor<S>
|
||||
where
|
||||
S: BlockExecutionStrategy<DB>,
|
||||
DB: Database,
|
||||
S: BlockExecutionStrategy,
|
||||
{
|
||||
/// Provides safe read access to the state
|
||||
pub fn with_state<F, R>(&self, f: F) -> R
|
||||
where
|
||||
F: FnOnce(&State<DB>) -> R,
|
||||
F: FnOnce(&State<S::DB>) -> R,
|
||||
{
|
||||
f(self.strategy.state_ref())
|
||||
}
|
||||
@ -154,13 +154,13 @@ where
|
||||
/// Provides safe write access to the state
|
||||
pub fn with_state_mut<F, R>(&mut self, f: F) -> R
|
||||
where
|
||||
F: FnOnce(&mut State<DB>) -> R,
|
||||
F: FnOnce(&mut State<S::DB>) -> R,
|
||||
{
|
||||
f(self.strategy.state_mut())
|
||||
}
|
||||
|
||||
/// Accessor for batch executor receipts.
|
||||
pub const fn receipts(&self) -> &Receipts {
|
||||
pub const fn receipts(&self) -> &Receipts<<S::Primitives as NodePrimitives>::Receipt> {
|
||||
self.batch_record.receipts()
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user