feat: integrate ExecutorProvider (#7798)

This commit is contained in:
Matthias Seitz
2024-05-03 13:39:46 +02:00
committed by GitHub
parent ec45ae679f
commit 067b0ff420
74 changed files with 1087 additions and 2027 deletions

View File

@ -15,6 +15,8 @@ workspace = true
reth-primitives.workspace = true
reth-interfaces.workspace = true
reth-db.workspace = true
reth-evm.workspace = true
reth-revm.workspace = true
reth-provider.workspace = true
reth-stages-api.workspace = true
reth-trie = { workspace = true, features = ["metrics"] }
@ -40,6 +42,7 @@ reth-db = { workspace = true, features = ["test-utils"] }
reth-interfaces = { workspace = true, features = ["test-utils"] }
reth-primitives = { workspace = true , features = ["test-utils"] }
reth-provider = { workspace = true, features = ["test-utils"] }
reth-evm = { workspace = true, features = ["test-utils"] }
reth-revm.workspace = true
reth-evm-ethereum.workspace = true
parking_lot.workspace = true

View File

@ -7,6 +7,7 @@ use crate::{
};
use reth_consensus::{Consensus, ConsensusError};
use reth_db::database::Database;
use reth_evm::execute::BlockExecutorProvider;
use reth_interfaces::{
blockchain_tree::{
error::{BlockchainTreeError, CanonicalError, InsertBlockError, InsertBlockErrorKind},
@ -24,7 +25,7 @@ use reth_provider::{
chain::{ChainSplit, ChainSplitTarget},
BlockExecutionWriter, BlockNumReader, BlockWriter, BundleStateWithReceipts,
CanonStateNotification, CanonStateNotificationSender, CanonStateNotifications, Chain,
ChainSpecProvider, DisplayBlocksChain, ExecutorFactory, HeaderProvider, ProviderError,
ChainSpecProvider, DisplayBlocksChain, HeaderProvider, ProviderError,
};
use reth_stages_api::{MetricEvent, MetricEventsSender};
use std::{
@ -57,13 +58,13 @@ use tracing::{debug, error, info, instrument, trace, warn};
/// * [BlockchainTree::make_canonical]: Check if we have the hash of a block that is the current
/// canonical head and commit it to db.
#[derive(Debug)]
pub struct BlockchainTree<DB, EVM> {
pub struct BlockchainTree<DB, E> {
/// The state of the tree
///
/// Tracks all the chains, the block indices, and the block buffer.
state: TreeState,
/// External components (the database, consensus engine etc.)
externals: TreeExternals<DB, EVM>,
externals: TreeExternals<DB, E>,
/// Tree configuration
config: BlockchainTreeConfig,
/// Broadcast channel for canon state changes notifications.
@ -75,7 +76,7 @@ pub struct BlockchainTree<DB, EVM> {
prune_modes: Option<PruneModes>,
}
impl<DB, EVM> BlockchainTree<DB, EVM> {
impl<DB, E> BlockchainTree<DB, E> {
/// Subscribe to new blocks events.
///
/// Note: Only canonical blocks are emitted by the tree.
@ -89,10 +90,10 @@ impl<DB, EVM> BlockchainTree<DB, EVM> {
}
}
impl<DB, EVM> BlockchainTree<DB, EVM>
impl<DB, E> BlockchainTree<DB, E>
where
DB: Database + Clone,
EVM: ExecutorFactory,
E: BlockExecutorProvider,
{
/// Builds the blockchain tree for the node.
///
@ -115,7 +116,7 @@ where
/// storage space efficiently. It's important to validate this configuration to ensure it does
/// not lead to unintended data loss.
pub fn new(
externals: TreeExternals<DB, EVM>,
externals: TreeExternals<DB, E>,
config: BlockchainTreeConfig,
prune_modes: Option<PruneModes>,
) -> RethResult<Self> {
@ -1273,7 +1274,8 @@ mod tests {
use linked_hash_set::LinkedHashSet;
use reth_consensus::test_utils::TestConsensus;
use reth_db::{tables, test_utils::TempDatabase, transaction::DbTxMut, DatabaseEnv};
use reth_evm_ethereum::EthEvmConfig;
use reth_evm::test_utils::MockExecutorProvider;
use reth_evm_ethereum::execute::EthExecutorProvider;
#[cfg(not(feature = "optimism"))]
use reth_primitives::proofs::calculate_receipt_root;
#[cfg(feature = "optimism")]
@ -1289,19 +1291,15 @@ mod tests {
MAINNET,
};
use reth_provider::{
test_utils::{
blocks::BlockchainTestData, create_test_provider_factory_with_chain_spec,
TestExecutorFactory,
},
test_utils::{blocks::BlockchainTestData, create_test_provider_factory_with_chain_spec},
ProviderFactory,
};
use reth_revm::EvmProcessorFactory;
use reth_trie::StateRoot;
use std::collections::HashMap;
fn setup_externals(
exec_res: Vec<BundleStateWithReceipts>,
) -> TreeExternals<Arc<TempDatabase<DatabaseEnv>>, TestExecutorFactory> {
) -> TreeExternals<Arc<TempDatabase<DatabaseEnv>>, MockExecutorProvider> {
let chain_spec = Arc::new(
ChainSpecBuilder::default()
.chain(MAINNET.chain)
@ -1311,7 +1309,7 @@ mod tests {
);
let provider_factory = create_test_provider_factory_with_chain_spec(chain_spec);
let consensus = Arc::new(TestConsensus::default());
let executor_factory = TestExecutorFactory::default();
let executor_factory = MockExecutorProvider::default();
executor_factory.extend(exec_res);
TreeExternals::new(provider_factory, consensus, executor_factory)
@ -1395,7 +1393,7 @@ mod tests {
self
}
fn assert<DB: Database, EF: ExecutorFactory>(self, tree: &BlockchainTree<DB, EF>) {
fn assert<DB: Database, E: BlockExecutorProvider>(self, tree: &BlockchainTree<DB, E>) {
if let Some(chain_num) = self.chain_num {
assert_eq!(tree.state.chains.len(), chain_num);
}
@ -1439,8 +1437,7 @@ mod tests {
);
let provider_factory = create_test_provider_factory_with_chain_spec(chain_spec.clone());
let consensus = Arc::new(TestConsensus::default());
let executor_factory =
EvmProcessorFactory::new(chain_spec.clone(), EthEvmConfig::default());
let executor_provider = EthExecutorProvider::ethereum(chain_spec.clone());
{
let provider_rw = provider_factory.provider_rw().unwrap();
@ -1548,7 +1545,7 @@ mod tests {
mock_block(3, Some(sidechain_block_1.hash()), Vec::from([mock_tx(2)]), 3);
let mut tree = BlockchainTree::new(
TreeExternals::new(provider_factory, consensus, executor_factory),
TreeExternals::new(provider_factory, consensus, executor_provider),
BlockchainTreeConfig::default(),
None,
)

View File

@ -7,6 +7,7 @@ use super::externals::TreeExternals;
use crate::BundleStateDataRef;
use reth_consensus::{Consensus, ConsensusError};
use reth_db::database::Database;
use reth_evm::execute::{BlockExecutionOutput, BlockExecutorProvider, Executor};
use reth_interfaces::{
blockchain_tree::{
error::{BlockchainTreeError, InsertBlockErrorKind},
@ -15,13 +16,14 @@ use reth_interfaces::{
RethResult,
};
use reth_primitives::{
BlockHash, BlockNumber, ForkBlock, GotExpected, SealedBlockWithSenders, SealedHeader, U256,
BlockHash, BlockNumber, ForkBlock, GotExpected, Receipts, SealedBlockWithSenders, SealedHeader,
U256,
};
use reth_provider::{
providers::{BundleStateProvider, ConsistentDbView},
BundleStateDataProvider, BundleStateWithReceipts, Chain, ExecutorFactory, ProviderError,
StateRootProvider,
BundleStateDataProvider, BundleStateWithReceipts, Chain, ProviderError, StateRootProvider,
};
use reth_revm::database::StateProviderDatabase;
use reth_trie::updates::TrieUpdates;
use reth_trie_parallel::parallel_root::ParallelStateRoot;
use std::{
@ -66,18 +68,18 @@ impl AppendableChain {
///
/// if [BlockValidationKind::Exhaustive] is specified, the method will verify the state root of
/// the block.
pub fn new_canonical_fork<DB, EF>(
pub fn new_canonical_fork<DB, E>(
block: SealedBlockWithSenders,
parent_header: &SealedHeader,
canonical_block_hashes: &BTreeMap<BlockNumber, BlockHash>,
canonical_fork: ForkBlock,
externals: &TreeExternals<DB, EF>,
externals: &TreeExternals<DB, E>,
block_attachment: BlockAttachment,
block_validation_kind: BlockValidationKind,
) -> Result<Self, InsertBlockErrorKind>
where
DB: Database + Clone,
EF: ExecutorFactory,
E: BlockExecutorProvider,
{
let state = BundleStateWithReceipts::default();
let empty = BTreeMap::new();
@ -104,18 +106,18 @@ impl AppendableChain {
/// Create a new chain that forks off of an existing sidechain.
///
/// This differs from [AppendableChain::new_canonical_fork] in that this starts a new fork.
pub(crate) fn new_chain_fork<DB, EF>(
pub(crate) fn new_chain_fork<DB, E>(
&self,
block: SealedBlockWithSenders,
side_chain_block_hashes: BTreeMap<BlockNumber, BlockHash>,
canonical_block_hashes: &BTreeMap<BlockNumber, BlockHash>,
canonical_fork: ForkBlock,
externals: &TreeExternals<DB, EF>,
externals: &TreeExternals<DB, E>,
block_validation_kind: BlockValidationKind,
) -> Result<Self, InsertBlockErrorKind>
where
DB: Database + Clone,
EF: ExecutorFactory,
E: BlockExecutorProvider,
{
let parent_number = block.number - 1;
let parent = self.blocks().get(&parent_number).ok_or(
@ -166,18 +168,18 @@ impl AppendableChain {
/// - [BlockAttachment] represents if the block extends the canonical chain, and thus we can
/// cache the trie state updates.
/// - [BlockValidationKind] determines if the state root __should__ be validated.
fn validate_and_execute<BSDP, DB, EVM>(
fn validate_and_execute<BSDP, DB, E>(
block: SealedBlockWithSenders,
parent_block: &SealedHeader,
bundle_state_data_provider: BSDP,
externals: &TreeExternals<DB, EVM>,
externals: &TreeExternals<DB, E>,
block_attachment: BlockAttachment,
block_validation_kind: BlockValidationKind,
) -> RethResult<(BundleStateWithReceipts, Option<TrieUpdates>)>
where
BSDP: BundleStateDataProvider,
DB: Database + Clone,
EVM: ExecutorFactory,
E: BlockExecutorProvider,
{
// some checks are done before blocks comes here.
externals.consensus.validate_header_against_parent(&block, parent_block)?;
@ -203,11 +205,17 @@ impl AppendableChain {
let provider = BundleStateProvider::new(state_provider, bundle_state_data_provider);
let mut executor = externals.executor_factory.with_state(&provider);
let db = StateProviderDatabase::new(&provider);
let executor = externals.executor_factory.executor(db);
let block_hash = block.hash();
let block = block.unseal();
executor.execute_and_verify_receipt(&block, U256::MAX)?;
let bundle_state = executor.take_output_state();
let state = executor.execute((&block, U256::MAX).into())?;
let BlockExecutionOutput { state, receipts, .. } = state;
let bundle_state = BundleStateWithReceipts::new(
state,
Receipts::from_block_receipt(receipts),
block.number,
);
// check state root if the block extends the canonical chain __and__ if state root
// validation was requested.
@ -259,19 +267,19 @@ impl AppendableChain {
/// __not__ the canonical head.
#[track_caller]
#[allow(clippy::too_many_arguments)]
pub(crate) fn append_block<DB, EF>(
pub(crate) fn append_block<DB, E>(
&mut self,
block: SealedBlockWithSenders,
side_chain_block_hashes: BTreeMap<BlockNumber, BlockHash>,
canonical_block_hashes: &BTreeMap<BlockNumber, BlockHash>,
externals: &TreeExternals<DB, EF>,
externals: &TreeExternals<DB, E>,
canonical_fork: ForkBlock,
block_attachment: BlockAttachment,
block_validation_kind: BlockValidationKind,
) -> Result<(), InsertBlockErrorKind>
where
DB: Database + Clone,
EF: ExecutorFactory,
E: BlockExecutorProvider,
{
let parent_block = self.chain.tip();

View File

@ -19,27 +19,27 @@ use std::{collections::BTreeMap, sync::Arc};
/// - The executor factory to execute blocks with
/// - The chain spec
#[derive(Debug)]
pub struct TreeExternals<DB, EVM> {
pub struct TreeExternals<DB, E> {
/// The provider factory, used to commit the canonical chain, or unwind it.
pub(crate) provider_factory: ProviderFactory<DB>,
/// The consensus engine.
pub(crate) consensus: Arc<dyn Consensus>,
/// The executor factory to execute blocks with.
pub(crate) executor_factory: EVM,
pub(crate) executor_factory: E,
}
impl<DB, EVM> TreeExternals<DB, EVM> {
impl<DB, E> TreeExternals<DB, E> {
/// Create new tree externals.
pub fn new(
provider_factory: ProviderFactory<DB>,
consensus: Arc<dyn Consensus>,
executor_factory: EVM,
executor_factory: E,
) -> Self {
Self { provider_factory, consensus, executor_factory }
}
}
impl<DB: Database, EVM> TreeExternals<DB, EVM> {
impl<DB: Database, E> TreeExternals<DB, E> {
/// Fetches the latest canonical block hashes by walking backwards from the head.
///
/// Returns the hashes sorted by increasing block numbers

View File

@ -3,6 +3,7 @@
use super::BlockchainTree;
use parking_lot::RwLock;
use reth_db::database::Database;
use reth_evm::execute::BlockExecutorProvider;
use reth_interfaces::{
blockchain_tree::{
error::{CanonicalError, InsertBlockError},
@ -17,7 +18,7 @@ use reth_primitives::{
};
use reth_provider::{
BlockchainTreePendingStateProvider, BundleStateDataProvider, CanonStateSubscriptions,
ExecutorFactory, ProviderError,
ProviderError,
};
use std::{
collections::{BTreeMap, HashSet},
@ -27,22 +28,22 @@ use tracing::trace;
/// Shareable blockchain tree that is behind a RwLock
#[derive(Clone, Debug)]
pub struct ShareableBlockchainTree<DB, EF> {
pub struct ShareableBlockchainTree<DB, E> {
/// BlockchainTree
pub tree: Arc<RwLock<BlockchainTree<DB, EF>>>,
pub tree: Arc<RwLock<BlockchainTree<DB, E>>>,
}
impl<DB, EF> ShareableBlockchainTree<DB, EF> {
impl<DB, E> ShareableBlockchainTree<DB, E> {
/// Create a new shareable database.
pub fn new(tree: BlockchainTree<DB, EF>) -> Self {
pub fn new(tree: BlockchainTree<DB, E>) -> Self {
Self { tree: Arc::new(RwLock::new(tree)) }
}
}
impl<DB, EF> BlockchainTreeEngine for ShareableBlockchainTree<DB, EF>
impl<DB, E> BlockchainTreeEngine for ShareableBlockchainTree<DB, E>
where
DB: Database + Clone,
EF: ExecutorFactory,
E: BlockExecutorProvider,
{
fn buffer_block(&self, block: SealedBlockWithSenders) -> Result<(), InsertBlockError> {
let mut tree = self.tree.write();
@ -99,10 +100,10 @@ where
}
}
impl<DB, EF> BlockchainTreeViewer for ShareableBlockchainTree<DB, EF>
impl<DB, E> BlockchainTreeViewer for ShareableBlockchainTree<DB, E>
where
DB: Database + Clone,
EF: ExecutorFactory,
E: BlockExecutorProvider,
{
fn blocks(&self) -> BTreeMap<BlockNumber, HashSet<BlockHash>> {
trace!(target: "blockchain_tree", "Returning all blocks in blockchain tree");
@ -181,10 +182,10 @@ where
}
}
impl<DB, EF> BlockchainTreePendingStateProvider for ShareableBlockchainTree<DB, EF>
impl<DB, E> BlockchainTreePendingStateProvider for ShareableBlockchainTree<DB, E>
where
DB: Database + Clone,
EF: ExecutorFactory,
E: BlockExecutorProvider,
{
fn find_pending_state_provider(
&self,
@ -196,10 +197,10 @@ where
}
}
impl<DB, EF> CanonStateSubscriptions for ShareableBlockchainTree<DB, EF>
impl<DB, E> CanonStateSubscriptions for ShareableBlockchainTree<DB, E>
where
DB: Send + Sync,
EF: Send + Sync,
E: Send + Sync,
{
fn subscribe_to_canonical_state(&self) -> reth_provider::CanonStateNotifications {
trace!(target: "blockchain_tree", "Registered subscriber for canonical state");

View File

@ -18,23 +18,18 @@
use reth_beacon_consensus::BeaconEngineMessage;
use reth_consensus::{Consensus, ConsensusError};
use reth_engine_primitives::EngineTypes;
use reth_evm::ConfigureEvm;
use reth_interfaces::executor::{BlockExecutionError, BlockValidationError};
use reth_primitives::{
constants::{EMPTY_RECEIPTS, EMPTY_TRANSACTIONS, ETHEREUM_BLOCK_GAS_LIMIT},
constants::{EMPTY_TRANSACTIONS, ETHEREUM_BLOCK_GAS_LIMIT},
eip4844::calculate_excess_blob_gas,
proofs, Block, BlockBody, BlockHash, BlockHashOrNumber, BlockNumber, BlockWithSenders, Bloom,
ChainSpec, Header, ReceiptWithBloom, SealedBlock, SealedHeader, TransactionSigned, Withdrawals,
B256, U256,
proofs, Block, BlockBody, BlockHash, BlockHashOrNumber, BlockNumber, ChainSpec, Header,
Receipts, SealedBlock, SealedHeader, TransactionSigned, Withdrawals, B256, U256,
};
use reth_provider::{
BlockExecutor, BlockReaderIdExt, BundleStateWithReceipts, CanonStateNotificationSender,
StateProviderFactory,
};
use reth_revm::{
database::StateProviderDatabase, db::states::bundle_state::BundleRetention,
processor::EVMProcessor, State,
BlockReaderIdExt, BundleStateWithReceipts, CanonStateNotificationSender, StateProviderFactory,
StateRootProvider,
};
use reth_revm::database::StateProviderDatabase;
use reth_transaction_pool::TransactionPool;
use std::{
collections::HashMap,
@ -50,6 +45,7 @@ mod task;
pub use crate::client::AutoSealClient;
pub use mode::{FixedBlockTimeMiner, MiningMode, ReadyTransactionMiner};
use reth_evm::execute::{BlockExecutionOutput, BlockExecutorProvider, Executor};
pub use task::MiningTask;
/// A consensus implementation intended for local development and testing purposes.
@ -281,6 +277,18 @@ impl StorageInner {
parent.next_block_base_fee(chain_spec.base_fee_params_at_timestamp(timestamp))
});
let blob_gas_used = if chain_spec.is_cancun_active_at_timestamp(timestamp) {
let mut sum_blob_gas_used = 0;
for tx in transactions {
if let Some(blob_tx) = tx.transaction.as_eip4844() {
sum_blob_gas_used += blob_tx.blob_gas();
}
}
Some(sum_blob_gas_used)
} else {
None
};
let mut header = Header {
parent_hash: self.best_hash,
ommers_hash: proofs::calculate_ommers_root(ommers),
@ -298,7 +306,7 @@ impl StorageInner {
mix_hash: Default::default(),
nonce: 0,
base_fee_per_gas,
blob_gas_used: None,
blob_gas_used,
excess_blob_gas: None,
extra_data: Default::default(),
parent_beacon_block_root: None,
@ -334,111 +342,26 @@ impl StorageInner {
header
}
/// Executes the block with the given block and senders, on the provided [EVMProcessor].
///
/// This returns the poststate from execution and post-block changes, as well as the gas used.
pub(crate) fn execute<EvmConfig>(
&self,
block: &BlockWithSenders,
executor: &mut EVMProcessor<'_, EvmConfig>,
) -> Result<(BundleStateWithReceipts, u64), BlockExecutionError>
where
EvmConfig: ConfigureEvm,
{
trace!(target: "consensus::auto", transactions=?&block.body, "executing transactions");
// TODO: there isn't really a parent beacon block root here, so not sure whether or not to
// call the 4788 beacon contract
// set the first block to find the correct index in bundle state
executor.set_first_block(block.number);
let (receipts, gas_used) = executor.execute_transactions(block, U256::ZERO)?;
// Save receipts.
executor.save_receipts(receipts)?;
// add post execution state change
// Withdrawals, rewards etc.
executor.apply_post_execution_state_change(block, U256::ZERO)?;
// merge transitions
executor.db_mut().merge_transitions(BundleRetention::Reverts);
// apply post block changes
Ok((executor.take_output_state(), gas_used))
}
/// Fills in the post-execution header fields based on the given BundleState and gas used.
/// In doing this, the state root is calculated and the final header is returned.
pub(crate) fn complete_header<S: StateProviderFactory>(
&self,
mut header: Header,
bundle_state: &BundleStateWithReceipts,
client: &S,
gas_used: u64,
blob_gas_used: Option<u64>,
#[cfg(feature = "optimism")] chain_spec: &ChainSpec,
) -> Result<Header, BlockExecutionError> {
let receipts = bundle_state.receipts_by_block(header.number);
header.receipts_root = if receipts.is_empty() {
EMPTY_RECEIPTS
} else {
let receipts_with_bloom = receipts
.iter()
.map(|r| (*r).clone().expect("receipts have not been pruned").into())
.collect::<Vec<ReceiptWithBloom>>();
header.logs_bloom =
receipts_with_bloom.iter().fold(Bloom::ZERO, |bloom, r| bloom | r.bloom);
#[cfg(feature = "optimism")]
{
proofs::calculate_receipt_root_optimism(
&receipts_with_bloom,
chain_spec,
header.timestamp,
)
}
#[cfg(not(feature = "optimism"))]
{
proofs::calculate_receipt_root(&receipts_with_bloom)
}
};
header.gas_used = gas_used;
header.blob_gas_used = blob_gas_used;
// calculate the state root
let state_root = client
.latest()
.map_err(BlockExecutionError::LatestBlock)?
.state_root(bundle_state.state())
.unwrap();
header.state_root = state_root;
Ok(header)
}
/// Builds and executes a new block with the given transactions, on the provided [EVMProcessor].
/// Builds and executes a new block with the given transactions, on the provided executor.
///
/// This returns the header of the executed block, as well as the poststate from execution.
pub(crate) fn build_and_execute<EvmConfig>(
pub(crate) fn build_and_execute<Provider, Executor>(
&mut self,
transactions: Vec<TransactionSigned>,
ommers: Vec<Header>,
withdrawals: Option<Withdrawals>,
client: &impl StateProviderFactory,
provider: &Provider,
chain_spec: Arc<ChainSpec>,
evm_config: &EvmConfig,
executor: &Executor,
) -> Result<(SealedHeader, BundleStateWithReceipts), BlockExecutionError>
where
EvmConfig: ConfigureEvm,
Executor: BlockExecutorProvider,
Provider: StateProviderFactory,
{
let header = self.build_header_template(
&transactions,
&ommers,
withdrawals.as_ref(),
chain_spec.clone(),
);
let header =
self.build_header_template(&transactions, &ommers, withdrawals.as_ref(), chain_spec);
let block = Block {
let mut block = Block {
header,
body: transactions,
ommers: ommers.clone(),
@ -449,45 +372,46 @@ impl StorageInner {
trace!(target: "consensus::auto", transactions=?&block.body, "executing transactions");
// now execute the block
let db = State::builder()
.with_database_boxed(Box::new(StateProviderDatabase::new(
client.latest().map_err(BlockExecutionError::LatestBlock)?,
)))
.with_bundle_update()
.build();
let mut executor = EVMProcessor::new_with_state(chain_spec.clone(), db, evm_config);
let mut db = StateProviderDatabase::new(
provider.latest().map_err(BlockExecutionError::LatestBlock)?,
);
let (bundle_state, gas_used) = self.execute(&block, &mut executor)?;
let Block { header, body, .. } = block.block;
let body = BlockBody { transactions: body, ommers, withdrawals };
let blob_gas_used = if chain_spec.is_cancun_active_at_timestamp(header.timestamp) {
let mut sum_blob_gas_used = 0;
for tx in &body.transactions {
if let Some(blob_tx) = tx.transaction.as_eip4844() {
sum_blob_gas_used += blob_tx.blob_gas();
// TODO(mattsse): At this point we don't know certain fields of the header, so we first
// execute it and then update the header this can be improved by changing the executor
// input, for now we intercept the errors and retry
loop {
match executor.executor(&mut db).execute((&block, U256::ZERO).into()) {
Err(BlockExecutionError::Validation(BlockValidationError::BlockGasUsed {
gas,
..
})) => {
block.block.header.gas_used = gas.got;
}
}
Some(sum_blob_gas_used)
} else {
None
};
Err(BlockExecutionError::Validation(BlockValidationError::ReceiptRootDiff(
err,
))) => {
block.block.header.receipts_root = err.got;
}
_ => break,
};
}
// now execute the block
let BlockExecutionOutput { state, receipts, .. } =
executor.executor(&mut db).execute((&block, U256::ZERO).into())?;
let bundle_state = BundleStateWithReceipts::new(
state,
Receipts::from_block_receipt(receipts),
block.number,
);
let Block { mut header, body, .. } = block.block;
let body = BlockBody { transactions: body, ommers, withdrawals };
trace!(target: "consensus::auto", ?bundle_state, ?header, ?body, "executed block, calculating state root and completing header");
// fill in the rest of the fields
let header = self.complete_header(
header,
&bundle_state,
client,
gas_used,
blob_gas_used,
#[cfg(feature = "optimism")]
chain_spec.as_ref(),
)?;
// calculate the state root
header.state_root = db.state_root(bundle_state.state())?;
trace!(target: "consensus::auto", root=?header.state_root, ?body, "calculated root");
// finally insert into storage

View File

@ -2,7 +2,7 @@ use crate::{mode::MiningMode, Storage};
use futures_util::{future::BoxFuture, FutureExt};
use reth_beacon_consensus::{BeaconEngineMessage, ForkchoiceStatus};
use reth_engine_primitives::EngineTypes;
use reth_evm::ConfigureEvm;
use reth_evm::execute::BlockExecutorProvider;
use reth_primitives::{
Block, ChainSpec, IntoRecoveredTransaction, SealedBlockWithSenders, Withdrawals,
};
@ -22,7 +22,7 @@ use tokio_stream::wrappers::UnboundedReceiverStream;
use tracing::{debug, error, warn};
/// A Future that listens for new ready transactions and puts new blocks into storage
pub struct MiningTask<Client, Pool: TransactionPool, EvmConfig, Engine: EngineTypes> {
pub struct MiningTask<Client, Pool: TransactionPool, Executor, Engine: EngineTypes> {
/// The configured chain spec
chain_spec: Arc<ChainSpec>,
/// The client used to interact with the state
@ -43,14 +43,14 @@ pub struct MiningTask<Client, Pool: TransactionPool, EvmConfig, Engine: EngineTy
canon_state_notification: CanonStateNotificationSender,
/// The pipeline events to listen on
pipe_line_events: Option<UnboundedReceiverStream<PipelineEvent>>,
/// The type that defines how to configure the EVM.
evm_config: EvmConfig,
/// The type used for block execution
block_executor: Executor,
}
// === impl MiningTask ===
impl<EvmConfig, Client, Pool: TransactionPool, Engine: EngineTypes>
MiningTask<Client, Pool, EvmConfig, Engine>
impl<Executor, Client, Pool: TransactionPool, Engine: EngineTypes>
MiningTask<Client, Pool, Executor, Engine>
{
/// Creates a new instance of the task
#[allow(clippy::too_many_arguments)]
@ -62,7 +62,7 @@ impl<EvmConfig, Client, Pool: TransactionPool, Engine: EngineTypes>
storage: Storage,
client: Client,
pool: Pool,
evm_config: EvmConfig,
block_executor: Executor,
) -> Self {
Self {
chain_spec,
@ -75,7 +75,7 @@ impl<EvmConfig, Client, Pool: TransactionPool, Engine: EngineTypes>
canon_state_notification,
queued: Default::default(),
pipe_line_events: None,
evm_config,
block_executor,
}
}
@ -85,13 +85,13 @@ impl<EvmConfig, Client, Pool: TransactionPool, Engine: EngineTypes>
}
}
impl<EvmConfig, Client, Pool, Engine> Future for MiningTask<Client, Pool, EvmConfig, Engine>
impl<Executor, Client, Pool, Engine> Future for MiningTask<Client, Pool, Executor, Engine>
where
Client: StateProviderFactory + CanonChainTracker + Clone + Unpin + 'static,
Pool: TransactionPool + Unpin + 'static,
<Pool as TransactionPool>::Transaction: IntoRecoveredTransaction,
Engine: EngineTypes + 'static,
EvmConfig: ConfigureEvm + Clone + Unpin + Send + Sync + 'static,
Executor: BlockExecutorProvider,
{
type Output = ();
@ -121,7 +121,7 @@ where
let pool = this.pool.clone();
let events = this.pipe_line_events.take();
let canon_state_notification = this.canon_state_notification.clone();
let evm_config = this.evm_config.clone();
let executor = this.block_executor.clone();
// Create the mining future that creates a block, notifies the engine that drives
// the pipeline
@ -145,7 +145,7 @@ where
withdrawals.clone(),
&client,
chain_spec,
&evm_config,
&executor,
) {
Ok((new_header, bundle_state)) => {
// clear all transactions from pool

View File

@ -51,6 +51,7 @@ reth-stages = { workspace = true, features = ["test-utils"] }
reth-blockchain-tree = { workspace = true, features = ["test-utils"] }
reth-db = { workspace = true, features = ["test-utils"] }
reth-provider = { workspace = true, features = ["test-utils"] }
reth-evm = { workspace = true, features = ["test-utils"] }
reth-rpc-types-compat.workspace = true
reth-rpc.workspace = true
reth-tracing.workspace = true

View File

@ -14,9 +14,9 @@ use reth_downloaders::{
headers::reverse_headers::ReverseHeadersDownloaderBuilder,
};
use reth_ethereum_engine_primitives::EthEngineTypes;
use reth_evm_ethereum::EthEvmConfig;
use reth_evm::{either::Either, test_utils::MockExecutorProvider};
use reth_evm_ethereum::execute::EthExecutorProvider;
use reth_interfaces::{
executor::BlockExecutionError,
p2p::{bodies::client::BodiesClient, either::EitherDownloader, headers::client::HeadersClient},
sync::NoopSyncStateUpdater,
test_utils::NoopFullBlockClient,
@ -24,13 +24,10 @@ use reth_interfaces::{
use reth_payload_builder::test_utils::spawn_test_payload_service;
use reth_primitives::{BlockNumber, ChainSpec, FinishedExExHeight, PruneModes, B256};
use reth_provider::{
providers::BlockchainProvider,
test_utils::{create_test_provider_factory_with_chain_spec, TestExecutorFactory},
BundleStateWithReceipts, ExecutorFactory, HeaderSyncMode, PrunableBlockExecutor,
StaticFileProviderFactory,
providers::BlockchainProvider, test_utils::create_test_provider_factory_with_chain_spec,
BundleStateWithReceipts, HeaderSyncMode, StaticFileProviderFactory,
};
use reth_prune::Pruner;
use reth_revm::EvmProcessorFactory;
use reth_rpc_types::engine::{
CancunPayloadFields, ExecutionPayload, ForkchoiceState, ForkchoiceUpdated, PayloadStatus,
};
@ -155,31 +152,6 @@ impl Default for TestExecutorConfig {
}
}
/// A type that represents one of two possible executor factories.
#[derive(Debug, Clone)]
pub enum EitherExecutorFactory<A: ExecutorFactory, B: ExecutorFactory> {
/// The first factory variant
Left(A),
/// The second factory variant
Right(B),
}
impl<A, B> ExecutorFactory for EitherExecutorFactory<A, B>
where
A: ExecutorFactory,
B: ExecutorFactory,
{
fn with_state<'a, SP: reth_provider::StateProvider + 'a>(
&'a self,
sp: SP,
) -> Box<dyn PrunableBlockExecutor<Error = BlockExecutionError> + 'a> {
match self {
EitherExecutorFactory::Left(a) => a.with_state::<'a, SP>(sp),
EitherExecutorFactory::Right(b) => b.with_state::<'a, SP>(sp),
}
}
}
/// The basic configuration for a `TestConsensusEngine`, without generics for the client or
/// consensus engine.
#[derive(Debug)]
@ -366,14 +338,13 @@ where
// use either test executor or real executor
let executor_factory = match self.base_config.executor_config {
TestExecutorConfig::Test(results) => {
let executor_factory = TestExecutorFactory::default();
let executor_factory = MockExecutorProvider::default();
executor_factory.extend(results);
EitherExecutorFactory::Left(executor_factory)
Either::Left(executor_factory)
}
TestExecutorConfig::Real => {
Either::Right(EthExecutorProvider::ethereum(self.base_config.chain_spec.clone()))
}
TestExecutorConfig::Real => EitherExecutorFactory::Right(EvmProcessorFactory::new(
self.base_config.chain_spec.clone(),
EthEvmConfig::default(),
)),
};
let static_file_producer = StaticFileProducer::new(

View File

@ -16,7 +16,6 @@ reth-evm.workspace = true
reth-primitives.workspace = true
reth-revm.workspace = true
reth-interfaces.workspace = true
reth-provider.workspace = true
# Ethereum
revm-primitives.workspace = true

View File

@ -1,10 +1,10 @@
//! Ethereum block executor.
use crate::EthEvmConfig;
use crate::{verify::verify_receipts, EthEvmConfig};
use reth_evm::{
execute::{
BatchBlockOutput, BatchExecutor, EthBlockExecutionInput, EthBlockOutput, Executor,
ExecutorProvider,
BatchBlockExecutionOutput, BatchExecutor, BlockExecutionInput, BlockExecutionOutput,
BlockExecutorProvider, Executor,
},
ConfigureEvm, ConfigureEvmEnv,
};
@ -13,15 +13,13 @@ use reth_interfaces::{
provider::ProviderError,
};
use reth_primitives::{
BlockWithSenders, ChainSpec, GotExpected, Hardfork, Header, PruneModes, Receipt, Receipts,
Withdrawals, U256,
BlockNumber, BlockWithSenders, ChainSpec, GotExpected, Hardfork, Header, PruneModes, Receipt,
Receipts, Withdrawals, MAINNET, U256,
};
use reth_provider::BundleStateWithReceipts;
use reth_revm::{
batch::{BlockBatchRecord, BlockExecutorStats},
db::states::bundle_state::BundleRetention,
eth_dao_fork::{DAO_HARDFORK_BENEFICIARY, DAO_HARDKFORK_ACCOUNTS},
processor::verify_receipt,
stack::InspectorStack,
state_change::{apply_beacon_root_contract_call, post_block_balance_increments},
Evm, State,
@ -35,35 +33,33 @@ use tracing::debug;
/// Provides executors to execute regular ethereum blocks
#[derive(Debug, Clone)]
pub struct EthExecutorProvider<EvmConfig> {
pub struct EthExecutorProvider<EvmConfig = EthEvmConfig> {
chain_spec: Arc<ChainSpec>,
evm_config: EvmConfig,
inspector: Option<InspectorStack>,
prune_modes: PruneModes,
}
impl EthExecutorProvider<EthEvmConfig> {
impl EthExecutorProvider {
/// Creates a new default ethereum executor provider.
pub fn ethereum(chain_spec: Arc<ChainSpec>) -> Self {
Self::new(chain_spec, Default::default())
}
/// Returns a new provider for the mainnet.
pub fn mainnet() -> Self {
Self::ethereum(MAINNET.clone())
}
}
impl<EvmConfig> EthExecutorProvider<EvmConfig> {
/// Creates a new executor provider.
pub fn new(chain_spec: Arc<ChainSpec>, evm_config: EvmConfig) -> Self {
Self { chain_spec, evm_config, inspector: None, prune_modes: PruneModes::none() }
Self { chain_spec, evm_config, inspector: None }
}
/// Configures an optional inspector stack for debugging.
pub fn with_inspector(mut self, inspector: InspectorStack) -> Self {
self.inspector = Some(inspector);
self
}
/// Configures the prune modes for the executor.
pub fn with_prune_modes(mut self, prune_modes: PruneModes) -> Self {
self.prune_modes = prune_modes;
pub fn with_inspector(mut self, inspector: Option<InspectorStack>) -> Self {
self.inspector = inspector;
self
}
}
@ -86,7 +82,7 @@ where
}
}
impl<EvmConfig> ExecutorProvider for EthExecutorProvider<EvmConfig>
impl<EvmConfig> BlockExecutorProvider for EthExecutorProvider<EvmConfig>
where
EvmConfig: ConfigureEvm,
EvmConfig: ConfigureEvmEnv<TxMeta = ()>,
@ -102,14 +98,14 @@ where
self.eth_executor(db)
}
fn batch_executor<DB>(&self, db: DB) -> Self::BatchExecutor<DB>
fn batch_executor<DB>(&self, db: DB, prune_modes: PruneModes) -> Self::BatchExecutor<DB>
where
DB: Database<Error = ProviderError>,
{
let executor = self.eth_executor(db);
EthBatchExecutor {
executor,
batch_record: BlockBatchRecord::new(self.prune_modes.clone()),
batch_record: BlockBatchRecord::new(prune_modes),
stats: BlockExecutorStats::default(),
}
}
@ -318,9 +314,11 @@ where
// transaction This was replaced with is_success flag.
// See more about EIP here: https://eips.ethereum.org/EIPS/eip-658
if self.chain_spec().is_byzantium_active_at_block(block.header.number) {
if let Err(error) =
verify_receipt(block.header.receipts_root, block.header.logs_bloom, receipts.iter())
{
if let Err(error) = verify_receipts(
block.header.receipts_root,
block.header.logs_bloom,
receipts.iter(),
) {
debug!(target: "evm", %error, ?receipts, "receipts verification failed");
return Err(error)
};
@ -382,8 +380,8 @@ where
EvmConfig: ConfigureEvmEnv<TxMeta = ()>,
DB: Database<Error = ProviderError>,
{
type Input<'a> = EthBlockExecutionInput<'a, BlockWithSenders>;
type Output = EthBlockOutput<Receipt>;
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BlockExecutionOutput<Receipt>;
type Error = BlockExecutionError;
/// Executes the block and commits the state changes.
@ -394,13 +392,13 @@ where
///
/// State changes are committed to the database.
fn execute(mut self, input: Self::Input<'_>) -> Result<Self::Output, Self::Error> {
let EthBlockExecutionInput { block, total_difficulty } = input;
let BlockExecutionInput { block, total_difficulty } = input;
let (receipts, gas_used) = self.execute_and_verify(block, total_difficulty)?;
// prepare the state for extraction
self.state.merge_transitions(BundleRetention::PlainState);
// NOTE: we need to merge keep the reverts for the bundle retention
self.state.merge_transitions(BundleRetention::Reverts);
Ok(EthBlockOutput { state: self.state.take_bundle(), receipts, gas_used })
Ok(BlockExecutionOutput { state: self.state.take_bundle(), receipts, gas_used })
}
}
@ -433,12 +431,12 @@ where
EvmConfig: ConfigureEvmEnv<TxMeta = ()>,
DB: Database<Error = ProviderError>,
{
type Input<'a> = EthBlockExecutionInput<'a, BlockWithSenders>;
type Output = BundleStateWithReceipts;
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BatchBlockExecutionOutput;
type Error = BlockExecutionError;
fn execute_one(&mut self, input: Self::Input<'_>) -> Result<BatchBlockOutput, Self::Error> {
let EthBlockExecutionInput { block, total_difficulty } = input;
fn execute_one(&mut self, input: Self::Input<'_>) -> Result<(), Self::Error> {
let BlockExecutionInput { block, total_difficulty } = input;
let (receipts, _gas_used) = self.executor.execute_and_verify(block, total_difficulty)?;
// prepare the state according to the prune mode
@ -448,18 +446,30 @@ where
// store receipts in the set
self.batch_record.save_receipts(receipts)?;
Ok(BatchBlockOutput { size_hint: Some(self.executor.state.bundle_size_hint()) })
if self.batch_record.first_block().is_none() {
self.batch_record.set_first_block(block.number);
}
Ok(())
}
fn finalize(mut self) -> Self::Output {
self.stats.log_debug();
BundleStateWithReceipts::new(
BatchBlockExecutionOutput::new(
self.executor.state.take_bundle(),
self.batch_record.take_receipts(),
self.batch_record.first_block().unwrap_or_default(),
)
}
fn set_tip(&mut self, tip: BlockNumber) {
self.batch_record.set_tip(tip);
}
fn size_hint(&self) -> Option<usize> {
Some(self.executor.state.bundle_state.size_hint())
}
}
#[cfg(test)]
@ -468,7 +478,7 @@ mod tests {
use reth_primitives::{
bytes,
constants::{BEACON_ROOTS_ADDRESS, SYSTEM_ADDRESS},
keccak256, Account, Block, Bytes, ChainSpecBuilder, ForkCondition, B256, MAINNET,
keccak256, Account, Block, Bytes, ChainSpecBuilder, ForkCondition, B256,
};
use reth_revm::{
database::StateProviderDatabase, test_utils::StateProviderTest, TransitionState,
@ -497,12 +507,7 @@ mod tests {
}
fn executor_provider(chain_spec: Arc<ChainSpec>) -> EthExecutorProvider<EthEvmConfig> {
EthExecutorProvider {
chain_spec,
evm_config: Default::default(),
inspector: None,
prune_modes: Default::default(),
}
EthExecutorProvider { chain_spec, evm_config: Default::default(), inspector: None }
}
#[test]
@ -697,7 +702,8 @@ mod tests {
let provider = executor_provider(chain_spec);
let mut executor = provider.batch_executor(StateProviderDatabase::new(&db));
let mut executor =
provider.batch_executor(StateProviderDatabase::new(&db), PruneModes::none());
// attempt to execute the genesis block with non-zero parent beacon block root, expect err
header.parent_beacon_block_root = Some(B256::with_last_byte(0x69));
@ -777,7 +783,8 @@ mod tests {
let provider = executor_provider(chain_spec);
// execute header
let mut executor = provider.batch_executor(StateProviderDatabase::new(&db));
let mut executor =
provider.batch_executor(StateProviderDatabase::new(&db), PruneModes::none());
// Now execute a block with the fixed header, ensure that it does not fail
executor

View File

@ -16,6 +16,7 @@ use reth_primitives::{
};
use reth_revm::{Database, EvmBuilder};
pub mod execute;
pub mod verify;
/// Ethereum-related EVM configuration.
#[derive(Debug, Clone, Copy, Default)]

View File

@ -0,0 +1,53 @@
//! Helpers for verifying the receipts.
use reth_interfaces::executor::{BlockExecutionError, BlockValidationError};
use reth_primitives::{Bloom, GotExpected, Receipt, ReceiptWithBloom, B256};
/// Calculate the receipts root, and compare it against against the expected receipts root and logs
/// bloom.
pub fn verify_receipts<'a>(
expected_receipts_root: B256,
expected_logs_bloom: Bloom,
receipts: impl Iterator<Item = &'a Receipt> + Clone,
) -> Result<(), BlockExecutionError> {
// Calculate receipts root.
let receipts_with_bloom = receipts.map(|r| r.clone().into()).collect::<Vec<ReceiptWithBloom>>();
let receipts_root = reth_primitives::proofs::calculate_receipt_root(&receipts_with_bloom);
// Create header log bloom.
let logs_bloom = receipts_with_bloom.iter().fold(Bloom::ZERO, |bloom, r| bloom | r.bloom);
compare_receipts_root_and_logs_bloom(
receipts_root,
logs_bloom,
expected_receipts_root,
expected_logs_bloom,
)?;
Ok(())
}
/// Compare the calculated receipts root with the expected receipts root, also compare
/// the calculated logs bloom with the expected logs bloom.
pub fn compare_receipts_root_and_logs_bloom(
calculated_receipts_root: B256,
calculated_logs_bloom: Bloom,
expected_receipts_root: B256,
expected_logs_bloom: Bloom,
) -> Result<(), BlockExecutionError> {
if calculated_receipts_root != expected_receipts_root {
return Err(BlockValidationError::ReceiptRootDiff(
GotExpected { got: calculated_receipts_root, expected: expected_receipts_root }.into(),
)
.into())
}
if calculated_logs_bloom != expected_logs_bloom {
return Err(BlockValidationError::BloomLogDiff(
GotExpected { got: calculated_logs_bloom, expected: expected_logs_bloom }.into(),
)
.into())
}
Ok(())
}

View File

@ -17,3 +17,11 @@ revm-primitives.workspace = true
revm.workspace = true
reth-interfaces.workspace = true
futures-util.workspace = true
parking_lot = { workspace = true, optional = true }
[dev-dependencies]
parking_lot.workspace = true
[features]
test-utils = ["dep:parking_lot"]

119
crates/evm/src/either.rs Normal file
View File

@ -0,0 +1,119 @@
//! Helper type that represents one of two possible executor types
use crate::execute::{
BatchBlockExecutionOutput, BatchExecutor, BlockExecutionInput, BlockExecutionOutput,
BlockExecutorProvider, Executor,
};
use reth_interfaces::{executor::BlockExecutionError, provider::ProviderError};
use reth_primitives::{BlockNumber, BlockWithSenders, PruneModes, Receipt};
use revm_primitives::db::Database;
// re-export Either
pub use futures_util::future::Either;
impl<A, B> BlockExecutorProvider for Either<A, B>
where
A: BlockExecutorProvider,
B: BlockExecutorProvider,
{
type Executor<DB: Database<Error = ProviderError>> = Either<A::Executor<DB>, B::Executor<DB>>;
type BatchExecutor<DB: Database<Error = ProviderError>> =
Either<A::BatchExecutor<DB>, B::BatchExecutor<DB>>;
fn executor<DB>(&self, db: DB) -> Self::Executor<DB>
where
DB: Database<Error = ProviderError>,
{
match self {
Either::Left(a) => Either::Left(a.executor(db)),
Either::Right(b) => Either::Right(b.executor(db)),
}
}
fn batch_executor<DB>(&self, db: DB, prune_modes: PruneModes) -> Self::BatchExecutor<DB>
where
DB: Database<Error = ProviderError>,
{
match self {
Either::Left(a) => Either::Left(a.batch_executor(db, prune_modes)),
Either::Right(b) => Either::Right(b.batch_executor(db, prune_modes)),
}
}
}
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,
>,
DB: Database<Error = ProviderError>,
{
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BlockExecutionOutput<Receipt>;
type Error = BlockExecutionError;
fn execute(self, input: Self::Input<'_>) -> Result<Self::Output, Self::Error> {
match self {
Either::Left(a) => a.execute(input),
Either::Right(b) => b.execute(input),
}
}
}
impl<A, B, DB> BatchExecutor<DB> for Either<A, B>
where
A: for<'a> BatchExecutor<
DB,
Input<'a> = BlockExecutionInput<'a, BlockWithSenders>,
Output = BatchBlockExecutionOutput,
Error = BlockExecutionError,
>,
B: for<'a> BatchExecutor<
DB,
Input<'a> = BlockExecutionInput<'a, BlockWithSenders>,
Output = BatchBlockExecutionOutput,
Error = BlockExecutionError,
>,
DB: Database<Error = ProviderError>,
{
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BatchBlockExecutionOutput;
type Error = BlockExecutionError;
fn execute_one(&mut self, input: Self::Input<'_>) -> Result<(), Self::Error> {
match self {
Either::Left(a) => a.execute_one(input),
Either::Right(b) => b.execute_one(input),
}
}
fn finalize(self) -> Self::Output {
match self {
Either::Left(a) => a.finalize(),
Either::Right(b) => b.finalize(),
}
}
fn set_tip(&mut self, tip: BlockNumber) {
match self {
Either::Left(a) => a.set_tip(tip),
Either::Right(b) => b.set_tip(tip),
}
}
fn size_hint(&self) -> Option<usize> {
match self {
Either::Left(a) => a.size_hint(),
Either::Right(b) => b.size_hint(),
}
}
}

View File

@ -1,7 +1,7 @@
//! Traits for execution.
use reth_interfaces::provider::ProviderError;
use reth_primitives::U256;
use reth_interfaces::{executor::BlockExecutionError, provider::ProviderError};
use reth_primitives::{BlockNumber, BlockWithSenders, PruneModes, Receipt, Receipts, U256};
use revm::db::BundleState;
use revm_primitives::db::Database;
@ -21,8 +21,8 @@ pub trait Executor<DB> {
fn execute(self, input: Self::Input<'_>) -> Result<Self::Output, Self::Error>;
}
/// An executor that can execute multiple blocks in a row and keep track of the state over the
/// entire batch.
/// A general purpose executor that can execute multiple inputs in sequence and keep track of the
/// state over the entire batch.
pub trait BatchExecutor<DB> {
/// The input type for the executor.
type Input<'a>;
@ -32,17 +32,20 @@ pub trait BatchExecutor<DB> {
type Error;
/// Executes the next block in the batch and update the state internally.
fn execute_one(&mut self, input: Self::Input<'_>) -> Result<BatchBlockOutput, Self::Error>;
fn execute_one(&mut self, input: Self::Input<'_>) -> Result<(), Self::Error>;
/// Finishes the batch and return the final state.
fn finalize(self) -> Self::Output;
}
/// The output of an executed block in a batch.
#[derive(Debug, Clone, Copy)]
pub struct BatchBlockOutput {
/// The size hint of the batch's tracked state.
pub size_hint: Option<usize>,
/// Set the expected tip of the batch.
///
/// This can be used to optimize state pruning during execution.
fn set_tip(&mut self, tip: BlockNumber);
/// 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) -> Option<usize>;
}
/// The output of an ethereum block.
@ -51,7 +54,7 @@ pub struct BatchBlockOutput {
///
/// TODO(mattsse): combine with BundleStateWithReceipts
#[derive(Debug)]
pub struct EthBlockOutput<T> {
pub struct BlockExecutionOutput<T> {
/// The changed state of the block after execution.
pub state: BundleState,
/// All the receipts of the transactions in the block.
@ -60,42 +63,94 @@ pub struct EthBlockOutput<T> {
pub gas_used: u64,
}
/// The output of a batch of ethereum blocks.
#[derive(Debug)]
pub struct BatchBlockExecutionOutput {
/// Bundle state with reverts.
pub bundle: BundleState,
/// The collection of receipts.
/// Outer vector stores receipts for each block sequentially.
/// The inner vector stores receipts ordered by transaction number.
///
/// If receipt is None it means it is pruned.
pub receipts: Receipts,
/// First block of bundle state.
pub first_block: BlockNumber,
}
impl BatchBlockExecutionOutput {
/// Create Bundle State.
pub fn new(bundle: BundleState, receipts: Receipts, first_block: BlockNumber) -> Self {
Self { bundle, receipts, first_block }
}
}
/// A helper type for ethereum block inputs that consists of a block and the total difficulty.
#[derive(Debug)]
pub struct EthBlockExecutionInput<'a, Block> {
pub struct BlockExecutionInput<'a, Block> {
/// The block to execute.
pub block: &'a Block,
/// The total difficulty of the block.
pub total_difficulty: U256,
}
impl<'a, Block> EthBlockExecutionInput<'a, Block> {
impl<'a, Block> BlockExecutionInput<'a, Block> {
/// Creates a new input.
pub fn new(block: &'a Block, total_difficulty: U256) -> Self {
Self { block, total_difficulty }
}
}
impl<'a, Block> From<(&'a Block, U256)> for EthBlockExecutionInput<'a, Block> {
impl<'a, Block> From<(&'a Block, U256)> for BlockExecutionInput<'a, Block> {
fn from((block, total_difficulty): (&'a Block, U256)) -> Self {
Self::new(block, total_difficulty)
}
}
/// A type that can create a new executor.
pub trait ExecutorProvider: Send + Sync + Clone {
/// A type that can create a new executor for block execution.
pub trait BlockExecutorProvider: Send + Sync + Clone + Unpin + 'static {
/// An executor that can execute a single block given a database.
type Executor<DB: Database<Error = ProviderError>>: Executor<DB>;
/// An executor that can execute a batch of blocks given a database.
///
/// # Verification
///
/// The on [Executor::execute] the executor is expected to validate the execution output of the
/// input, this includes:
/// - Cumulative gas used must match the input's gas used.
/// - Receipts must match the input's receipts root.
///
/// 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 = ProviderError>>: for<'a> Executor<
DB,
Input<'a> = BlockExecutionInput<'a, BlockWithSenders>,
Output = BlockExecutionOutput<Receipt>,
Error = BlockExecutionError,
>;
/// An executor that can execute a batch of blocks given a database.
type BatchExecutor<DB: Database<Error = ProviderError>>: for<'a> BatchExecutor<
DB,
Input<'a> = BlockExecutionInput<'a, BlockWithSenders>,
// TODO: change to bundle state with receipts
Output = BatchBlockExecutionOutput,
Error = BlockExecutionError,
>;
type BatchExecutor<DB: Database<Error = ProviderError>>: BatchExecutor<DB>;
/// Creates a new executor for single block execution.
///
/// 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 = ProviderError>;
/// Creates a new batch executor
fn batch_executor<DB>(&self, db: DB) -> Self::BatchExecutor<DB>
/// Creates a new batch executor with the given database and pruning modes.
///
/// Batch executor is used to execute multiple blocks in sequence and keep track of the state
/// during historical sync which involves executing multiple blocks in sequence.
///
/// The pruning modes are used to determine which parts of the state should be kept during
/// execution.
fn batch_executor<DB>(&self, db: DB, prune_modes: PruneModes) -> Self::BatchExecutor<DB>
where
DB: Database<Error = ProviderError>;
}
@ -103,13 +158,14 @@ pub trait ExecutorProvider: Send + Sync + Clone {
#[cfg(test)]
mod tests {
use super::*;
use reth_primitives::Block;
use revm::db::{CacheDB, EmptyDBTyped};
use std::marker::PhantomData;
#[derive(Clone, Default)]
struct TestExecutorProvider;
impl ExecutorProvider for TestExecutorProvider {
impl BlockExecutorProvider for TestExecutorProvider {
type Executor<DB: Database<Error = ProviderError>> = TestExecutor<DB>;
type BatchExecutor<DB: Database<Error = ProviderError>> = TestExecutor<DB>;
@ -120,7 +176,7 @@ mod tests {
TestExecutor(PhantomData)
}
fn batch_executor<DB>(&self, _db: DB) -> Self::BatchExecutor<DB>
fn batch_executor<DB>(&self, _db: DB, _prune_modes: PruneModes) -> Self::BatchExecutor<DB>
where
DB: Database<Error = ProviderError>,
{
@ -131,28 +187,35 @@ mod tests {
struct TestExecutor<DB>(PhantomData<DB>);
impl<DB> Executor<DB> for TestExecutor<DB> {
type Input<'a> = &'static str;
type Output = ();
type Error = String;
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BlockExecutionOutput<Receipt>;
type Error = BlockExecutionError;
fn execute(self, _input: Self::Input<'_>) -> Result<Self::Output, Self::Error> {
Ok(())
Err(BlockExecutionError::UnavailableForTest)
}
}
impl<DB> BatchExecutor<DB> for TestExecutor<DB> {
type Input<'a> = &'static str;
type Output = ();
type Error = String;
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BatchBlockExecutionOutput;
type Error = BlockExecutionError;
fn execute_one(
&mut self,
_input: Self::Input<'_>,
) -> Result<BatchBlockOutput, Self::Error> {
Ok(BatchBlockOutput { size_hint: None })
fn execute_one(&mut self, _input: Self::Input<'_>) -> Result<(), Self::Error> {
Ok(())
}
fn finalize(self) -> Self::Output {}
fn finalize(self) -> Self::Output {
todo!()
}
fn set_tip(&mut self, _tip: BlockNumber) {
todo!()
}
fn size_hint(&self) -> Option<usize> {
None
}
}
#[test]
@ -160,6 +223,9 @@ mod tests {
let provider = TestExecutorProvider;
let db = CacheDB::<EmptyDBTyped<ProviderError>>::default();
let executor = provider.executor(db);
executor.execute("test").unwrap();
let block =
Block { header: Default::default(), body: vec![], ommers: vec![], withdrawals: None };
let block = BlockWithSenders::new(block, Default::default()).unwrap();
let _ = executor.execute(BlockExecutionInput::new(&block, U256::ZERO));
}
}

View File

@ -12,8 +12,13 @@ use reth_primitives::{revm::env::fill_block_env, Address, ChainSpec, Header, Tra
use revm::{inspector_handle_register, Database, Evm, EvmBuilder, GetInspector};
use revm_primitives::{BlockEnv, CfgEnvWithHandlerCfg, EnvWithHandlerCfg, SpecId, TxEnv};
pub mod either;
pub mod execute;
#[cfg(any(test, feature = "test-utils"))]
/// test helpers for mocking executor
pub mod test_utils;
/// Trait for configuring the EVM for executing full blocks.
pub trait ConfigureEvm: ConfigureEvmEnv {
/// Associated type for the default external context that should be configured for the EVM.

View File

@ -0,0 +1,80 @@
//! Helpers for testing.
use crate::execute::{
BatchBlockExecutionOutput, BatchExecutor, BlockExecutionInput, BlockExecutionOutput,
BlockExecutorProvider, Executor,
};
use parking_lot::Mutex;
use reth_interfaces::{executor::BlockExecutionError, provider::ProviderError};
use reth_primitives::{BlockNumber, BlockWithSenders, PruneModes, Receipt};
use revm_primitives::db::Database;
use std::sync::Arc;
/// A [BlockExecutorProvider] that returns mocked execution results.
#[derive(Clone, Debug, Default)]
pub struct MockExecutorProvider {
exec_results: Arc<Mutex<Vec<BatchBlockExecutionOutput>>>,
}
impl MockExecutorProvider {
/// Extend the mocked execution results
pub fn extend(&self, results: impl IntoIterator<Item = impl Into<BatchBlockExecutionOutput>>) {
self.exec_results.lock().extend(results.into_iter().map(Into::into));
}
}
impl BlockExecutorProvider for MockExecutorProvider {
type Executor<DB: Database<Error = ProviderError>> = Self;
type BatchExecutor<DB: Database<Error = ProviderError>> = Self;
fn executor<DB>(&self, _: DB) -> Self::Executor<DB>
where
DB: Database<Error = ProviderError>,
{
self.clone()
}
fn batch_executor<DB>(&self, _: DB, _: PruneModes) -> Self::BatchExecutor<DB>
where
DB: Database<Error = ProviderError>,
{
self.clone()
}
}
impl<DB> Executor<DB> for MockExecutorProvider {
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BlockExecutionOutput<Receipt>;
type Error = BlockExecutionError;
fn execute(self, _: Self::Input<'_>) -> Result<Self::Output, Self::Error> {
let BatchBlockExecutionOutput { bundle, receipts, .. } =
self.exec_results.lock().pop().unwrap();
Ok(BlockExecutionOutput {
state: bundle,
receipts: receipts.into_iter().flatten().flatten().collect(),
gas_used: 0,
})
}
}
impl<DB> BatchExecutor<DB> for MockExecutorProvider {
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BatchBlockExecutionOutput;
type Error = BlockExecutionError;
fn execute_one(&mut self, _: Self::Input<'_>) -> Result<(), Self::Error> {
Ok(())
}
fn finalize(self) -> Self::Output {
self.exec_results.lock().pop().unwrap()
}
fn set_tip(&mut self, _: BlockNumber) {}
fn size_hint(&self) -> Option<usize> {
None
}
}

View File

@ -53,11 +53,20 @@ impl<Node: FullNodeComponents> FullNodeTypes for ExExContext<Node> {
impl<Node: FullNodeComponents> FullNodeComponents for ExExContext<Node> {
type Pool = Node::Pool;
type Evm = Node::Evm;
type Executor = Node::Executor;
fn pool(&self) -> &Self::Pool {
self.components.pool()
}
fn evm_config(&self) -> &Self::Evm {
self.components.evm_config()
}
fn block_executor(&self) -> &Self::Executor {
self.components.block_executor()
}
fn provider(&self) -> &Self::Provider {
self.components.provider()
}
@ -73,8 +82,4 @@ impl<Node: FullNodeComponents> FullNodeComponents for ExExContext<Node> {
fn task_executor(&self) -> &TaskExecutor {
self.components.task_executor()
}
fn evm_config(&self) -> &Self::Evm {
self.components.evm_config()
}
}

View File

@ -1,4 +1,6 @@
//! Ethereum EVM support
#[doc(inline)]
pub use reth_evm_ethereum::execute::EthExecutorProvider;
#[doc(inline)]
pub use reth_evm_ethereum::EthEvmConfig;

View File

@ -11,7 +11,7 @@
pub use reth_ethereum_engine_primitives::EthEngineTypes;
pub mod evm;
pub use evm::EthEvmConfig;
pub use evm::{EthEvmConfig, EthExecutorProvider};
pub mod node;
pub use node::EthereumNode;

View File

@ -2,6 +2,7 @@
use crate::{EthEngineTypes, EthEvmConfig};
use reth_basic_payload_builder::{BasicPayloadJobGenerator, BasicPayloadJobGeneratorConfig};
use reth_evm_ethereum::execute::EthExecutorProvider;
use reth_network::NetworkHandle;
use reth_node_builder::{
components::{
@ -76,9 +77,18 @@ where
Node: FullNodeTypes,
{
type EVM = EthEvmConfig;
type Executor = EthExecutorProvider<Self::EVM>;
async fn build_evm(self, _ctx: &BuilderContext<Node>) -> eyre::Result<Self::EVM> {
Ok(EthEvmConfig::default())
async fn build_evm(
self,
ctx: &BuilderContext<Node>,
) -> eyre::Result<(Self::EVM, Self::Executor)> {
let chain_spec = ctx.chain_spec();
let evm_config = EthEvmConfig::default();
let executor =
EthExecutorProvider::new(chain_spec, evm_config).with_inspector(ctx.inspector_stack());
Ok((evm_config, executor))
}
}

View File

@ -5,6 +5,7 @@ use reth_db::{
database::Database,
database_metrics::{DatabaseMetadata, DatabaseMetrics},
};
use reth_evm::execute::BlockExecutorProvider;
use reth_network::NetworkHandle;
use reth_payload_builder::PayloadBuilderHandle;
use reth_provider::FullProvider;
@ -88,12 +89,18 @@ pub trait FullNodeComponents: FullNodeTypes + 'static {
/// The node's EVM configuration, defining settings for the Ethereum Virtual Machine.
type Evm: ConfigureEvm;
/// The type that knows how to execute blocks.
type Executor: BlockExecutorProvider;
/// Returns the transaction pool of the node.
fn pool(&self) -> &Self::Pool;
/// Returns the node's evm config.
fn evm_config(&self) -> &Self::Evm;
/// Returns the node's executor type.
fn block_executor(&self) -> &Self::Executor;
/// Returns the provider of the node.
fn provider(&self) -> &Self::Provider;

View File

@ -17,6 +17,7 @@ reth-auto-seal-consensus.workspace = true
reth-beacon-consensus.workspace = true
reth-blockchain-tree.workspace = true
reth-exex.workspace = true
reth-evm.workspace = true
reth-provider.workspace = true
reth-revm.workspace = true
reth-db.workspace = true

View File

@ -27,6 +27,7 @@ use reth_node_core::{
};
use reth_primitives::{constants::eip4844::MAINNET_KZG_TRUSTED_SETUP, ChainSpec};
use reth_provider::{providers::BlockchainProvider, ChainSpecProvider};
use reth_revm::stack::{InspectorStack, InspectorStackConfig};
use reth_tasks::TaskExecutor;
use reth_transaction_pool::{PoolConfig, TransactionPool};
pub use states::*;
@ -460,6 +461,28 @@ impl<Node: FullNodeTypes> BuilderContext<Node> {
&self.config
}
/// Returns an inspector stack if configured.
///
/// This can be used to debug block execution.
pub fn inspector_stack(&self) -> Option<InspectorStack> {
use reth_revm::stack::Hook;
let stack_config = InspectorStackConfig {
use_printer_tracer: self.config.debug.print_inspector,
hook: if let Some(hook_block) = self.config.debug.hook_block {
Hook::Block(hook_block)
} else if let Some(tx) = self.config.debug.hook_transaction {
Hook::Transaction(tx)
} else if self.config.debug.hook_all {
Hook::All
} else {
// no inspector
return None
},
};
Some(InspectorStack::new(stack_config))
}
/// Returns the data dir of the node.
///
/// This gives access to all relevant files and directories of the node's datadir.

View File

@ -98,6 +98,7 @@ impl<T: FullNodeTypes, C: NodeComponents<T>> FullNodeTypes for NodeAdapter<T, C>
impl<T: FullNodeTypes, C: NodeComponents<T>> FullNodeComponents for NodeAdapter<T, C> {
type Pool = C::Pool;
type Evm = C::Evm;
type Executor = C::Executor;
fn pool(&self) -> &Self::Pool {
self.components.pool()
@ -107,6 +108,10 @@ impl<T: FullNodeTypes, C: NodeComponents<T>> FullNodeComponents for NodeAdapter<
self.components.evm_config()
}
fn block_executor(&self) -> &Self::Executor {
self.components.block_executor()
}
fn provider(&self) -> &Self::Provider {
&self.provider
}

View File

@ -7,6 +7,7 @@ use crate::{
},
BuilderContext, ConfigureEvm, FullNodeTypes,
};
use reth_evm::execute::BlockExecutorProvider;
use reth_transaction_pool::TransactionPool;
use std::{future::Future, marker::PhantomData};
@ -232,7 +233,7 @@ where
PayloadB: PayloadServiceBuilder<Node, PoolB::Pool>,
ExecB: ExecutorBuilder<Node>,
{
type Components = Components<Node, PoolB::Pool, ExecB::EVM>;
type Components = Components<Node, PoolB::Pool, ExecB::EVM, ExecB::Executor>;
async fn build_components(
self,
@ -246,12 +247,12 @@ where
_marker,
} = self;
let evm_config = evm_builder.build_evm(context).await?;
let (evm_config, executor) = evm_builder.build_evm(context).await?;
let pool = pool_builder.build_pool(context).await?;
let network = network_builder.build_network(context, pool.clone()).await?;
let payload_builder = payload_builder.spawn_payload_service(context, pool.clone()).await?;
Ok(Components { transaction_pool: pool, evm_config, network, payload_builder })
Ok(Components { transaction_pool: pool, evm_config, network, payload_builder, executor })
}
}
@ -287,15 +288,16 @@ pub trait NodeComponentsBuilder<Node: FullNodeTypes>: Send {
) -> impl Future<Output = eyre::Result<Self::Components>> + Send;
}
impl<Node, F, Fut, Pool, EVM> NodeComponentsBuilder<Node> for F
impl<Node, F, Fut, Pool, EVM, Executor> NodeComponentsBuilder<Node> for F
where
Node: FullNodeTypes,
F: FnOnce(&BuilderContext<Node>) -> Fut + Send,
Fut: Future<Output = eyre::Result<Components<Node, Pool, EVM>>> + Send,
Fut: Future<Output = eyre::Result<Components<Node, Pool, EVM, Executor>>> + Send,
Pool: TransactionPool + Unpin + 'static,
EVM: ConfigureEvm,
Executor: BlockExecutorProvider,
{
type Components = Components<Node, Pool, EVM>;
type Components = Components<Node, Pool, EVM, Executor>;
fn build_components(
self,

View File

@ -1,34 +1,41 @@
//! EVM component for the node builder.
use crate::{BuilderContext, FullNodeTypes};
use reth_evm::execute::BlockExecutorProvider;
use reth_node_api::ConfigureEvm;
use std::future::Future;
/// A type that knows how to build the executor types.
pub trait ExecutorBuilder<Node: FullNodeTypes>: Send {
/// The EVM config to build.
/// The EVM config to use.
///
/// This provides the node with the necessary configuration to configure an EVM.
type EVM: ConfigureEvm;
// TODO(mattsse): integrate `Executor`
/// The type that knows how to execute blocks.
type Executor: BlockExecutorProvider;
/// Creates the EVM config.
fn build_evm(
self,
ctx: &BuilderContext<Node>,
) -> impl Future<Output = eyre::Result<Self::EVM>> + Send;
) -> impl Future<Output = eyre::Result<(Self::EVM, Self::Executor)>> + Send;
}
impl<Node, F, Fut, EVM> ExecutorBuilder<Node> for F
impl<Node, F, Fut, EVM, Executor> ExecutorBuilder<Node> for F
where
Node: FullNodeTypes,
EVM: ConfigureEvm,
Executor: BlockExecutorProvider,
F: FnOnce(&BuilderContext<Node>) -> Fut + Send,
Fut: Future<Output = eyre::Result<EVM>> + Send,
Fut: Future<Output = eyre::Result<(EVM, Executor)>> + Send,
{
type EVM = EVM;
type Executor = Executor;
fn build_evm(
self,
ctx: &BuilderContext<Node>,
) -> impl Future<Output = eyre::Result<Self::EVM>> {
) -> impl Future<Output = eyre::Result<(Self::EVM, Self::Executor)>> {
self(ctx)
}
}

View File

@ -13,6 +13,7 @@ pub use execute::*;
pub use network::*;
pub use payload::*;
pub use pool::*;
use reth_evm::execute::BlockExecutorProvider;
use reth_network::NetworkHandle;
use reth_payload_builder::PayloadBuilderHandle;
use reth_transaction_pool::TransactionPool;
@ -35,12 +36,18 @@ pub trait NodeComponents<NodeTypes: FullNodeTypes>: Clone + Send + Sync + 'stati
/// The node's EVM configuration, defining settings for the Ethereum Virtual Machine.
type Evm: ConfigureEvm;
/// The type that knows how to execute blocks.
type Executor: BlockExecutorProvider;
/// Returns the transaction pool of the node.
fn pool(&self) -> &Self::Pool;
/// Returns the node's evm config.
fn evm_config(&self) -> &Self::Evm;
/// Returns the node's executor type.
fn block_executor(&self) -> &Self::Executor;
/// Returns the handle to the network
fn network(&self) -> &NetworkHandle;
@ -52,25 +59,29 @@ pub trait NodeComponents<NodeTypes: FullNodeTypes>: Clone + Send + Sync + 'stati
///
/// This provides access to all the components of the node.
#[derive(Debug)]
pub struct Components<Node: FullNodeTypes, Pool, EVM> {
pub struct Components<Node: FullNodeTypes, Pool, EVM, Executor> {
/// The transaction pool of the node.
pub transaction_pool: Pool,
/// The node's EVM configuration, defining settings for the Ethereum Virtual Machine.
pub evm_config: EVM,
/// The node's executor type used to execute individual blocks and batches of blocks.
pub executor: Executor,
/// The network implementation of the node.
pub network: NetworkHandle,
/// The handle to the payload builder service.
pub payload_builder: PayloadBuilderHandle<Node::Engine>,
}
impl<Node, Pool, EVM> NodeComponents<Node> for Components<Node, Pool, EVM>
impl<Node, Pool, EVM, Executor> NodeComponents<Node> for Components<Node, Pool, EVM, Executor>
where
Node: FullNodeTypes,
Pool: TransactionPool + Unpin + 'static,
EVM: ConfigureEvm,
Executor: BlockExecutorProvider,
{
type Pool = Pool;
type Evm = EVM;
type Executor = Executor;
fn pool(&self) -> &Self::Pool {
&self.transaction_pool
@ -80,6 +91,10 @@ where
&self.evm_config
}
fn block_executor(&self) -> &Self::Executor {
&self.executor
}
fn network(&self) -> &NetworkHandle {
&self.network
}
@ -89,16 +104,18 @@ where
}
}
impl<Node, Pool, EVM> Clone for Components<Node, Pool, EVM>
impl<Node, Pool, EVM, Executor> Clone for Components<Node, Pool, EVM, Executor>
where
Node: FullNodeTypes,
Pool: TransactionPool,
EVM: ConfigureEvm,
Executor: BlockExecutorProvider,
{
fn clone(&self) -> Self {
Self {
transaction_pool: self.transaction_pool.clone(),
evm_config: self.evm_config.clone(),
executor: self.executor.clone(),
network: self.network.clone(),
payload_builder: self.payload_builder.clone(),
}

View File

@ -30,7 +30,6 @@ use reth_node_core::{
use reth_node_events::{cl::ConsensusLayerHealthEvents, node};
use reth_primitives::format_ether;
use reth_provider::{providers::BlockchainProvider, CanonStateSubscriptions};
use reth_revm::EvmProcessorFactory;
use reth_rpc_engine_api::EngineApi;
use reth_tasks::TaskExecutor;
use reth_tracing::tracing::{debug, info};
@ -157,7 +156,7 @@ where
let tree_externals = TreeExternals::new(
ctx.provider_factory().clone(),
consensus.clone(),
EvmProcessorFactory::new(ctx.chain_spec(), components.evm_config().clone()),
components.block_executor().clone(),
);
let tree = BlockchainTree::new(tree_externals, tree_config, ctx.prune_modes())?
.with_sync_metrics_tx(sync_metrics_tx.clone())
@ -303,7 +302,7 @@ where
consensus_engine_tx.clone(),
canon_state_notification_sender,
mining_mode,
node_adapter.components.evm_config().clone(),
node_adapter.components.block_executor().clone(),
)
.build();
@ -318,7 +317,7 @@ where
ctx.prune_config(),
max_block,
static_file_producer,
node_adapter.components.evm_config().clone(),
node_adapter.components.block_executor().clone(),
pipeline_exex_handle,
)
.await?;
@ -341,7 +340,7 @@ where
ctx.prune_config(),
max_block,
static_file_producer,
node_adapter.components.evm_config().clone(),
node_adapter.components.block_executor().clone(),
pipeline_exex_handle,
)
.await?;

View File

@ -1,6 +1,5 @@
//! Helpers for setting up parts of the node.
use crate::ConfigureEvm;
use reth_config::{config::StageConfig, PruneConfig};
use reth_consensus::Consensus;
use reth_db::database::Database;
@ -8,6 +7,7 @@ use reth_downloaders::{
bodies::bodies::BodiesDownloaderBuilder,
headers::reverse_headers::ReverseHeadersDownloaderBuilder,
};
use reth_evm::execute::BlockExecutorProvider;
use reth_exex::ExExManagerHandle;
use reth_interfaces::p2p::{
bodies::{client::BodiesClient, downloader::BodyDownloader},
@ -18,7 +18,6 @@ use reth_node_core::{
primitives::{BlockNumber, B256},
};
use reth_provider::{HeaderSyncMode, ProviderFactory};
use reth_revm::stack::{Hook, InspectorStackConfig};
use reth_stages::{
prelude::DefaultStages,
stages::{
@ -36,7 +35,7 @@ use tokio::sync::watch;
/// Constructs a [Pipeline] that's wired to the network
#[allow(clippy::too_many_arguments)]
pub async fn build_networked_pipeline<DB, Client, EvmConfig>(
pub async fn build_networked_pipeline<DB, Client, Executor>(
node_config: &NodeConfig,
config: &StageConfig,
client: Client,
@ -47,13 +46,13 @@ pub async fn build_networked_pipeline<DB, Client, EvmConfig>(
prune_config: Option<PruneConfig>,
max_block: Option<BlockNumber>,
static_file_producer: StaticFileProducer<DB>,
evm_config: EvmConfig,
executor: Executor,
exex_manager_handle: ExExManagerHandle,
) -> eyre::Result<Pipeline<DB>>
where
DB: Database + Unpin + Clone + 'static,
Client: HeadersClient + BodiesClient + Clone + 'static,
EvmConfig: ConfigureEvm + Clone + 'static,
Executor: BlockExecutorProvider,
{
// building network downloaders using the fetch client
let header_downloader = ReverseHeadersDownloaderBuilder::new(config.headers)
@ -75,7 +74,7 @@ where
metrics_tx,
prune_config,
static_file_producer,
evm_config,
executor,
exex_manager_handle,
)
.await?;
@ -85,7 +84,7 @@ where
/// Builds the [Pipeline] with the given [ProviderFactory] and downloaders.
#[allow(clippy::too_many_arguments)]
pub async fn build_pipeline<DB, H, B, EvmConfig>(
pub async fn build_pipeline<DB, H, B, Executor>(
node_config: &NodeConfig,
provider_factory: ProviderFactory<DB>,
stage_config: &StageConfig,
@ -96,14 +95,14 @@ pub async fn build_pipeline<DB, H, B, EvmConfig>(
metrics_tx: reth_stages::MetricEventsSender,
prune_config: Option<PruneConfig>,
static_file_producer: StaticFileProducer<DB>,
evm_config: EvmConfig,
executor: Executor,
exex_manager_handle: ExExManagerHandle,
) -> eyre::Result<Pipeline<DB>>
where
DB: Database + Clone + 'static,
H: HeaderDownloader + 'static,
B: BodyDownloader + 'static,
EvmConfig: ConfigureEvm + Clone + 'static,
Executor: BlockExecutorProvider,
{
let mut builder = Pipeline::builder();
@ -113,22 +112,6 @@ where
}
let (tip_tx, tip_rx) = watch::channel(B256::ZERO);
let factory = reth_revm::EvmProcessorFactory::new(node_config.chain.clone(), evm_config);
let stack_config = InspectorStackConfig {
use_printer_tracer: node_config.debug.print_inspector,
hook: if let Some(hook_block) = node_config.debug.hook_block {
Hook::Block(hook_block)
} else if let Some(tx) = node_config.debug.hook_transaction {
Hook::Transaction(tx)
} else if node_config.debug.hook_all {
Hook::All
} else {
Hook::None
},
};
let factory = factory.with_stack_config(stack_config);
let prune_modes = prune_config.map(|prune| prune.segments).unwrap_or_default();
@ -147,7 +130,7 @@ where
Arc::clone(&consensus),
header_downloader,
body_downloader,
factory.clone(),
executor.clone(),
stage_config.etl.clone(),
)
.set(SenderRecoveryStage {
@ -155,7 +138,7 @@ where
})
.set(
ExecutionStage::new(
factory,
executor,
ExecutionStageThresholds {
max_blocks: stage_config.execution.max_blocks,
max_changes: stage_config.execution.max_changes,

View File

@ -19,6 +19,7 @@ reth-interfaces.workspace = true
reth-provider.workspace = true
# Optimism
revm.workspace = true
revm-primitives.workspace = true
# misc

View File

@ -1,10 +1,10 @@
//! Optimism block executor.
use crate::OptimismEvmConfig;
use crate::{l1::ensure_create2_deployer, verify::verify_receipts, OptimismEvmConfig};
use reth_evm::{
execute::{
BatchBlockOutput, BatchExecutor, EthBlockExecutionInput, EthBlockOutput, Executor,
ExecutorProvider,
BatchBlockExecutionOutput, BatchExecutor, BlockExecutionInput, BlockExecutionOutput,
BlockExecutorProvider, Executor,
},
ConfigureEvm, ConfigureEvmEnv,
};
@ -13,16 +13,12 @@ use reth_interfaces::{
provider::ProviderError,
};
use reth_primitives::{
proofs::calculate_receipt_root_optimism, BlockWithSenders, Bloom, Bytes, ChainSpec,
GotExpected, Hardfork, Header, PruneModes, Receipt, ReceiptWithBloom, Receipts, TxType,
Withdrawals, B256, U256,
BlockNumber, BlockWithSenders, Bytes, ChainSpec, GotExpected, Hardfork, Header, PruneModes,
Receipt, Receipts, TxType, Withdrawals, U256,
};
use reth_provider::BundleStateWithReceipts;
use reth_revm::{
batch::{BlockBatchRecord, BlockExecutorStats},
db::states::bundle_state::BundleRetention,
optimism::ensure_create2_deployer,
processor::compare_receipts_root_and_logs_bloom,
stack::InspectorStack,
state_change::{apply_beacon_root_contract_call, post_block_balance_increments},
Evm, State,
@ -36,14 +32,13 @@ use tracing::{debug, trace};
/// Provides executors to execute regular ethereum blocks
#[derive(Debug, Clone)]
pub struct OpExecutorProvider<EvmConfig> {
pub struct OpExecutorProvider<EvmConfig = OptimismEvmConfig> {
chain_spec: Arc<ChainSpec>,
evm_config: EvmConfig,
inspector: Option<InspectorStack>,
prune_modes: PruneModes,
}
impl OpExecutorProvider<OptimismEvmConfig> {
impl OpExecutorProvider {
/// Creates a new default optimism executor provider.
pub fn optimism(chain_spec: Arc<ChainSpec>) -> Self {
Self::new(chain_spec, Default::default())
@ -53,7 +48,7 @@ impl OpExecutorProvider<OptimismEvmConfig> {
impl<EvmConfig> OpExecutorProvider<EvmConfig> {
/// Creates a new executor provider.
pub fn new(chain_spec: Arc<ChainSpec>, evm_config: EvmConfig) -> Self {
Self { chain_spec, evm_config, inspector: None, prune_modes: PruneModes::none() }
Self { chain_spec, evm_config, inspector: None }
}
/// Configures an optional inspector stack for debugging.
@ -61,12 +56,6 @@ impl<EvmConfig> OpExecutorProvider<EvmConfig> {
self.inspector = inspector;
self
}
/// Configures the prune modes for the executor.
pub fn with_prune_modes(mut self, prune_modes: PruneModes) -> Self {
self.prune_modes = prune_modes;
self
}
}
impl<EvmConfig> OpExecutorProvider<EvmConfig>
@ -87,7 +76,7 @@ where
}
}
impl<EvmConfig> ExecutorProvider for OpExecutorProvider<EvmConfig>
impl<EvmConfig> BlockExecutorProvider for OpExecutorProvider<EvmConfig>
where
EvmConfig: ConfigureEvm,
EvmConfig: ConfigureEvmEnv<TxMeta = Bytes>,
@ -102,14 +91,14 @@ where
self.op_executor(db)
}
fn batch_executor<DB>(&self, db: DB) -> Self::BatchExecutor<DB>
fn batch_executor<DB>(&self, db: DB, prune_modes: PruneModes) -> Self::BatchExecutor<DB>
where
DB: Database<Error = ProviderError>,
{
let executor = self.op_executor(db);
OpBatchExecutor {
executor,
batch_record: BlockBatchRecord::new(self.prune_modes.clone()),
batch_record: BlockBatchRecord::new(prune_modes),
stats: BlockExecutorStats::default(),
}
}
@ -370,7 +359,7 @@ where
// transaction This was replaced with is_success flag.
// See more about EIP here: https://eips.ethereum.org/EIPS/eip-658
if self.chain_spec().is_byzantium_active_at_block(block.header.number) {
if let Err(error) = verify_receipt_optimism(
if let Err(error) = verify_receipts(
block.header.receipts_root,
block.header.logs_bloom,
receipts.iter(),
@ -424,8 +413,8 @@ where
EvmConfig: ConfigureEvmEnv<TxMeta = Bytes>,
DB: Database<Error = ProviderError>,
{
type Input<'a> = EthBlockExecutionInput<'a, BlockWithSenders>;
type Output = EthBlockOutput<Receipt>;
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BlockExecutionOutput<Receipt>;
type Error = BlockExecutionError;
/// Executes the block and commits the state changes.
@ -436,13 +425,13 @@ where
///
/// State changes are committed to the database.
fn execute(mut self, input: Self::Input<'_>) -> Result<Self::Output, Self::Error> {
let EthBlockExecutionInput { block, total_difficulty } = input;
let BlockExecutionInput { block, total_difficulty } = input;
let (receipts, gas_used) = self.execute_and_verify(block, total_difficulty)?;
// prepare the state for extraction
self.state.merge_transitions(BundleRetention::PlainState);
// NOTE: we need to merge keep the reverts for the bundle retention
self.state.merge_transitions(BundleRetention::Reverts);
Ok(EthBlockOutput { state: self.state.take_bundle(), receipts, gas_used })
Ok(BlockExecutionOutput { state: self.state.take_bundle(), receipts, gas_used })
}
}
@ -478,12 +467,12 @@ where
EvmConfig: ConfigureEvmEnv<TxMeta = Bytes>,
DB: Database<Error = ProviderError>,
{
type Input<'a> = EthBlockExecutionInput<'a, BlockWithSenders>;
type Output = BundleStateWithReceipts;
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BatchBlockExecutionOutput;
type Error = BlockExecutionError;
fn execute_one(&mut self, input: Self::Input<'_>) -> Result<BatchBlockOutput, Self::Error> {
let EthBlockExecutionInput { block, total_difficulty } = input;
fn execute_one(&mut self, input: Self::Input<'_>) -> Result<(), Self::Error> {
let BlockExecutionInput { block, total_difficulty } = input;
let (receipts, _gas_used) = self.executor.execute_and_verify(block, total_difficulty)?;
// prepare the state according to the prune mode
@ -493,45 +482,30 @@ where
// store receipts in the set
self.batch_record.save_receipts(receipts)?;
Ok(BatchBlockOutput { size_hint: Some(self.executor.state.bundle_size_hint()) })
if self.batch_record.first_block().is_none() {
self.batch_record.set_first_block(block.number);
}
Ok(())
}
fn finalize(mut self) -> Self::Output {
// TODO: track stats
self.stats.log_debug();
BundleStateWithReceipts::new(
BatchBlockExecutionOutput::new(
self.executor.state.take_bundle(),
self.batch_record.take_receipts(),
self.batch_record.first_block().unwrap_or_default(),
)
}
}
/// Verify the calculated receipts root against the expected receipts root.
pub fn verify_receipt_optimism<'a>(
expected_receipts_root: B256,
expected_logs_bloom: Bloom,
receipts: impl Iterator<Item = &'a Receipt> + Clone,
chain_spec: &ChainSpec,
timestamp: u64,
) -> Result<(), BlockExecutionError> {
// Calculate receipts root.
let receipts_with_bloom = receipts.map(|r| r.clone().into()).collect::<Vec<ReceiptWithBloom>>();
let receipts_root =
calculate_receipt_root_optimism(&receipts_with_bloom, chain_spec, timestamp);
fn set_tip(&mut self, tip: BlockNumber) {
self.batch_record.set_tip(tip);
}
// Create header log bloom.
let logs_bloom = receipts_with_bloom.iter().fold(Bloom::ZERO, |bloom, r| bloom | r.bloom);
compare_receipts_root_and_logs_bloom(
receipts_root,
logs_bloom,
expected_receipts_root,
expected_logs_bloom,
)?;
Ok(())
fn size_hint(&self) -> Option<usize> {
Some(self.executor.state.bundle_state.size_hint())
}
}
#[cfg(test)]
@ -574,12 +548,7 @@ mod tests {
}
fn executor_provider(chain_spec: Arc<ChainSpec>) -> OpExecutorProvider<OptimismEvmConfig> {
OpExecutorProvider {
chain_spec,
evm_config: Default::default(),
inspector: None,
prune_modes: Default::default(),
}
OpExecutorProvider { chain_spec, evm_config: Default::default(), inspector: None }
}
#[test]
@ -626,7 +595,8 @@ mod tests {
);
let provider = executor_provider(chain_spec);
let mut executor = provider.batch_executor(StateProviderDatabase::new(&db));
let mut executor =
provider.batch_executor(StateProviderDatabase::new(&db), PruneModes::none());
executor.state_mut().load_cache_account(L1_BLOCK_CONTRACT).unwrap();
@ -706,7 +676,8 @@ mod tests {
);
let provider = executor_provider(chain_spec);
let mut executor = provider.batch_executor(StateProviderDatabase::new(&db));
let mut executor =
provider.batch_executor(StateProviderDatabase::new(&db), PruneModes::none());
executor.state_mut().load_cache_account(L1_BLOCK_CONTRACT).unwrap();

View File

@ -1,3 +1,5 @@
//! Optimism-specific implementation and utilities for the executor
use reth_interfaces::{
executor::{self as reth_executor, BlockExecutionError},
RethError,
@ -10,14 +12,13 @@ use revm::{
use std::sync::Arc;
use tracing::trace;
/// Optimism-specific processor implementation for the `EVMProcessor`
pub mod processor;
/// The address of the create2 deployer
const CREATE_2_DEPLOYER_ADDR: Address = address!("13b0D85CcB8bf860b6b79AF3029fCA081AE9beF2");
/// The codehash of the create2 deployer contract.
const CREATE_2_DEPLOYER_CODEHASH: B256 =
b256!("b0550b5b431e30d38000efb7107aaa0ade03d48a7198a140edda9d27134468b2");
/// The raw bytecode of the create2 deployer contract.
const CREATE_2_DEPLOYER_BYTECODE: [u8; 1584] = hex!("6080604052600436106100435760003560e01c8063076c37b21461004f578063481286e61461007157806356299481146100ba57806366cfa057146100da57600080fd5b3661004a57005b600080fd5b34801561005b57600080fd5b5061006f61006a366004610327565b6100fa565b005b34801561007d57600080fd5b5061009161008c366004610327565b61014a565b60405173ffffffffffffffffffffffffffffffffffffffff909116815260200160405180910390f35b3480156100c657600080fd5b506100916100d5366004610349565b61015d565b3480156100e657600080fd5b5061006f6100f53660046103ca565b610172565b61014582826040518060200161010f9061031a565b7fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffe082820381018352601f90910116604052610183565b505050565b600061015683836102e7565b9392505050565b600061016a8484846102f0565b949350505050565b61017d838383610183565b50505050565b6000834710156101f4576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601d60248201527f437265617465323a20696e73756666696369656e742062616c616e636500000060448201526064015b60405180910390fd5b815160000361025f576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820181905260248201527f437265617465323a2062797465636f6465206c656e677468206973207a65726f60448201526064016101eb565b8282516020840186f5905073ffffffffffffffffffffffffffffffffffffffff8116610156576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601960248201527f437265617465323a204661696c6564206f6e206465706c6f790000000000000060448201526064016101eb565b60006101568383305b6000604051836040820152846020820152828152600b8101905060ff815360559020949350505050565b61014e806104ad83390190565b6000806040838503121561033a57600080fd5b50508035926020909101359150565b60008060006060848603121561035e57600080fd5b8335925060208401359150604084013573ffffffffffffffffffffffffffffffffffffffff8116811461039057600080fd5b809150509250925092565b7f4e487b7100000000000000000000000000000000000000000000000000000000600052604160045260246000fd5b6000806000606084860312156103df57600080fd5b8335925060208401359150604084013567ffffffffffffffff8082111561040557600080fd5b818601915086601f83011261041957600080fd5b81358181111561042b5761042b61039b565b604051601f82017fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffe0908116603f011681019083821181831017156104715761047161039b565b8160405282815289602084870101111561048a57600080fd5b826020860160208301376000602084830101528095505050505050925092509256fe608060405234801561001057600080fd5b5061012e806100206000396000f3fe6080604052348015600f57600080fd5b506004361060285760003560e01c8063249cb3fa14602d575b600080fd5b603c603836600460b1565b604e565b60405190815260200160405180910390f35b60008281526020818152604080832073ffffffffffffffffffffffffffffffffffffffff8516845290915281205460ff16608857600060aa565b7fa2ef4600d742022d532d4747cb3547474667d6f13804902513b2ec01c848f4b45b9392505050565b6000806040838503121560c357600080fd5b82359150602083013573ffffffffffffffffffffffffffffffffffffffff8116811460ed57600080fd5b80915050925092905056fea26469706673582212205ffd4e6cede7d06a5daf93d48d0541fc68189eeb16608c1999a82063b666eb1164736f6c63430008130033a2646970667358221220fdc4a0fe96e3b21c108ca155438d37c9143fb01278a3c1d274948bad89c564ba64736f6c63430008130033");
@ -75,21 +76,21 @@ pub fn parse_l1_info_tx_bedrock(data: &[u8]) -> Result<L1BlockInfo, BlockExecuti
}
let l1_base_fee = U256::try_from_be_slice(&data[64..96]).ok_or(
reth_executor::BlockExecutionError::OptimismBlockExecution(
BlockExecutionError::OptimismBlockExecution(
reth_executor::OptimismBlockExecutionError::L1BlockInfoError {
message: "could not convert l1 base fee".to_string(),
},
),
)?;
let l1_fee_overhead = U256::try_from_be_slice(&data[192..224]).ok_or(
reth_executor::BlockExecutionError::OptimismBlockExecution(
BlockExecutionError::OptimismBlockExecution(
reth_executor::OptimismBlockExecutionError::L1BlockInfoError {
message: "could not convert l1 fee overhead".to_string(),
},
),
)?;
let l1_fee_scalar = U256::try_from_be_slice(&data[224..256]).ok_or(
reth_executor::BlockExecutionError::OptimismBlockExecution(
BlockExecutionError::OptimismBlockExecution(
reth_executor::OptimismBlockExecutionError::L1BlockInfoError {
message: "could not convert l1 fee scalar".to_string(),
},
@ -288,10 +289,11 @@ where
}
#[cfg(test)]
mod test_l1_fee {
mod tests {
use super::*;
#[test]
fn sanity_l1_block() {
use super::*;
use reth_primitives::{hex_literal::hex, Bytes, Header, TransactionSigned};
let bytes = Bytes::from_static(&hex!("7ef9015aa044bae9d41b8380d781187b426c6fe43df5fb2fb57bd4466ef6a701e1f01e015694deaddeaddeaddeaddeaddeaddeaddeaddead000194420000000000000000000000000000000000001580808408f0d18001b90104015d8eb900000000000000000000000000000000000000000000000000000000008057650000000000000000000000000000000000000000000000000000000063d96d10000000000000000000000000000000000000000000000000000000000009f35273d89754a1e0387b89520d989d3be9c37c1f32495a88faf1ea05c61121ab0d1900000000000000000000000000000000000000000000000000000000000000010000000000000000000000002d679b567db6187c0c8323fa982cfb88b74dbcc7000000000000000000000000000000000000000000000000000000000000083400000000000000000000000000000000000000000000000000000000000f4240"));
@ -303,7 +305,7 @@ mod test_l1_fee {
withdrawals: None,
};
let l1_info: L1BlockInfo = super::extract_l1_info(&mock_block).unwrap();
let l1_info: L1BlockInfo = extract_l1_info(&mock_block).unwrap();
assert_eq!(l1_info.l1_base_fee, U256::from(652_114));
assert_eq!(l1_info.l1_fee_overhead, Some(U256::from(2100)));
assert_eq!(l1_info.l1_base_fee_scalar, U256::from(1_000_000));
@ -313,7 +315,6 @@ mod test_l1_fee {
#[test]
fn sanity_l1_block_ecotone() {
use super::*;
use reth_primitives::{hex_literal::hex, Bytes, Header, TransactionSigned};
let bytes = Bytes::from_static(&hex!("7ef8f8a0b84fa363879a2159e341c50a32da3ea0d21765b7bd43db37f2e5e04e8848b1ee94deaddeaddeaddeaddeaddeaddeaddeaddead00019442000000000000000000000000000000000000158080830f424080b8a4440a5e20000f42400000000000000000000000040000000065c41f680000000000a03f6b00000000000000000000000000000000000000000000000000000000000000080000000000000000000000000000000000000000000000000000000535f4d983dea59eac60478a64ecfdcde8571e611404295350de7ed4ccb404296c1a84ab7a00000000000000000000000073b4168cc87f35cc239200a20eb841cded23493b"));
@ -325,7 +326,7 @@ mod test_l1_fee {
withdrawals: None,
};
let l1_info: L1BlockInfo = super::extract_l1_info(&mock_block).unwrap();
let l1_info: L1BlockInfo = extract_l1_info(&mock_block).unwrap();
assert_eq!(l1_info.l1_base_fee, U256::from(8));
assert_eq!(l1_info.l1_base_fee_scalar, U256::from(4));
assert_eq!(l1_info.l1_blob_base_fee, Some(U256::from(22_380_075_395u64)));

View File

@ -19,6 +19,10 @@ use reth_revm::{inspector_handle_register, Database, Evm, EvmBuilder, GetInspect
mod execute;
pub use execute::*;
pub mod l1;
pub use l1::*;
pub mod verify;
/// Optimism-related EVM configuration.
#[derive(Debug, Default, Clone, Copy)]

View File

@ -0,0 +1,58 @@
//! Helpers for verifying the receipts.
use reth_interfaces::executor::{BlockExecutionError, BlockValidationError};
use reth_primitives::{
proofs::calculate_receipt_root_optimism, Bloom, ChainSpec, GotExpected, Receipt,
ReceiptWithBloom, B256,
};
/// Verify the calculated receipts root against the expected receipts root.
pub fn verify_receipts<'a>(
expected_receipts_root: B256,
expected_logs_bloom: Bloom,
receipts: impl Iterator<Item = &'a Receipt> + Clone,
chain_spec: &ChainSpec,
timestamp: u64,
) -> Result<(), BlockExecutionError> {
// Calculate receipts root.
let receipts_with_bloom = receipts.map(|r| r.clone().into()).collect::<Vec<ReceiptWithBloom>>();
let receipts_root =
calculate_receipt_root_optimism(&receipts_with_bloom, chain_spec, timestamp);
// Create header log bloom.
let logs_bloom = receipts_with_bloom.iter().fold(Bloom::ZERO, |bloom, r| bloom | r.bloom);
compare_receipts_root_and_logs_bloom(
receipts_root,
logs_bloom,
expected_receipts_root,
expected_logs_bloom,
)?;
Ok(())
}
/// Compare the calculated receipts root with the expected receipts root, also compare
/// the calculated logs bloom with the expected logs bloom.
pub fn compare_receipts_root_and_logs_bloom(
calculated_receipts_root: B256,
calculated_logs_bloom: Bloom,
expected_receipts_root: B256,
expected_logs_bloom: Bloom,
) -> Result<(), BlockExecutionError> {
if calculated_receipts_root != expected_receipts_root {
return Err(BlockValidationError::ReceiptRootDiff(
GotExpected { got: calculated_receipts_root, expected: expected_receipts_root }.into(),
)
.into())
}
if calculated_logs_bloom != expected_logs_bloom {
return Err(BlockValidationError::BloomLogDiff(
GotExpected { got: calculated_logs_bloom, expected: expected_logs_bloom }.into(),
)
.into())
}
Ok(())
}

View File

@ -7,7 +7,7 @@ use crate::{
};
use reth_basic_payload_builder::{BasicPayloadJobGenerator, BasicPayloadJobGeneratorConfig};
use reth_evm::ConfigureEvm;
use reth_evm_optimism::OptimismEvmConfig;
use reth_evm_optimism::{OpExecutorProvider, OptimismEvmConfig};
use reth_network::{NetworkHandle, NetworkManager};
use reth_node_builder::{
components::{
@ -97,9 +97,18 @@ where
Node: FullNodeTypes,
{
type EVM = OptimismEvmConfig;
type Executor = OpExecutorProvider<Self::EVM>;
async fn build_evm(self, _ctx: &BuilderContext<Node>) -> eyre::Result<Self::EVM> {
Ok(OptimismEvmConfig::default())
async fn build_evm(
self,
ctx: &BuilderContext<Node>,
) -> eyre::Result<(Self::EVM, Self::Executor)> {
let chain_spec = ctx.chain_spec();
let evm_config = OptimismEvmConfig::default();
let executor =
OpExecutorProvider::new(chain_spec, evm_config).with_inspector(ctx.inspector_stack());
Ok((evm_config, executor))
}
}

View File

@ -1,8 +1,9 @@
//! OP transaction pool types
use parking_lot::RwLock;
use reth_evm_optimism::RethL1BlockInfo;
use reth_primitives::{Block, ChainSpec, GotExpected, InvalidTransactionError, SealedBlock};
use reth_provider::{BlockReaderIdExt, StateProviderFactory};
use reth_revm::{optimism::RethL1BlockInfo, L1BlockInfo};
use reth_revm::L1BlockInfo;
use reth_transaction_pool::{
CoinbaseTipOrdering, EthPoolTransaction, EthPooledTransaction, EthTransactionValidator, Pool,
TransactionOrigin, TransactionValidationOutcome, TransactionValidationTaskExecutor,
@ -75,7 +76,7 @@ where
/// Update the L1 block info.
fn update_l1_block_info(&self, block: &Block) {
self.block_info.timestamp.store(block.timestamp, Ordering::Relaxed);
if let Ok(cost_addition) = reth_revm::optimism::extract_l1_info(block) {
if let Ok(cost_addition) = reth_evm_optimism::extract_l1_info(block) {
*self.block_info.l1_block_info.write() = cost_addition;
}
}

View File

@ -21,6 +21,7 @@ reth-rpc-types.workspace = true
reth-rpc-types-compat.workspace = true
reth-engine-primitives.workspace = true
reth-evm.workspace = true
reth-evm-optimism.workspace = true
reth-payload-builder.workspace = true
reth-basic-payload-builder.workspace = true
@ -39,4 +40,5 @@ optimism = [
"reth-revm/optimism",
"reth-provider/optimism",
"reth-rpc-types-compat/optimism",
"reth-evm-optimism/optimism",
]

View File

@ -303,7 +303,7 @@ where
// blocks will always have at least a single transaction in them (the L1 info transaction),
// so we can safely assume that this will always be triggered upon the transition and that
// the above check for empty blocks will never be hit on OP chains.
reth_revm::optimism::ensure_create2_deployer(
reth_evm_optimism::ensure_create2_deployer(
chain_spec.clone(),
attributes.payload_attributes.timestamp,
&mut db,

View File

@ -17,7 +17,7 @@ reth-primitives.workspace = true
reth-interfaces.workspace = true
reth-provider.workspace = true
reth-consensus-common.workspace = true
reth-evm.workspace = true
reth-evm = { workspace = true, optional = true }
reth-trie = { workspace = true, optional = true }
# revm
@ -28,10 +28,11 @@ revm-inspectors.workspace = true
tracing.workspace = true
[dev-dependencies]
reth-evm.workspace = true
reth-trie.workspace = true
[features]
test-utils = ["dep:reth-trie"]
test-utils = ["dep:reth-trie", "dep:reth-evm"]
optimism = [
"revm/optimism",
"reth-primitives/optimism",

View File

@ -1,56 +0,0 @@
use crate::{
database::StateProviderDatabase,
processor::EVMProcessor,
stack::{InspectorStack, InspectorStackConfig},
};
use reth_evm::ConfigureEvm;
use reth_interfaces::executor::BlockExecutionError;
use reth_primitives::ChainSpec;
use reth_provider::{ExecutorFactory, PrunableBlockExecutor, StateProvider};
use std::sync::Arc;
/// Factory for creating [EVMProcessor].
#[derive(Clone, Debug)]
pub struct EvmProcessorFactory<EvmConfig> {
chain_spec: Arc<ChainSpec>,
stack: Option<InspectorStack>,
/// Type that defines how the produced EVM should be configured.
evm_config: EvmConfig,
}
impl<EvmConfig> EvmProcessorFactory<EvmConfig> {
/// Create new factory
pub fn new(chain_spec: Arc<ChainSpec>, evm_config: EvmConfig) -> Self {
Self { chain_spec, stack: None, evm_config }
}
/// Sets the inspector stack for all generated executors.
pub fn with_stack(mut self, stack: InspectorStack) -> Self {
self.stack = Some(stack);
self
}
/// Sets the inspector stack for all generated executors using the provided config.
pub fn with_stack_config(mut self, config: InspectorStackConfig) -> Self {
self.stack = Some(InspectorStack::new(config));
self
}
}
impl<EvmConfig> ExecutorFactory for EvmProcessorFactory<EvmConfig>
where
EvmConfig: ConfigureEvm + Send + Sync + Clone + 'static,
{
fn with_state<'a, SP: StateProvider + 'a>(
&'a self,
sp: SP,
) -> Box<dyn PrunableBlockExecutor<Error = BlockExecutionError> + 'a> {
let database_state = StateProviderDatabase::new(sp);
let mut evm =
EVMProcessor::new_with_db(self.chain_spec.clone(), database_state, &self.evm_config);
if let Some(stack) = &self.stack {
evm.set_stack(stack.clone());
}
Box::new(evm)
}
}

View File

@ -11,20 +11,11 @@
/// Contains glue code for integrating reth database into revm's [Database].
pub mod database;
/// revm implementation of reth block and transaction executors.
mod factory;
pub mod batch;
/// new revm account state executor
pub mod processor;
/// State changes that are not related to transactions.
pub mod state_change;
/// revm executor factory.
pub use factory::EvmProcessorFactory;
/// Ethereum DAO hardfork state change data.
pub mod eth_dao_fork;
@ -33,10 +24,6 @@ pub mod eth_dao_fork;
/// used in the main Reth executor.
pub mod stack;
/// Optimism-specific implementation and utilities for the executor
#[cfg(feature = "optimism")]
pub mod optimism;
/// Common test helpers
#[cfg(any(test, feature = "test-utils"))]
pub mod test_utils;

View File

@ -1,401 +0,0 @@
use crate::processor::{compare_receipts_root_and_logs_bloom, EVMProcessor};
use reth_evm::ConfigureEvm;
use reth_interfaces::executor::{
BlockExecutionError, BlockValidationError, OptimismBlockExecutionError,
};
use reth_primitives::{
proofs::calculate_receipt_root_optimism, revm_primitives::ResultAndState, BlockWithSenders,
Bloom, ChainSpec, Hardfork, Receipt, ReceiptWithBloom, TxType, B256, U256,
};
use reth_provider::{BlockExecutor, BundleStateWithReceipts};
use revm::DatabaseCommit;
use std::time::Instant;
use tracing::{debug, trace};
/// Verify the calculated receipts root against the expected receipts root.
pub fn verify_receipt_optimism<'a>(
expected_receipts_root: B256,
expected_logs_bloom: Bloom,
receipts: impl Iterator<Item = &'a Receipt> + Clone,
chain_spec: &ChainSpec,
timestamp: u64,
) -> Result<(), BlockExecutionError> {
// Calculate receipts root.
let receipts_with_bloom = receipts.map(|r| r.clone().into()).collect::<Vec<ReceiptWithBloom>>();
let receipts_root =
calculate_receipt_root_optimism(&receipts_with_bloom, chain_spec, timestamp);
// Create header log bloom.
let logs_bloom = receipts_with_bloom.iter().fold(Bloom::ZERO, |bloom, r| bloom | r.bloom);
compare_receipts_root_and_logs_bloom(
receipts_root,
logs_bloom,
expected_receipts_root,
expected_logs_bloom,
)?;
Ok(())
}
impl<'a, EvmConfig> BlockExecutor for EVMProcessor<'a, EvmConfig>
where
EvmConfig: ConfigureEvm,
{
type Error = BlockExecutionError;
fn execute_and_verify_receipt(
&mut self,
block: &BlockWithSenders,
total_difficulty: U256,
) -> Result<(), BlockExecutionError> {
// execute block
let receipts = self.execute_inner(block, total_difficulty)?;
// TODO Before Byzantium, receipts contained state root that would mean that expensive
// operation as hashing that is needed for state root got calculated in every
// transaction This was replaced with is_success flag.
// See more about EIP here: https://eips.ethereum.org/EIPS/eip-658
if self.chain_spec.fork(Hardfork::Byzantium).active_at_block(block.header.number) {
let time = Instant::now();
if let Err(error) = verify_receipt_optimism(
block.header.receipts_root,
block.header.logs_bloom,
receipts.iter(),
self.chain_spec.as_ref(),
block.timestamp,
) {
debug!(target: "evm", %error, ?receipts, "receipts verification failed");
return Err(error)
};
self.stats.receipt_root_duration += time.elapsed();
}
self.batch_record.save_receipts(receipts)
}
fn execute_transactions(
&mut self,
block: &BlockWithSenders,
total_difficulty: U256,
) -> Result<(Vec<Receipt>, u64), BlockExecutionError> {
self.init_env(&block.header, total_difficulty);
// perf: do not execute empty blocks
if block.body.is_empty() {
return Ok((Vec::new(), 0))
}
let is_regolith =
self.chain_spec.fork(Hardfork::Regolith).active_at_timestamp(block.timestamp);
// Ensure that the create2deployer is force-deployed at the canyon transition. Optimism
// blocks will always have at least a single transaction in them (the L1 info transaction),
// so we can safely assume that this will always be triggered upon the transition and that
// the above check for empty blocks will never be hit on OP chains.
super::ensure_create2_deployer(self.chain_spec().clone(), block.timestamp, self.db_mut())
.map_err(|_| {
BlockExecutionError::OptimismBlockExecution(
OptimismBlockExecutionError::ForceCreate2DeployerFail,
)
})?;
let mut cumulative_gas_used = 0;
let mut receipts = Vec::with_capacity(block.body.len());
for (sender, transaction) in block.transactions_with_sender() {
let time = Instant::now();
// The sum of the transactions gas limit, Tg, and the gas utilized in this block prior,
// must be no greater than the blocks gasLimit.
let block_available_gas = block.header.gas_limit - cumulative_gas_used;
if transaction.gas_limit() > block_available_gas &&
(is_regolith || !transaction.is_system_transaction())
{
return Err(BlockValidationError::TransactionGasLimitMoreThanAvailableBlockGas {
transaction_gas_limit: transaction.gas_limit(),
block_available_gas,
}
.into())
}
// An optimism block should never contain blob transactions.
if matches!(transaction.tx_type(), TxType::Eip4844) {
return Err(BlockExecutionError::OptimismBlockExecution(
OptimismBlockExecutionError::BlobTransactionRejected,
))
}
// Cache the depositor account prior to the state transition for the deposit nonce.
//
// Note that this *only* needs to be done post-regolith hardfork, as deposit nonces
// were not introduced in Bedrock. In addition, regular transactions don't have deposit
// nonces, so we don't need to touch the DB for those.
let depositor = (is_regolith && transaction.is_deposit())
.then(|| {
self.db_mut()
.load_cache_account(*sender)
.map(|acc| acc.account_info().unwrap_or_default())
})
.transpose()
.map_err(|_| {
BlockExecutionError::OptimismBlockExecution(
OptimismBlockExecutionError::AccountLoadFailed(*sender),
)
})?;
// Execute transaction.
let ResultAndState { result, state } = self.transact(transaction, *sender)?;
trace!(
target: "evm",
?transaction, ?result, ?state,
"Executed transaction"
);
self.stats.execution_duration += time.elapsed();
let time = Instant::now();
self.db_mut().commit(state);
self.stats.apply_state_duration += time.elapsed();
// append gas used
cumulative_gas_used += result.gas_used();
// Push transaction changeset and calculate header bloom filter for receipt.
receipts.push(Receipt {
tx_type: transaction.tx_type(),
// Success flag was added in `EIP-658: Embedding transaction status code in
// receipts`.
success: result.is_success(),
cumulative_gas_used,
// convert to reth log
logs: result.into_logs().into_iter().map(Into::into).collect(),
#[cfg(feature = "optimism")]
deposit_nonce: depositor.map(|account| account.nonce),
// The deposit receipt version was introduced in Canyon to indicate an update to how
// receipt hashes should be computed when set. The state transition process ensures
// this is only set for post-Canyon deposit transactions.
#[cfg(feature = "optimism")]
deposit_receipt_version: (transaction.is_deposit() &&
self.chain_spec()
.is_fork_active_at_timestamp(Hardfork::Canyon, block.timestamp))
.then_some(1),
});
}
Ok((receipts, cumulative_gas_used))
}
fn take_output_state(&mut self) -> BundleStateWithReceipts {
BundleStateWithReceipts::new(
self.evm.context.evm.db.take_bundle(),
self.batch_record.take_receipts(),
self.batch_record.first_block().unwrap_or_default(),
)
}
fn size_hint(&self) -> Option<usize> {
Some(self.evm.context.evm.db.bundle_size_hint())
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::{
database::StateProviderDatabase,
test_utils::{StateProviderTest, TestEvmConfig},
};
use reth_primitives::{
b256, Account, Address, Block, ChainSpecBuilder, Header, Signature, StorageKey,
StorageValue, Transaction, TransactionSigned, TxEip1559, BASE_MAINNET,
};
use revm::L1_BLOCK_CONTRACT;
use std::{collections::HashMap, str::FromStr, sync::Arc};
fn create_op_state_provider() -> StateProviderTest {
let mut db = StateProviderTest::default();
let l1_block_contract_account =
Account { balance: U256::ZERO, bytecode_hash: None, nonce: 1 };
let mut l1_block_storage = HashMap::new();
// base fee
l1_block_storage.insert(StorageKey::with_last_byte(1), StorageValue::from(1000000000));
// l1 fee overhead
l1_block_storage.insert(StorageKey::with_last_byte(5), StorageValue::from(188));
// l1 fee scalar
l1_block_storage.insert(StorageKey::with_last_byte(6), StorageValue::from(684000));
// l1 free scalars post ecotone
l1_block_storage.insert(
StorageKey::with_last_byte(3),
StorageValue::from_str(
"0x0000000000000000000000000000000000001db0000d27300000000000000005",
)
.unwrap(),
);
db.insert_account(L1_BLOCK_CONTRACT, l1_block_contract_account, None, l1_block_storage);
db
}
fn create_op_evm_processor<'a>(
chain_spec: Arc<ChainSpec>,
db: StateProviderTest,
) -> EVMProcessor<'a, TestEvmConfig> {
static CONFIG: std::sync::OnceLock<TestEvmConfig> = std::sync::OnceLock::new();
let mut executor = EVMProcessor::new_with_db(
chain_spec,
StateProviderDatabase::new(db),
CONFIG.get_or_init(TestEvmConfig::default),
);
executor.evm.context.evm.db.load_cache_account(L1_BLOCK_CONTRACT).unwrap();
executor
}
#[test]
fn op_deposit_fields_pre_canyon() {
let header = Header {
timestamp: 1,
number: 1,
gas_limit: 1_000_000,
gas_used: 42_000,
receipts_root: b256!(
"83465d1e7d01578c0d609be33570f91242f013e9e295b0879905346abbd63731"
),
..Default::default()
};
let mut db = create_op_state_provider();
let addr = Address::ZERO;
let account = Account { balance: U256::MAX, ..Account::default() };
db.insert_account(addr, account, None, HashMap::new());
let chain_spec =
Arc::new(ChainSpecBuilder::from(&*BASE_MAINNET).regolith_activated().build());
let tx = TransactionSigned::from_transaction_and_signature(
Transaction::Eip1559(TxEip1559 {
chain_id: chain_spec.chain.id(),
nonce: 0,
gas_limit: 21_000,
to: addr.into(),
..Default::default()
}),
Signature::default(),
);
let tx_deposit = TransactionSigned::from_transaction_and_signature(
Transaction::Deposit(reth_primitives::TxDeposit {
from: addr,
to: addr.into(),
gas_limit: 21_000,
..Default::default()
}),
Signature::default(),
);
let mut executor = create_op_evm_processor(chain_spec, db);
// Attempt to execute a block with one deposit and one non-deposit transaction
executor
.execute_and_verify_receipt(
&BlockWithSenders {
block: Block {
header,
body: vec![tx, tx_deposit],
ommers: vec![],
withdrawals: None,
},
senders: vec![addr, addr],
},
U256::ZERO,
)
.unwrap();
let tx_receipt = executor.receipts()[0][0].as_ref().unwrap();
let deposit_receipt = executor.receipts()[0][1].as_ref().unwrap();
// deposit_receipt_version is not present in pre canyon transactions
assert!(deposit_receipt.deposit_receipt_version.is_none());
assert!(tx_receipt.deposit_receipt_version.is_none());
// deposit_nonce is present only in deposit transactions
assert!(deposit_receipt.deposit_nonce.is_some());
assert!(tx_receipt.deposit_nonce.is_none());
}
#[test]
fn op_deposit_fields_post_canyon() {
// ensure_create2_deployer will fail if timestamp is set to less then 2
let header = Header {
timestamp: 2,
number: 1,
gas_limit: 1_000_000,
gas_used: 42_000,
receipts_root: b256!(
"fffc85c4004fd03c7bfbe5491fae98a7473126c099ac11e8286fd0013f15f908"
),
..Default::default()
};
let mut db = create_op_state_provider();
let addr = Address::ZERO;
let account = Account { balance: U256::MAX, ..Account::default() };
db.insert_account(addr, account, None, HashMap::new());
let chain_spec =
Arc::new(ChainSpecBuilder::from(&*BASE_MAINNET).canyon_activated().build());
let tx = TransactionSigned::from_transaction_and_signature(
Transaction::Eip1559(TxEip1559 {
chain_id: chain_spec.chain.id(),
nonce: 0,
gas_limit: 21_000,
to: addr.into(),
..Default::default()
}),
Signature::default(),
);
let tx_deposit = TransactionSigned::from_transaction_and_signature(
Transaction::Deposit(reth_primitives::TxDeposit {
from: addr,
to: addr.into(),
gas_limit: 21_000,
..Default::default()
}),
Signature::optimism_deposit_tx_signature(),
);
let mut executor = create_op_evm_processor(chain_spec, db);
// attempt to execute an empty block with parent beacon block root, this should not fail
executor
.execute_and_verify_receipt(
&BlockWithSenders {
block: Block {
header,
body: vec![tx, tx_deposit],
ommers: vec![],
withdrawals: None,
},
senders: vec![addr, addr],
},
U256::ZERO,
)
.expect("Executing a block while canyon is active should not fail");
let tx_receipt = executor.receipts()[0][0].as_ref().unwrap();
let deposit_receipt = executor.receipts()[0][1].as_ref().unwrap();
// deposit_receipt_version is set to 1 for post canyon deposit transactions
assert_eq!(deposit_receipt.deposit_receipt_version, Some(1));
assert!(tx_receipt.deposit_receipt_version.is_none());
// deposit_nonce is present only in deposit transactions
assert!(deposit_receipt.deposit_nonce.is_some());
assert!(tx_receipt.deposit_nonce.is_none());
}
}

View File

@ -1,865 +0,0 @@
#[cfg(not(feature = "optimism"))]
use revm::DatabaseCommit;
use revm::{
db::StateDBBox,
inspector_handle_register,
interpreter::Host,
primitives::{CfgEnvWithHandlerCfg, ResultAndState},
Evm, State,
};
use std::{marker::PhantomData, sync::Arc, time::Instant};
#[cfg(not(feature = "optimism"))]
use tracing::{debug, trace};
use reth_evm::ConfigureEvm;
use reth_interfaces::executor::{BlockExecutionError, BlockValidationError};
#[cfg(feature = "optimism")]
use reth_primitives::revm::env::fill_op_tx_env;
#[cfg(not(feature = "optimism"))]
use reth_primitives::revm::env::fill_tx_env;
use reth_primitives::{
Address, Block, BlockNumber, BlockWithSenders, Bloom, ChainSpec, GotExpected, Hardfork, Header,
PruneModes, Receipt, ReceiptWithBloom, Receipts, TransactionSigned, Withdrawals, B256, U256,
};
#[cfg(not(feature = "optimism"))]
use reth_provider::BundleStateWithReceipts;
use reth_provider::{BlockExecutor, ProviderError, PrunableBlockExecutor, StateProvider};
use crate::{
batch::{BlockBatchRecord, BlockExecutorStats},
database::StateProviderDatabase,
eth_dao_fork::{DAO_HARDFORK_BENEFICIARY, DAO_HARDKFORK_ACCOUNTS},
stack::{InspectorStack, InspectorStackConfig},
state_change::{apply_beacon_root_contract_call, post_block_balance_increments},
};
/// EVMProcessor is a block executor that uses revm to execute blocks or multiple blocks.
///
/// Output is obtained by calling `take_output_state` function.
///
/// It is capable of pruning the data that will be written to the database
/// and implemented [PrunableBlockExecutor] traits.
///
/// It implemented the [BlockExecutor] that give it the ability to take block
/// apply pre state (Cancun system contract call), execute transaction and apply
/// state change and then apply post execution changes (block reward, withdrawals, irregular DAO
/// hardfork state change). And if `execute_and_verify_receipt` is called it will verify the
/// receipt.
///
/// InspectorStack are used for optional inspecting execution. And it contains
/// various duration of parts of execution.
#[allow(missing_debug_implementations)]
pub struct EVMProcessor<'a, EvmConfig> {
/// The configured chain-spec
pub(crate) chain_spec: Arc<ChainSpec>,
/// revm instance that contains database and env environment.
pub(crate) evm: Evm<'a, InspectorStack, StateDBBox<'a, ProviderError>>,
/// Keeps track of the recorded receipts and pruning configuration.
pub(crate) batch_record: BlockBatchRecord,
/// Execution stats
pub(crate) stats: BlockExecutorStats,
/// The type that is able to configure the EVM environment.
_phantom: PhantomData<EvmConfig>,
}
impl<'a, EvmConfig> EVMProcessor<'a, EvmConfig>
where
EvmConfig: ConfigureEvm,
{
/// Return chain spec.
pub fn chain_spec(&self) -> &Arc<ChainSpec> {
&self.chain_spec
}
/// Creates a new executor from the given chain spec and database.
pub fn new_with_db<DB: StateProvider + 'a>(
chain_spec: Arc<ChainSpec>,
db: StateProviderDatabase<DB>,
evm_config: &'a EvmConfig,
) -> Self {
let state = State::builder()
.with_database_boxed(Box::new(db))
.with_bundle_update()
.without_state_clear()
.build();
EVMProcessor::new_with_state(chain_spec, state, evm_config)
}
/// Create a new EVM processor with the given revm state.
pub fn new_with_state(
chain_spec: Arc<ChainSpec>,
revm_state: StateDBBox<'a, ProviderError>,
evm_config: &'a EvmConfig,
) -> Self {
let stack = InspectorStack::new(InspectorStackConfig::default());
let evm = evm_config.evm_with_inspector(revm_state, stack);
EVMProcessor {
chain_spec,
evm,
batch_record: BlockBatchRecord::default(),
stats: BlockExecutorStats::default(),
_phantom: PhantomData,
}
}
/// Configures the executor with the given inspectors.
pub fn set_stack(&mut self, stack: InspectorStack) {
self.evm.context.external = stack;
}
/// Configure the executor with the given block.
pub fn set_first_block(&mut self, num: BlockNumber) {
self.batch_record.set_first_block(num);
}
/// Saves the receipts to the batch record.
pub fn save_receipts(&mut self, receipts: Vec<Receipt>) -> Result<(), BlockExecutionError> {
self.batch_record.save_receipts(receipts)
}
/// Returns the recorded receipts.
pub fn receipts(&self) -> &Receipts {
self.batch_record.receipts()
}
/// Returns a reference to the database
pub fn db_mut(&mut self) -> &mut StateDBBox<'a, ProviderError> {
&mut self.evm.context.evm.db
}
/// Initializes the config and block env.
pub(crate) fn init_env(&mut self, header: &Header, total_difficulty: U256) {
// Set state clear flag.
let state_clear_flag =
self.chain_spec.fork(Hardfork::SpuriousDragon).active_at_block(header.number);
self.db_mut().set_state_clear_flag(state_clear_flag);
let mut cfg =
CfgEnvWithHandlerCfg::new_with_spec_id(self.evm.cfg().clone(), self.evm.spec_id());
EvmConfig::fill_cfg_and_block_env(
&mut cfg,
self.evm.block_mut(),
&self.chain_spec,
header,
total_difficulty,
);
*self.evm.cfg_mut() = cfg.cfg_env;
// This will update the spec in case it changed
self.evm.modify_spec_id(cfg.handler_cfg.spec_id);
}
/// Applies the pre-block call to the EIP-4788 beacon block root contract.
///
/// If cancun is not activated or the block is the genesis block, then this is a no-op, and no
/// state changes are made.
fn apply_beacon_root_contract_call(
&mut self,
block: &Block,
) -> Result<(), BlockExecutionError> {
apply_beacon_root_contract_call(
&self.chain_spec,
block.timestamp,
block.number,
block.parent_beacon_block_root,
&mut self.evm,
)?;
Ok(())
}
/// Apply post execution state changes, including block rewards, withdrawals, and irregular DAO
/// hardfork state change.
pub fn apply_post_execution_state_change(
&mut self,
block: &Block,
total_difficulty: U256,
) -> Result<(), BlockExecutionError> {
let mut balance_increments = post_block_balance_increments(
&self.chain_spec,
block.number,
block.difficulty,
block.beneficiary,
block.timestamp,
total_difficulty,
&block.ommers,
block.withdrawals.as_ref().map(Withdrawals::as_ref),
);
// Irregular state change at Ethereum DAO hardfork
if self.chain_spec.fork(Hardfork::Dao).transitions_at_block(block.number) {
// drain balances from hardcoded addresses.
let drained_balance: u128 = self
.db_mut()
.drain_balances(DAO_HARDKFORK_ACCOUNTS)
.map_err(|_| BlockValidationError::IncrementBalanceFailed)?
.into_iter()
.sum();
// return balance to DAO beneficiary.
*balance_increments.entry(DAO_HARDFORK_BENEFICIARY).or_default() += drained_balance;
}
// increment balances
self.db_mut()
.increment_balances(balance_increments)
.map_err(|_| BlockValidationError::IncrementBalanceFailed)?;
Ok(())
}
/// Runs a single transaction in the configured environment and proceeds
/// to return the result and state diff (without applying it).
///
/// Assumes the rest of the block environment has been filled via `init_block_env`.
pub fn transact(
&mut self,
transaction: &TransactionSigned,
sender: Address,
) -> Result<ResultAndState, BlockExecutionError> {
// Fill revm structure.
#[cfg(not(feature = "optimism"))]
fill_tx_env(self.evm.tx_mut(), transaction, sender);
#[cfg(feature = "optimism")]
{
let mut envelope_buf = Vec::with_capacity(transaction.length_without_header());
transaction.encode_enveloped(&mut envelope_buf);
fill_op_tx_env(self.evm.tx_mut(), transaction, sender, envelope_buf.into());
}
let hash = transaction.hash_ref();
let should_inspect = self.evm.context.external.should_inspect(self.evm.env(), hash);
let out = if should_inspect {
// push inspector handle register.
self.evm.handler.append_handler_register_plain(inspector_handle_register);
let output = self.evm.transact();
tracing::trace!(
target: "evm",
%hash, ?output, ?transaction, env = ?self.evm.context.evm.env,
"Executed transaction"
);
// pop last handle register
self.evm.handler.pop_handle_register();
output
} else {
// Main execution without needing the hash
self.evm.transact()
};
out.map_err(move |e| {
// Ensure hash is calculated for error log, if not already done
BlockValidationError::EVM { hash: transaction.recalculate_hash(), error: e.into() }
.into()
})
}
/// Execute the block, verify gas usage and apply post-block state changes.
pub(crate) fn execute_inner(
&mut self,
block: &BlockWithSenders,
total_difficulty: U256,
) -> Result<Vec<Receipt>, BlockExecutionError> {
self.init_env(&block.header, total_difficulty);
self.apply_beacon_root_contract_call(block)?;
let (receipts, cumulative_gas_used) = self.execute_transactions(block, total_difficulty)?;
// Check if gas used matches the value set in header.
if block.gas_used != cumulative_gas_used {
let receipts = Receipts::from_block_receipt(receipts);
return Err(BlockValidationError::BlockGasUsed {
gas: GotExpected { got: cumulative_gas_used, expected: block.gas_used },
gas_spent_by_tx: receipts.gas_spent_by_tx()?,
}
.into())
}
let time = Instant::now();
self.apply_post_execution_state_change(block, total_difficulty)?;
self.stats.apply_post_execution_state_changes_duration += time.elapsed();
let time = Instant::now();
let retention = self.batch_record.bundle_retention(block.number);
self.db_mut().merge_transitions(retention);
self.stats.merge_transitions_duration += time.elapsed();
if self.batch_record.first_block().is_none() {
self.batch_record.set_first_block(block.number);
}
Ok(receipts)
}
}
/// Default Ethereum implementation of the [BlockExecutor] trait for the [EVMProcessor].
#[cfg(not(feature = "optimism"))]
impl<'a, EvmConfig> BlockExecutor for EVMProcessor<'a, EvmConfig>
where
EvmConfig: ConfigureEvm,
{
type Error = BlockExecutionError;
fn execute_and_verify_receipt(
&mut self,
block: &BlockWithSenders,
total_difficulty: U256,
) -> Result<(), BlockExecutionError> {
// execute block
let receipts = self.execute_inner(block, total_difficulty)?;
// TODO Before Byzantium, receipts contained state root that would mean that expensive
// operation as hashing that is needed for state root got calculated in every
// transaction This was replaced with is_success flag.
// See more about EIP here: https://eips.ethereum.org/EIPS/eip-658
if self.chain_spec.fork(Hardfork::Byzantium).active_at_block(block.header.number) {
let time = Instant::now();
if let Err(error) =
verify_receipt(block.header.receipts_root, block.header.logs_bloom, receipts.iter())
{
debug!(target: "evm", %error, ?receipts, "receipts verification failed");
return Err(error)
};
self.stats.receipt_root_duration += time.elapsed();
}
self.batch_record.save_receipts(receipts)?;
Ok(())
}
fn execute_transactions(
&mut self,
block: &BlockWithSenders,
total_difficulty: U256,
) -> Result<(Vec<Receipt>, u64), BlockExecutionError> {
self.init_env(&block.header, total_difficulty);
// perf: do not execute empty blocks
if block.body.is_empty() {
return Ok((Vec::new(), 0))
}
let mut cumulative_gas_used = 0;
let mut receipts = Vec::with_capacity(block.body.len());
for (sender, transaction) in block.transactions_with_sender() {
let time = Instant::now();
// The sum of the transactions gas limit, Tg, and the gas utilized in this block prior,
// must be no greater than the blocks gasLimit.
let block_available_gas = block.header.gas_limit - cumulative_gas_used;
if transaction.gas_limit() > block_available_gas {
return Err(BlockValidationError::TransactionGasLimitMoreThanAvailableBlockGas {
transaction_gas_limit: transaction.gas_limit(),
block_available_gas,
}
.into())
}
// Execute transaction.
let ResultAndState { result, state } = self.transact(transaction, *sender)?;
trace!(
target: "evm",
?transaction, ?result, ?state,
"Executed transaction"
);
self.stats.execution_duration += time.elapsed();
let time = Instant::now();
self.db_mut().commit(state);
self.stats.apply_state_duration += time.elapsed();
// append gas used
cumulative_gas_used += result.gas_used();
// Push transaction changeset and calculate header bloom filter for receipt.
receipts.push(Receipt {
tx_type: transaction.tx_type(),
// Success flag was added in `EIP-658: Embedding transaction status code in
// receipts`.
success: result.is_success(),
cumulative_gas_used,
// convert to reth log
logs: result.into_logs().into_iter().map(Into::into).collect(),
});
}
Ok((receipts, cumulative_gas_used))
}
fn take_output_state(&mut self) -> BundleStateWithReceipts {
self.stats.log_debug();
BundleStateWithReceipts::new(
self.evm.context.evm.db.take_bundle(),
self.batch_record.take_receipts(),
self.batch_record.first_block().unwrap_or_default(),
)
}
fn size_hint(&self) -> Option<usize> {
Some(self.evm.context.evm.db.bundle_size_hint())
}
}
impl<'a, EvmConfig> PrunableBlockExecutor for EVMProcessor<'a, EvmConfig>
where
EvmConfig: ConfigureEvm,
{
fn set_tip(&mut self, tip: BlockNumber) {
self.batch_record.set_tip(tip);
}
fn set_prune_modes(&mut self, prune_modes: PruneModes) {
self.batch_record.set_prune_modes(prune_modes);
}
}
/// Calculate the receipts root, and copmare it against against the expected receipts root and logs
/// bloom.
pub fn verify_receipt<'a>(
expected_receipts_root: B256,
expected_logs_bloom: Bloom,
receipts: impl Iterator<Item = &'a Receipt> + Clone,
) -> Result<(), BlockExecutionError> {
// Calculate receipts root.
let receipts_with_bloom = receipts.map(|r| r.clone().into()).collect::<Vec<ReceiptWithBloom>>();
let receipts_root = reth_primitives::proofs::calculate_receipt_root(&receipts_with_bloom);
// Create header log bloom.
let logs_bloom = receipts_with_bloom.iter().fold(Bloom::ZERO, |bloom, r| bloom | r.bloom);
compare_receipts_root_and_logs_bloom(
receipts_root,
logs_bloom,
expected_receipts_root,
expected_logs_bloom,
)?;
Ok(())
}
/// Compare the calculated receipts root with the expected receipts root, also copmare
/// the calculated logs bloom with the expected logs bloom.
pub fn compare_receipts_root_and_logs_bloom(
calculated_receipts_root: B256,
calculated_logs_bloom: Bloom,
expected_receipts_root: B256,
expected_logs_bloom: Bloom,
) -> Result<(), BlockExecutionError> {
if calculated_receipts_root != expected_receipts_root {
return Err(BlockValidationError::ReceiptRootDiff(
GotExpected { got: calculated_receipts_root, expected: expected_receipts_root }.into(),
)
.into())
}
if calculated_logs_bloom != expected_logs_bloom {
return Err(BlockValidationError::BloomLogDiff(
GotExpected { got: calculated_logs_bloom, expected: expected_logs_bloom }.into(),
)
.into())
}
Ok(())
}
#[cfg(test)]
mod tests {
use super::*;
use crate::test_utils::{StateProviderTest, TestEvmConfig};
use reth_primitives::{
bytes,
constants::{BEACON_ROOTS_ADDRESS, EIP1559_INITIAL_BASE_FEE, SYSTEM_ADDRESS},
keccak256, Account, Bytes, ChainSpecBuilder, ForkCondition, Signature, Transaction,
TxEip1559, MAINNET,
};
use revm::{Database, TransitionState};
use std::collections::HashMap;
static BEACON_ROOT_CONTRACT_CODE: Bytes = bytes!("3373fffffffffffffffffffffffffffffffffffffffe14604d57602036146024575f5ffd5b5f35801560495762001fff810690815414603c575f5ffd5b62001fff01545f5260205ff35b5f5ffd5b62001fff42064281555f359062001fff015500");
fn create_state_provider_with_beacon_root_contract() -> StateProviderTest {
let mut db = StateProviderTest::default();
let beacon_root_contract_account = Account {
balance: U256::ZERO,
bytecode_hash: Some(keccak256(BEACON_ROOT_CONTRACT_CODE.clone())),
nonce: 1,
};
db.insert_account(
BEACON_ROOTS_ADDRESS,
beacon_root_contract_account,
Some(BEACON_ROOT_CONTRACT_CODE.clone()),
HashMap::new(),
);
db
}
#[test]
fn eip_4788_non_genesis_call() {
let mut header =
Header { timestamp: 1, number: 1, excess_blob_gas: Some(0), ..Header::default() };
let db = create_state_provider_with_beacon_root_contract();
let chain_spec = Arc::new(
ChainSpecBuilder::from(&*MAINNET)
.shanghai_activated()
.with_fork(Hardfork::Cancun, ForkCondition::Timestamp(1))
.build(),
);
// execute invalid header (no parent beacon block root)
let evm_config = TestEvmConfig::default();
let mut executor =
EVMProcessor::new_with_db(chain_spec, StateProviderDatabase::new(db), &evm_config);
// attempt to execute a block without parent beacon block root, expect err
let err = executor
.execute_and_verify_receipt(
&BlockWithSenders {
block: Block {
header: header.clone(),
body: vec![],
ommers: vec![],
withdrawals: None,
},
senders: vec![],
},
U256::ZERO,
)
.expect_err(
"Executing cancun block without parent beacon block root field should fail",
);
assert_eq!(
err,
BlockExecutionError::Validation(BlockValidationError::MissingParentBeaconBlockRoot)
);
// fix header, set a gas limit
header.parent_beacon_block_root = Some(B256::with_last_byte(0x69));
// Now execute a block with the fixed header, ensure that it does not fail
executor
.execute_and_verify_receipt(
&BlockWithSenders {
block: Block {
header: header.clone(),
body: vec![],
ommers: vec![],
withdrawals: None,
},
senders: vec![],
},
U256::ZERO,
)
.unwrap();
// check the actual storage of the contract - it should be:
// * The storage value at header.timestamp % HISTORY_BUFFER_LENGTH should be
// header.timestamp
// * The storage value at header.timestamp % HISTORY_BUFFER_LENGTH + HISTORY_BUFFER_LENGTH
// should be parent_beacon_block_root
let history_buffer_length = 8191u64;
let timestamp_index = header.timestamp % history_buffer_length;
let parent_beacon_block_root_index =
timestamp_index % history_buffer_length + history_buffer_length;
// get timestamp storage and compare
let timestamp_storage =
executor.db_mut().storage(BEACON_ROOTS_ADDRESS, U256::from(timestamp_index)).unwrap();
assert_eq!(timestamp_storage, U256::from(header.timestamp));
// get parent beacon block root storage and compare
let parent_beacon_block_root_storage = executor
.db_mut()
.storage(BEACON_ROOTS_ADDRESS, U256::from(parent_beacon_block_root_index))
.expect("storage value should exist");
assert_eq!(parent_beacon_block_root_storage, U256::from(0x69));
}
#[test]
fn eip_4788_no_code_cancun() {
// This test ensures that we "silently fail" when cancun is active and there is no code at
// BEACON_ROOTS_ADDRESS
let header = Header {
timestamp: 1,
number: 1,
parent_beacon_block_root: Some(B256::with_last_byte(0x69)),
excess_blob_gas: Some(0),
..Header::default()
};
let db = StateProviderTest::default();
// DON'T deploy the contract at genesis
let chain_spec = Arc::new(
ChainSpecBuilder::from(&*MAINNET)
.shanghai_activated()
.with_fork(Hardfork::Cancun, ForkCondition::Timestamp(1))
.build(),
);
let evm_config = TestEvmConfig::default();
let mut executor =
EVMProcessor::new_with_db(chain_spec, StateProviderDatabase::new(db), &evm_config);
executor.init_env(&header, U256::ZERO);
// get the env
let previous_env = executor.evm.context.evm.env.clone();
// attempt to execute an empty block with parent beacon block root, this should not fail
executor
.execute_and_verify_receipt(
&BlockWithSenders {
block: Block {
header: header.clone(),
body: vec![],
ommers: vec![],
withdrawals: None,
},
senders: vec![],
},
U256::ZERO,
)
.expect(
"Executing a block with no transactions while cancun is active should not fail",
);
// ensure that the env has not changed
assert_eq!(executor.evm.context.evm.env, previous_env);
}
#[test]
fn eip_4788_empty_account_call() {
// This test ensures that we do not increment the nonce of an empty SYSTEM_ADDRESS account
// during the pre-block call
let mut db = create_state_provider_with_beacon_root_contract();
// insert an empty SYSTEM_ADDRESS
db.insert_account(SYSTEM_ADDRESS, Account::default(), None, HashMap::new());
let chain_spec = Arc::new(
ChainSpecBuilder::from(&*MAINNET)
.shanghai_activated()
.with_fork(Hardfork::Cancun, ForkCondition::Timestamp(1))
.build(),
);
let evm_config = TestEvmConfig::default();
let mut executor =
EVMProcessor::new_with_db(chain_spec, StateProviderDatabase::new(db), &evm_config);
// construct the header for block one
let header = Header {
timestamp: 1,
number: 1,
parent_beacon_block_root: Some(B256::with_last_byte(0x69)),
excess_blob_gas: Some(0),
..Header::default()
};
executor.init_env(&header, U256::ZERO);
// attempt to execute an empty block with parent beacon block root, this should not fail
executor
.execute_and_verify_receipt(
&BlockWithSenders {
block: Block {
header: header.clone(),
body: vec![],
ommers: vec![],
withdrawals: None,
},
senders: vec![],
},
U256::ZERO,
)
.expect(
"Executing a block with no transactions while cancun is active should not fail",
);
// ensure that the nonce of the system address account has not changed
let nonce = executor.db_mut().basic(SYSTEM_ADDRESS).unwrap().unwrap().nonce;
assert_eq!(nonce, 0);
}
#[test]
fn eip_4788_genesis_call() {
let db = create_state_provider_with_beacon_root_contract();
// activate cancun at genesis
let chain_spec = Arc::new(
ChainSpecBuilder::from(&*MAINNET)
.shanghai_activated()
.with_fork(Hardfork::Cancun, ForkCondition::Timestamp(0))
.build(),
);
let mut header = chain_spec.genesis_header();
let evm_config = TestEvmConfig::default();
let mut executor =
EVMProcessor::new_with_db(chain_spec, StateProviderDatabase::new(db), &evm_config);
executor.init_env(&header, U256::ZERO);
// attempt to execute the genesis block with non-zero parent beacon block root, expect err
header.parent_beacon_block_root = Some(B256::with_last_byte(0x69));
let _err = executor
.execute_and_verify_receipt(
&BlockWithSenders {
block: Block {
header: header.clone(),
body: vec![],
ommers: vec![],
withdrawals: None,
},
senders: vec![],
},
U256::ZERO,
)
.expect_err(
"Executing genesis cancun block with non-zero parent beacon block root field should fail",
);
// fix header
header.parent_beacon_block_root = Some(B256::ZERO);
// now try to process the genesis block again, this time ensuring that a system contract
// call does not occur
executor
.execute_and_verify_receipt(
&BlockWithSenders {
block: Block {
header: header.clone(),
body: vec![],
ommers: vec![],
withdrawals: None,
},
senders: vec![],
},
U256::ZERO,
)
.unwrap();
// there is no system contract call so there should be NO STORAGE CHANGES
// this means we'll check the transition state
let state = executor.evm.context.evm.inner.db;
let transition_state =
state.transition_state.expect("the evm should be initialized with bundle updates");
// assert that it is the default (empty) transition state
assert_eq!(transition_state, TransitionState::default());
}
#[test]
fn eip_4788_high_base_fee() {
// This test ensures that if we have a base fee, then we don't return an error when the
// system contract is called, due to the gas price being less than the base fee.
let header = Header {
timestamp: 1,
number: 1,
parent_beacon_block_root: Some(B256::with_last_byte(0x69)),
base_fee_per_gas: Some(u64::MAX),
excess_blob_gas: Some(0),
..Header::default()
};
let db = create_state_provider_with_beacon_root_contract();
let chain_spec = Arc::new(
ChainSpecBuilder::from(&*MAINNET)
.shanghai_activated()
.with_fork(Hardfork::Cancun, ForkCondition::Timestamp(1))
.build(),
);
// execute header
let evm_config = TestEvmConfig::default();
let mut executor =
EVMProcessor::new_with_db(chain_spec, StateProviderDatabase::new(db), &evm_config);
executor.init_env(&header, U256::ZERO);
// ensure that the env is configured with a base fee
assert_eq!(executor.evm.block().basefee, U256::from(u64::MAX));
// Now execute a block with the fixed header, ensure that it does not fail
executor
.execute_and_verify_receipt(
&BlockWithSenders {
block: Block {
header: header.clone(),
body: vec![],
ommers: vec![],
withdrawals: None,
},
senders: vec![],
},
U256::ZERO,
)
.unwrap();
// check the actual storage of the contract - it should be:
// * The storage value at header.timestamp % HISTORY_BUFFER_LENGTH should be
// header.timestamp
// * The storage value at header.timestamp % HISTORY_BUFFER_LENGTH + HISTORY_BUFFER_LENGTH
// should be parent_beacon_block_root
let history_buffer_length = 8191u64;
let timestamp_index = header.timestamp % history_buffer_length;
let parent_beacon_block_root_index =
timestamp_index % history_buffer_length + history_buffer_length;
// get timestamp storage and compare
let timestamp_storage =
executor.db_mut().storage(BEACON_ROOTS_ADDRESS, U256::from(timestamp_index)).unwrap();
assert_eq!(timestamp_storage, U256::from(header.timestamp));
// get parent beacon block root storage and compare
let parent_beacon_block_root_storage = executor
.db_mut()
.storage(BEACON_ROOTS_ADDRESS, U256::from(parent_beacon_block_root_index))
.unwrap();
assert_eq!(parent_beacon_block_root_storage, U256::from(0x69));
}
#[test]
fn test_transact_error_includes_correct_hash() {
let chain_spec = Arc::new(
ChainSpecBuilder::from(&*MAINNET)
.shanghai_activated()
.with_fork(Hardfork::Cancun, ForkCondition::Timestamp(1))
.build(),
);
let db = StateProviderTest::default();
let chain_id = chain_spec.chain.id();
// execute header
let evm_config = TestEvmConfig::default();
let mut executor =
EVMProcessor::new_with_db(chain_spec, StateProviderDatabase::new(db), &evm_config);
// Create a test transaction that gonna fail
let transaction = TransactionSigned::from_transaction_and_signature(
Transaction::Eip1559(TxEip1559 {
chain_id,
nonce: 1,
gas_limit: 21_000,
to: Address::ZERO.into(),
max_fee_per_gas: EIP1559_INITIAL_BASE_FEE as u128,
..Default::default()
}),
Signature::default(),
);
let result = executor.transact(&transaction, Address::random());
let expected_hash = transaction.recalculate_hash();
// Check the error
match result {
Err(BlockExecutionError::Validation(BlockValidationError::EVM { hash, error: _ })) => {
assert_eq!(hash, expected_hash, "The EVM error does not include the correct transaction hash.");
},
_ => panic!("Expected a BlockExecutionError::Validation error, but transaction did not fail as expected."),
}
}
}

View File

@ -44,7 +44,8 @@ tracing.workspace = true
reth-beacon-consensus.workspace = true
reth-interfaces = { workspace = true, features = ["test-utils"] }
reth-network-api.workspace = true
reth-node-ethereum.workspace = true
reth-evm-ethereum.workspace = true
reth-ethereum-engine-primitives.workspace = true
reth-payload-builder = { workspace = true, features = ["test-utils"] }
reth-primitives.workspace = true
reth-provider = { workspace = true, features = ["test-utils"] }

View File

@ -2,7 +2,7 @@
use crate::utils::launch_auth;
use jsonrpsee::core::client::{ClientT, SubscriptionClientT};
use reth_node_ethereum::EthEngineTypes;
use reth_ethereum_engine_primitives::EthEngineTypes;
use reth_primitives::{Block, U64};
use reth_rpc::JwtSecret;
use reth_rpc_api::clients::EngineApiClient;

View File

@ -1,6 +1,7 @@
use reth_beacon_consensus::BeaconConsensusEngineHandle;
use reth_ethereum_engine_primitives::EthEngineTypes;
use reth_evm_ethereum::EthEvmConfig;
use reth_network_api::noop::NoopNetwork;
use reth_node_ethereum::{EthEngineTypes, EthEvmConfig};
use reth_payload_builder::test_utils::spawn_test_payload_service;
use reth_primitives::MAINNET;
use reth_provider::test_utils::{NoopProvider, TestCanonStateSubscriptions};

View File

@ -29,6 +29,8 @@ revm-inspectors.workspace = true
reth-evm.workspace = true
reth-network-types.workspace = true
reth-evm-optimism = { workspace = true, optional = true }
# eth
alloy-rlp.workspace = true
alloy-dyn-abi = { workspace = true, features = ["eip712"] }
@ -89,4 +91,6 @@ optimism = [
"reth-primitives/optimism",
"reth-rpc-types-compat/optimism",
"reth-provider/optimism",
"dep:reth-evm-optimism",
"reth-evm-optimism/optimism",
]

View File

@ -84,7 +84,7 @@ where
#[cfg(feature = "optimism")]
let (block_timestamp, l1_block_info) = {
let body = reth_revm::optimism::extract_l1_info(&block);
let body = reth_evm_optimism::extract_l1_info(&block);
(block.timestamp, body.ok())
};

View File

@ -49,17 +49,9 @@ use revm::{
};
use std::future::Future;
#[cfg(feature = "optimism")]
use crate::eth::api::optimism::OptimismTxMeta;
#[cfg(feature = "optimism")]
use crate::eth::optimism::OptimismEthApiError;
use crate::eth::revm_utils::FillableTransaction;
#[cfg(feature = "optimism")]
use reth_revm::optimism::RethL1BlockInfo;
#[cfg(feature = "optimism")]
use reth_rpc_types::OptimismTransactionReceiptFields;
#[cfg(feature = "optimism")]
use revm::L1BlockInfo;
use revm_primitives::db::{Database, DatabaseRef};
/// Helper alias type for the state's [CacheDB]
@ -1498,7 +1490,7 @@ where
.ok_or(EthApiError::UnknownBlockNumber)?;
let block = block.unseal();
let l1_block_info = reth_revm::optimism::extract_l1_info(&block).ok();
let l1_block_info = reth_evm_optimism::extract_l1_info(&block).ok();
let optimism_tx_meta = self.build_op_tx_meta(&tx, l1_block_info, block.timestamp)?;
build_transaction_receipt_with_block_receipts(
@ -1510,17 +1502,19 @@ where
)
}
/// Builds [OptimismTxMeta] object using the provided [TransactionSigned],
/// [L1BlockInfo] and `block_timestamp`. The [L1BlockInfo] is used to calculate
/// the l1 fee and l1 data gas for the transaction.
/// If the [L1BlockInfo] is not provided, the [OptimismTxMeta] will be empty.
/// Builds op metadata object using the provided [TransactionSigned], L1 block info and
/// `block_timestamp`. The L1BlockInfo is used to calculate the l1 fee and l1 data gas for the
/// transaction. If the L1BlockInfo is not provided, the meta info will be empty.
#[cfg(feature = "optimism")]
pub(crate) fn build_op_tx_meta(
&self,
tx: &TransactionSigned,
l1_block_info: Option<L1BlockInfo>,
l1_block_info: Option<revm::L1BlockInfo>,
block_timestamp: u64,
) -> EthResult<OptimismTxMeta> {
) -> EthResult<crate::eth::api::optimism::OptimismTxMeta> {
use crate::eth::{api::optimism::OptimismTxMeta, optimism::OptimismEthApiError};
use reth_evm_optimism::RethL1BlockInfo;
let Some(l1_block_info) = l1_block_info else { return Ok(OptimismTxMeta::default()) };
let (l1_fee, l1_data_gas) = if !tx.is_deposit() {
@ -1711,7 +1705,7 @@ pub(crate) fn build_transaction_receipt_with_block_receipts(
meta: TransactionMeta,
receipt: Receipt,
all_receipts: &[Receipt],
#[cfg(feature = "optimism")] optimism_tx_meta: OptimismTxMeta,
#[cfg(feature = "optimism")] optimism_tx_meta: crate::eth::api::optimism::OptimismTxMeta,
) -> EthResult<AnyTransactionReceipt> {
// Note: we assume this transaction is valid, because it's mined (or part of pending block) and
// we don't need to check for pre EIP-2

View File

@ -24,6 +24,8 @@ reth-etl.workspace = true
reth-config.workspace = true
reth-stages-api = { workspace = true, features = ["test-utils"] }
reth-consensus.workspace = true
reth-evm.workspace = true
reth-revm.workspace = true
# async
tokio = { workspace = true, features = ["sync"] }

View File

@ -16,7 +16,7 @@
//! # use reth_downloaders::bodies::bodies::BodiesDownloaderBuilder;
//! # use reth_downloaders::headers::reverse_headers::ReverseHeadersDownloaderBuilder;
//! # use reth_interfaces::test_utils::{TestBodiesClient, TestHeadersClient};
//! # use reth_revm::EvmProcessorFactory;
//! # use reth_evm_ethereum::execute::EthExecutorProvider;
//! # use reth_primitives::{MAINNET, B256, PruneModes};
//! # use reth_network_types::PeerId;
//! # use reth_stages::Pipeline;
@ -45,7 +45,7 @@
//! # provider_factory.clone()
//! # );
//! # let (tip_tx, tip_rx) = watch::channel(B256::default());
//! # let executor_factory = EvmProcessorFactory::new(chain_spec.clone(), EthEvmConfig::default());
//! # let executor_provider = EthExecutorProvider::mainnet();
//! # let static_file_producer = StaticFileProducer::new(
//! # provider_factory.clone(),
//! # provider_factory.static_file_provider(),
@ -55,17 +55,15 @@
//! # let pipeline =
//! Pipeline::builder()
//! .with_tip_sender(tip_tx)
//! .add_stages(
//! DefaultStages::new(
//! provider_factory.clone(),
//! HeaderSyncMode::Tip(tip_rx),
//! consensus,
//! headers_downloader,
//! bodies_downloader,
//! executor_factory,
//! EtlConfig::default(),
//! )
//! )
//! .add_stages(DefaultStages::new(
//! provider_factory.clone(),
//! HeaderSyncMode::Tip(tip_rx),
//! consensus,
//! headers_downloader,
//! bodies_downloader,
//! executor_provider,
//! EtlConfig::default(),
//! ))
//! .build(provider_factory, static_file_producer);
//! ```
//!

View File

@ -12,44 +12,29 @@
//! ```no_run
//! # use reth_stages::Pipeline;
//! # use reth_stages::sets::{OfflineStages};
//! # use reth_revm::EvmProcessorFactory;
//! # use reth_primitives::{PruneModes, MAINNET};
//! # use reth_evm_ethereum::EthEvmConfig;
//! # use reth_provider::StaticFileProviderFactory;
//! # use reth_provider::test_utils::create_test_provider_factory;
//! # use reth_static_file::StaticFileProducer;
//! # use reth_config::config::EtlConfig;
//! # use reth_evm::execute::BlockExecutorProvider;
//!
//! # let executor_factory = EvmProcessorFactory::new(MAINNET.clone(), EthEvmConfig::default());
//! # let provider_factory = create_test_provider_factory();
//! # let static_file_producer = StaticFileProducer::new(
//! # fn create(exec: impl BlockExecutorProvider) {
//!
//! let provider_factory = create_test_provider_factory();
//! let static_file_producer = StaticFileProducer::new(
//! provider_factory.clone(),
//! provider_factory.static_file_provider(),
//! PruneModes::default(),
//! );
//! // Build a pipeline with all offline stages.
//! # let pipeline = Pipeline::builder()
//! .add_stages(OfflineStages::new(executor_factory, EtlConfig::default()))
//! let pipeline = Pipeline::builder()
//! .add_stages(OfflineStages::new(exec, EtlConfig::default()))
//! .build(provider_factory, static_file_producer);
//! ```
//!
//! ```ignore
//! # use reth_stages::Pipeline;
//! # use reth_stages::{StageSet, sets::OfflineStages};
//! # use reth_revm::EvmProcessorFactory;
//! # use reth_node_ethereum::EthEvmConfig;
//! # use reth_primitives::MAINNET;
//! # use reth_config::config::EtlConfig;
//!
//! // Build a pipeline with all offline stages and a custom stage at the end.
//! # let executor_factory = EvmProcessorFactory::new(MAINNET.clone(), EthEvmConfig::default());
//! Pipeline::builder()
//! .add_stages(
//! OfflineStages::new(executor_factory, EtlConfig::default()).builder().add_stage(MyCustomStage)
//! )
//! .build();
//! # }
//! ```
use crate::{
stages::{
AccountHashingStage, BodyStage, ExecutionStage, FinishStage, HeaderStage,
@ -61,10 +46,11 @@ use crate::{
use reth_config::config::EtlConfig;
use reth_consensus::Consensus;
use reth_db::database::Database;
use reth_evm::execute::BlockExecutorProvider;
use reth_interfaces::p2p::{
bodies::downloader::BodyDownloader, headers::downloader::HeaderDownloader,
};
use reth_provider::{ExecutorFactory, HeaderSyncGapProvider, HeaderSyncMode};
use reth_provider::{HeaderSyncGapProvider, HeaderSyncMode};
use std::sync::Arc;
/// A set containing all stages to run a fully syncing instance of reth.
@ -98,7 +84,7 @@ pub struct DefaultStages<Provider, H, B, EF> {
etl_config: EtlConfig,
}
impl<Provider, H, B, EF> DefaultStages<Provider, H, B, EF> {
impl<Provider, H, B, E> DefaultStages<Provider, H, B, E> {
/// Create a new set of default stages with default values.
pub fn new(
provider: Provider,
@ -106,11 +92,11 @@ impl<Provider, H, B, EF> DefaultStages<Provider, H, B, EF> {
consensus: Arc<dyn Consensus>,
header_downloader: H,
body_downloader: B,
executor_factory: EF,
executor_factory: E,
etl_config: EtlConfig,
) -> Self
where
EF: ExecutorFactory,
E: BlockExecutorProvider,
{
Self {
online: OnlineStages::new(
@ -127,14 +113,14 @@ impl<Provider, H, B, EF> DefaultStages<Provider, H, B, EF> {
}
}
impl<Provider, H, B, EF> DefaultStages<Provider, H, B, EF>
impl<Provider, H, B, E> DefaultStages<Provider, H, B, E>
where
EF: ExecutorFactory,
E: BlockExecutorProvider,
{
/// Appends the default offline stages and default finish stage to the given builder.
pub fn add_offline_stages<DB: Database>(
default_offline: StageSetBuilder<DB>,
executor_factory: EF,
executor_factory: E,
etl_config: EtlConfig,
) -> StageSetBuilder<DB> {
StageSetBuilder::default()
@ -144,12 +130,12 @@ where
}
}
impl<Provider, H, B, EF, DB> StageSet<DB> for DefaultStages<Provider, H, B, EF>
impl<Provider, H, B, E, DB> StageSet<DB> for DefaultStages<Provider, H, B, E>
where
Provider: HeaderSyncGapProvider + 'static,
H: HeaderDownloader + 'static,
B: BodyDownloader + 'static,
EF: ExecutorFactory,
E: BlockExecutorProvider,
DB: Database + 'static,
{
fn builder(self) -> StageSetBuilder<DB> {
@ -269,7 +255,11 @@ impl<EF> OfflineStages<EF> {
}
}
impl<EF: ExecutorFactory, DB: Database> StageSet<DB> for OfflineStages<EF> {
impl<E, DB> StageSet<DB> for OfflineStages<E>
where
E: BlockExecutorProvider,
DB: Database,
{
fn builder(self) -> StageSetBuilder<DB> {
ExecutionStages::new(self.executor_factory)
.builder()
@ -281,23 +271,27 @@ impl<EF: ExecutorFactory, DB: Database> StageSet<DB> for OfflineStages<EF> {
/// A set containing all stages that are required to execute pre-existing block data.
#[derive(Debug)]
#[non_exhaustive]
pub struct ExecutionStages<EF> {
pub struct ExecutionStages<E> {
/// Executor factory that will create executors.
executor_factory: EF,
executor_factory: E,
}
impl<EF> ExecutionStages<EF> {
impl<E> ExecutionStages<E> {
/// Create a new set of execution stages with default values.
pub fn new(executor_factory: EF) -> Self {
pub fn new(executor_factory: E) -> Self {
Self { executor_factory }
}
}
impl<EF: ExecutorFactory, DB: Database> StageSet<DB> for ExecutionStages<EF> {
impl<E, DB> StageSet<DB> for ExecutionStages<E>
where
DB: Database,
E: BlockExecutorProvider,
{
fn builder(self) -> StageSetBuilder<DB> {
StageSetBuilder::default()
.add_stage(SenderRecoveryStage::default())
.add_stage(ExecutionStage::new_with_factory(self.executor_factory))
.add_stage(ExecutionStage::new_with_executor(self.executor_factory))
}
}

View File

@ -3,6 +3,7 @@ use num_traits::Zero;
use reth_db::{
cursor::DbCursorRO, database::Database, static_file::HeaderMask, tables, transaction::DbTx,
};
use reth_evm::execute::{BatchBlockExecutionOutput, BatchExecutor, BlockExecutorProvider};
use reth_exex::{ExExManagerHandle, ExExNotification};
use reth_primitives::{
stage::{
@ -12,9 +13,10 @@ use reth_primitives::{
};
use reth_provider::{
providers::{StaticFileProvider, StaticFileProviderRWRefMut, StaticFileWriter},
BlockReader, Chain, DatabaseProviderRW, ExecutorFactory, HeaderProvider,
BlockReader, BundleStateWithReceipts, Chain, DatabaseProviderRW, HeaderProvider,
LatestStateProviderRef, OriginalValuesKnown, ProviderError, StatsReader, TransactionVariant,
};
use reth_revm::database::StateProviderDatabase;
use reth_stages_api::{
BlockErrorKind, ExecInput, ExecOutput, MetricEvent, MetricEventsSender, Stage, StageError,
UnwindInput, UnwindOutput,
@ -59,10 +61,10 @@ use tracing::*;
/// to [tables::PlainStorageState]
// false positive, we cannot derive it if !DB: Debug.
#[allow(missing_debug_implementations)]
pub struct ExecutionStage<EF> {
pub struct ExecutionStage<E> {
metrics_tx: Option<MetricEventsSender>,
/// The stage's internal executor
executor_factory: EF,
/// The stage's internal block executor
executor_provider: E,
/// The commit thresholds of the execution stage.
thresholds: ExecutionStageThresholds,
/// The highest threshold (in number of blocks) for switching between incremental
@ -76,10 +78,10 @@ pub struct ExecutionStage<EF> {
exex_manager_handle: ExExManagerHandle,
}
impl<EF> ExecutionStage<EF> {
impl<E> ExecutionStage<E> {
/// Create new execution stage with specified config.
pub fn new(
executor_factory: EF,
executor_provider: E,
thresholds: ExecutionStageThresholds,
external_clean_threshold: u64,
prune_modes: PruneModes,
@ -88,19 +90,19 @@ impl<EF> ExecutionStage<EF> {
Self {
metrics_tx: None,
external_clean_threshold,
executor_factory,
executor_provider,
thresholds,
prune_modes,
exex_manager_handle,
}
}
/// Create an execution stage with the provided executor factory.
/// Create an execution stage with the provided executor.
///
/// The commit threshold will be set to 10_000.
pub fn new_with_factory(executor_factory: EF) -> Self {
pub fn new_with_executor(executor_provider: E) -> Self {
Self::new(
executor_factory,
executor_provider,
ExecutionStageThresholds::default(),
MERKLE_STAGE_DEFAULT_CLEAN_THRESHOLD,
PruneModes::none(),
@ -144,7 +146,10 @@ impl<EF> ExecutionStage<EF> {
}
}
impl<EF: ExecutorFactory> ExecutionStage<EF> {
impl<E> ExecutionStage<E>
where
E: BlockExecutorProvider,
{
/// Execute the stage.
pub fn execute_inner<DB: Database>(
&mut self,
@ -169,12 +174,11 @@ impl<EF: ExecutorFactory> ExecutionStage<EF> {
None
};
// Build executor
let mut executor = self.executor_factory.with_state(LatestStateProviderRef::new(
let db = StateProviderDatabase(LatestStateProviderRef::new(
provider.tx_ref(),
provider.static_file_provider().clone(),
));
executor.set_prune_modes(prune_modes);
let mut executor = self.executor_provider.batch_executor(db, prune_modes);
executor.set_tip(max_block);
// Progress tracking
@ -213,7 +217,8 @@ impl<EF: ExecutorFactory> ExecutionStage<EF> {
// Execute the block
let execute_start = Instant::now();
executor.execute_and_verify_receipt(&block, td).map_err(|error| StageError::Block {
executor.execute_one((&block, td).into()).map_err(|error| StageError::Block {
block: Box::new(block.header.clone().seal_slow()),
error: BlockErrorKind::Execution(error),
})?;
@ -245,7 +250,8 @@ impl<EF: ExecutorFactory> ExecutionStage<EF> {
}
}
let time = Instant::now();
let state = executor.take_output_state();
let BatchBlockExecutionOutput { bundle, receipts, first_block } = executor.finalize();
let state = BundleStateWithReceipts::new(bundle, receipts, first_block);
let write_preparation_duration = time.elapsed();
// Check if we should send a [`ExExNotification`] to execution extensions.
@ -383,7 +389,11 @@ fn calculate_gas_used_from_headers(
Ok(gas_total)
}
impl<EF: ExecutorFactory, DB: Database> Stage<DB> for ExecutionStage<EF> {
impl<E, DB> Stage<DB> for ExecutionStage<E>
where
DB: Database,
E: BlockExecutorProvider,
{
/// Return the id of the stage
fn id(&self) -> StageId {
StageId::Execution
@ -609,7 +619,7 @@ mod tests {
use alloy_rlp::Decodable;
use assert_matches::assert_matches;
use reth_db::{models::AccountBeforeTx, transaction::DbTxMut};
use reth_evm_ethereum::EthEvmConfig;
use reth_evm_ethereum::execute::EthExecutorProvider;
use reth_interfaces::executor::BlockValidationError;
use reth_primitives::{
address, hex_literal::hex, keccak256, stage::StageUnitCheckpoint, Account, Address,
@ -620,16 +630,14 @@ mod tests {
test_utils::create_test_provider_factory, AccountReader, ReceiptProvider,
StaticFileProviderFactory,
};
use reth_revm::EvmProcessorFactory;
use std::collections::BTreeMap;
fn stage() -> ExecutionStage<EvmProcessorFactory<EthEvmConfig>> {
let executor_factory = EvmProcessorFactory::new(
Arc::new(ChainSpecBuilder::mainnet().berlin_activated().build()),
EthEvmConfig::default(),
);
fn stage() -> ExecutionStage<EthExecutorProvider> {
let executor_provider = EthExecutorProvider::ethereum(Arc::new(
ChainSpecBuilder::mainnet().berlin_activated().build(),
));
ExecutionStage::new(
executor_factory,
executor_provider,
ExecutionStageThresholds {
max_blocks: Some(100),
max_changes: None,
@ -864,7 +872,7 @@ mod tests {
mode.receipts_log_filter = random_filter.clone();
}
let mut execution_stage: ExecutionStage<EvmProcessorFactory<EthEvmConfig>> = stage();
let mut execution_stage = stage();
execution_stage.prune_modes = mode.clone().unwrap_or_default();
let output = execution_stage.execute(&provider, input).unwrap();

View File

@ -50,7 +50,7 @@ mod tests {
transaction::{DbTx, DbTxMut},
AccountsHistory, DatabaseEnv,
};
use reth_evm_ethereum::EthEvmConfig;
use reth_evm_ethereum::execute::EthExecutorProvider;
use reth_exex::ExExManagerHandle;
use reth_interfaces::test_utils::generators::{self, random_block};
use reth_primitives::{
@ -61,7 +61,6 @@ mod tests {
providers::StaticFileWriter, AccountExtReader, ProviderFactory, ReceiptProvider,
StorageReader,
};
use reth_revm::EvmProcessorFactory;
use reth_stages_api::{ExecInput, Stage};
use std::sync::Arc;
@ -140,10 +139,9 @@ mod tests {
// Check execution and create receipts and changesets according to the pruning
// configuration
let mut execution_stage = ExecutionStage::new(
EvmProcessorFactory::new(
Arc::new(ChainSpecBuilder::mainnet().berlin_activated().build()),
EthEvmConfig::default(),
),
EthExecutorProvider::ethereum(Arc::new(
ChainSpecBuilder::mainnet().berlin_activated().build(),
)),
ExecutionStageThresholds {
max_blocks: Some(100),
max_changes: None,

View File

@ -4,6 +4,7 @@ use reth_db::{
tables,
transaction::{DbTx, DbTxMut},
};
use reth_evm::execute::BatchBlockExecutionOutput;
use reth_interfaces::provider::{ProviderError, ProviderResult};
use reth_primitives::{
logs_bloom,
@ -34,6 +35,22 @@ pub struct BundleStateWithReceipts {
first_block: BlockNumber,
}
// TODO(mattsse): unify the types, currently there's a cyclic dependency between
impl From<BatchBlockExecutionOutput> for BundleStateWithReceipts {
fn from(value: BatchBlockExecutionOutput) -> Self {
let BatchBlockExecutionOutput { bundle, receipts, first_block } = value;
Self { bundle, receipts, first_block }
}
}
// TODO(mattsse): unify the types, currently there's a cyclic dependency between
impl From<BundleStateWithReceipts> for BatchBlockExecutionOutput {
fn from(value: BundleStateWithReceipts) -> Self {
let BundleStateWithReceipts { bundle, receipts, first_block } = value;
Self { bundle, receipts, first_block }
}
}
/// Type used to initialize revms bundle state.
pub type BundleStateInit =
HashMap<Address, (Option<Account>, Option<Account>, HashMap<B256, (U256, U256)>)>;