mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: abstract over Evm::Error (#14085)
Co-authored-by: Matthias Seitz <matthias.seitz@outlook.de>
This commit is contained in:
@ -19,7 +19,6 @@ reth-prune-types.workspace = true
|
||||
alloy-primitives.workspace = true
|
||||
alloy-rlp.workspace = true
|
||||
alloy-eips.workspace = true
|
||||
revm-primitives.workspace = true
|
||||
nybbles.workspace = true
|
||||
|
||||
thiserror.workspace = true
|
||||
@ -30,7 +29,6 @@ std = [
|
||||
"reth-consensus/std",
|
||||
"alloy-eips/std",
|
||||
"alloy-primitives/std",
|
||||
"revm-primitives/std",
|
||||
"alloy-rlp/std",
|
||||
"thiserror/std",
|
||||
"nybbles/std",
|
||||
|
||||
@ -20,14 +20,13 @@ use alloy_primitives::B256;
|
||||
use reth_consensus::ConsensusError;
|
||||
use reth_prune_types::PruneSegmentError;
|
||||
use reth_storage_errors::provider::ProviderError;
|
||||
use revm_primitives::EVMError;
|
||||
use thiserror::Error;
|
||||
|
||||
pub mod trie;
|
||||
pub use trie::*;
|
||||
|
||||
/// Transaction validation errors
|
||||
#[derive(Error, Clone, Debug)]
|
||||
#[derive(Error, Debug)]
|
||||
pub enum BlockValidationError {
|
||||
/// EVM error with transaction hash and message
|
||||
#[error("EVM reported invalid transaction ({hash}): {error}")]
|
||||
@ -35,7 +34,7 @@ pub enum BlockValidationError {
|
||||
/// The hash of the transaction
|
||||
hash: B256,
|
||||
/// The EVM error.
|
||||
error: Box<EVMError<ProviderError>>,
|
||||
error: Box<dyn core::error::Error + Send + Sync>,
|
||||
},
|
||||
/// Error when recovering the sender for a transaction
|
||||
#[error("failed to recover sender for transaction")]
|
||||
|
||||
@ -1,13 +1,10 @@
|
||||
//! Helper type that represents one of two possible executor types
|
||||
|
||||
use core::fmt::Display;
|
||||
|
||||
use crate::{
|
||||
execute::{BatchExecutor, BlockExecutorProvider, Executor},
|
||||
system_calls::OnStateHook,
|
||||
Database,
|
||||
};
|
||||
use reth_storage_errors::provider::ProviderError;
|
||||
use revm_primitives::db::Database;
|
||||
|
||||
// re-export Either
|
||||
pub use futures_util::future::Either;
|
||||
@ -20,15 +17,13 @@ where
|
||||
{
|
||||
type Primitives = A::Primitives;
|
||||
|
||||
type Executor<DB: Database<Error: Into<ProviderError> + Display>> =
|
||||
Either<A::Executor<DB>, B::Executor<DB>>;
|
||||
type Executor<DB: Database> = Either<A::Executor<DB>, B::Executor<DB>>;
|
||||
|
||||
type BatchExecutor<DB: Database<Error: Into<ProviderError> + Display>> =
|
||||
Either<A::BatchExecutor<DB>, B::BatchExecutor<DB>>;
|
||||
type BatchExecutor<DB: Database> = Either<A::BatchExecutor<DB>, B::BatchExecutor<DB>>;
|
||||
|
||||
fn executor<DB>(&self, db: DB) -> Self::Executor<DB>
|
||||
where
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
DB: Database,
|
||||
{
|
||||
match self {
|
||||
Self::Left(a) => Either::Left(a.executor(db)),
|
||||
@ -38,7 +33,7 @@ where
|
||||
|
||||
fn batch_executor<DB>(&self, db: DB) -> Self::BatchExecutor<DB>
|
||||
where
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
DB: Database,
|
||||
{
|
||||
match self {
|
||||
Self::Left(a) => Either::Left(a.batch_executor(db)),
|
||||
@ -51,7 +46,7 @@ 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>,
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
DB: Database,
|
||||
{
|
||||
type Input<'a> = A::Input<'a>;
|
||||
type Output = A::Output;
|
||||
@ -97,7 +92,7 @@ impl<A, B, DB> BatchExecutor<DB> for Either<A, B>
|
||||
where
|
||||
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>,
|
||||
DB: Database,
|
||||
{
|
||||
type Input<'a> = A::Input<'a>;
|
||||
type Output = A::Output;
|
||||
|
||||
49
crates/evm/src/error.rs
Normal file
49
crates/evm/src/error.rs
Normal file
@ -0,0 +1,49 @@
|
||||
use revm_primitives::{EVMError, InvalidTransaction};
|
||||
|
||||
/// Abstraction over transaction validation error.
|
||||
pub trait InvalidTxError: core::error::Error + Send + Sync + 'static {
|
||||
/// Returns whether the error cause by transaction having a nonce lower than expected.
|
||||
fn is_nonce_too_low(&self) -> bool;
|
||||
}
|
||||
|
||||
impl InvalidTxError for InvalidTransaction {
|
||||
fn is_nonce_too_low(&self) -> bool {
|
||||
matches!(self, Self::NonceTooLow { .. })
|
||||
}
|
||||
}
|
||||
|
||||
/// Abstraction over errors that can occur during EVM execution.
|
||||
///
|
||||
/// It's assumed that errors can occur either because of an invalid transaction, meaning that other
|
||||
/// transaction might still result in successful execution, or because of a general EVM
|
||||
/// misconfiguration.
|
||||
///
|
||||
/// If caller occurs a error different from [`EvmError::InvalidTransaction`], it should most likely
|
||||
/// be treated as fatal error flagging some EVM misconfiguration.
|
||||
pub trait EvmError: core::error::Error + Send + Sync + 'static {
|
||||
/// Errors which might occur as a result of an invalid transaction. i.e unrelated to general EVM
|
||||
/// configuration.
|
||||
type InvalidTransaction: InvalidTxError;
|
||||
|
||||
/// Returns the [`EvmError::InvalidTransaction`] if the error is an invalid transaction error.
|
||||
fn as_invalid_tx_err(&self) -> Option<&Self::InvalidTransaction>;
|
||||
|
||||
/// Returns `true` if the error is an invalid transaction error.
|
||||
fn is_invalid_tx_err(&self) -> bool {
|
||||
self.as_invalid_tx_err().is_some()
|
||||
}
|
||||
}
|
||||
|
||||
impl<DBError> EvmError for EVMError<DBError>
|
||||
where
|
||||
DBError: core::error::Error + Send + Sync + 'static,
|
||||
{
|
||||
type InvalidTransaction = InvalidTransaction;
|
||||
|
||||
fn as_invalid_tx_err(&self) -> Option<&Self::InvalidTransaction> {
|
||||
match self {
|
||||
Self::Transaction(err) => Some(err),
|
||||
_ => None,
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -8,14 +8,13 @@ pub use reth_execution_errors::{
|
||||
pub use reth_execution_types::{BlockExecutionOutput, ExecutionOutcome};
|
||||
pub use reth_storage_errors::provider::ProviderError;
|
||||
|
||||
use crate::system_calls::OnStateHook;
|
||||
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 core::fmt::Display;
|
||||
use reth_consensus::ConsensusError;
|
||||
use reth_primitives::{NodePrimitives, Receipt, RecoveredBlock};
|
||||
use reth_revm::batch::BlockBatchRecord;
|
||||
@ -23,7 +22,7 @@ use revm::{
|
||||
db::{states::bundle_state::BundleRetention, BundleState},
|
||||
State,
|
||||
};
|
||||
use revm_primitives::{db::Database, Account, AccountStatus, EvmState};
|
||||
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).
|
||||
@ -134,7 +133,7 @@ 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<Error: Into<ProviderError> + Display>>: for<'a> Executor<
|
||||
type Executor<DB: Database>: for<'a> Executor<
|
||||
DB,
|
||||
Input<'a> = &'a RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
Output = BlockExecutionOutput<<Self::Primitives as NodePrimitives>::Receipt>,
|
||||
@ -142,7 +141,7 @@ pub trait BlockExecutorProvider: Send + Sync + Clone + Unpin + 'static {
|
||||
>;
|
||||
|
||||
/// An executor that can execute a batch of blocks given a database.
|
||||
type BatchExecutor<DB: Database<Error: Into<ProviderError> + Display>>: for<'a> BatchExecutor<
|
||||
type BatchExecutor<DB: Database>: for<'a> BatchExecutor<
|
||||
DB,
|
||||
Input<'a> = &'a RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
Output = ExecutionOutcome<<Self::Primitives as NodePrimitives>::Receipt>,
|
||||
@ -154,7 +153,7 @@ pub trait BlockExecutorProvider: Send + Sync + Clone + Unpin + 'static {
|
||||
/// This is used to execute a single block and get the changed state.
|
||||
fn executor<DB>(&self, db: DB) -> Self::Executor<DB>
|
||||
where
|
||||
DB: Database<Error: Into<ProviderError> + Display>;
|
||||
DB: Database;
|
||||
|
||||
/// Creates a new batch executor with the given database and pruning modes.
|
||||
///
|
||||
@ -162,7 +161,7 @@ pub trait BlockExecutorProvider: Send + Sync + Clone + Unpin + 'static {
|
||||
/// during historical sync which involves executing multiple blocks in sequence.
|
||||
fn batch_executor<DB>(&self, db: DB) -> Self::BatchExecutor<DB>
|
||||
where
|
||||
DB: Database<Error: Into<ProviderError> + Display>;
|
||||
DB: Database;
|
||||
}
|
||||
|
||||
/// Helper type for the output of executing a block.
|
||||
@ -177,13 +176,13 @@ pub struct ExecuteOutput<R = Receipt> {
|
||||
/// Defines the strategy for executing a single block.
|
||||
pub trait BlockExecutionStrategy {
|
||||
/// Database this strategy operates on.
|
||||
type DB: Database;
|
||||
type DB: revm::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;
|
||||
type Error: core::error::Error;
|
||||
|
||||
/// Applies any necessary changes before executing the block's transactions.
|
||||
fn apply_pre_execution_changes(
|
||||
@ -236,7 +235,7 @@ pub trait BlockExecutionStrategyFactory: Send + Sync + Clone + Unpin + 'static {
|
||||
type Primitives: NodePrimitives;
|
||||
|
||||
/// Associated strategy type.
|
||||
type Strategy<DB: Database<Error: Into<ProviderError> + Display>>: BlockExecutionStrategy<
|
||||
type Strategy<DB: Database>: BlockExecutionStrategy<
|
||||
DB = DB,
|
||||
Primitives = Self::Primitives,
|
||||
Error = BlockExecutionError,
|
||||
@ -245,7 +244,7 @@ pub trait BlockExecutionStrategyFactory: Send + Sync + Clone + Unpin + 'static {
|
||||
/// Creates a strategy using the give database.
|
||||
fn create_strategy<DB>(&self, db: DB) -> Self::Strategy<DB>
|
||||
where
|
||||
DB: Database<Error: Into<ProviderError> + Display>;
|
||||
DB: Database;
|
||||
}
|
||||
|
||||
impl<F> Clone for BasicBlockExecutorProvider<F>
|
||||
@ -276,15 +275,13 @@ where
|
||||
{
|
||||
type Primitives = F::Primitives;
|
||||
|
||||
type Executor<DB: Database<Error: Into<ProviderError> + Display>> =
|
||||
BasicBlockExecutor<F::Strategy<DB>>;
|
||||
type Executor<DB: Database> = BasicBlockExecutor<F::Strategy<DB>>;
|
||||
|
||||
type BatchExecutor<DB: Database<Error: Into<ProviderError> + Display>> =
|
||||
BasicBatchExecutor<F::Strategy<DB>>;
|
||||
type BatchExecutor<DB: Database> = BasicBatchExecutor<F::Strategy<DB>>;
|
||||
|
||||
fn executor<DB>(&self, db: DB) -> Self::Executor<DB>
|
||||
where
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
DB: Database,
|
||||
{
|
||||
let strategy = self.strategy_factory.create_strategy(db);
|
||||
BasicBlockExecutor::new(strategy)
|
||||
@ -292,7 +289,7 @@ where
|
||||
|
||||
fn batch_executor<DB>(&self, db: DB) -> Self::BatchExecutor<DB>
|
||||
where
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
DB: Database,
|
||||
{
|
||||
let strategy = self.strategy_factory.create_strategy(db);
|
||||
let batch_record = BlockBatchRecord::default();
|
||||
@ -318,7 +315,7 @@ impl<S> BasicBlockExecutor<S> {
|
||||
impl<S, DB> Executor<DB> for BasicBlockExecutor<S>
|
||||
where
|
||||
S: BlockExecutionStrategy<DB = DB>,
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
DB: Database,
|
||||
{
|
||||
type Input<'a> = &'a RecoveredBlock<<S::Primitives as NodePrimitives>::Block>;
|
||||
type Output = BlockExecutionOutput<<S::Primitives as NodePrimitives>::Receipt>;
|
||||
@ -401,7 +398,7 @@ where
|
||||
impl<S, DB> BatchExecutor<DB> for BasicBatchExecutor<S>
|
||||
where
|
||||
S: BlockExecutionStrategy<DB = DB, Error = BlockExecutionError>,
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
DB: Database,
|
||||
{
|
||||
type Input<'a> = &'a RecoveredBlock<<S::Primitives as NodePrimitives>::Block>;
|
||||
type Output = ExecutionOutcome<<S::Primitives as NodePrimitives>::Receipt>;
|
||||
@ -495,19 +492,19 @@ mod tests {
|
||||
|
||||
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>;
|
||||
type Executor<DB: Database> = TestExecutor<DB>;
|
||||
type BatchExecutor<DB: Database> = TestExecutor<DB>;
|
||||
|
||||
fn executor<DB>(&self, _db: DB) -> Self::Executor<DB>
|
||||
where
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
DB: Database,
|
||||
{
|
||||
TestExecutor(PhantomData)
|
||||
}
|
||||
|
||||
fn batch_executor<DB>(&self, _db: DB) -> Self::BatchExecutor<DB>
|
||||
where
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
DB: Database,
|
||||
{
|
||||
TestExecutor(PhantomData)
|
||||
}
|
||||
@ -585,12 +582,11 @@ mod tests {
|
||||
|
||||
impl BlockExecutionStrategyFactory for TestExecutorStrategyFactory {
|
||||
type Primitives = EthPrimitives;
|
||||
type Strategy<DB: Database<Error: Into<ProviderError> + Display>> =
|
||||
TestExecutorStrategy<DB, TestEvmConfig>;
|
||||
type Strategy<DB: Database> = TestExecutorStrategy<DB, TestEvmConfig>;
|
||||
|
||||
fn create_strategy<DB>(&self, db: DB) -> Self::Strategy<DB>
|
||||
where
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
DB: Database,
|
||||
{
|
||||
let state = State::builder()
|
||||
.with_database(db)
|
||||
|
||||
@ -22,12 +22,15 @@ use alloy_eips::eip2930::AccessList;
|
||||
use alloy_primitives::{Address, Bytes, B256, U256};
|
||||
use core::fmt::Debug;
|
||||
use reth_primitives_traits::{BlockHeader, SignedTransaction};
|
||||
use revm::{Database, DatabaseCommit, GetInspector};
|
||||
use revm_primitives::{BlockEnv, EVMError, ResultAndState, TxEnv, TxKind};
|
||||
use revm::{DatabaseCommit, GetInspector};
|
||||
use revm_primitives::{BlockEnv, ResultAndState, TxEnv, TxKind};
|
||||
|
||||
pub mod either;
|
||||
/// EVM environment configuration.
|
||||
pub mod env;
|
||||
/// EVM error types.
|
||||
mod error;
|
||||
pub use error::*;
|
||||
pub mod execute;
|
||||
pub use env::EvmEnv;
|
||||
|
||||
@ -82,6 +85,9 @@ pub trait Evm {
|
||||
Ok(result)
|
||||
}
|
||||
}
|
||||
/// Helper trait to bound [`revm::Database::Error`] with common requirements.
|
||||
pub trait Database: revm::Database<Error: core::error::Error + Send + Sync + 'static> {}
|
||||
impl<T> Database for T where T: revm::Database<Error: core::error::Error + Send + Sync + 'static> {}
|
||||
|
||||
/// Trait for configuring the EVM for executing full blocks.
|
||||
pub trait ConfigureEvm: ConfigureEvmEnv {
|
||||
@ -89,9 +95,12 @@ pub trait ConfigureEvm: ConfigureEvmEnv {
|
||||
type Evm<'a, DB: Database + 'a, I: 'a>: Evm<
|
||||
Tx = Self::TxEnv,
|
||||
DB = DB,
|
||||
Error = EVMError<DB::Error>,
|
||||
Error = Self::EvmError<DB::Error>,
|
||||
>;
|
||||
|
||||
/// The error type returned by the EVM.
|
||||
type EvmError<DBError: core::error::Error + Send + Sync + 'static>: EvmError;
|
||||
|
||||
/// Returns a new EVM with the given database configured with the given environment settings,
|
||||
/// including the spec id and transaction environment.
|
||||
///
|
||||
@ -137,6 +146,7 @@ where
|
||||
&'b T: ConfigureEvmEnv<Header = T::Header, TxEnv = T::TxEnv, Spec = T::Spec>,
|
||||
{
|
||||
type Evm<'a, DB: Database + 'a, I: 'a> = T::Evm<'a, DB, I>;
|
||||
type EvmError<DBError: core::error::Error + Send + Sync + 'static> = T::EvmError<DBError>;
|
||||
|
||||
fn evm_for_block<DB: Database>(&self, db: DB, header: &Self::Header) -> Self::Evm<'_, DB, ()> {
|
||||
(*self).evm_for_block(db, header)
|
||||
|
||||
@ -1,16 +1,14 @@
|
||||
//! A no operation block executor implementation.
|
||||
|
||||
use core::fmt::Display;
|
||||
use reth_execution_errors::BlockExecutionError;
|
||||
use reth_execution_types::{BlockExecutionOutput, ExecutionOutcome};
|
||||
use reth_primitives::{NodePrimitives, RecoveredBlock};
|
||||
use reth_storage_errors::provider::ProviderError;
|
||||
use revm::State;
|
||||
use revm_primitives::db::Database;
|
||||
|
||||
use crate::{
|
||||
execute::{BatchExecutor, BlockExecutorProvider, Executor},
|
||||
system_calls::OnStateHook,
|
||||
Database,
|
||||
};
|
||||
|
||||
const UNAVAILABLE_FOR_NOOP: &str = "execution unavailable for noop";
|
||||
@ -23,20 +21,20 @@ pub struct NoopBlockExecutorProvider<P>(core::marker::PhantomData<P>);
|
||||
impl<P: NodePrimitives> BlockExecutorProvider for NoopBlockExecutorProvider<P> {
|
||||
type Primitives = P;
|
||||
|
||||
type Executor<DB: Database<Error: Into<ProviderError> + Display>> = Self;
|
||||
type Executor<DB: Database> = Self;
|
||||
|
||||
type BatchExecutor<DB: Database<Error: Into<ProviderError> + Display>> = Self;
|
||||
type BatchExecutor<DB: Database> = Self;
|
||||
|
||||
fn executor<DB>(&self, _: DB) -> Self::Executor<DB>
|
||||
where
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
DB: Database,
|
||||
{
|
||||
Self::default()
|
||||
}
|
||||
|
||||
fn batch_executor<DB>(&self, _: DB) -> Self::BatchExecutor<DB>
|
||||
where
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
DB: Database,
|
||||
{
|
||||
Self::default()
|
||||
}
|
||||
|
||||
@ -1,6 +1,6 @@
|
||||
//! System contract call functions.
|
||||
|
||||
use crate::{ConfigureEvm, Evm, EvmEnv};
|
||||
use crate::{ConfigureEvm, Database, Evm, EvmEnv};
|
||||
use alloc::{boxed::Box, sync::Arc};
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_eips::{
|
||||
@ -10,7 +10,7 @@ use alloy_primitives::Bytes;
|
||||
use core::fmt::Display;
|
||||
use reth_chainspec::EthereumHardforks;
|
||||
use reth_execution_errors::BlockExecutionError;
|
||||
use revm::{Database, DatabaseCommit};
|
||||
use revm::DatabaseCommit;
|
||||
use revm_primitives::{EvmState, B256};
|
||||
|
||||
mod eip2935;
|
||||
@ -260,7 +260,6 @@ where
|
||||
) -> Result<Bytes, BlockExecutionError>
|
||||
where
|
||||
DB: Database + DatabaseCommit,
|
||||
DB::Error: Display,
|
||||
{
|
||||
let evm_config = self.evm_config.clone();
|
||||
let mut evm = evm_config.evm_with_env(db, evm_env.clone());
|
||||
|
||||
@ -6,16 +6,15 @@ use crate::{
|
||||
BlockExecutionStrategy, BlockExecutorProvider, Executor,
|
||||
},
|
||||
system_calls::OnStateHook,
|
||||
Database,
|
||||
};
|
||||
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, Receipts, RecoveredBlock};
|
||||
use reth_storage_errors::provider::ProviderError;
|
||||
use revm::State;
|
||||
use revm_primitives::db::Database;
|
||||
use std::{fmt::Display, sync::Arc};
|
||||
use std::sync::Arc;
|
||||
|
||||
/// A [`BlockExecutorProvider`] that returns mocked execution results.
|
||||
#[derive(Clone, Debug, Default)]
|
||||
@ -33,20 +32,20 @@ impl MockExecutorProvider {
|
||||
impl BlockExecutorProvider for MockExecutorProvider {
|
||||
type Primitives = EthPrimitives;
|
||||
|
||||
type Executor<DB: Database<Error: Into<ProviderError> + Display>> = Self;
|
||||
type Executor<DB: Database> = Self;
|
||||
|
||||
type BatchExecutor<DB: Database<Error: Into<ProviderError> + Display>> = Self;
|
||||
type BatchExecutor<DB: Database> = Self;
|
||||
|
||||
fn executor<DB>(&self, _: DB) -> Self::Executor<DB>
|
||||
where
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
DB: Database,
|
||||
{
|
||||
self.clone()
|
||||
}
|
||||
|
||||
fn batch_executor<DB>(&self, _: DB) -> Self::BatchExecutor<DB>
|
||||
where
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
DB: Database,
|
||||
{
|
||||
self.clone()
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user